diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index c7d36bdc0078..8b8d5d38b747 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -27,6 +27,7 @@ import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.Schema; 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.io.FileIO; @@ -394,4 +395,19 @@ public static Schema schemaFor(Table table, Long snapshotId, Long timestampMilli return table.schema(); } + + /** + * Fetch the snapshot at the head of the given branch in the given table. + * + * @param table a {@link Table} + * @param branch a branch + * @return the latest snapshot for the given branch + */ + public static Snapshot latestSnapshot(Table table, String branch) { + if (SnapshotRef.MAIN_BRANCH.equals(branch)) { + return table.currentSnapshot(); + } else { + return table.snapshot(branch); + } + } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index ff09cf754a42..64132f322de5 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -43,6 +43,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.Files; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.parquet.GenericParquetWriter; @@ -66,6 +67,7 @@ public abstract class SparkRowLevelOperationsTestBase extends SparkExtensionsTestBase { private static final Random RANDOM = ThreadLocalRandom.current(); + protected static final String TEST_BRANCH = "test_branch"; protected final String fileFormat; protected final boolean vectorized; @@ -173,14 +175,23 @@ protected void createAndInitTable(String schema, String jsonData) { createAndInitTable(schema, "", jsonData); } + protected void createAndInitTableBranch(String schema, String jsonData, String branch) { + createAndInitTableBranch(schema, "", jsonData, branch); + } + protected void createAndInitTable(String schema, String partitioning, String jsonData) { + createAndInitTableBranch(schema, partitioning, jsonData, SnapshotRef.MAIN_BRANCH); + } + + protected void createAndInitTableBranch( + String schema, String partitioning, String jsonData, String branch) { 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(); + ds.coalesce(1).writeTo(tableNameWithBranch(branch)).append(); } catch (NoSuchTableException e) { throw new RuntimeException("Failed to write data", e); } @@ -315,4 +326,12 @@ protected DataFile writeDataFile(Table table, List records) { throw new UncheckedIOException(e); } } + + protected String tableNameWithBranch(String branch) { + if (branch.equals(SnapshotRef.MAIN_BRANCH)) { + return tableName; + } + + return tableName + "." + branch; + } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchDelete.java new file mode 100644 index 000000000000..f0ff93a5dc34 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchDelete.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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_DISTRIBUTION_MODE_NONE; + +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestCopyOnWriteBranchDelete extends TestCopyOnWriteDelete { + + public TestCopyOnWriteBranchDelete( + String catalogName, + String implementation, + Map config, + String fileFormat, + Boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + @Parameterized.Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + "parquet", + true, + WRITE_DISTRIBUTION_MODE_NONE + }, + }; + } + + @Test + public void testDeleteWithoutScanningTable() throws Exception { + testDeleteWithoutScanningTable(TEST_BRANCH); + } + + @Test + public void testDeleteFileThenMetadataDelete() throws Exception { + testDeleteFileThenMetadataDelete(TEST_BRANCH); + } + + @Test + public void testDeleteWithFalseCondition() { + testDeleteWithFalseCondition(TEST_BRANCH); + } + + @Test + public void testDeleteFromEmptyTable() { + testDeleteFromEmptyTable(TEST_BRANCH); + } + + @Test + public void testExplain() { + testExplain(TEST_BRANCH); + } + + @Test + public void testDeleteWithAlias() { + testDeleteWithAlias(TEST_BRANCH); + } + + @Test + public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { + testDeleteWithDynamicFileFiltering(TEST_BRANCH); + } + + @Test + public void testDeleteNonExistingRecords() { + testDeleteNonExistingRecords(TEST_BRANCH); + } + + @Test + public void testDeleteWithoutCondition() { + testDeleteWithoutCondition(TEST_BRANCH); + } + + @Test + public void testDeleteUsingMetadataWithComplexCondition() { + testDeleteUsingMetadataWithComplexCondition(TEST_BRANCH); + } + + @Test + public void testDeleteWithArbitraryPartitionPredicates() { + testDeleteWithArbitraryPartitionPredicates(TEST_BRANCH); + } + + @Test + public void testDeleteWithNonDeterministicCondition() { + testDeleteWithNonDeterministicCondition(TEST_BRANCH); + } + + @Test + public void testDeleteWithFoldableConditions() { + testDeleteWithFoldableConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithNullConditions() { + testDeleteWithNullConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithInAndNotInConditions() { + testDeleteWithInAndNotInConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException { + testDeleteWithMultipleRowGroupsParquet(TEST_BRANCH); + } + + @Test + public void testDeleteWithConditionOnNestedColumn() { + testDeleteWithConditionOnNestedColumn(TEST_BRANCH); + } + + @Test + public void testDeleteWithInSubquery() throws NoSuchTableException { + testDeleteWithInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { + testDeleteWithNotInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithNotInSubquery() throws NoSuchTableException { + testDeleteWithNotInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteOnNonIcebergTableNotSupported() { + testDeleteOnNonIcebergTableNotSupported(TEST_BRANCH); + } + + @Test + public void testDeleteWithExistSubquery() throws NoSuchTableException { + testDeleteWithExistSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { + testDeleteWithNotExistsSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithScalarSubquery() throws NoSuchTableException { + testDeleteWithScalarSubquery(TEST_BRANCH); + } + + @Test + public synchronized void testDeleteWithSerializableIsolation() throws InterruptedException { + testDeleteWithSerializableIsolation(TEST_BRANCH); + } + + @Test + public synchronized void testDeleteWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + testDeleteWithSnapshotIsolation(TEST_BRANCH); + } + + @Test + public void testDeleteRefreshesRelationCache() throws NoSuchTableException { + testDeleteRefreshesRelationCache(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultipleSpecs() { + testDeleteWithMultipleSpecs(TEST_BRANCH); + } +} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchMerge.java new file mode 100644 index 000000000000..d566d2b033bc --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteBranchMerge.java @@ -0,0 +1,429 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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_DISTRIBUTION_MODE_NONE; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +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.SparkCatalog; +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.junit.Assert; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestCopyOnWriteBranchMerge extends TestCopyOnWriteMerge { + + public TestCopyOnWriteBranchMerge( + String catalogName, + String implementation, + Map config, + String fileFormat, + boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + @Parameterized.Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + "parquet", + true, + WRITE_DISTRIBUTION_MODE_NONE + }, + }; + } + + @Test + public void testMergeConditionSplitIntoTargetPredicateAndJoinCondition() { + testMergeConditionSplitIntoTargetPredicateAndJoinCondition(TEST_BRANCH); + } + + @Test + public void testMergeWithStaticPredicatePushDown() { + testMergeWithStaticPredicatePushDown(TEST_BRANCH); + } + + @Test + public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { + testMergeIntoEmptyTargetInsertAllNonMatchingRows(TEST_BRANCH); + } + + @Test + public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { + testMergeIntoEmptyTargetInsertOnlyMatchingRows(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyUpdateClause() { + testMergeWithOnlyUpdateClause(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyUpdateClauseAndNullValues() { + testMergeWithOnlyUpdateClauseAndNullValues(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyDeleteClause() { + testMergeWithOnlyDeleteClause(TEST_BRANCH); + } + + @Test + public void testMergeWithAllCauses() { + testMergeWithAllCauses(TEST_BRANCH); + } + + @Test + public void testMergeWithAllCausesWithExplicitColumnSpecification() { + testMergeWithAllCausesWithExplicitColumnSpecification(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceCTE() { + testMergeWithSourceCTE(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceFromSetOps() { + testMergeWithSourceFromSetOps(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource(TEST_BRANCH); + } + + @Test + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqualityCondition() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions(TEST_BRANCH); + } + + @Test + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRow() { + testMergeWithMultipleUpdatesForTargetRow(TEST_BRANCH); + } + + @Test + public void testMergeWithUnconditionalDelete() { + testMergeWithUnconditionalDelete(TEST_BRANCH); + } + + @Test + public void testMergeWithSingleConditionalDelete() { + testMergeWithSingleConditionalDelete(TEST_BRANCH); + } + + @Test + public void testMergeWithIdentityTransform() { + testMergeWithIdentityTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithDaysTransform() { + testMergeWithDaysTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithBucketTransform() { + testMergeWithBucketTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithTruncateTransform() { + testMergeWithTruncateTransform(TEST_BRANCH); + } + + @Test + public void testMergeIntoPartitionedAndOrderedTable() { + testMergeIntoPartitionedAndOrderedTable(TEST_BRANCH); + } + + @Test + public void testSelfMerge() { + testSelfMerge(TEST_BRANCH); + } + + @Test + public void testSelfMergeWithCaching() { + testSelfMergeWithCaching(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceAsSelfSubquery() { + testMergeWithSourceAsSelfSubquery(TEST_BRANCH); + } + + @Test + public synchronized void testMergeWithSerializableIsolation() throws InterruptedException { + testMergeWithSerializableIsolation(TEST_BRANCH); + } + + @Test + public synchronized void testMergeWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + testMergeWithSnapshotIsolation(TEST_BRANCH); + } + + @Test + public void testMergeWithExtraColumnsInSource() { + testMergeWithExtraColumnsInSource(TEST_BRANCH); + } + + @Test + public void testMergeWithNullsInTargetAndSource() { + testMergeWithNullsInTargetAndSource(TEST_BRANCH); + } + + @Test + public void testMergeWithNullSafeEquals() { + testMergeWithNullSafeEquals(TEST_BRANCH); + } + + @Test + public void testMergeWithNullCondition() { + testMergeWithNullCondition(TEST_BRANCH); + } + + @Test + public void testMergeWithNullActionConditions() { + testMergeWithNullActionConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleMatchingActions() { + testMergeWithMultipleMatchingActions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException { + testMergeWithMultipleRowGroupsParquet(TEST_BRANCH); + } + + @Test + public void testMergeInsertOnly() { + testMergeInsertOnly(TEST_BRANCH); + } + + @Test + public void testMergeInsertOnlyWithCondition() { + testMergeInsertOnlyWithCondition(TEST_BRANCH); + } + + @Test + public void testMergeAlignsUpdateAndInsertActions() { + testMergeAlignsUpdateAndInsertActions(TEST_BRANCH); + } + + @Test + public void testMergeMixedCaseAlignsUpdateAndInsertActions() { + testMergeMixedCaseAlignsUpdateAndInsertActions(TEST_BRANCH); + } + + @Test + public void testMergeUpdatesNestedStructFields() { + testMergeUpdatesNestedStructFields(TEST_BRANCH); + } + + @Test + public void testMergeWithInferredCasts() { + testMergeWithInferredCasts(TEST_BRANCH); + } + + @Test + public void testMergeModifiesNullStruct() { + testMergeModifiesNullStruct(TEST_BRANCH); + } + + @Test + public void testMergeRefreshesRelationCache() { + testMergeRefreshesRelationCache(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleNotMatchedActions() { + testMergeWithMultipleNotMatchedActions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleConditionalNotMatchedActions() { + testMergeWithMultipleConditionalNotMatchedActions(TEST_BRANCH); + } + + @Test + public void testMergeResolvesColumnsByName() { + testMergeResolvesColumnsByName(TEST_BRANCH); + } + + @Test + public void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns() { + testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns(TEST_BRANCH); + } + + @Test + public void testMergeWithTableWithNonNullableColumn() { + testMergeWithTableWithNonNullableColumn(TEST_BRANCH); + } + + @Test + public void testMergeWithNonExistingColumns() { + testMergeWithNonExistingColumns(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidColumnsInInsert() { + testMergeWithInvalidColumnsInInsert(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidUpdates() { + testMergeWithInvalidUpdates(TEST_BRANCH); + } + + @Test + public void testMergeWithConflictingUpdates() { + testMergeWithConflictingUpdates(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidAssignments() { + testMergeWithInvalidAssignments(TEST_BRANCH); + } + + @Test + public void testMergeWithNonDeterministicConditions() { + testMergeWithNonDeterministicConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithAggregateExpressions() { + testMergeWithAggregateExpressions(TEST_BRANCH); + } + + @Test + public void testMergeWithSubqueriesInConditions() { + testMergeWithSubqueriesInConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithTargetColumnsInInsertConditions() { + testMergeWithTargetColumnsInInsertConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithNonIcebergTargetTableNotSupported() { + testMergeWithNonIcebergTargetTableNotSupported(TEST_BRANCH); + } + + /** + * 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. + */ + @Test + public void testMergeSinglePartitionPartitioning() { + testMergeSinglePartitionPartitioning(TEST_BRANCH); + } + + @Test + public void testMergeEmptyTable() { + testMergeEmptyTable(TEST_BRANCH); + } + + @Test + public void testRuntimeFilteringWithReportedPartitioning() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append( + tableNameWithBranch(TEST_BRANCH), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + tableNameWithBranch(TEST_BRANCH), + "{ \"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", + tableNameWithBranch(TEST_BRANCH))); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, TEST_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", tableNameWithBranch(TEST_BRANCH))); + } +} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index 5c9d547a6aba..8afbeef318e8 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -33,6 +33,7 @@ import org.apache.iceberg.DataFile; 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; @@ -153,8 +154,8 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception public void testRuntimeFilteringWithPreservedDataGrouping() throws NoSuchTableException { createAndInitPartitionedTable(); - append(new Employee(1, "hr"), new Employee(3, "hr")); - append(new Employee(1, "hardware"), new Employee(2, "hardware")); + append(SnapshotRef.MAIN_BRANCH, new Employee(1, "hr"), new Employee(3, "hr")); + append(SnapshotRef.MAIN_BRANCH, new Employee(1, "hardware"), new Employee(2, "hardware")); Map sqlConf = ImmutableMap.of( diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 0b73821c617d..bc3b469491e6 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -18,62 +18,15 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; -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_SIZE; -import static org.apache.spark.sql.functions.lit; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; import java.util.Map; -import java.util.Set; 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.AssertHelpers; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.RowLevelOperationMode; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericRecord; -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.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.Spark3Util; -import org.apache.iceberg.spark.data.TestHelpers; -import org.apache.spark.SparkException; -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.iceberg.SnapshotRef; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.catalyst.parser.ParseException; -import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable; -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromIcebergTable; -import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; -public abstract class TestDelete extends SparkRowLevelOperationsTestBase { +public abstract class TestDelete extends TestDeleteBase { public TestDelete( String catalogName, @@ -99,946 +52,142 @@ public void removeTables() { @Test public void testDeleteWithoutScanningTable() throws Exception { - createAndInitPartitionedTable(); - - append(new Employee(1, "hr"), new Employee(3, "hr")); - append(new Employee(1, "hardware"), new Employee(2, "hardware")); - - Table table = validationCatalog.loadTable(tableIdent); - - List manifestLocations = - table.currentSnapshot().allManifests(table.io()).stream() - .map(ManifestFile::path) - .collect(Collectors.toList()); - - withUnavailableLocations( - manifestLocations, - () -> { - LogicalPlan parsed = parsePlan("DELETE FROM %s WHERE dep = 'hr'", tableName); - - DeleteFromIcebergTable analyzed = - (DeleteFromIcebergTable) spark.sessionState().analyzer().execute(parsed); - Assert.assertTrue("Should have rewrite plan", analyzed.rewritePlan().isDefined()); - - DeleteFromIcebergTable optimized = - (DeleteFromIcebergTable) OptimizeMetadataOnlyDeleteFromIcebergTable.apply(analyzed); - Assert.assertTrue("Should discard rewrite plan", optimized.rewritePlan().isEmpty()); - }); - - sql("DELETE FROM %s WHERE dep = 'hr'", tableName); - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hardware"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testDeleteWithoutScanningTable(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteFileThenMetadataDelete() throws Exception { - Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - // MOR mode: writes a delete file as null cannot be deleted by metadata - sql("DELETE FROM %s AS t WHERE t.id IS NULL", tableName); - - // Metadata Delete - Table table = Spark3Util.loadIcebergTable(spark, tableName); - Set dataFilesBefore = TestHelpers.dataFiles(table); - - sql("DELETE FROM %s AS t WHERE t.id = 1", tableName); - - Set dataFilesAfter = TestHelpers.dataFiles(table); - Assert.assertTrue( - "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testDeleteFileThenMetadataDelete(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithFalseCondition() { - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); - - sql("DELETE FROM %s WHERE id = 1 AND id > 20", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testDeleteWithFalseCondition(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteFromEmptyTable() { - createAndInitUnpartitionedTable(); - - sql("DELETE FROM %s WHERE id IN (1)", tableName); - sql("DELETE FROM %s WHERE dep = 'hr'", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); - - assertEquals( - "Should have expected rows", - ImmutableList.of(), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testDeleteFromEmptyTable(SnapshotRef.MAIN_BRANCH); } @Test public void testExplain() { - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - sql("EXPLAIN DELETE FROM %s WHERE id <=> 1", tableName); - - sql("EXPLAIN DELETE FROM %s WHERE true", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); - - 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", tableName)); + testExplain(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithAlias() { - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - sql("DELETE FROM %s AS t WHERE t.id IS NULL", tableName); - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testDeleteWithAlias(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { - createAndInitPartitionedTable(); - - append(new Employee(1, "hr"), new Employee(3, "hr")); - append(new Employee(1, "hardware"), new Employee(2, "hardware")); - - sql("DELETE FROM %s WHERE id = 2", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); - - Snapshot currentSnapshot = table.currentSnapshot(); - 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", tableName)); + testDeleteWithDynamicFileFiltering(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteNonExistingRecords() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - sql("DELETE FROM %s AS t WHERE t.id > 10", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); - - Snapshot currentSnapshot = table.currentSnapshot(); - - if (fileFormat.equals("orc") || fileFormat.equals("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", tableName)); + testDeleteNonExistingRecords(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithoutCondition() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); - sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", tableName); - sql("INSERT INTO TABLE %s VALUES (null, 'hr')", tableName); - - sql("DELETE FROM %s", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); - - // should be a delete instead of an overwrite as it is done through a metadata operation - Snapshot currentSnapshot = table.currentSnapshot(); - validateDelete(currentSnapshot, "2", "3"); - - assertEquals( - "Should have expected rows", ImmutableList.of(), sql("SELECT * FROM %s", tableName)); + testDeleteWithoutCondition(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteUsingMetadataWithComplexCondition() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'dep1')", tableName); - sql("INSERT INTO TABLE %s VALUES (2, 'dep2')", tableName); - sql("INSERT INTO TABLE %s VALUES (null, 'dep3')", tableName); - - sql("DELETE FROM %s WHERE dep > 'dep2' OR dep = CAST(4 AS STRING) OR dep = 'dep2'", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); - - // should be a delete instead of an overwrite as it is done through a metadata operation - Snapshot currentSnapshot = table.currentSnapshot(); - validateDelete(currentSnapshot, "2", "2"); - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "dep1")), - sql("SELECT * FROM %s", tableName)); + testDeleteUsingMetadataWithComplexCondition(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithArbitraryPartitionPredicates() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); - sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", tableName); - sql("INSERT INTO TABLE %s VALUES (null, 'hr')", tableName); - - // %% is an escaped version of % - sql("DELETE FROM %s WHERE id = 10 OR dep LIKE '%%ware'", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); - - // should be an overwrite since cannot be executed using a metadata operation - Snapshot currentSnapshot = table.currentSnapshot(); - 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", tableName)); + testDeleteWithArbitraryPartitionPredicates(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithNonDeterministicCondition() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); - - AssertHelpers.assertThrows( - "Should complain about non-deterministic expressions", - AnalysisException.class, - "nondeterministic expressions are only allowed", - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", tableName)); + testDeleteWithNonDeterministicCondition(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithFoldableConditions() { - createAndInitPartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); - - // should keep all rows and don't trigger execution - sql("DELETE FROM %s WHERE false", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - // should keep all rows and don't trigger execution - sql("DELETE FROM %s WHERE 50 <> 50", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - // should keep all rows and don't trigger execution - sql("DELETE FROM %s WHERE 1 > null", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - // should remove all rows - sql("DELETE FROM %s WHERE 21 = 21", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + testDeleteWithFoldableConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithNullConditions() { - createAndInitPartitionedTable(); - - sql( - "INSERT INTO TABLE %s VALUES (0, null), (1, 'hr'), (2, 'hardware'), (null, 'hr')", - tableName); - - // should keep all rows as null is never equal to null - sql("DELETE FROM %s WHERE dep = null", tableName); - 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", tableName)); - - // null = 'software' -> null - // should delete using metadata operation only - sql("DELETE FROM %s WHERE dep = 'software'", tableName); - 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", tableName)); - - // should delete using metadata operation only - sql("DELETE FROM %s WHERE dep <=> NULL", tableName); - 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", tableName)); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); - - Snapshot currentSnapshot = table.currentSnapshot(); - validateDelete(currentSnapshot, "1", "1"); + testDeleteWithNullConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithInAndNotInConditions() { - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - sql("DELETE FROM %s WHERE id IN (1, null)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - sql("DELETE FROM %s WHERE id NOT IN (null, 1)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - sql("DELETE FROM %s WHERE id NOT IN (1, 10)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testDeleteWithInAndNotInConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException { - Assume.assumeTrue(fileFormat.equalsIgnoreCase("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(); - - Assert.assertEquals(200, spark.table(tableName).count()); - - // delete a record from one of two row groups and copy over the second one - sql("DELETE FROM %s WHERE id IN (200, 201)", tableName); - - Assert.assertEquals(199, spark.table(tableName).count()); + testDeleteWithMultipleRowGroupsParquet(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithConditionOnNestedColumn() { - 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 = id + 2", 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 = id", tableName); - assertEquals( - "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", tableName)); + testDeleteWithConditionOnNestedColumn(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithInSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - - 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)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - 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", tableName)); - - sql("DELETE FROM %s WHERE id IS NULL OR id IN (SELECT value + 2 FROM deleted_id)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(-1, "hr"), row(1, "hr")), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - 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", tableName)); - - sql("DELETE FROM %s WHERE id IN (SELECT value + 2 FROM deleted_id) AND dep = 'hr'", tableName); - 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", tableName)); + testDeleteWithInSubquery(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - append(new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); - - 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)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testDeleteWithNotInSubquery(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithNotInSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - append(new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); - - 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)", tableName); - 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", tableName)); - - sql( - "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id WHERE value IS NOT NULL)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); - 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", tableName)); - - sql( - "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id) OR dep IN ('software', 'hr')", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - 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)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - 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)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testDeleteWithNotInSubquery(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteOnNonIcebergTableNotSupported() { - createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Delete is supported only for Iceberg tables", - AnalysisException.class, - "DELETE is only supported with v2 tables.", - () -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")); + testDeleteOnNonIcebergTableNotSupported(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithExistSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - append(new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); - - 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)", - tableName); - 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", tableName)); - - sql( - "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - sql( - "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value) OR t.id IS NULL", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware")), - sql("SELECT * FROM %s", tableName)); - - 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)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware")), - sql("SELECT * FROM %s", tableName)); + testDeleteWithExistSubquery(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - append(new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); - - 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)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - sql( - "DELETE FROM %s t WHERE NOT EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", - tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(1, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - - 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", tableName, subquery); - assertEquals( - "Should have expected rows", - ImmutableList.of(), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testDeleteWithNotExistsSubquery(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithScalarSubquery() throws NoSuchTableException { - createAndInitUnpartitionedTable(); - - append(new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); - - 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)", tableName); - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hardware"), row(null, "hr")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); - }); - } - - @Test - public void testDeleteThatRequiresGroupingBeforeWrite() throws NoSuchTableException { - createAndInitPartitionedTable(); - - 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")); - 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)", tableName); - Assert.assertEquals("Should have expected num of rows", 8L, spark.table(tableName).count()); - } finally { - spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); - } + testDeleteWithScalarSubquery(SnapshotRef.MAIN_BRANCH); } @Test public synchronized void testDeleteWithSerializableIsolation() throws InterruptedException { - // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); - - 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); - - 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++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } - - sql("DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", tableName); - - 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++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - table.newFastAppend().appendFile(dataFile).commit(); - sleep(10); - } - - barrier.incrementAndGet(); - } - }); - - try { - Assertions.assertThatThrownBy(deleteFuture::get) - .isInstanceOf(ExecutionException.class) - .cause() - .isInstanceOf(SparkException.class) - .cause() - .isInstanceOf(ValidationException.class) - .hasMessageContaining("Found conflicting files that can contain"); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + testDeleteWithSerializableIsolation(SnapshotRef.MAIN_BRANCH); } @Test public synchronized void testDeleteWithSnapshotIsolation() throws InterruptedException, ExecutionException { - // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); - - 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); - - 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++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } - - sql("DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", 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(table.schema()); - record.set(0, 1); // id - record.set(1, "hr"); // dep - - for (int numOperations = 0; numOperations < 20; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - table.newFastAppend().appendFile(dataFile).commit(); - sleep(10); - } - - barrier.incrementAndGet(); - } - }); - - try { - deleteFuture.get(); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + testDeleteWithSnapshotIsolation(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteRefreshesRelationCache() throws NoSuchTableException { - createAndInitPartitionedTable(); - - append(new Employee(1, "hr"), new Employee(3, "hr")); - append(new Employee(1, "hardware"), new Employee(2, "hardware")); - - Dataset query = spark.sql("SELECT * FROM " + tableName + " 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", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); - - Snapshot currentSnapshot = table.currentSnapshot(); - 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", tableName)); - - assertEquals( - "Should refresh the relation cache", - ImmutableList.of(), - sql("SELECT * FROM tmp ORDER BY id, dep")); - - spark.sql("UNCACHE TABLE tmp"); + testDeleteRefreshesRelationCache(SnapshotRef.MAIN_BRANCH); } @Test public void testDeleteWithMultipleSpecs() { - createAndInitTable("id INT, dep STRING, category STRING"); - - // write an unpartitioned file - append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}"); - - // write a file partitioned by dep - sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); - append( - tableName, - "{ \"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(tableName, "{ \"id\": 5, \"dep\": \"hr\", \"category\": \"c1\"}"); - - // write another file partitioned by dep - sql("ALTER TABLE %s DROP PARTITION FIELD category", tableName); - append(tableName, "{ \"id\": 7, \"dep\": \"hr\", \"category\": \"c1\"}"); - - sql("DELETE FROM %s WHERE id IN (1, 3, 5, 7)", tableName); - - Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 5 snapshots", 5, Iterables.size(table.snapshots())); - - Snapshot currentSnapshot = table.currentSnapshot(); - if (mode(table) == COPY_ON_WRITE) { - // copy-on-write is tested against v1 and such tables have different partition evolution - // behavior - // that's why the number of changed partitions is 4 for copy-on-write - validateCopyOnWrite(currentSnapshot, "4", "4", "1"); - } else { - validateMergeOnRead(currentSnapshot, "3", "3", null); - } - - assertEquals( - "Should have expected rows", - ImmutableList.of(row(2, "hr", "c1")), - sql("SELECT * FROM %s ORDER BY id", 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 { - List input = Arrays.asList(employees); - Dataset inputDF = spark.createDataFrame(input, Employee.class); - inputDF.coalesce(1).writeTo(tableName).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); - } + testDeleteWithMultipleSpecs(SnapshotRef.MAIN_BRANCH); } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDeleteBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDeleteBase.java new file mode 100644 index 000000000000..ce2d5ef62527 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDeleteBase.java @@ -0,0 +1,1046 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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.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_SIZE; +import static org.apache.spark.sql.functions.lit; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +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.AssertHelpers; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericRecord; +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.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.Spark3Util; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.SparkException; +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.DeleteFromIcebergTable; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromIcebergTable; +import org.apache.spark.sql.internal.SQLConf; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; + +abstract class TestDeleteBase extends SparkRowLevelOperationsTestBase { + + protected TestDeleteBase( + String catalogName, + String implementation, + Map config, + String fileFormat, + boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + protected void testDeleteWithoutScanningTable(String branch) throws Exception { + createAndInitPartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(3, "hr")); + append(branch, 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'", tableNameWithBranch(branch)); + + DeleteFromIcebergTable analyzed = + (DeleteFromIcebergTable) spark.sessionState().analyzer().execute(parsed); + Assert.assertTrue("Should have rewrite plan", analyzed.rewritePlan().isDefined()); + + DeleteFromIcebergTable optimized = + (DeleteFromIcebergTable) OptimizeMetadataOnlyDeleteFromIcebergTable.apply(analyzed); + Assert.assertTrue("Should discard rewrite plan", optimized.rewritePlan().isEmpty()); + }); + + sql("DELETE FROM %s WHERE dep = 'hr'", tableNameWithBranch(branch)); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hardware"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected void testDeleteFileThenMetadataDelete(String branch) throws Exception { + Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); + createAndInitUnpartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + // MOR mode: writes a delete file as null cannot be deleted by metadata + sql("DELETE FROM %s AS t WHERE t.id IS NULL", tableNameWithBranch(branch)); + + // Metadata Delete + Table table = Spark3Util.loadIcebergTable(spark, tableNameWithBranch(branch)); + Set dataFilesBefore = TestHelpers.dataFiles(table, branch); + + sql("DELETE FROM %s AS t WHERE t.id = 1", tableNameWithBranch(branch)); + + Set dataFilesAfter = TestHelpers.dataFiles(table, branch); + Assert.assertTrue( + "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected void testDeleteWithFalseCondition(String branch) { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableNameWithBranch(branch)); + + sql("DELETE FROM %s WHERE id = 1 AND id > 20", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected void testDeleteFromEmptyTable(String branch) { + createAndInitUnpartitionedTable(); + + sql("DELETE FROM %s WHERE id IN (1)", tableNameWithBranch(branch)); + sql("DELETE FROM %s WHERE dep = 'hr'", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected void testExplain(String branch) { + createAndInitUnpartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + sql("EXPLAIN DELETE FROM %s WHERE id <=> 1", tableNameWithBranch(branch)); + + sql("EXPLAIN DELETE FROM %s WHERE true", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); + + 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", tableNameWithBranch(branch))); + } + + protected void testDeleteWithAlias(String branch) { + createAndInitUnpartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + sql("DELETE FROM %s AS t WHERE t.id IS NULL", tableNameWithBranch(branch)); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected void testDeleteWithDynamicFileFiltering(String branch) throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(3, "hr")); + append(branch, new Employee(1, "hardware"), new Employee(2, "hardware")); + + sql("DELETE FROM %s WHERE id = 2", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + + 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", tableNameWithBranch(branch))); + } + + protected void testDeleteNonExistingRecords(String branch) { + createAndInitPartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + sql("DELETE FROM %s AS t WHERE t.id > 10", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (fileFormat.equals("orc") || fileFormat.equals("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", tableNameWithBranch(branch))); + } + + protected void testDeleteWithoutCondition(String branch) { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (null, 'hr')", tableNameWithBranch(branch)); + + sql("DELETE FROM %s", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + + // 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", tableNameWithBranch(branch))); + } + + protected void testDeleteUsingMetadataWithComplexCondition(String branch) { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'dep1')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (2, 'dep2')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (null, 'dep3')", tableNameWithBranch(branch)); + + sql( + "DELETE FROM %s WHERE dep > 'dep2' OR dep = CAST(4 AS STRING) OR dep = 'dep2'", + tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + + // 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", tableNameWithBranch(branch))); + } + + protected void testDeleteWithArbitraryPartitionPredicates(String branch) { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", tableNameWithBranch(branch)); + sql("INSERT INTO TABLE %s VALUES (null, 'hr')", tableNameWithBranch(branch)); + + // %% is an escaped version of % + sql("DELETE FROM %s WHERE id = 10 OR dep LIKE '%%ware'", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + + // should be an overwrite since cannot be executed using a metadata operation + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + 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", tableNameWithBranch(branch))); + } + + protected void testDeleteWithNonDeterministicCondition(String branch) { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableNameWithBranch(branch)); + + AssertHelpers.assertThrows( + "Should complain about non-deterministic expressions", + AnalysisException.class, + "nondeterministic expressions are only allowed", + () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", tableNameWithBranch(branch))); + } + + protected void testDeleteWithFoldableConditions(String branch) { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableNameWithBranch(branch)); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE false", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE 50 <> 50", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE 1 > null", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + // should remove all rows + sql("DELETE FROM %s WHERE 21 = 21", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + } + + protected void testDeleteWithNullConditions(String branch) { + createAndInitPartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (0, null), (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + // should keep all rows as null is never equal to null + sql("DELETE FROM %s WHERE dep = null", tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + // null = 'software' -> null + // should delete using metadata operation only + sql("DELETE FROM %s WHERE dep = 'software'", tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + // should delete using metadata operation only + sql("DELETE FROM %s WHERE dep <=> NULL", tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateDelete(currentSnapshot, "1", "1"); + } + + protected void testDeleteWithInAndNotInConditions(String branch) { + createAndInitUnpartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + sql("DELETE FROM %s WHERE id IN (1, null)", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + sql("DELETE FROM %s WHERE id NOT IN (null, 1)", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + sql("DELETE FROM %s WHERE id NOT IN (1, 10)", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected void testDeleteWithMultipleRowGroupsParquet(String branch) throws NoSuchTableException { + Assume.assumeTrue(fileFormat.equalsIgnoreCase("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(tableNameWithBranch(branch)).append(); + + Assert.assertEquals(200, spark.table(tableNameWithBranch(branch)).count()); + + // delete a record from one of two row groups and copy over the second one + sql("DELETE FROM %s WHERE id IN (200, 201)", tableNameWithBranch(branch)); + + Assert.assertEquals(199, spark.table(tableNameWithBranch(branch)).count()); + } + + protected void testDeleteWithConditionOnNestedColumn(String branch) { + createAndInitNestedColumnsTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", + tableNameWithBranch(branch)); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", + tableNameWithBranch(branch)); + + sql("DELETE FROM %s WHERE complex.c1 = id + 2", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2)), + sql("SELECT id FROM %s", tableNameWithBranch(branch))); + + sql("DELETE FROM %s t WHERE t.complex.c1 = id", tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT id FROM %s", tableNameWithBranch(branch))); + } + + protected void testDeleteWithInSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + append(branch, 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", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s WHERE id IS NULL OR id IN (SELECT value + 2 FROM deleted_id)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + append(branch, 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", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s WHERE id IN (SELECT value + 2 FROM deleted_id) AND dep = 'hr'", + tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + } + + protected void testDeleteWithMultiColumnInSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected void testDeleteWithNotInSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + + 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)", tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id WHERE value IS NOT NULL)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + sql( + "INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id) OR dep IN ('software', 'hr')", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected void testDeleteOnNonIcebergTableNotSupported(String branch) { + createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Delete is supported only for Iceberg tables", + AnalysisException.class, + "DELETE is only supported with v2 tables.", + () -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")); + } + + protected void testDeleteWithExistSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + + 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)", + tableNameWithBranch(branch)); + 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", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value) OR t.id IS NULL", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s", tableNameWithBranch(branch))); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s", tableNameWithBranch(branch))); + } + + protected void testDeleteWithNotExistsSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + sql( + "DELETE FROM %s t WHERE NOT EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + + 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", + tableNameWithBranch(branch), subquery); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected void testDeleteWithScalarSubquery(String branch) throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + + 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)", + tableNameWithBranch(branch)); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + }); + } + + protected void testDeleteThatRequiresGroupingBeforeWrite(String branch) + throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(branch, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + append(branch, new Employee(0, "ops"), new Employee(1, "ops"), new Employee(2, "ops")); + append(branch, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + append(branch, 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)", tableNameWithBranch(branch)); + Assert.assertEquals( + "Should have expected num of rows", 8L, spark.table(tableNameWithBranch(branch)).count()); + } finally { + spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); + } + } + + protected synchronized void testDeleteWithSerializableIsolation(String branch) + throws InterruptedException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + + 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')", tableNameWithBranch(branch)); + + 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++) { + while (barrier.get() < numOperations * 2) { + sleep(10); + } + + sql( + "DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", + tableNameWithBranch(branch)); + + 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++) { + while (shouldAppend.get() && barrier.get() < numOperations * 2) { + sleep(10); + } + + if (!shouldAppend.get()) { + return; + } + + for (int numAppends = 0; numAppends < 5; numAppends++) { + DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); + table.newFastAppend().appendFile(dataFile).toBranch(branch).commit(); + sleep(10); + } + + barrier.incrementAndGet(); + } + }); + + try { + Assertions.assertThatThrownBy(deleteFuture::get) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Found conflicting files that can contain"); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + } + + protected synchronized void testDeleteWithSnapshotIsolation(String branch) + throws InterruptedException, ExecutionException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + + 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')", tableNameWithBranch(branch)); + + 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++) { + while (barrier.get() < numOperations * 2) { + sleep(10); + } + + sql( + "DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", + tableNameWithBranch(branch)); + + 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++) { + while (shouldAppend.get() && barrier.get() < numOperations * 2) { + sleep(10); + } + + if (!shouldAppend.get()) { + return; + } + + for (int numAppends = 0; numAppends < 5; numAppends++) { + DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); + table.newFastAppend().appendFile(dataFile).commit(); + sleep(10); + } + + barrier.incrementAndGet(); + } + }); + + try { + deleteFuture.get(); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + } + + protected void testDeleteRefreshesRelationCache(String branch) throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(branch, new Employee(1, "hr"), new Employee(3, "hr")); + append(branch, new Employee(1, "hardware"), new Employee(2, "hardware")); + + Dataset query = + spark.sql("SELECT * FROM " + tableNameWithBranch(branch) + " 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", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + + 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", tableNameWithBranch(branch))); + + assertEquals( + "Should refresh the relation cache", + ImmutableList.of(), + sql("SELECT * FROM tmp ORDER BY id, dep")); + + spark.sql("UNCACHE TABLE tmp"); + } + + protected void testDeleteWithMultipleSpecs(String branch) { + createAndInitTable("id INT, dep STRING, category STRING"); + + // write an unpartitioned file + append(tableNameWithBranch(branch), "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}"); + + // write a file partitioned by dep + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + append( + tableNameWithBranch(branch), + "{ \"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(tableNameWithBranch(branch), "{ \"id\": 5, \"dep\": \"hr\", \"category\": \"c1\"}"); + + // write another file partitioned by dep + sql("ALTER TABLE %s DROP PARTITION FIELD category", tableName); + append(tableNameWithBranch(branch), "{ \"id\": 7, \"dep\": \"hr\", \"category\": \"c1\"}"); + + sql("DELETE FROM %s WHERE id IN (1, 3, 5, 7)", tableNameWithBranch(branch)); + + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertEquals("Should have 5 snapshots", 5, Iterables.size(table.snapshots())); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + // copy-on-write is tested against v1 and such tables have different partition evolution + // behavior + // that's why the number of changed partitions is 4 for copy-on-write + validateCopyOnWrite(currentSnapshot, "4", "4", "1"); + } else { + validateMergeOnRead(currentSnapshot, "3", "3", null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hr", "c1")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + // 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(String branch, Employee... employees) throws NoSuchTableException { + List input = Arrays.asList(employees); + Dataset inputDF = spark.createDataFrame(input, Employee.class); + inputDF.coalesce(1).writeTo(tableNameWithBranch(branch)).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/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index 9581748e324e..71e6ccf0f9d6 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -18,56 +18,15 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; -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_SIZE; -import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; -import static org.apache.spark.sql.functions.lit; - -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.AssertHelpers; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.RowLevelOperationMode; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotSummary; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericRecord; -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.spark.SparkException; -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.iceberg.SnapshotRef; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.execution.SparkPlan; -import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; import org.junit.BeforeClass; import org.junit.Test; -public abstract class TestMerge extends SparkRowLevelOperationsTestBase { +public abstract class TestMerge extends TestMergeBase { public TestMerge( String catalogName, @@ -88,2231 +47,309 @@ public static void setupSparkConf() { public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS source"); + validationCatalog.dropTable(tableIdent, true); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeConditionSplitIntoTargetPredicateAndJoinCondition(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - // add a data file to the 'hr' partition - append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }"); - - Table table = validationCatalog.loadTable(tableIdent); - - Snapshot snapshot = table.currentSnapshot(); - String dataFilesCount = snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP); - Assert.assertEquals("Must have 2 files before MERGE", "2", dataFilesCount); - - 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"), - () -> { - 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)", - tableName); - }); - }); - - 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", tableName)); + testMergeWithStaticPredicatePushDown(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { - 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", tableName)); + testMergeIntoEmptyTargetInsertAllNonMatchingRows(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { - 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", tableName)); + testMergeIntoEmptyTargetInsertOnlyMatchingRows(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithOnlyUpdateClause(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithOnlyUpdateClauseAndNullValues(SnapshotRef.MAIN_BRANCH); } @Test 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", - tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(1, "emp-id-one") // kept - ); - assertEquals( - "Should have expected rows", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testMergeWithOnlyDeleteClause(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithAllCauses() { - 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 *", - tableName); - - 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", tableName)); + testMergeWithAllCauses(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithAllCausesWithExplicitColumnSpecification(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithSourceCTE(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName, 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", tableName)); + testMergeWithSourceFromSetOps(SnapshotRef.MAIN_BRANCH); } @Test 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 = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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)", - tableName); - }); - - 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", tableName)); + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource(SnapshotRef.MAIN_BRANCH); } @Test 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 = - "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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)", - tableName); - }); - }); - - 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", tableName)); + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + SnapshotRef.MAIN_BRANCH); } @Test 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 = - "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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)", - tableName); - }); - }); - - assertEquals( - "Target should be unchanged", - ImmutableList.of(row(1, "emp-id-one")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + SnapshotRef.MAIN_BRANCH); } @Test 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 = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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", - tableName); - }); - - 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", tableName)); + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions( + SnapshotRef.MAIN_BRANCH); } @Test 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 = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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", - tableName); - }); - - assertEquals( - "Target should be unchanged", - ImmutableList.of(row(1, "emp-id-one")), - sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName)); + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition( + SnapshotRef.MAIN_BRANCH); } @Test 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 = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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 *", - tableName); - }); - - 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", tableName)); + testMergeWithMultipleUpdatesForTargetRow(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(2, "emp-id-2") // new - ); - assertEquals( - "Should have expected rows", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testMergeWithUnconditionalDelete(SnapshotRef.MAIN_BRANCH); } @Test 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 = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - 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 *", - tableName); - }); - - 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", tableName)); + testMergeWithSingleConditionalDelete(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - 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 *", - tableName); - - 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", tableName)); - - removeTables(); - } + testMergeWithIdentityTransform(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - 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 *", - tableName); - - 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", tableName)); - - removeTables(); - } + testMergeWithDaysTransform(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - 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 *", - tableName); - - 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", tableName)); - - removeTables(); - } + testMergeWithBucketTransform(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - 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 *", - tableName); - - 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", tableName)); - - removeTables(); - } + testMergeWithTruncateTransform(SnapshotRef.MAIN_BRANCH); } @Test 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\" }"); - - 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 *", - tableName); - - 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", tableName)); - - removeTables(); - } + testMergeIntoPartitionedAndOrderedTable(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName, tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(1, "x"), // updated - row(2, "v2") // kept - ); - assertEquals( - "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testSelfMerge(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName, tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(1, "x"), // updated - row(2, "v2") // kept - ); - assertEquals( - "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testSelfMergeWithCaching(SnapshotRef.MAIN_BRANCH); } @Test 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) ", - tableName, tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(1, "x"), // updated - row(2, "v2") // kept - ); - assertEquals( - "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testMergeWithSourceAsSelfSubquery(SnapshotRef.MAIN_BRANCH); } @Test public synchronized void testMergeWithSerializableIsolation() throws InterruptedException { - // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); - - 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); - - 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++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } - - 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( - () -> { - // 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++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - table.newFastAppend().appendFile(dataFile).commit(); - sleep(10); - } - - barrier.incrementAndGet(); - } - }); - - try { - Assertions.assertThatThrownBy(mergeFuture::get) - .isInstanceOf(ExecutionException.class) - .cause() - .isInstanceOf(SparkException.class) - .cause() - .isInstanceOf(ValidationException.class) - .hasMessageContaining("Found conflicting files that can contain"); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + testMergeWithSerializableIsolation(SnapshotRef.MAIN_BRANCH); } @Test public synchronized void testMergeWithSnapshotIsolation() throws InterruptedException, ExecutionException { - // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); - - 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); - - 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++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } - - 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( - () -> { - // 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++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } - - if (!shouldAppend.get()) { - return; - } - - for (int numAppends = 0; numAppends < 5; numAppends++) { - DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); - table.newFastAppend().appendFile(dataFile).commit(); - sleep(10); - } - - barrier.incrementAndGet(); - } - }); - - try { - mergeFuture.get(); - } finally { - shouldAppend.set(false); - appendFuture.cancel(true); - } - - executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + testMergeWithSnapshotIsolation(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithExtraColumnsInSource(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithNullsInTargetAndSource(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithNullSafeEquals(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithNullCondition(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - ImmutableList expectedRows1 = - ImmutableList.of( - row(1, "v1"), // kept - row(2, "v2") // kept - ); - assertEquals( - "Output should match", expectedRows1, sql("SELECT * FROM %s ORDER BY v", tableName)); - - // 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)", - tableName); - - ImmutableList expectedRows2 = - ImmutableList.of( - row(2, "v2") // kept - ); - assertEquals( - "Output should match", expectedRows2, sql("SELECT * FROM %s ORDER BY v", tableName)); + testMergeWithNullActionConditions(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeWithMultipleMatchingActions(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException { - Assume.assumeTrue(fileFormat.equalsIgnoreCase("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(); - - Assert.assertEquals(200, spark.table(tableName).count()); - - // 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'", - tableName); - - Assert.assertEquals(200, spark.table(tableName).count()); + testMergeWithMultipleRowGroupsParquet(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeInsertOnly(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - ImmutableList expectedRows = - ImmutableList.of( - row(1, 1), // kept - row(2, 121) // new - ); - assertEquals( - "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", tableName)); + testMergeInsertOnlyWithCondition(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); + testMergeAlignsUpdateAndInsertActions(SnapshotRef.MAIN_BRANCH); } @Test 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)", - tableName); - - 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", tableName)); - - assertEquals( - "Output should match", - ImmutableList.of(row(1, -2, "new_str_1")), - sql("SELECT * FROM %s WHERE id = 1 ORDER BY id", tableName)); - assertEquals( - "Output should match", - ImmutableList.of(row(2, -20, "new_str_2")), - sql("SELECT * FROM %s WHERE b = 'new_str_2'ORDER BY id", tableName)); + testMergeMixedCaseAlignsUpdateAndInsertActions(SnapshotRef.MAIN_BRANCH); } @Test 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')", - tableName); - - 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", tableName)); - - // 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", - tableName); - - assertEquals( - "Output should match", - ImmutableList.of(row(1, row(null, null))), - sql("SELECT * FROM %s ORDER BY id", tableName)); - - // 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')))", - tableName); - - 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", tableName)); + testMergeUpdatesNestedStructFields(SnapshotRef.MAIN_BRANCH); } @Test 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", - tableName); - - assertEquals( - "Output should match", - ImmutableList.of(row(1, "-2")), - sql("SELECT * FROM %s ORDER BY id", tableName)); + testMergeWithInferredCasts(SnapshotRef.MAIN_BRANCH); } @Test 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", - tableName); - - assertEquals( - "Output should match", - ImmutableList.of(row(1, row(-10, null))), - sql("SELECT * FROM %s", tableName)); + testMergeModifiesNullStruct(SnapshotRef.MAIN_BRANCH); } @Test 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 " + tableName); - 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", - tableName); - - assertEquals( - "View should have correct data", ImmutableList.of(row("n2")), sql("SELECT * FROM tmp")); - - spark.sql("UNCACHE TABLE tmp"); + testMergeRefreshesRelationCache(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithMultipleNotMatchedActions(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithMultipleConditionalNotMatchedActions(SnapshotRef.MAIN_BRANCH); } @Test 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 * ", - tableName); - - 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", tableName)); + testMergeResolvesColumnsByName(SnapshotRef.MAIN_BRANCH); } @Test 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); - - 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", tableName)); + testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns(SnapshotRef.MAIN_BRANCH); } @Test 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 *", - tableName); - - 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", tableName)); + testMergeWithTableWithNonNullableColumn(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithNonExistingColumns() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about the invalid top-level column", - AnalysisException.class, - "cannot resolve t.invalid_col", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.invalid_col = s.c2", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about the invalid nested column", - AnalysisException.class, - "No such struct field invalid_col", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about the invalid top-level column", - AnalysisException.class, - "cannot resolve invalid_col", - () -> { - 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)", - tableName); - }); + testMergeWithNonExistingColumns(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithInvalidColumnsInInsert() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about the nested column", - AnalysisException.class, - "Nested fields are not supported inside INSERT clauses", - () -> { - 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)", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about duplicate columns", - AnalysisException.class, - "Duplicate column names inside INSERT clause", - () -> { - 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)", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about missing columns", - AnalysisException.class, - "must provide values for all columns of the target table", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN NOT MATCHED THEN " - + " INSERT (id) VALUES (s.c1)", - tableName); - }); + testMergeWithInvalidColumnsInInsert(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithInvalidUpdates() { - createAndInitTable("id INT, a ARRAY>, m MAP"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about updating an array column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.a.c1 = s.c2", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about updating a map column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.m.key = 'new_key'", - tableName); - }); + testMergeWithInvalidUpdates(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithConflictingUpdates() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a top-level column", - AnalysisException.class, - "Updates are in conflict", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict for these columns", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict", - () -> { - 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))", - tableName); - }); + testMergeWithConflictingUpdates(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithInvalidAssignments() { - createAndInitTable( - "id INT NOT NULL, s STRUCT> NOT NULL"); - createOrReplaceView( - "source", - "c1 INT, c2 STRUCT NOT NULL, c3 STRING NOT NULL, c4 STRUCT", - "{ \"c1\": -100, \"c2\": { \"n1\" : 1 }, \"c3\" : 'str', \"c4\": { \"dn2\": 1, \"dn2\": 2 } }"); - - for (String policy : new String[] {"ansi", "strict"}) { - withSQLConf( - ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), - () -> { - AssertHelpers.assertThrows( - "Should complain about writing nulls to a top-level column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.id = NULL", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about writing nulls to a nested column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n1 = NULL", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about writing missing fields in structs", - AnalysisException.class, - "missing fields", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s = s.c2", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about writing invalid data types", - AnalysisException.class, - "Cannot safely cast", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n1 = s.c3", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about writing incompatible structs", - AnalysisException.class, - "field name does not match", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n2 = s.c4", - tableName); - }); - }); - } + testMergeWithInvalidAssignments(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithNonDeterministicConditions() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about non-deterministic search conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about non-deterministic update conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about non-deterministic delete conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND rand() > t.id THEN " - + " DELETE", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about non-deterministic insert conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - 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)", - tableName); - }); + testMergeWithNonDeterministicConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithAggregateExpressions() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about agg expressions in search conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about agg expressions in update conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about agg expressions in delete conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND sum(t.id) THEN " - + " DELETE", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about agg expressions in insert conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - 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)", - tableName); - }); + testMergeWithAggregateExpressions(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithSubqueriesInConditions() { - createAndInitTable("id INT, c STRUCT>"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - 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", - tableName); - }); - - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - 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)", - tableName); - }); + testMergeWithSubqueriesInConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithTargetColumnsInInsertConditions() { - createAndInitTable("id INT, c2 INT"); - createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); - - AssertHelpers.assertThrows( - "Should complain about the target column", - AnalysisException.class, - "Cannot resolve [c2]", - () -> { - 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)", - tableName); - }); + testMergeWithTargetColumnsInInsertConditions(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeWithNonIcebergTargetTableNotSupported() { - createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); - createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - - AssertHelpers.assertThrows( - "Should complain non iceberg target table", - UnsupportedOperationException.class, - "MERGE INTO TABLE is not supported temporarily.", - () -> { - sql( - "MERGE INTO target t USING source s " - + "ON t.c1 == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET *"); - }); + testMergeWithNonIcebergTargetTableNotSupported(SnapshotRef.MAIN_BRANCH); } /** @@ -2321,63 +358,11 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { */ @Test 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 *", - tableName); - - 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", tableName); - assertEquals("Should correctly add the non-matching rows", expectedRows, result); + testMergeSinglePartitionPartitioning(SnapshotRef.MAIN_BRANCH); } @Test public void testMergeEmptyTable() { - // 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 *", - tableName); - - ImmutableList expectedRows = ImmutableList.of(row(0), row(1), row(2), row(3), row(4)); - - List result = sql("SELECT * FROM %s ORDER BY id", tableName); - assertEquals("Should correctly add the non-matching rows", expectedRows, result); - } - - 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"), - () -> { - SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); - String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); - - Assertions.assertThat(planAsString).as("Join should match").contains(join + "\n"); - - Assertions.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); + testMergeEmptyTable(SnapshotRef.MAIN_BRANCH); } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeBase.java new file mode 100644 index 000000000000..141207123b39 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeBase.java @@ -0,0 +1,2422 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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.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_SIZE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; +import static org.apache.spark.sql.functions.lit; + +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.AssertHelpers; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericRecord; +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.util.SnapshotUtil; +import org.apache.spark.SparkException; +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.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; + +abstract class TestMergeBase extends SparkRowLevelOperationsTestBase { + + TestMergeBase( + String catalogName, + String implementation, + Map config, + String fileFormat, + boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + protected synchronized void testMergeConditionSplitIntoTargetPredicateAndJoinCondition( + String branch) { + createAndInitTableBranch( + "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\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithStaticPredicatePushDown(String branch) { + createAndInitTable("id BIGINT, dep STRING"); + + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + // add a data file to the 'software' partition + append(tableNameWithBranch(branch), "{ \"id\": 1, \"dep\": \"software\" }"); + + // add a data file to the 'hr' partition + append(tableNameWithBranch(branch), "{ \"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); + Assert.assertEquals("Must have 2 files before MERGE", "2", dataFilesCount); + + 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"), + () -> { + 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)", + tableNameWithBranch(branch)); + }); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeIntoEmptyTargetInsertAllNonMatchingRows(String branch) { + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeIntoEmptyTargetInsertOnlyMatchingRows(String branch) { + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithOnlyUpdateClause(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-six\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithOnlyUpdateClauseAndNullValues(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": null, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-six\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithOnlyDeleteClause(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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", + tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-one") // kept + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithAllCauses(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithAllCausesWithExplicitColumnSpecification(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithSourceCTE(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-two\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithSourceFromSetOps(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 *", + tableNameWithBranch(branch), 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource( + String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 = "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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)", + tableNameWithBranch(branch)); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 = + "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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)", + tableNameWithBranch(branch)); + }); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqualityCondition( + String branch) { + createAndInitTableBranch( + "id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }", branch); + + 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 = + "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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)", + tableNameWithBranch(branch)); + }); + }); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected synchronized void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions( + String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 = "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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", + tableNameWithBranch(branch)); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition( + String branch) { + createAndInitTableBranch( + "id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }", branch); + + 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 = "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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", + tableNameWithBranch(branch)); + }); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithMultipleUpdatesForTargetRow(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 = "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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 *", + tableNameWithBranch(branch)); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithUnconditionalDelete(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithSingleConditionalDelete(String branch) { + createAndInitTableBranch( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 = "a single row from the target table with multiple rows of the source table"; + AssertHelpers.assertThrowsCause( + "Should complain about multiple matches", + SparkException.class, + errorMsg, + () -> { + 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 *", + tableNameWithBranch(branch)); + }); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithIdentityTransform(String branch) { + 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( + tableNameWithBranch(branch), + "{ \"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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + removeTables(); + } + } + + protected synchronized void testMergeWithDaysTransform(String branch) { + 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( + tableNameWithBranch(branch), + "id INT, ts TIMESTAMP", + "{ \"id\": 1, \"ts\": \"2000-01-01 00:00:00\" }\n" + + "{ \"id\": 6, \"ts\": \"2000-01-06 00:00:00\" }"); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + removeTables(); + } + } + + protected synchronized void testMergeWithBucketTransform(String branch) { + 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( + tableNameWithBranch(branch), + "{ \"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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + removeTables(); + } + } + + protected synchronized void testMergeWithTruncateTransform(String branch) { + 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( + tableNameWithBranch(branch), + "{ \"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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + removeTables(); + } + } + + protected synchronized void testMergeIntoPartitionedAndOrderedTable(String branch) { + 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( + tableNameWithBranch(branch), + "{ \"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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + removeTables(); + } + } + + protected synchronized void testSelfMerge(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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 *", + tableNameWithBranch(branch), tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testSelfMergeWithCaching(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + sql("CACHE TABLE %s", tableNameWithBranch(branch)); + + 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 *", + tableNameWithBranch(branch), tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + sql("UNCACHE TABLE %s", tableNameWithBranch(branch)); + } + + protected synchronized void testMergeWithSourceAsSelfSubquery(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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) ", + tableNameWithBranch(branch), tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithSerializableIsolation(String branch) + throws InterruptedException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + + 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')", tableNameWithBranch(branch)); + + 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++) { + while (barrier.get() < numOperations * 2) { + sleep(10); + } + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " UPDATE SET dep = 'x'", + tableNameWithBranch(branch)); + + 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++) { + while (shouldAppend.get() && barrier.get() < numOperations * 2) { + sleep(10); + } + + if (!shouldAppend.get()) { + return; + } + + for (int numAppends = 0; numAppends < 5; numAppends++) { + DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); + table.newFastAppend().appendFile(dataFile).toBranch(branch).commit(); + sleep(10); + } + + barrier.incrementAndGet(); + } + }); + + try { + Assertions.assertThatThrownBy(mergeFuture::get) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Found conflicting files that can contain"); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + } + + protected synchronized void testMergeWithSnapshotIsolation(String branch) + throws InterruptedException, ExecutionException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + + 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')", tableNameWithBranch(branch)); + + 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++) { + while (barrier.get() < numOperations * 2) { + sleep(10); + } + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " UPDATE SET dep = 'x'", + tableNameWithBranch(branch)); + + 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++) { + while (shouldAppend.get() && barrier.get() < numOperations * 2) { + sleep(10); + } + + if (!shouldAppend.get()) { + return; + } + + for (int numAppends = 0; numAppends < 5; numAppends++) { + DataFile dataFile = writeDataFile(table, ImmutableList.of(record)); + table.newFastAppend().appendFile(dataFile).toBranch(branch).commit(); + sleep(10); + } + + barrier.incrementAndGet(); + } + }); + + try { + mergeFuture.get(); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + } + + protected synchronized void testMergeWithExtraColumnsInSource(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithNullsInTargetAndSource(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithNullSafeEquals(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithNullCondition(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithNullActionConditions(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + ImmutableList expectedRows1 = + ImmutableList.of( + row(1, "v1"), // kept + row(2, "v2") // kept + ); + assertEquals( + "Output should match", + expectedRows1, + sql("SELECT * FROM %s ORDER BY v", tableNameWithBranch(branch))); + + // 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)", + tableNameWithBranch(branch)); + + ImmutableList expectedRows2 = + ImmutableList.of( + row(2, "v2") // kept + ); + assertEquals( + "Output should match", + expectedRows2, + sql("SELECT * FROM %s ORDER BY v", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithMultipleMatchingActions(String branch) { + createAndInitTableBranch( + "id INT, v STRING", + "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }", + branch); + + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithMultipleRowGroupsParquet(String branch) + throws NoSuchTableException { + Assume.assumeTrue(fileFormat.equalsIgnoreCase("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(tableNameWithBranch(branch)).append(); + + Assert.assertEquals(200, spark.table(tableNameWithBranch(branch)).count()); + + // 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'", + tableNameWithBranch(branch)); + + Assert.assertEquals(200, spark.table(tableNameWithBranch(branch)).count()); + } + + protected synchronized void testMergeInsertOnly(String branch) { + createAndInitTableBranch( + "id STRING, v STRING", + "{ \"id\": \"a\", \"v\": \"v1\" }\n" + "{ \"id\": \"b\", \"v\": \"v2\" }", + branch); + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeInsertOnlyWithCondition(String branch) { + 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)", + tableNameWithBranch(branch)); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, 1), // kept + row(2, 121) // new + ); + assertEquals( + "Output should match", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeAlignsUpdateAndInsertActions(String branch) { + createAndInitTableBranch( + "id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }", branch); + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeMixedCaseAlignsUpdateAndInsertActions(String branch) { + createAndInitTableBranch( + "id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }", branch); + 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)", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, -2, "new_str_1")), + sql("SELECT * FROM %s WHERE id = 1 ORDER BY id", tableNameWithBranch(branch))); + assertEquals( + "Output should match", + ImmutableList.of(row(2, -20, "new_str_2")), + sql("SELECT * FROM %s WHERE b = 'new_str_2'ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeUpdatesNestedStructFields(String branch) { + createAndInitTableBranch( + "id INT, s STRUCT,m:MAP>>", + "{ \"id\": 1, \"s\": { \"c1\": 2, \"c2\": { \"a\": [1,2], \"m\": { \"a\": \"b\"} } } } }", + branch); + 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')", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + + // 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", + tableNameWithBranch(branch)); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(null, null))), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + + // 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')))", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithInferredCasts(String branch) { + createAndInitTableBranch("id INT, s STRING", "{ \"id\": 1, \"s\": \"value\" }", branch); + 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", + tableNameWithBranch(branch)); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, "-2")), + sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeModifiesNullStruct(String branch) { + createAndInitTableBranch( + "id INT, s STRUCT", "{ \"id\": 1, \"s\": null }", branch); + 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", + tableNameWithBranch(branch)); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(-10, null))), + sql("SELECT * FROM %s", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeRefreshesRelationCache(String branch) { + createAndInitTableBranch("id INT, name STRING", "{ \"id\": 1, \"name\": \"n1\" }", branch); + createOrReplaceView("source", "{ \"id\": 1, \"name\": \"n2\" }"); + + Dataset query = spark.sql("SELECT name FROM " + tableNameWithBranch(branch)); + 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", + tableNameWithBranch(branch)); + + assertEquals( + "View should have correct data", ImmutableList.of(row("n2")), sql("SELECT * FROM tmp")); + + spark.sql("UNCACHE TABLE tmp"); + } + + protected synchronized void testMergeWithMultipleNotMatchedActions(String branch) { + createAndInitTableBranch("id INT, dep STRING", "{ \"id\": 0, \"dep\": \"emp-id-0\" }", branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithMultipleConditionalNotMatchedActions(String branch) { + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeResolvesColumnsByName(String branch) { + createAndInitTableBranch( + "id INT, badge INT, dep STRING", + "{ \"id\": 1, \"badge\": 1000, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 6, \"badge\": 6000, \"dep\": \"emp-id-6\" }", + branch); + + 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 * ", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns( + String branch) { + // 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithTableWithNonNullableColumn(String branch) { + createAndInitTableBranch( + "id INT NOT NULL, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }", + branch); + + 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch))); + } + + protected synchronized void testMergeWithNonExistingColumns(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about the invalid top-level column", + AnalysisException.class, + "cannot resolve t.invalid_col", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.invalid_col = s.c2", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about the invalid nested column", + AnalysisException.class, + "No such struct field invalid_col", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about the invalid top-level column", + AnalysisException.class, + "cannot resolve invalid_col", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithInvalidColumnsInInsert(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about the nested column", + AnalysisException.class, + "Nested fields are not supported inside INSERT clauses", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about duplicate columns", + AnalysisException.class, + "Duplicate column names inside INSERT clause", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about missing columns", + AnalysisException.class, + "must provide values for all columns of the target table", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id) VALUES (s.c1)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithInvalidUpdates(String branch) { + createAndInitTable("id INT, a ARRAY>, m MAP"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about updating an array column", + AnalysisException.class, + "Updating nested fields is only supported for structs", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.a.c1 = s.c2", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about updating a map column", + AnalysisException.class, + "Updating nested fields is only supported for structs", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.m.key = 'new_key'", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithConflictingUpdates(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about conflicting updates to a top-level column", + AnalysisException.class, + "Updates are in conflict", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about conflicting updates to a nested column", + AnalysisException.class, + "Updates are in conflict for these columns", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about conflicting updates to a nested column", + AnalysisException.class, + "Updates are in conflict", + () -> { + 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))", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithInvalidAssignments(String branch) { + createAndInitTable( + "id INT NOT NULL, s STRUCT> NOT NULL"); + createOrReplaceView( + "source", + "c1 INT, c2 STRUCT NOT NULL, c3 STRING NOT NULL, c4 STRUCT", + "{ \"c1\": -100, \"c2\": { \"n1\" : 1 }, \"c3\" : 'str', \"c4\": { \"dn2\": 1, \"dn2\": 2 } }"); + + for (String policy : new String[] {"ansi", "strict"}) { + withSQLConf( + ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), + () -> { + AssertHelpers.assertThrows( + "Should complain about writing nulls to a top-level column", + AnalysisException.class, + "Cannot write nullable values to non-null column", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = NULL", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about writing nulls to a nested column", + AnalysisException.class, + "Cannot write nullable values to non-null column", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = NULL", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about writing missing fields in structs", + AnalysisException.class, + "missing fields", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = s.c2", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about writing invalid data types", + AnalysisException.class, + "Cannot safely cast", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = s.c3", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about writing incompatible structs", + AnalysisException.class, + "field name does not match", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n2 = s.c4", + tableNameWithBranch(branch)); + }); + }); + } + } + + protected synchronized void testMergeWithNonDeterministicConditions(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about non-deterministic search conditions", + AnalysisException.class, + "Non-deterministic functions are not supported", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about non-deterministic update conditions", + AnalysisException.class, + "Non-deterministic functions are not supported", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about non-deterministic delete conditions", + AnalysisException.class, + "Non-deterministic functions are not supported", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND rand() > t.id THEN " + + " DELETE", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about non-deterministic insert conditions", + AnalysisException.class, + "Non-deterministic functions are not supported", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithAggregateExpressions(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about agg expressions in search conditions", + AnalysisException.class, + "Agg functions are not supported", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about agg expressions in update conditions", + AnalysisException.class, + "Agg functions are not supported", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about agg expressions in delete conditions", + AnalysisException.class, + "Agg functions are not supported", + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND sum(t.id) THEN " + + " DELETE", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about agg expressions in insert conditions", + AnalysisException.class, + "Agg functions are not supported", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithSubqueriesInConditions(String branch) { + createAndInitTable("id INT, c STRUCT>"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain about subquery expressions", + AnalysisException.class, + "Subqueries are not supported in conditions", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about subquery expressions", + AnalysisException.class, + "Subqueries are not supported in conditions", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about subquery expressions", + AnalysisException.class, + "Subqueries are not supported in conditions", + () -> { + 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", + tableNameWithBranch(branch)); + }); + + AssertHelpers.assertThrows( + "Should complain about subquery expressions", + AnalysisException.class, + "Subqueries are not supported in conditions", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithTargetColumnsInInsertConditions(String branch) { + createAndInitTable("id INT, c2 INT"); + createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); + + AssertHelpers.assertThrows( + "Should complain about the target column", + AnalysisException.class, + "Cannot resolve [c2]", + () -> { + 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)", + tableNameWithBranch(branch)); + }); + } + + protected synchronized void testMergeWithNonIcebergTargetTableNotSupported(String branch) { + createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + AssertHelpers.assertThrows( + "Should complain non iceberg target table", + UnsupportedOperationException.class, + "MERGE INTO TABLE is not supported temporarily.", + () -> { + sql( + "MERGE INTO target t USING source s " + + "ON t.c1 == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET *"); + }); + } + + /** + * 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. + */ + protected synchronized void testMergeSinglePartitionPartitioning(String branch) { + // This table will only have a single file and a single partition + createAndInitTableBranch("id INT", "{\"id\": -1}", branch); + + // 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 *", + tableNameWithBranch(branch)); + + 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", tableNameWithBranch(branch)); + assertEquals("Should correctly add the non-matching rows", expectedRows, result); + } + + protected synchronized void testMergeEmptyTable(String branch) { + // This table will only have a single file and a single partition + createAndInitTableBranch("id INT", null, branch); + + // 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 *", + tableNameWithBranch(branch)); + + ImmutableList expectedRows = ImmutableList.of(row(0), row(1), row(2), row(3), row(4)); + + List result = sql("SELECT * FROM %s ORDER BY id", tableNameWithBranch(branch)); + assertEquals("Should correctly add the non-matching rows", expectedRows, result); + } + + 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"), + () -> { + SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + Assertions.assertThat(planAsString).as("Join should match").contains(join + "\n"); + + Assertions.assertThat(planAsString) + .as("Pushed filters must match") + .contains("[filters=" + icebergFilters + ","); + }); + } + + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS source"); + } + + private RowLevelOperationMode mode(Table table) { + String modeName = table.properties().getOrDefault(MERGE_MODE, MERGE_MODE_DEFAULT); + return RowLevelOperationMode.fromName(modeName); + } +} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchDelete.java new file mode 100644 index 000000000000..4bce00776224 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchDelete.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * 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_DISTRIBUTION_MODE_NONE; + +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestMergeOnReadBranchDelete extends TestMergeOnReadDelete { + + public TestMergeOnReadBranchDelete( + String catalogName, + String implementation, + Map config, + String fileFormat, + Boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + @Parameterized.Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + "parquet", + true, + WRITE_DISTRIBUTION_MODE_NONE + }, + }; + } + + @Test + public void testDeleteWithoutScanningTable() throws Exception { + testDeleteWithoutScanningTable(TEST_BRANCH); + } + + @Test + public void testDeleteFileThenMetadataDelete() throws Exception { + testDeleteFileThenMetadataDelete(TEST_BRANCH); + } + + @Test + public void testDeleteWithFalseCondition() { + testDeleteWithFalseCondition(TEST_BRANCH); + } + + @Test + public void testDeleteFromEmptyTable() { + testDeleteFromEmptyTable(TEST_BRANCH); + } + + @Test + public void testExplain() { + testExplain(TEST_BRANCH); + } + + @Test + public void testDeleteWithAlias() { + testDeleteWithAlias(TEST_BRANCH); + } + + @Test + public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { + testDeleteWithDynamicFileFiltering(TEST_BRANCH); + } + + @Test + public void testDeleteNonExistingRecords() { + testDeleteNonExistingRecords("test_branch"); + } + + @Test + public void testDeleteWithoutCondition() { + testDeleteWithoutCondition("test_branch"); + } + + @Test + public void testDeleteUsingMetadataWithComplexCondition() { + testDeleteUsingMetadataWithComplexCondition("test_branch"); + } + + @Test + public void testDeleteWithArbitraryPartitionPredicates() { + testDeleteWithArbitraryPartitionPredicates(TEST_BRANCH); + } + + @Test + public void testDeleteWithNonDeterministicCondition() { + testDeleteWithNonDeterministicCondition(TEST_BRANCH); + } + + @Test + public void testDeleteWithFoldableConditions() { + testDeleteWithFoldableConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithNullConditions() { + testDeleteWithNullConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithInAndNotInConditions() { + testDeleteWithInAndNotInConditions(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException { + testDeleteWithMultipleRowGroupsParquet(TEST_BRANCH); + } + + @Test + public void testDeleteWithConditionOnNestedColumn() { + testDeleteWithConditionOnNestedColumn(TEST_BRANCH); + } + + @Test + public void testDeleteWithInSubquery() throws NoSuchTableException { + testDeleteWithInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { + testDeleteWithNotInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithNotInSubquery() throws NoSuchTableException { + testDeleteWithNotInSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteOnNonIcebergTableNotSupported() { + testDeleteOnNonIcebergTableNotSupported(TEST_BRANCH); + } + + @Test + public void testDeleteWithExistSubquery() throws NoSuchTableException { + testDeleteWithExistSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { + testDeleteWithNotExistsSubquery(TEST_BRANCH); + } + + @Test + public void testDeleteWithScalarSubquery() throws NoSuchTableException { + testDeleteWithScalarSubquery(TEST_BRANCH); + } + + @Test + public synchronized void testDeleteWithSerializableIsolation() throws InterruptedException { + testDeleteWithSerializableIsolation(TEST_BRANCH); + } + + @Test + public synchronized void testDeleteWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + testDeleteWithSnapshotIsolation(TEST_BRANCH); + } + + @Test + public void testDeleteRefreshesRelationCache() throws NoSuchTableException { + testDeleteRefreshesRelationCache(TEST_BRANCH); + } + + @Test + public void testDeleteWithMultipleSpecs() { + testDeleteWithMultipleSpecs(TEST_BRANCH); + } +} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchMerge.java new file mode 100644 index 000000000000..7e4d5e2a52ed --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadBranchMerge.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.extensions; + +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE; + +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestMergeOnReadBranchMerge extends TestMergeOnReadMerge { + + public TestMergeOnReadBranchMerge( + String catalogName, + String implementation, + Map config, + String fileFormat, + boolean vectorized, + String distributionMode) { + super(catalogName, implementation, config, fileFormat, vectorized, distributionMode); + } + + @Parameterized.Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + "orc", + true, + WRITE_DISTRIBUTION_MODE_NONE + }, + }; + } + + @Test + public void testMergeConditionSplitIntoTargetPredicateAndJoinCondition() { + testMergeConditionSplitIntoTargetPredicateAndJoinCondition(TEST_BRANCH); + } + + @Test + public void testMergeWithStaticPredicatePushDown() { + testMergeWithStaticPredicatePushDown(TEST_BRANCH); + } + + @Test + public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { + testMergeIntoEmptyTargetInsertAllNonMatchingRows(TEST_BRANCH); + } + + @Test + public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { + testMergeIntoEmptyTargetInsertOnlyMatchingRows(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyUpdateClause() { + testMergeWithOnlyUpdateClause(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyUpdateClauseAndNullValues() { + testMergeWithOnlyUpdateClauseAndNullValues(TEST_BRANCH); + } + + @Test + public void testMergeWithOnlyDeleteClause() { + testMergeWithOnlyDeleteClause(TEST_BRANCH); + } + + @Test + public void testMergeWithAllCauses() { + testMergeWithAllCauses(TEST_BRANCH); + } + + @Test + public void testMergeWithAllCausesWithExplicitColumnSpecification() { + testMergeWithAllCausesWithExplicitColumnSpecification(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceCTE() { + testMergeWithSourceCTE(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceFromSetOps() { + testMergeWithSourceFromSetOps(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource(TEST_BRANCH); + } + + @Test + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqualityCondition() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions(TEST_BRANCH); + } + + @Test + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition() { + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition( + TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleUpdatesForTargetRow() { + testMergeWithMultipleUpdatesForTargetRow(TEST_BRANCH); + } + + @Test + public void testMergeWithUnconditionalDelete() { + testMergeWithUnconditionalDelete(TEST_BRANCH); + } + + @Test + public void testMergeWithSingleConditionalDelete() { + testMergeWithSingleConditionalDelete(TEST_BRANCH); + } + + @Test + public void testMergeWithIdentityTransform() { + testMergeWithIdentityTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithDaysTransform() { + testMergeWithDaysTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithBucketTransform() { + testMergeWithBucketTransform(TEST_BRANCH); + } + + @Test + public void testMergeWithTruncateTransform() { + testMergeWithTruncateTransform(TEST_BRANCH); + } + + @Test + public void testMergeIntoPartitionedAndOrderedTable() { + testMergeIntoPartitionedAndOrderedTable(TEST_BRANCH); + } + + @Test + public void testSelfMerge() { + testSelfMerge(TEST_BRANCH); + } + + @Test + public void testSelfMergeWithCaching() { + testSelfMergeWithCaching(TEST_BRANCH); + } + + @Test + public void testMergeWithSourceAsSelfSubquery() { + testMergeWithSourceAsSelfSubquery(TEST_BRANCH); + } + + @Test + public synchronized void testMergeWithSerializableIsolation() throws InterruptedException { + testMergeWithSerializableIsolation(TEST_BRANCH); + } + + @Test + public synchronized void testMergeWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + testMergeWithSnapshotIsolation(TEST_BRANCH); + } + + @Test + public void testMergeWithExtraColumnsInSource() { + testMergeWithExtraColumnsInSource(TEST_BRANCH); + } + + @Test + public void testMergeWithNullsInTargetAndSource() { + testMergeWithNullsInTargetAndSource(TEST_BRANCH); + } + + @Test + public void testMergeWithNullSafeEquals() { + testMergeWithNullSafeEquals(TEST_BRANCH); + } + + @Test + public void testMergeWithNullCondition() { + testMergeWithNullCondition(TEST_BRANCH); + } + + @Test + public void testMergeWithNullActionConditions() { + testMergeWithNullActionConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleMatchingActions() { + testMergeWithMultipleMatchingActions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException { + testMergeWithMultipleRowGroupsParquet(TEST_BRANCH); + } + + @Test + public void testMergeInsertOnly() { + testMergeInsertOnly(TEST_BRANCH); + } + + @Test + public void testMergeInsertOnlyWithCondition() { + testMergeInsertOnlyWithCondition(TEST_BRANCH); + } + + @Test + public void testMergeAlignsUpdateAndInsertActions() { + testMergeAlignsUpdateAndInsertActions(TEST_BRANCH); + } + + @Test + public void testMergeMixedCaseAlignsUpdateAndInsertActions() { + testMergeMixedCaseAlignsUpdateAndInsertActions(TEST_BRANCH); + } + + @Test + public void testMergeUpdatesNestedStructFields() { + testMergeUpdatesNestedStructFields(TEST_BRANCH); + } + + @Test + public void testMergeWithInferredCasts() { + testMergeWithInferredCasts(TEST_BRANCH); + } + + @Test + public void testMergeModifiesNullStruct() { + testMergeModifiesNullStruct(TEST_BRANCH); + } + + @Test + public void testMergeRefreshesRelationCache() { + testMergeRefreshesRelationCache(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleNotMatchedActions() { + testMergeWithMultipleNotMatchedActions(TEST_BRANCH); + } + + @Test + public void testMergeWithMultipleConditionalNotMatchedActions() { + testMergeWithMultipleConditionalNotMatchedActions(TEST_BRANCH); + } + + @Test + public void testMergeResolvesColumnsByName() { + testMergeResolvesColumnsByName(TEST_BRANCH); + } + + @Test + public void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns() { + testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns(TEST_BRANCH); + } + + @Test + public void testMergeWithTableWithNonNullableColumn() { + testMergeWithTableWithNonNullableColumn(TEST_BRANCH); + } + + @Test + public void testMergeWithNonExistingColumns() { + testMergeWithNonExistingColumns(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidColumnsInInsert() { + testMergeWithInvalidColumnsInInsert(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidUpdates() { + testMergeWithInvalidUpdates(TEST_BRANCH); + } + + @Test + public void testMergeWithConflictingUpdates() { + testMergeWithConflictingUpdates(TEST_BRANCH); + } + + @Test + public void testMergeWithInvalidAssignments() { + testMergeWithInvalidAssignments(TEST_BRANCH); + } + + @Test + public void testMergeWithNonDeterministicConditions() { + testMergeWithNonDeterministicConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithAggregateExpressions() { + testMergeWithAggregateExpressions(TEST_BRANCH); + } + + @Test + public void testMergeWithSubqueriesInConditions() { + testMergeWithSubqueriesInConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithTargetColumnsInInsertConditions() { + testMergeWithTargetColumnsInInsertConditions(TEST_BRANCH); + } + + @Test + public void testMergeWithNonIcebergTargetTableNotSupported() { + testMergeWithNonIcebergTargetTableNotSupported(TEST_BRANCH); + } + + /** + * 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. + */ + @Test + public void testMergeSinglePartitionPartitioning() { + testMergeSinglePartitionPartitioning(TEST_BRANCH); + } + + @Test + public void testMergeEmptyTable() { + testMergeEmptyTable(TEST_BRANCH); + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java index 2533b3bd75b5..e27299a95858 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java @@ -185,9 +185,13 @@ private Pair load(Identifier ident) throws NoSuchTableException { return Pair.of(table, SnapshotUtil.snapshotIdAsOfTime(table, asOfTimestamp)); } else if (branch != null) { Snapshot branchSnapshot = table.snapshot(branch); - Preconditions.checkArgument( - branchSnapshot != null, "Cannot find snapshot associated with branch name: %s", branch); - return Pair.of(table, branchSnapshot.snapshotId()); + + // It's possible that the branch does not exist when performing writes to new branches. + // Load table should still succeed when spark is performing the write. + // Reads with invalid branches will fail at a later point + Long branchSnapshotId = branchSnapshot == null ? null : branchSnapshot.snapshotId(); + + return Pair.of(table, branchSnapshotId); } else if (tag != null) { Snapshot tagSnapshot = table.snapshot(tag); Preconditions.checkArgument( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index f9a4787fc16d..8802b7c152b4 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -657,14 +657,6 @@ private Table load(Identifier ident) { return new SparkTable(table, snapshotId, !cacheEnabled); } - Matcher branch = BRANCH.matcher(ident.name()); - if (branch.matches()) { - Snapshot branchSnapshot = table.snapshot(branch.group(1)); - if (branchSnapshot != null) { - return new SparkTable(table, branchSnapshot.snapshotId(), !cacheEnabled); - } - } - Matcher tag = TAG.matcher(ident.name()); if (tag.matches()) { Snapshot tagSnapshot = table.snapshot(tag.group(1)); @@ -673,6 +665,11 @@ private Table load(Identifier ident) { } } + String branch = ident.name(); + if (branch != null) { + return new SparkTable(table, branch, !cacheEnabled); + } + // the name wasn't a valid snapshot selector and did not point to the changelog // throw the original exception throw e; @@ -760,9 +757,12 @@ private Table loadFromPathIdentifier(PathIdentifier ident) { } else if (branch != null) { Snapshot branchSnapshot = table.snapshot(branch); - Preconditions.checkArgument( - branchSnapshot != null, "Cannot find snapshot associated with branch name: %s", branch); - return new SparkTable(table, branchSnapshot.snapshotId(), !cacheEnabled); + + // It's possible that the branch does not exist when performing writes to new branches. + // Load table should still succeed when spark is performing the write. + // Reads performed on non-existing branches will fail at a later point + Long branchSnapshotId = branchSnapshot == null ? null : branchSnapshot.snapshotId(); + return new SparkTable(table, branchSnapshotId, !cacheEnabled); } else if (tag != null) { Snapshot tagSnapshot = table.snapshot(tag); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index a44929aa30ab..ac368403be37 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -82,6 +82,8 @@ public Long endSnapshotId() { return confParser.longConf().option(SparkReadOptions.END_SNAPSHOT_ID).parseOptional(); } + /** @deprecated will be removed in 1.3.0; specify read branch in the table identifier */ + @Deprecated public String branch() { return confParser.stringConf().option(SparkReadOptions.BRANCH).parseOptional(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index 6f4649642c57..b6c0b4e8d5fa 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -77,4 +77,7 @@ private SparkWriteOptions() {} // Isolation Level for DataFrame calls. Currently supported by overwritePartitions public static final String ISOLATION_LEVEL = "isolation-level"; + + // Branch to write to + public static final String BRANCH = "branch"; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java index 68c99440441d..4d3ce3cc445b 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java @@ -42,17 +42,24 @@ class SparkCopyOnWriteOperation implements RowLevelOperation { private final Command command; private final IsolationLevel isolationLevel; + private final String writeBranch; + // lazy vars private ScanBuilder lazyScanBuilder; private Scan configuredScan; private WriteBuilder lazyWriteBuilder; SparkCopyOnWriteOperation( - SparkSession spark, Table table, RowLevelOperationInfo info, IsolationLevel isolationLevel) { + SparkSession spark, + Table table, + RowLevelOperationInfo info, + IsolationLevel isolationLevel, + String writeBranch) { this.spark = spark; this.table = table; this.command = info.command(); this.isolationLevel = isolationLevel; + this.writeBranch = writeBranch; } @Override @@ -64,10 +71,10 @@ public Command command() { public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { if (lazyScanBuilder == null) { lazyScanBuilder = - new SparkScanBuilder(spark, table, options) { + new SparkScanBuilder(spark, table, table.schema(), options, writeBranch) { @Override public Scan build() { - Scan scan = super.buildCopyOnWriteScan(); + Scan scan = super.buildCopyOnWriteScan(writeBranch); SparkCopyOnWriteOperation.this.configuredScan = scan; return scan; } @@ -80,7 +87,7 @@ public Scan build() { @Override public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { if (lazyWriteBuilder == null) { - SparkWriteBuilder writeBuilder = new SparkWriteBuilder(spark, table, info); + SparkWriteBuilder writeBuilder = new SparkWriteBuilder(spark, table, info, writeBranch); lazyWriteBuilder = writeBuilder.overwriteFiles(configuredScan, command, isolationLevel); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java index 9a411c213484..d05be0e1c9e2 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -28,11 +28,13 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; 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.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; @@ -49,6 +51,7 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan private static final Logger LOG = LoggerFactory.getLogger(SparkCopyOnWriteScan.class); private final Snapshot snapshot; + private final String branch; private Set filteredLocations = null; SparkCopyOnWriteScan( @@ -57,7 +60,7 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan SparkReadConf readConf, Schema expectedSchema, List filters) { - this(spark, table, null, null, readConf, expectedSchema, filters); + this(spark, table, null, null, readConf, expectedSchema, filters, SnapshotRef.MAIN_BRANCH); } SparkCopyOnWriteScan( @@ -67,15 +70,17 @@ class SparkCopyOnWriteScan extends SparkPartitioningAwareScan Snapshot snapshot, SparkReadConf readConf, Schema expectedSchema, - List filters) { + List filters, + String branch) { super(spark, table, scan, readConf, expectedSchema, filters); - this.snapshot = snapshot; - if (scan == null) { this.filteredLocations = Collections.emptySet(); } + + this.snapshot = snapshot; + this.branch = branch; } Long snapshotId() { @@ -100,12 +105,12 @@ public NamedReference[] filterAttributes() { @Override public void filter(Filter[] filters) { Preconditions.checkState( - Objects.equals(snapshotId(), currentSnapshotId()), + Objects.equals(snapshotId(), latestSnapshotId()), "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 multiple threads modify the table, use independent Spark sessions in each thread.", snapshotId(), - currentSnapshotId()); + latestSnapshotId()); for (Filter filter : filters) { // Spark can only pass In filters at the moment @@ -178,8 +183,8 @@ public String toString() { table(), expectedSchema().asStruct(), filterExpressions(), caseSensitive()); } - private Long currentSnapshotId() { - Snapshot currentSnapshot = table().currentSnapshot(); - return currentSnapshot != null ? currentSnapshot.snapshotId() : null; + private Long latestSnapshotId() { + Snapshot latestSnapshot = SnapshotUtil.latestSnapshot(table(), branch); + return latestSnapshot != null ? latestSnapshot.snapshotId() : null; } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java index 72948dedb2bf..1aeeea98bde9 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java @@ -41,6 +41,7 @@ class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta { private final Table table; private final Command command; private final IsolationLevel isolationLevel; + private final String branch; // lazy vars private ScanBuilder lazyScanBuilder; @@ -48,11 +49,16 @@ class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta { private DeltaWriteBuilder lazyWriteBuilder; SparkPositionDeltaOperation( - SparkSession spark, Table table, RowLevelOperationInfo info, IsolationLevel isolationLevel) { + SparkSession spark, + Table table, + RowLevelOperationInfo info, + IsolationLevel isolationLevel, + String branch) { this.spark = spark; this.table = table; this.command = info.command(); this.isolationLevel = isolationLevel; + this.branch = branch; } @Override @@ -64,10 +70,10 @@ public Command command() { public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { if (lazyScanBuilder == null) { this.lazyScanBuilder = - new SparkScanBuilder(spark, table, options) { + new SparkScanBuilder(spark, table, table.schema(), options, branch) { @Override public Scan build() { - Scan scan = super.buildMergeOnReadScan(); + Scan scan = super.buildMergeOnReadScan(branch); SparkPositionDeltaOperation.this.configuredScan = scan; return scan; } @@ -91,7 +97,8 @@ public DeltaWriteBuilder newWriteBuilder(LogicalWriteInfo info) { command, configuredScan, isolationLevel, - (ExtendedLogicalWriteInfo) info); + (ExtendedLogicalWriteInfo) info, + branch); } return lazyWriteBuilder; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 5eba7166c98b..1770baadaa93 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -100,6 +100,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde private final Map extraSnapshotMetadata; private final Distribution requiredDistribution; private final SortOrder[] requiredOrdering; + private final String branch; private boolean cleanupOnAbort = true; @@ -113,7 +114,8 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde ExtendedLogicalWriteInfo info, Schema dataSchema, Distribution requiredDistribution, - SortOrder[] requiredOrdering) { + SortOrder[] requiredOrdering, + String branch) { this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; this.command = command; @@ -126,6 +128,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata(); this.requiredDistribution = requiredDistribution; this.requiredOrdering = requiredOrdering; + this.branch = branch; } @Override @@ -275,6 +278,7 @@ private void commitOperation(SnapshotUpdate operation, String description) { try { long start = System.currentTimeMillis(); + operation.toBranch(branch); operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java index ebac7e2515cc..2142666b2530 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java @@ -55,6 +55,7 @@ class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { private final boolean handleTimestampWithoutZone; private final boolean checkNullability; private final boolean checkOrdering; + private final String branch; SparkPositionDeltaWriteBuilder( SparkSession spark, @@ -62,7 +63,8 @@ class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { Command command, Scan scan, IsolationLevel isolationLevel, - ExtendedLogicalWriteInfo info) { + ExtendedLogicalWriteInfo info, + String branch) { this.spark = spark; this.table = table; this.command = command; @@ -73,6 +75,7 @@ class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone(); this.checkNullability = writeConf.checkNullability(); this.checkOrdering = writeConf.checkOrdering(); + this.branch = branch; } @Override @@ -115,7 +118,8 @@ public DeltaWrite build() { info, dataSchema, distribution, - ordering); + ordering, + branch); } private Schema dataSchema() { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java index 0673d647703c..5a8cc45299a0 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java @@ -40,6 +40,7 @@ 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; +import org.jetbrains.annotations.NotNull; class SparkRowLevelOperationBuilder implements RowLevelOperationBuilder { @@ -48,22 +49,25 @@ class SparkRowLevelOperationBuilder implements RowLevelOperationBuilder { private final RowLevelOperationInfo info; private final RowLevelOperationMode mode; private final IsolationLevel isolationLevel; + private final String branch; - SparkRowLevelOperationBuilder(SparkSession spark, Table table, RowLevelOperationInfo info) { + SparkRowLevelOperationBuilder( + SparkSession spark, @NotNull Table table, RowLevelOperationInfo info, String branch) { this.spark = spark; this.table = table; this.info = info; this.mode = mode(table.properties(), info.command()); this.isolationLevel = isolationLevel(table.properties(), info.command()); + this.branch = branch; } @Override public RowLevelOperation build() { switch (mode) { case COPY_ON_WRITE: - return new SparkCopyOnWriteOperation(spark, table, info, isolationLevel); + return new SparkCopyOnWriteOperation(spark, table, info, isolationLevel, branch); case MERGE_ON_READ: - return new SparkPositionDeltaOperation(spark, table, info, isolationLevel); + return new SparkPositionDeltaOperation(spark, table, info, isolationLevel, branch); default: throw new IllegalArgumentException("Unsupported operation mode: " + mode); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 1bc751e30b86..4c219fbd3238 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; import org.apache.iceberg.BatchScan; import org.apache.iceberg.FileScanTask; @@ -32,6 +33,7 @@ import org.apache.iceberg.MetricsModes; 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.TableProperties; @@ -82,6 +84,7 @@ public class SparkScanBuilder private static final Logger LOG = LoggerFactory.getLogger(SparkScanBuilder.class); private static final Filter[] NO_FILTERS = new Filter[0]; + private final String branch; private StructType pushedAggregateSchema; private Scan localScan; @@ -97,17 +100,27 @@ public class SparkScanBuilder private Filter[] pushedFilters = NO_FILTERS; SparkScanBuilder( - SparkSession spark, Table table, Schema schema, CaseInsensitiveStringMap options) { + SparkSession spark, + Table table, + Schema schema, + CaseInsensitiveStringMap options, + String branch) { this.spark = spark; this.table = table; this.schema = schema; this.options = options; this.readConf = new SparkReadConf(spark, table, options); this.caseSensitive = readConf.caseSensitive(); + this.branch = branch; + } + + SparkScanBuilder( + SparkSession spark, Table table, Schema schema, CaseInsensitiveStringMap options) { + this(spark, table, schema, options, SnapshotRef.MAIN_BRANCH); } SparkScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { - this(spark, table, table.schema(), options); + this(spark, table, table.schema(), options, SnapshotRef.MAIN_BRANCH); } private Expression filterExpression() { @@ -356,9 +369,19 @@ public Scan build() { } private Scan buildBatchScan() { - Long snapshotId = readConf.snapshotId(); + Preconditions.checkArgument( + readConf.branch() == null || readConf.branch().equals(branch), + "Specified different branch %s in read option and branch %s in identifier", + readConf.branch(), + branch); + + Snapshot branchSnapshot = null; + if (!(table instanceof BaseMetadataTable)) { + branchSnapshot = SnapshotUtil.latestSnapshot(table, branch); + } + + Long snapshotId = branchSnapshot != null ? branchSnapshot.snapshotId() : null; Long asOfTimestamp = readConf.asOfTimestamp(); - String branch = readConf.branch(); String tag = readConf.tag(); Preconditions.checkArgument( @@ -398,11 +421,11 @@ private Scan buildBatchScan() { if (startSnapshotId != null) { return buildIncrementalAppendScan(startSnapshotId, endSnapshotId); } else { - return buildBatchScan(snapshotId, asOfTimestamp, branch, tag); + return buildBatchScan(snapshotId, asOfTimestamp, tag); } } - private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, String tag) { + private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String tag) { Schema expectedSchema = schemaWithMetadataColumns(); BatchScan scan = @@ -420,10 +443,6 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = scan.asOfTime(asOfTimestamp); } - if (branch != null) { - scan = scan.useRef(branch); - } - if (tag != null) { scan = scan.useRef(tag); } @@ -536,6 +555,10 @@ private Long getStartSnapshotId(Long startTimestamp) { } public Scan buildMergeOnReadScan() { + return buildMergeOnReadScan(SnapshotRef.MAIN_BRANCH); + } + + public Scan buildMergeOnReadScan(String writeBranch) { Preconditions.checkArgument( readConf.snapshotId() == null && readConf.asOfTimestamp() == null @@ -553,7 +576,7 @@ public Scan buildMergeOnReadScan() { SparkReadOptions.START_SNAPSHOT_ID, SparkReadOptions.END_SNAPSHOT_ID); - Snapshot snapshot = table.currentSnapshot(); + Snapshot snapshot = table.snapshot(writeBranch); if (snapshot == null) { return new SparkBatchQueryScan( @@ -584,11 +607,22 @@ public Scan buildMergeOnReadScan() { } public Scan buildCopyOnWriteScan() { - Snapshot snapshot = table.currentSnapshot(); + return buildCopyOnWriteScan(SnapshotRef.MAIN_BRANCH); + } + + public Scan buildCopyOnWriteScan(String writeBranch) { + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, writeBranch); if (snapshot == null) { return new SparkCopyOnWriteScan( - spark, table, readConf, schemaWithMetadataColumns(), filterExpressions); + spark, + table, + null, + null, + readConf, + schemaWithMetadataColumns(), + filterExpressions, + writeBranch); } Schema expectedSchema = schemaWithMetadataColumns(); @@ -605,7 +639,7 @@ public Scan buildCopyOnWriteScan() { scan = configureSplitPlanning(scan); return new SparkCopyOnWriteScan( - spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions); + spark, table, scan, snapshot, readConf, expectedSchema, filterExpressions, writeBranch); } private > T configureSplitPlanning(T scan) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 004be8b95951..009f37c10c49 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -31,6 +31,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Partitioning; 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; @@ -113,18 +114,26 @@ public class SparkTable private final Long snapshotId; private final boolean refreshEagerly; private final Set capabilities; + private String branch = SnapshotRef.MAIN_BRANCH; private StructType lazyTableSchema = null; private SparkSession lazySpark = null; public SparkTable(Table icebergTable, boolean refreshEagerly) { - this(icebergTable, null, refreshEagerly); + this(icebergTable, (Long) null, refreshEagerly); + } + + public SparkTable(Table icebergTable, String branch, boolean refreshEagerly) { + this( + icebergTable, + icebergTable.snapshot(branch) == null ? null : icebergTable.snapshot(branch).snapshotId(), + refreshEagerly); + this.branch = branch; } public SparkTable(Table icebergTable, Long snapshotId, boolean refreshEagerly) { this.icebergTable = icebergTable; this.snapshotId = snapshotId; this.refreshEagerly = refreshEagerly; - boolean acceptAnySchema = PropertyUtil.propertyAsBoolean( icebergTable.properties(), @@ -154,6 +163,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } @@ -244,27 +257,22 @@ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { } CaseInsensitiveStringMap scanOptions = addSnapshotId(options, snapshotId); - return new SparkScanBuilder(sparkSession(), icebergTable, snapshotSchema(), scanOptions); + return new SparkScanBuilder( + sparkSession(), icebergTable, snapshotSchema(), scanOptions, branch); } @Override public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - Preconditions.checkArgument( - snapshotId == null, "Cannot write to table at a specific snapshot: %s", snapshotId); - - return new SparkWriteBuilder(sparkSession(), icebergTable, info); + return new SparkWriteBuilder(sparkSession(), icebergTable, info, branch); } @Override public RowLevelOperationBuilder newRowLevelOperationBuilder(RowLevelOperationInfo info) { - return new SparkRowLevelOperationBuilder(sparkSession(), icebergTable, info); + return new SparkRowLevelOperationBuilder(sparkSession(), icebergTable, info, branch); } @Override public boolean canDeleteWhere(Filter[] filters) { - Preconditions.checkArgument( - snapshotId == null, "Cannot delete from table at a specific snapshot: %s", snapshotId); - Expression deleteExpr = Expressions.alwaysTrue(); for (Filter filter : filters) { @@ -287,11 +295,16 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { return true; } + if (!table().refs().containsKey(branch)) { + return false; + } + TableScan scan = table() .newScan() .filter(deleteExpr) .caseSensitive(caseSensitive) + .useRef(branch) .includeColumnStats() .ignoreResiduals(); @@ -333,6 +346,7 @@ public void deleteWhere(Filter[] filters) { .newDelete() .set("spark.app.id", sparkSession().sparkContext().applicationId()) .deleteFromRowFilter(deleteExpr) + .toBranch(branch) .commit(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index f68898e27b3d..daa31d5af066 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -57,6 +57,7 @@ import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; import org.apache.spark.api.java.JavaSparkContext; @@ -91,6 +92,7 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { private final String applicationId; private final boolean wapEnabled; private final String wapId; + private final String branch; private final long targetFileSize; private final Schema writeSchema; private final StructType dsSchema; @@ -110,7 +112,8 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { Schema writeSchema, StructType dsSchema, Distribution requiredDistribution, - SortOrder[] requiredOrdering) { + SortOrder[] requiredOrdering, + String branch) { this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); this.table = table; this.writeConf = writeConf; @@ -119,6 +122,7 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { this.applicationId = applicationId; this.wapEnabled = writeConf.wapEnabled(); this.wapId = writeConf.wapId(); + this.branch = branch; this.targetFileSize = writeConf.targetDataFileSize(); this.writeSchema = writeSchema; this.dsSchema = dsSchema; @@ -204,6 +208,7 @@ private void commitOperation(SnapshotUpdate operation, String description) { try { long start = System.currentTimeMillis(); + operation.toBranch(branch); operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); @@ -499,7 +504,7 @@ protected void commit(SnapshotUpdate snapshotUpdate, long epochId, String } private Long findLastCommittedEpochId() { - Snapshot snapshot = table.currentSnapshot(); + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); Long lastCommittedEpochId = null; while (snapshot != null) { Map summary = snapshot.summary(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index 55cf7961e92f..965e3c188144 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -72,11 +72,13 @@ class SparkWriteBuilder implements WriteBuilder, SupportsDynamicOverwrite, Suppo private SparkCopyOnWriteScan copyOnWriteScan = null; private Command copyOnWriteCommand = null; private IsolationLevel copyOnWriteIsolationLevel = null; + private final String writeBranch; - SparkWriteBuilder(SparkSession spark, Table table, LogicalWriteInfo info) { + SparkWriteBuilder(SparkSession spark, Table table, LogicalWriteInfo info, String writeBranch) { this.spark = spark; this.table = table; this.writeConf = new SparkWriteConf(spark, table, info.options()); + this.writeBranch = writeBranch; this.writeInfo = info; this.dsSchema = info.schema(); this.overwriteMode = writeConf.overwriteMode(); @@ -162,7 +164,16 @@ public Write build() { } return new SparkWrite( - spark, table, writeConf, writeInfo, appId, writeSchema, dsSchema, distribution, ordering) { + spark, + table, + writeConf, + writeInfo, + appId, + writeSchema, + dsSchema, + distribution, + ordering, + writeBranch) { @Override public BatchWrite toBatch() { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 59074bbd923b..21eb71bf85cf 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -806,6 +806,16 @@ public static Set dataFiles(Table table) { return dataFiles; } + public static Set dataFiles(Table table, String branch) { + Set dataFiles = Sets.newHashSet(); + + for (FileScanTask task : table.newScan().useRef(branch).planFiles()) { + dataFiles.add(task.file()); + } + + return dataFiles; + } + public static Set deleteFiles(Table table) { Set deleteFiles = Sets.newHashSet(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java index a9b4f0d3ad2f..f5c5a8c16fc5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -24,6 +24,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -193,6 +194,46 @@ public void testMergeSchemaIcebergProperty() throws Exception { sql("select * from %s order by id", tableName)); } + @Test + public void testMergeSchemaOnBranch() throws Exception { + String branch = "test-branch"; + + 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).option(SparkWriteOptions.BRANCH, branch).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s version as of '%s' order by id", tableName, branch)); + + 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(SparkWriteOptions.MERGE_SCHEMA, "true") + .option(SparkWriteOptions.BRANCH, branch) + .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 version as of '%s' order by id", tableName, branch)); + } + @Test public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, ParseException { SparkSession sparkSession = spark.cloneSession(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 310e69b827a9..7ff9d2153703 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -36,6 +36,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.TableProperties; import org.apache.iceberg.avro.Avro; @@ -44,6 +45,7 @@ import org.apache.iceberg.io.FileAppender; 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.SparkSQLProperties; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkWriteOptions; @@ -51,6 +53,7 @@ import org.apache.iceberg.spark.data.RandomData; import org.apache.iceberg.spark.data.SparkAvroReader; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.SparkException; import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; @@ -156,6 +159,18 @@ public void testWriteWithCustomDataLocation() throws IOException { writeAndValidateWithLocations(table, location, tablePropertyDataLocation); } + @Test + public void testBranchWriteWithCustomDataLocation() throws IOException { + File location = createTableFolder(); + File tablePropertyDataLocation = temp.newFolder("test-table-property-data-dir"); + Table table = createTable(new Schema(SUPPORTED_PRIMITIVES.fields()), location); + table + .updateProperties() + .set(TableProperties.WRITE_DATA_LOCATION, tablePropertyDataLocation.getAbsolutePath()) + .commit(); + writeAndValidateWithLocations(table, location, tablePropertyDataLocation, "test-branch"); + } + private File createTableFolder() throws IOException { File parent = temp.newFolder("parquet"); File location = new File(parent, "test"); @@ -170,16 +185,21 @@ private Table createTable(Schema schema, File location) { private void writeAndValidateWithLocations(Table table, File location, File expectedDataDir) throws IOException { + writeAndValidateWithLocations(table, location, expectedDataDir, SnapshotRef.MAIN_BRANCH); + } + + private void writeAndValidateWithLocations( + Table table, File location, File expectedDataDir, String branch) throws IOException { Schema tableSchema = table.schema(); // use the table schema because ids are reassigned table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit(); Iterable expected = RandomData.generate(tableSchema, 100, 0L); - writeData(expected, tableSchema, location.toString()); + writeData(expected, tableSchema, location.toString(), branch); table.refresh(); - List actual = readTable(location.toString()); + List actual = readTable(location.toString(), branch); Iterator expectedIter = expected.iterator(); Iterator actualIter = actual.iterator(); @@ -189,8 +209,7 @@ private void writeAndValidateWithLocations(Table table, File location, File expe Assert.assertEquals( "Both iterators should be exhausted", expectedIter.hasNext(), actualIter.hasNext()); - table - .currentSnapshot() + SnapshotUtil.latestSnapshot(table, branch) .addedDataFiles(table.io()) .forEach( dataFile -> @@ -204,15 +223,26 @@ private void writeAndValidateWithLocations(Table table, File location, File expe } private List readTable(String location) { - Dataset result = spark.read().format("iceberg").load(location); + return readTable(location, SnapshotRef.MAIN_BRANCH); + } + + private List readTable(String location, String branch) { + Dataset result = + spark.read().format("iceberg").option(SparkReadOptions.BRANCH, branch).load(location); return result.collectAsList(); } private void writeData(Iterable records, Schema schema, String location) throws IOException { + writeData(records, schema, location, SnapshotRef.MAIN_BRANCH); + } + + private void writeData(Iterable records, Schema schema, String location, String branch) + throws IOException { Dataset df = createDataset(records, schema); - DataFrameWriter writer = df.write().format("iceberg").mode("append"); + DataFrameWriter writer = + df.write().format("iceberg").option(SparkWriteOptions.BRANCH, branch).mode("append"); writer.save(location); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index 276fbcd592ae..b92c8b488efc 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -177,6 +177,26 @@ public void testSnapshotSelectionByInvalidSnapshotId() throws IOException { .hasMessage("Cannot find snapshot with ID -10"); } + @Test + public void testSnapshotSelectionByInvalidBranch() throws IOException { + String tableLocation = temp.newFolder("iceberg-table").toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + tables.create(SCHEMA, spec, tableLocation); + + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.BRANCH, "non-existing-branch") + .load(tableLocation); + + Assertions.assertThatThrownBy(df::collectAsList) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find ref non-existing-branch"); + } + @Test public void testSnapshotSelectionByInvalidTimestamp() throws IOException { long timestamp = System.currentTimeMillis(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index dac1c150cdb6..390302451c84 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -46,6 +46,7 @@ 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.SparkException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -73,9 +74,16 @@ public class TestSparkDataWrite { @Rule public TemporaryFolder temp = new TemporaryFolder(); - @Parameterized.Parameters(name = "format = {0}") + private String targetBranch; + + @Parameterized.Parameters(name = "format = {0}, branch = {1}") public static Object[] parameters() { - return new Object[] {"parquet", "avro", "orc"}; + return new Object[][] { + new Object[] {"parquet", "main"}, + new Object[] {"parquet", "testBranch"}, + new Object[] {"avro", "main"}, + new Object[] {"orc", "testBranch"}, + }; } @BeforeClass @@ -95,8 +103,9 @@ public static void stopSpark() { currentSpark.stop(); } - public TestSparkDataWrite(String format) { + public TestSparkDataWrite(String format, String targetBranch) { this.format = FileFormat.fromString(format); + this.targetBranch = targetBranch; } @Test @@ -119,17 +128,24 @@ public void testBasicWrite() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); Assert.assertEquals("Number of rows should match", expected.size(), actual.size()); Assert.assertEquals("Result rows should match", expected, actual); - for (ManifestFile manifest : table.currentSnapshot().allManifests(table.io())) { + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, targetBranch).allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { // TODO: avro not support split if (!format.equals(FileFormat.AVRO)) { @@ -177,6 +193,7 @@ public void testAppend() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); df.withColumn("id", df.col("id").plus(3)) @@ -185,11 +202,17 @@ public void testAppend() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -218,6 +241,7 @@ public void testEmptyOverwrite() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); Dataset empty = spark.createDataFrame(ImmutableList.of(), SimpleRecord.class); @@ -228,11 +252,17 @@ public void testEmptyOverwrite() throws IOException { .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Overwrite) .option("overwrite-mode", "dynamic") + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -268,6 +298,7 @@ public void testOverwrite() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); // overwrite with 2*id to replace record 2, append 4 and 6 @@ -278,11 +309,17 @@ public void testOverwrite() throws IOException { .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Overwrite) .option("overwrite-mode", "dynamic") + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -310,6 +347,7 @@ public void testUnpartitionedOverwrite() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); // overwrite with the same data; should not produce two copies @@ -318,11 +356,17 @@ public void testUnpartitionedOverwrite() throws IOException { .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Overwrite) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -356,11 +400,17 @@ public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws .format("iceberg") .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .mode(SaveMode.Append) + .option(SparkWriteOptions.BRANCH, targetBranch) .save(location.toString()); table.refresh(); - Dataset result = spark.read().format("iceberg").load(location.toString()); + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkWriteOptions.BRANCH, targetBranch) + .load(location.toString()); List actual = result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); @@ -368,7 +418,8 @@ public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() throws Assert.assertEquals("Result rows should match", expected, actual); List files = Lists.newArrayList(); - for (ManifestFile manifest : table.currentSnapshot().allManifests(table.io())) { + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, targetBranch).allManifests(table.io())) { for (DataFile file : ManifestFiles.read(manifest, table.io())) { files.add(file); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java index 51c56ac79d4d..3e5382c37344 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java @@ -20,8 +20,11 @@ import java.util.List; import java.util.Map; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkCatalogTestBase; +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; @@ -31,11 +34,35 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runners.Parameterized; public class TestPartitionedWrites extends SparkCatalogTestBase { + + private String targetBranch; + public TestPartitionedWrites( - String catalogName, String implementation, Map config) { + String catalogName, String implementation, Map config, String targetBranch) { super(catalogName, implementation, config); + this.targetBranch = targetBranch; + } + + @Parameterized.Parameters( + name = "catalogName = {0}, implementation = {1}, config = {2}, branch = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + SnapshotRef.MAIN_BRANCH + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + "test-branch" + } + }; } @Before @@ -91,16 +118,20 @@ public void testDataFrameV2Append() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).append(); + ds.writeTo(tableName).option(SparkWriteOptions.BRANCH, targetBranch).append(); Assert.assertEquals( - "Should have 5 rows after insert", 5L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 5 rows after insert", + 5L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); 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", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } @Test @@ -110,16 +141,20 @@ public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).overwritePartitions(); + ds.writeTo(tableName).option(SparkWriteOptions.BRANCH, targetBranch).overwritePartitions(); Assert.assertEquals( - "Should have 4 rows after overwrite", 4L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 4 rows after overwrite", + 4L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); 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", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } @Test @@ -129,15 +164,21 @@ public void testDataFrameV2Overwrite() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).overwrite(functions.col("id").$less(3)); + ds.writeTo(tableName) + .option(SparkWriteOptions.BRANCH, targetBranch) + .overwrite(functions.col("id").$less(3)); Assert.assertEquals( - "Should have 3 rows after overwrite", 3L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 3 rows after overwrite", + 3L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); 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", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java index 0849602c3b92..636985064567 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java @@ -21,8 +21,11 @@ import java.util.List; import java.util.Map; import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkCatalogTestBase; +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; @@ -32,11 +35,34 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runners.Parameterized; public class TestUnpartitionedWrites extends SparkCatalogTestBase { + private String targetBranch; + public TestUnpartitionedWrites( - String catalogName, String implementation, Map config) { + String catalogName, String implementation, Map config, String targetBranch) { super(catalogName, implementation, config); + this.targetBranch = targetBranch; + } + + @Parameterized.Parameters( + name = "catalogName = {0}, implementation = {1}, config = {2}, branch = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + SnapshotRef.MAIN_BRANCH + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties(), + "test-branch" + } + }; } @Before @@ -113,16 +139,20 @@ public void testDataFrameV2Append() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).append(); + ds.writeTo(tableName).option(SparkWriteOptions.BRANCH, targetBranch).append(); Assert.assertEquals( - "Should have 5 rows after insert", 5L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 5 rows after insert", + 5L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); 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", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } @Test @@ -132,15 +162,19 @@ public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).overwritePartitions(); + ds.writeTo(tableName).option(SparkWriteOptions.BRANCH, targetBranch).overwritePartitions(); Assert.assertEquals( - "Should have 2 rows after overwrite", 2L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 2 rows after overwrite", + 2L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); assertEquals( - "Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } @Test @@ -150,14 +184,20 @@ public void testDataFrameV2Overwrite() throws NoSuchTableException { List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); Dataset ds = spark.createDataFrame(data, SimpleRecord.class); - ds.writeTo(tableName).overwrite(functions.col("id").$less$eq(3)); + ds.writeTo(tableName) + .option(SparkWriteOptions.BRANCH, targetBranch) + .overwrite(functions.col("id").$less$eq(3)); Assert.assertEquals( - "Should have 2 rows after overwrite", 2L, scalarSql("SELECT count(*) FROM %s", tableName)); + "Should have 2 rows after overwrite", + 2L, + scalarSql("SELECT count(*) FROM %s VERSION AS OF '%s'", tableName, targetBranch)); List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); assertEquals( - "Row data should match expected", expected, sql("SELECT * FROM %s ORDER BY id", tableName)); + "Row data should match expected", + expected, + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, targetBranch)); } }