diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java index 48838f09c998..b3fd4f2b6657 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java @@ -13,6 +13,7 @@ */ package io.trino.plugin.iceberg; +import com.google.common.collect.ImmutableList; import org.apache.iceberg.FileContent; import org.apache.iceberg.SortOrder; @@ -33,7 +34,8 @@ public record CommitTaskData( Optional referencedDataFile, Optional> fileSplitOffsets, int sortOrderId, - Optional serializedDeletionVector) + Optional serializedDeletionVector, + Optional rewriteInfo) { public CommitTaskData { @@ -47,5 +49,65 @@ public record CommitTaskData( requireNonNull(fileSplitOffsets, "fileSplitOffsets is null"); checkArgument(content == FileContent.DATA || sortOrderId == SortOrder.unsorted().orderId(), "Sorted order id can be present only for data files"); requireNonNull(serializedDeletionVector, "serializedDeletionVector is null"); + requireNonNull(rewriteInfo, "rewriteInfo is null"); + checkArgument(rewriteInfo.isEmpty() || content == FileContent.DATA, "rewriteInfo can only be present for DATA content, got %s", content); + } + + public CommitTaskData( + String path, + IcebergFileFormat fileFormat, + long fileSizeInBytes, + MetricsWrapper metrics, + String partitionSpecJson, + Optional partitionDataJson, + FileContent content, + Optional referencedDataFile, + Optional> fileSplitOffsets, + int sortOrderId, + Optional serializedDeletionVector) + { + this(path, fileFormat, fileSizeInBytes, metrics, partitionSpecJson, partitionDataJson, + content, referencedDataFile, fileSplitOffsets, sortOrderId, serializedDeletionVector, + Optional.empty()); + } + + public record RewriteInfo( + String oldFilePath, + long oldFileSizeInBytes, + long oldRecordCount, + IcebergFileFormat oldFileFormat, + List danglingDeleteFiles) + { + public RewriteInfo + { + requireNonNull(oldFilePath, "oldFilePath is null"); + requireNonNull(oldFileFormat, "oldFileFormat is null"); + checkArgument(oldFileSizeInBytes >= 0, "oldFileSizeInBytes is negative"); + checkArgument(oldRecordCount >= 0, "oldRecordCount is negative"); + danglingDeleteFiles = ImmutableList.copyOf(requireNonNull(danglingDeleteFiles, "danglingDeleteFiles is null")); + } + + public RewriteInfo(String oldFilePath, long oldFileSizeInBytes, long oldRecordCount, IcebergFileFormat oldFileFormat) + { + this(oldFilePath, oldFileSizeInBytes, oldRecordCount, oldFileFormat, ImmutableList.of()); + } + } + + public record DanglingDeleteFile( + String path, + long fileSizeInBytes, + long recordCount, + String partitionSpecJson, + Optional partitionDataJson, + Long contentOffset, + Long contentSizeInBytes, + String referencedDataFile) + { + public DanglingDeleteFile + { + requireNonNull(path, "path is null"); + requireNonNull(partitionSpecJson, "partitionSpecJson is null"); + requireNonNull(partitionDataJson, "partitionDataJson is null"); + } } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CopyOnWriteFileRewriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CopyOnWriteFileRewriter.java new file mode 100644 index 000000000000..f0445abbd819 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CopyOnWriteFileRewriter.java @@ -0,0 +1,264 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import com.google.inject.Inject; +import io.airlift.log.Logger; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; +import io.trino.plugin.iceberg.delete.DeleteFile; +import io.trino.plugin.iceberg.delete.DeletionVector; +import io.trino.spi.Page; +import io.trino.spi.block.Block; +import io.trino.spi.connector.ConnectorPageSource; +import io.trino.spi.connector.ConnectorSession; +import io.trino.spi.connector.DynamicFilter; +import io.trino.spi.connector.SourcePage; +import io.trino.spi.predicate.TupleDomain; +import io.trino.spi.type.TypeManager; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.mapping.NameMapping; + +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; + +import static com.google.common.collect.ImmutableList.toImmutableList; +import static io.trino.plugin.iceberg.IcebergUtil.getColumnHandle; +import static io.trino.spi.type.BigintType.BIGINT; +import static java.util.Objects.requireNonNull; +import static java.util.UUID.randomUUID; + +/** + * Reads an Iceberg data file, filters out deleted positions, and writes the + * surviving rows to a new data file. Used by workers during copy-on-write + * DML operations to rewrite affected data files before the coordinator commits. + * + *

This class is stateless and thread-safe; a single instance is shared + * across all queries in the connector. + */ +public class CopyOnWriteFileRewriter +{ + private static final Logger log = Logger.get(CopyOnWriteFileRewriter.class); + private final IcebergPageSourceProviderFactory pageSourceProviderFactory; + private final IcebergFileWriterFactory fileWriterFactory; + private final IcebergFileSystemFactory fileSystemFactory; + private final TypeManager typeManager; + + @Inject + public CopyOnWriteFileRewriter( + IcebergPageSourceProviderFactory pageSourceProviderFactory, + IcebergFileWriterFactory fileWriterFactory, + IcebergFileSystemFactory fileSystemFactory, + TypeManager typeManager) + { + this.pageSourceProviderFactory = requireNonNull(pageSourceProviderFactory, "pageSourceProviderFactory is null"); + this.fileWriterFactory = requireNonNull(fileWriterFactory, "fileWriterFactory is null"); + this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + public record RewriteMetrics( + long originalRecordCount, + long newRecordCount, + long deletedRowCount, + long originalFileSizeBytes, + long newFileSizeBytes, + long rewriteDurationMs) {} + + public record RewriteResult( + Optional newFile, + Closeable rollbackAction, + RewriteMetrics metrics, + Metrics newFileMetrics) {} + + /** + * Reads {@code originalPath}, filters rows marked in {@code deletionVector}, + * and writes surviving rows to a new data file. + * + * @return a {@link RewriteResult} containing the new DataFile (empty if + * all rows were deleted), metrics, and a rollback action that + * deletes the newly written file on failure + */ + public RewriteResult rewriteFile( + ConnectorSession session, + String originalPath, + long originalFileSize, + DeletionVector deletionVector, + Schema schema, + PartitionSpec partitionSpec, + Optional partitionData, + List preExistingDeletes, + long sourceDataSequenceNumber, + OptionalLong sourceFileFirstRowId, + IcebergFileFormat sourceFileFormat, + IcebergFileFormat writeFileFormat, + MetricsConfig metricsConfig, + Map fileIoProperties, + Map tableProperties, + LocationProvider locationProvider, + Optional nameMapping) + { + long startTimeMs = System.currentTimeMillis(); + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), requireNonNull(fileIoProperties, "fileIoProperties is null")); + + String fileName = writeFileFormat.toIceberg().addExtension(session.getQueryId() + "-" + randomUUID()); + String newFilePath = partitionData + .map(partition -> locationProvider.newDataLocation(partitionSpec, partition, fileName)) + .orElseGet(() -> locationProvider.newDataLocation(fileName)); + + IcebergFileWriter writer = fileWriterFactory.createDataFileWriter( + fileSystem, Location.of(newFilePath), schema, session, + writeFileFormat, metricsConfig, tableProperties); + + long originalRecordCount = readFilterWrite( + session, writer, originalPath, originalFileSize, deletionVector, + schema, partitionSpec, partitionData, + preExistingDeletes, sourceDataSequenceNumber, sourceFileFirstRowId, + sourceFileFormat, fileIoProperties, nameMapping); + + Closeable rollbackAction = writer.commit(); + IcebergFileWriter.FileMetrics fileMetrics = writer.getFileMetrics(); + Metrics fullMetrics = fileMetrics.metrics(); + long newRecordCount = fullMetrics.recordCount(); + long newFileSize = writer.getWrittenBytes(); + Optional newFile = Optional.empty(); + if (newRecordCount > 0) { + DataFiles.Builder newBuilder = DataFiles.builder(partitionSpec) + .withPath(newFilePath) + .withFormat(writeFileFormat.toIceberg()) + .withFileSizeInBytes(newFileSize) + .withMetrics(fullMetrics); + fileMetrics.splitOffsets().ifPresent(newBuilder::withSplitOffsets); + partitionData.ifPresent(newBuilder::withPartition); + newFile = Optional.of(newBuilder.build()); + } + else { + // All rows were deleted -- clean up the empty output file to avoid orphans + try { + rollbackAction.close(); + } + catch (IOException e) { + log.warn(e, "Failed to delete empty CoW rewrite output file: %s", newFilePath); + } + rollbackAction = () -> {}; + } + + long durationMs = System.currentTimeMillis() - startTimeMs; + long deletedRowCount = originalRecordCount - newRecordCount; + RewriteMetrics metrics = new RewriteMetrics( + originalRecordCount, newRecordCount, deletedRowCount, + originalFileSize, newFileSize, durationMs); + + log.debug("CoW rewrite: %s -> %s (%d rows -> %d rows, %d deleted, %d bytes -> %d bytes, %d ms)", + originalPath, newFile.isPresent() ? newFilePath : "", + originalRecordCount, newRecordCount, deletedRowCount, + originalFileSize, newFileSize, durationMs); + + return new RewriteResult(newFile, rollbackAction, metrics, fullMetrics); + } + + private long readFilterWrite( + ConnectorSession session, + IcebergFileWriter writer, + String originalPath, + long originalFileSize, + DeletionVector deletionVector, + Schema schema, + PartitionSpec partitionSpec, + Optional partitionData, + List preExistingDeletes, + long sourceDataSequenceNumber, + OptionalLong sourceFileFirstRowId, + IcebergFileFormat fileFormat, + Map fileIoProperties, + Optional nameMapping) + { + List dataColumns = schema.columns().stream() + .map(column -> getColumnHandle(column, typeManager)) + .collect(toImmutableList()); + List readColumns = new ArrayList<>(dataColumns); + readColumns.add(getColumnHandle(MetadataColumns.ROW_POSITION, typeManager)); + + IcebergPageSourceProvider pageSourceProvider = pageSourceProviderFactory.createPageSourceProvider(); + + long originalRecordCount = 0; + try (ConnectorPageSource source = pageSourceProvider.createPageSource( + session, readColumns, schema, partitionSpec, partitionData.orElseGet(() -> new PartitionData(new Object[0])), + preExistingDeletes, DynamicFilter.EMPTY, TupleDomain.all(), TupleDomain.all(), + originalPath, 0, originalFileSize, originalFileSize, 0, + fileFormat, fileIoProperties, sourceDataSequenceNumber, sourceFileFirstRowId, nameMapping)) { + int dataColumnCount = dataColumns.size(); + int[] dataChannels = new int[dataColumnCount]; + for (int channel = 0; channel < dataColumnCount; channel++) { + dataChannels[channel] = channel; + } + while (!source.isFinished()) { + SourcePage sourcePage = source.getNextSourcePage(); + if (sourcePage == null) { + continue; + } + Page page = sourcePage.getPage(); + Page dataPage = page.getColumns(dataChannels); + int positionCount = dataPage.getPositionCount(); + originalRecordCount += positionCount; + Block rowPositionBlock = page.getBlock(dataColumnCount); + + int[] kept = new int[positionCount]; + int keptCount = 0; + for (int i = 0; i < positionCount; i++) { + long rowPosition = BIGINT.getLong(rowPositionBlock, i); + if (!deletionVector.isRowDeleted(rowPosition)) { + kept[keptCount++] = i; + } + } + + if (keptCount > 0) { + writer.appendRows(dataPage.getPositions(kept, 0, keptCount)); + } + } + } + catch (Throwable t) { + try { + writer.rollback(); + } + catch (Throwable rollbackFailure) { + t.addSuppressed(rollbackFailure); + } + if (t instanceof IOException ioException) { + throw new UncheckedIOException("Failed to rewrite data file: " + originalPath, ioException); + } + if (t instanceof RuntimeException runtimeException) { + throw runtimeException; + } + if (t instanceof Error error) { + throw error; + } + throw new RuntimeException("Failed to rewrite data file: " + originalPath, t); + } + return originalRecordCount; + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/ForIcebergCopyOnWriteRewrite.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/ForIcebergCopyOnWriteRewrite.java new file mode 100644 index 000000000000..b9ceeea862bc --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/ForIcebergCopyOnWriteRewrite.java @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.trino.plugin.iceberg; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@BindingAnnotation +public @interface ForIcebergCopyOnWriteRewrite {} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergColumnHandle.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergColumnHandle.java index ead4920e6f08..eb6a0824a193 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergColumnHandle.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergColumnHandle.java @@ -62,6 +62,11 @@ public class IcebergColumnHandle public static final int DATA_CHANGE_ORDINAL_ID = Integer.MIN_VALUE + 6; public static final int TRINO_MERGE_SOURCE_ROW_ID = Integer.MIN_VALUE + 7; + public static final int TRINO_MERGE_FILE_FORMAT = Integer.MIN_VALUE + 8; + public static final int TRINO_MERGE_FILE_SIZE = Integer.MIN_VALUE + 9; + public static final int TRINO_MERGE_FILE_RECORD_COUNT = Integer.MIN_VALUE + 10; + public static final int TRINO_MERGE_DATA_SEQUENCE_NUMBER = Integer.MIN_VALUE + 11; + public static final int TRINO_MERGE_FILE_FIRST_ROW_ID = Integer.MIN_VALUE + 12; private final ColumnIdentity baseColumnIdentity; private final Type baseType; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java index 59cdb6fac9b6..f92d474a9316 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java @@ -95,6 +95,7 @@ public class IcebergConfig private int splitManagerThreads = Math.min(Runtime.getRuntime().availableProcessors() * 2, 32); private int planningThreads = Runtime.getRuntime().availableProcessors() * 2; private int fileDeleteThreads = Runtime.getRuntime().availableProcessors() * 2; + private int copyOnWriteRewriteThreads = Runtime.getRuntime().availableProcessors(); private List allowedExtraProperties = ImmutableList.of(); private boolean incrementalRefreshEnabled = true; private int materializedViewRefreshMaxSnapshotsToExpire = 200; @@ -568,6 +569,20 @@ public IcebergConfig setFileDeleteThreads(String fileDeleteThreads) return this; } + @Min(1) + public int getCopyOnWriteRewriteThreads() + { + return copyOnWriteRewriteThreads; + } + + @Config("iceberg.copy-on-write-rewrite-threads") + @ConfigDescription("Number of threads per worker for parallel file rewrites during copy-on-write operations") + public IcebergConfig setCopyOnWriteRewriteThreads(String copyOnWriteRewriteThreads) + { + this.copyOnWriteRewriteThreads = ThreadCountParser.DEFAULT.parse(copyOnWriteRewriteThreads); + return this; + } + public List getAllowedExtraProperties() { return allowedExtraProperties; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergExecutorModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergExecutorModule.java index 9d401f33b808..0fa3c8d3870c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergExecutorModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergExecutorModule.java @@ -40,6 +40,7 @@ public void configure(Binder binder) closingBinder(binder).registerExecutor(Key.get(ListeningExecutorService.class, ForIcebergSplitSource.class)); closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForIcebergSplitManager.class)); closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForIcebergPlanning.class)); + closingBinder(binder).registerExecutor(Key.get(ExecutorService.class, ForIcebergCopyOnWriteRewrite.class)); } @Singleton @@ -96,4 +97,14 @@ public ExecutorService createFileDeleteExecutor(CatalogName catalogName, Iceberg config.getFileDeleteThreads(), daemonThreadsNamed("iceberg-file-delete-" + catalogName + "-%s")); } + + @Provides + @Singleton + @ForIcebergCopyOnWriteRewrite + public ExecutorService createCopyOnWriteRewriteExecutor(CatalogName catalogName, IcebergConfig config) + { + return newFixedThreadPool( + config.getCopyOnWriteRewriteThreads(), + daemonThreadsNamed("iceberg-cow-rewrite-" + catalogName + "-%s")); + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java index e6549444600a..ed62dc52aa02 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java @@ -16,10 +16,13 @@ import com.google.common.base.VerifyException; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.trino.filesystem.TrinoFileSystem; +import io.trino.plugin.iceberg.delete.DeleteFile; import io.trino.plugin.iceberg.delete.DeletionVector; import io.trino.plugin.iceberg.delete.PositionDeleteWriter; +import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.Page; import io.trino.spi.block.Block; import io.trino.spi.block.LongArrayBlock; @@ -29,26 +32,51 @@ import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.MergePage; import io.trino.spi.type.VarcharType; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StaticTableOperations; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutorService; import static com.google.common.base.Verify.verify; import static io.airlift.slice.Slices.wrappedBuffer; import static io.trino.plugin.base.util.Closables.closeAllSuppress; +import static io.trino.plugin.iceberg.IcebergSessionProperties.isUseFileSizeFromMetadata; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.TinyintType.TINYINT; @@ -58,18 +86,29 @@ public class IcebergMergeSink implements ConnectorMergeSink { + private static final Logger log = Logger.get(IcebergMergeSink.class); + private final int formatVersion; private final LocationProvider locationProvider; private final IcebergFileWriterFactory fileWriterFactory; private final TrinoFileSystem fileSystem; + private final ForwardingFileIoFactory fileIoFactory; + private final ExecutorService copyOnWriteRewriteExecutor; private final JsonCodec jsonCodec; private final ConnectorSession session; private final IcebergFileFormat fileFormat; + private final Map fileIoProperties; private final Map storageProperties; private final Schema schema; + private final Schema rewriteSchema; private final Map partitionsSpecs; private final ConnectorPageSink insertPageSink; private final int columnCount; + private final int writeSortOrderId; + private final RowLevelOperationMode rowLevelOperationMode; + private final CopyOnWriteFileRewriter copyOnWriteFileRewriter; + private final Optional baseTableMetadataLocation; + private final OptionalLong snapshotId; private final Map fileDeletions = new HashMap<>(); private long writtenBytes; @@ -78,27 +117,44 @@ public IcebergMergeSink( LocationProvider locationProvider, IcebergFileWriterFactory fileWriterFactory, TrinoFileSystem fileSystem, + ForwardingFileIoFactory fileIoFactory, + ExecutorService copyOnWriteRewriteExecutor, JsonCodec jsonCodec, ConnectorSession session, IcebergFileFormat fileFormat, + Map fileIoProperties, Map storageProperties, Schema schema, Map partitionsSpecs, ConnectorPageSink insertPageSink, - int columnCount) + int columnCount, + int writeSortOrderId, + RowLevelOperationMode rowLevelOperationMode, + CopyOnWriteFileRewriter copyOnWriteFileRewriter, + Optional baseTableMetadataLocation, + OptionalLong snapshotId) { this.formatVersion = formatVersion; this.locationProvider = requireNonNull(locationProvider, "locationProvider is null"); this.fileWriterFactory = requireNonNull(fileWriterFactory, "fileWriterFactory is null"); this.fileSystem = requireNonNull(fileSystem, "fileSystem is null"); + this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); + this.copyOnWriteRewriteExecutor = requireNonNull(copyOnWriteRewriteExecutor, "copyOnWriteRewriteExecutor is null"); this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null"); this.session = requireNonNull(session, "session is null"); this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); + this.fileIoProperties = ImmutableMap.copyOf(requireNonNull(fileIoProperties, "fileIoProperties is null")); this.storageProperties = ImmutableMap.copyOf(requireNonNull(storageProperties, "storageProperties is null")); this.schema = requireNonNull(schema, "schema is null"); + this.rewriteSchema = getRewriteSchema(this.schema, formatVersion); this.partitionsSpecs = ImmutableMap.copyOf(requireNonNull(partitionsSpecs, "partitionsSpecs is null")); this.insertPageSink = requireNonNull(insertPageSink, "insertPageSink is null"); this.columnCount = columnCount; + this.writeSortOrderId = writeSortOrderId; + this.rowLevelOperationMode = requireNonNull(rowLevelOperationMode, "rowLevelOperationMode is null"); + this.copyOnWriteFileRewriter = requireNonNull(copyOnWriteFileRewriter, "copyOnWriteFileRewriter is null"); + this.baseTableMetadataLocation = requireNonNull(baseTableMetadataLocation, "baseTableMetadataLocation is null"); + this.snapshotId = requireNonNull(snapshotId, "snapshotId is null"); } @Override @@ -109,7 +165,7 @@ public void storeMergedRows(Page page) mergePage.getDeletionsPage().ifPresent(this::processRemovals); mergePage.getInsertionsPage().ifPresent(insertionsPage -> { if (formatVersion >= 3) { - insertPageSink.appendPage(createInsertionsPageWithRowId(insertionsPage, page)); + insertPageSink.appendPage(createInsertionsPageWithRowLineage(insertionsPage, page)); } else { insertPageSink.appendPage(insertionsPage); @@ -126,6 +182,12 @@ private void processRemovals(Page removals) Block rowPositionBlock = fields.get(1); Block partitionSpecIdBlock = fields.get(2); Block partitionDataBlock = fields.get(3); + // fields[4] = sourceRowId (not needed for CoW) + Block fileFormatBlock = fields.get(5); + Block fileSizeBlock = fields.get(6); + Block fileRecordCountBlock = fields.get(7); + Block dataSequenceNumberBlock = fields.get(8); + Block fileFirstRowIdBlock = fields.get(9); for (int position = 0; position < filePathBlock.getPositionCount(); position++) { Slice filePath = VarcharType.VARCHAR.getSlice(filePathBlock, position); long rowPosition = BIGINT.getLong(rowPositionBlock, position); @@ -134,7 +196,17 @@ private void processRemovals(Page removals) FileDeletion deletion = fileDeletions.computeIfAbsent(filePath, _ -> { int partitionSpecId = INTEGER.getInt(partitionSpecIdBlock, index); String partitionData = VarcharType.VARCHAR.getSlice(partitionDataBlock, index).toStringUtf8(); - return new FileDeletion(partitionSpecId, partitionData); + int fileFormatOrdinal = INTEGER.getInt(fileFormatBlock, index); + IcebergFileFormat[] formats = IcebergFileFormat.values(); + verify(fileFormatOrdinal >= 0 && fileFormatOrdinal < formats.length, "Invalid file format ordinal: %s", fileFormatOrdinal); + IcebergFileFormat fileFormat = formats[fileFormatOrdinal]; + long fileSize = BIGINT.getLong(fileSizeBlock, index); + long fileRecordCount = BIGINT.getLong(fileRecordCountBlock, index); + long dataSequenceNumber = BIGINT.getLong(dataSequenceNumberBlock, index); + OptionalLong fileFirstRowId = fileFirstRowIdBlock.isNull(index) + ? OptionalLong.empty() + : OptionalLong.of(BIGINT.getLong(fileFirstRowIdBlock, index)); + return new FileDeletion(partitionSpecId, partitionData, fileFormat, fileSize, fileRecordCount, dataSequenceNumber, fileFirstRowId); }); deletion.rowsToDelete().add(rowPosition); @@ -149,6 +221,236 @@ public long getCompletedBytes() @Override public CompletableFuture> finish() + { + if (rowLevelOperationMode == RowLevelOperationMode.COPY_ON_WRITE) { + return finishCopyOnWrite(); + } + return finishMergeOnRead(); + } + + /** + * For copy-on-write mode, each merge-writer task has the complete deletion + * vector for its files (guaranteed by file-path-based update partitioning + * via IcebergUpdateBucketFunction). This method rewrites affected files in + * parallel using CopyOnWriteFileRewriter while applying pre-existing delete + * files (position/equality) in read path, and produces rewrite fragments for + * the coordinator to commit. + */ + private CompletableFuture> finishCopyOnWrite() + { + List fragments = new ArrayList<>(insertPageSink.finish().join()); + writtenBytes = insertPageSink.getCompletedBytes(); + + if (fileDeletions.isEmpty()) { + return completedFuture(fragments); + } + + FrozenTableMetadata frozenTableMetadata = loadBaseTableMetadata(); + Map tableProperties = frozenTableMetadata.tableMetadata().properties(); + + long startNanos = System.nanoTime(); + List inputs = buildRewriteInputs(frozenTableMetadata); + List results = executeParallelRewrites(inputs, tableProperties); + collectRewriteFragments(inputs, results, getWriteFileFormat(tableProperties), fragments, startNanos); + + return completedFuture(fragments); + } + + private record RewriteInput( + String originalPath, + DeletionVector deletionVector, + PartitionSpec partitionSpec, + Optional partitionData, + String partitionDataJson, + IcebergFileFormat sourceFileFormat, + long sourceFileSizeInBytes, + long sourceRecordCount, + long sourceDataSequenceNumber, + OptionalLong sourceFileFirstRowId, + List preExistingDeletes) {} + + private List buildRewriteInputs(FrozenTableMetadata frozenTableMetadata) + { + Map deletionVectors = new HashMap<>(); + Map deletionDetails = new HashMap<>(); + fileDeletions.forEach((dataFilePath, deletion) -> { + String path = dataFilePath.toStringUtf8(); + deletionVectors.put(path, deletion.rowsToDelete()); + deletionDetails.put(path, deletion); + }); + + Set rewrittenDataFiles = deletionVectors.keySet(); + + BaseTable frozenIcebergTable = createFrozenIcebergTable(frozenTableMetadata); + Map> preExistingDeletesByDataFile = collectPreExistingDeletesByDataFile( + frozenIcebergTable, + snapshotId.orElseThrow(() -> new VerifyException("Missing snapshot ID for copy-on-write merge")), + rewrittenDataFiles); + + List inputs = new ArrayList<>(); + for (Map.Entry entry : deletionVectors.entrySet()) { + String originalPath = entry.getKey(); + DeletionVector deletionVector = entry.getValue().build() + .orElseThrow(() -> new VerifyException("Empty aggregated deletion vector")); + FileDeletion detail = deletionDetails.get(originalPath); + verify(detail != null, "Missing deletion detail for file: %s", originalPath); + PartitionSpec partitionSpec = partitionsSpecs.get(detail.partitionSpecId()); + verify(partitionSpec != null, "Unknown partition spec ID %s for file: %s", detail.partitionSpecId(), originalPath); + Optional partitionData = createPartitionData(partitionSpec, detail.partitionDataJson()); + List preExistingDeletes = preExistingDeletesByDataFile.getOrDefault(originalPath, List.of()); + inputs.add(new RewriteInput( + originalPath, + deletionVector, + partitionSpec, + partitionData, + detail.partitionDataJson(), + detail.fileFormat(), + detail.fileSizeInBytes(), + detail.fileRecordCount(), + detail.dataSequenceNumber(), + detail.fileFirstRowId(), + preExistingDeletes)); + } + return inputs; + } + + private List executeParallelRewrites( + List inputs, Map tableProperties) + { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + Optional nameMapping = Optional.ofNullable(tableProperties.get(TableProperties.DEFAULT_NAME_MAPPING)) + .map(NameMappingParser::fromJson); + IcebergFileFormat writeFileFormat = getWriteFileFormat(tableProperties); + + List> futures = new ArrayList<>(inputs.size()); + for (RewriteInput input : inputs) { + futures.add(CompletableFuture.supplyAsync(() -> + copyOnWriteFileRewriter.rewriteFile( + session, input.originalPath(), input.sourceFileSizeInBytes(), + input.deletionVector(), + rewriteSchema, input.partitionSpec(), input.partitionData(), + input.preExistingDeletes(), + input.sourceDataSequenceNumber(), + input.sourceFileFirstRowId(), + input.sourceFileFormat(), writeFileFormat, + metricsConfig, fileIoProperties, tableProperties, locationProvider, nameMapping), + copyOnWriteRewriteExecutor)); + } + + try { + CompletableFuture.allOf(futures.toArray(CompletableFuture[]::new)).join(); + List results = new ArrayList<>(futures.size()); + for (CompletableFuture future : futures) { + results.add(future.join()); + } + return results; + } + catch (CompletionException e) { + futures.forEach(f -> f.cancel(true)); + Throwable cause = e.getCause(); + for (CompletableFuture future : futures) { + if (future.isDone() && !future.isCompletedExceptionally() && !future.isCancelled()) { + try { + future.join().rollbackAction().close(); + } + catch (Exception suppressed) { + cause.addSuppressed(suppressed); + } + } + } + throw cause instanceof RuntimeException re ? re : new RuntimeException(cause); + } + } + + private void collectRewriteFragments( + List inputs, + List results, + IcebergFileFormat writeFileFormat, + List fragments, + long startNanos) + { + List rollbackActions = new ArrayList<>(); + long totalBytesRead = 0; + long totalBytesWritten = 0; + long totalRowsDeleted = 0; + int filesRemoved = 0; + try { + for (int i = 0; i < results.size(); i++) { + CopyOnWriteFileRewriter.RewriteResult result = results.get(i); + RewriteInput input = inputs.get(i); + rollbackActions.add(result.rollbackAction()); + + CopyOnWriteFileRewriter.RewriteMetrics rewriteMetrics = result.metrics(); + totalBytesRead += rewriteMetrics.originalFileSizeBytes(); + totalBytesWritten += rewriteMetrics.newFileSizeBytes(); + totalRowsDeleted += rewriteMetrics.deletedRowCount(); + + List danglingDeleteFiles = buildDanglingDeleteFiles( + input.originalPath(), input.partitionSpec(), input.partitionData(), input.preExistingDeletes()); + CommitTaskData.RewriteInfo rewriteInfo = new CommitTaskData.RewriteInfo( + input.originalPath(), + input.sourceFileSizeInBytes(), + input.sourceRecordCount(), + input.sourceFileFormat(), + danglingDeleteFiles); + + if (result.newFile().isPresent()) { + DataFile newFile = result.newFile().get(); + writtenBytes += newFile.fileSizeInBytes(); + CommitTaskData task = new CommitTaskData( + newFile.location(), + writeFileFormat, + newFile.fileSizeInBytes(), + new MetricsWrapper(result.newFileMetrics()), + PartitionSpecParser.toJson(input.partitionSpec()), + input.partitionData().map(PartitionData::toJson), + FileContent.DATA, + Optional.empty(), + Optional.ofNullable(newFile.splitOffsets()), + writeSortOrderId, + Optional.empty(), + Optional.of(rewriteInfo)); + fragments.add(wrappedBuffer(jsonCodec.toJsonBytes(task))); + } + else { + filesRemoved++; + CommitTaskData task = new CommitTaskData( + "", + input.sourceFileFormat(), + 0, + new MetricsWrapper(new Metrics(0L)), + PartitionSpecParser.toJson(input.partitionSpec()), + input.partitionData().map(PartitionData::toJson), + FileContent.DATA, + Optional.empty(), + Optional.empty(), + SortOrder.unsorted().orderId(), + Optional.empty(), + Optional.of(rewriteInfo)); + fragments.add(wrappedBuffer(jsonCodec.toJsonBytes(task))); + } + } + } + catch (Throwable t) { + for (Closeable rollbackAction : rollbackActions) { + try { + rollbackAction.close(); + } + catch (IOException ex) { + t.addSuppressed(new UncheckedIOException("Failed to rollback rewritten file", ex)); + } + } + throw t instanceof RuntimeException re ? re : new RuntimeException(t); + } + + long rewriteDurationMs = (System.nanoTime() - startNanos) / 1_000_000; + log.info("CoW rewrite on worker: %d files rewritten (%d removed), %d rows deleted, " + + "%d bytes read, %d bytes written, %d ms elapsed", + results.size(), filesRemoved, totalRowsDeleted, + totalBytesRead, totalBytesWritten, rewriteDurationMs); + } + + private CompletableFuture> finishMergeOnRead() { List fragments = new ArrayList<>(insertPageSink.finish().join()); writtenBytes = insertPageSink.getCompletedBytes(); @@ -237,16 +539,119 @@ private Optional createPartitionData(PartitionSpec partitionSpec, return Optional.of(PartitionData.fromJson(partitionDataAsJson, columnTypes)); } - private Page createInsertionsPageWithRowId(Page insertionsPage, Page inputPage) + private FrozenTableMetadata loadBaseTableMetadata() + { + String metadataLocation = baseTableMetadataLocation.orElseThrow(() -> new VerifyException("Missing base table metadata location for copy-on-write merge")); + FileIO fileIo = fileIoFactory.create(fileSystem, isUseFileSizeFromMetadata(session), fileIoProperties); + TableMetadata tableMetadata = TableMetadataParser.read(fileIo, metadataLocation); + return new FrozenTableMetadata(fileIo, tableMetadata); + } + + private static IcebergFileFormat getWriteFileFormat(Map tableProperties) + { + return IcebergFileFormat.fromIceberg(FileFormat.fromString(tableProperties.getOrDefault(TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT))); + } + + private static Schema getRewriteSchema(Schema schema, int formatVersion) + { + if (formatVersion < 3) { + return schema; + } + + List columns = new ArrayList<>(schema.columns()); + if (schema.findField(MetadataColumns.ROW_ID.fieldId()) == null) { + columns.add(MetadataColumns.ROW_ID); + } + if (schema.findField(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId()) == null) { + columns.add(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER); + } + return new Schema(columns); + } + + private static BaseTable createFrozenIcebergTable(FrozenTableMetadata frozenTableMetadata) + { + return new BaseTable(new StaticTableOperations(frozenTableMetadata.tableMetadata(), frozenTableMetadata.fileIo()), "copy-on-write"); + } + + private static Map> collectPreExistingDeletesByDataFile( + BaseTable icebergTable, + long snapshotId, + Set targetDataFiles) + { + Map> result = new HashMap<>(); + try (CloseableIterable fileScanTasks = icebergTable.newScan() + .useSnapshot(snapshotId) + .planFiles()) { + for (FileScanTask fileScanTask : fileScanTasks) { + String dataFilePath = fileScanTask.file().location(); + if (!targetDataFiles.contains(dataFilePath)) { + continue; + } + + List preExistingDeletes = fileScanTask.deletes().stream() + .filter(deleteFile -> deleteFile.content() == FileContent.POSITION_DELETES || deleteFile.content() == FileContent.EQUALITY_DELETES) + .map(DeleteFile::fromIceberg) + .toList(); + if (!preExistingDeletes.isEmpty()) { + result.put(dataFilePath, preExistingDeletes); + } + } + } + catch (IOException e) { + throw new UncheckedIOException("Failed to plan data files for pre-existing delete lookup", e); + } + return result; + } + + private static List buildDanglingDeleteFiles( + String originalPath, + PartitionSpec partitionSpec, + Optional partitionData, + List preExistingDeletes) { - Block[] blocks = new Block[columnCount + 1]; + if (preExistingDeletes.isEmpty()) { + return List.of(); + } + List danglingDeleteFiles = new ArrayList<>(); + for (DeleteFile deleteFile : preExistingDeletes) { + // Deletion vectors are always file-scoped (1:1 with a data file). When we rewrite + // the data file, its DV becomes dangling and must be removed from the commit. + // This matches Spark's danglingDVs() which only cleans up DVs. + if (deleteFile.isDeletionVector()) { + danglingDeleteFiles.add(new CommitTaskData.DanglingDeleteFile( + deleteFile.path(), + deleteFile.fileSizeInBytes(), + deleteFile.recordCount(), + PartitionSpecParser.toJson(partitionSpec), + partitionData.map(PartitionData::toJson), + deleteFile.contentOffset().isPresent() ? deleteFile.contentOffset().getAsLong() : null, + deleteFile.contentSizeInBytes().isPresent() ? (long) deleteFile.contentSizeInBytes().get() : null, + originalPath)); + } + } + return danglingDeleteFiles; + } + + private record FrozenTableMetadata(FileIO fileIo, TableMetadata tableMetadata) {} + + private Page createInsertionsPageWithRowLineage(Page insertionsPage, Page inputPage) + { + Block[] blocks = new Block[columnCount + 2]; for (int channel = 0; channel < columnCount; channel++) { blocks[channel] = insertionsPage.getBlock(channel); } blocks[columnCount] = createRowIdBlock(inputPage, columnCount, insertionsPage.getPositionCount()); + blocks[columnCount + 1] = createNullLongBlock(insertionsPage.getPositionCount()); return new Page(insertionsPage.getPositionCount(), blocks); } + private static Block createNullLongBlock(int positionCount) + { + boolean[] nulls = new boolean[positionCount]; + Arrays.fill(nulls, true); + return new LongArrayBlock(positionCount, Optional.of(nulls), new long[positionCount]); + } + private static Block createRowIdBlock(Page inputPage, int dataColumnCount, int additionCount) { // For V3, preserve source_row_id on UPDATE_INSERT rows when it is available. @@ -297,12 +702,29 @@ private static class FileDeletion { private final int partitionSpecId; private final String partitionDataJson; + private final IcebergFileFormat fileFormat; + private final long fileSizeInBytes; + private final long fileRecordCount; + private final long dataSequenceNumber; + private final OptionalLong fileFirstRowId; private final DeletionVector.Builder rowsToDelete = DeletionVector.builder(); - public FileDeletion(int partitionSpecId, String partitionDataJson) + public FileDeletion( + int partitionSpecId, + String partitionDataJson, + IcebergFileFormat fileFormat, + long fileSizeInBytes, + long fileRecordCount, + long dataSequenceNumber, + OptionalLong fileFirstRowId) { this.partitionSpecId = partitionSpecId; this.partitionDataJson = requireNonNull(partitionDataJson, "partitionDataJson is null"); + this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); + this.fileSizeInBytes = fileSizeInBytes; + this.fileRecordCount = fileRecordCount; + this.dataSequenceNumber = dataSequenceNumber; + this.fileFirstRowId = requireNonNull(fileFirstRowId, "fileFirstRowId is null"); } public int partitionSpecId() @@ -315,6 +737,31 @@ public String partitionDataJson() return partitionDataJson; } + public IcebergFileFormat fileFormat() + { + return fileFormat; + } + + public long fileSizeInBytes() + { + return fileSizeInBytes; + } + + public long fileRecordCount() + { + return fileRecordCount; + } + + public long dataSequenceNumber() + { + return dataSequenceNumber; + } + + public OptionalLong fileFirstRowId() + { + return fileFirstRowId; + } + public DeletionVector.Builder rowsToDelete() { return rowsToDelete; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeTableHandle.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeTableHandle.java index d99cbc166241..5b74abbce771 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeTableHandle.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeTableHandle.java @@ -16,22 +16,40 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.trino.spi.connector.ConnectorMergeTableHandle; +import org.apache.iceberg.RowLevelOperationMode; + +import java.util.Optional; import static java.util.Objects.requireNonNull; public class IcebergMergeTableHandle implements ConnectorMergeTableHandle { + public enum CopyOnWriteOperationType + { + DELETE, + UPDATE_OR_MERGE + } + private final IcebergTableHandle tableHandle; private final IcebergWritableTableHandle insertTableHandle; + private final RowLevelOperationMode rowLevelOperationMode; + private final Optional baseTableMetadataLocation; + private final CopyOnWriteOperationType copyOnWriteOperationType; @JsonCreator public IcebergMergeTableHandle( @JsonProperty("tableHandle") IcebergTableHandle tableHandle, - @JsonProperty("insertTableHandle") IcebergWritableTableHandle insertTableHandle) + @JsonProperty("insertTableHandle") IcebergWritableTableHandle insertTableHandle, + @JsonProperty("rowLevelOperationMode") RowLevelOperationMode rowLevelOperationMode, + @JsonProperty("baseTableMetadataLocation") Optional baseTableMetadataLocation, + @JsonProperty("copyOnWriteOperationType") CopyOnWriteOperationType copyOnWriteOperationType) { this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); this.insertTableHandle = requireNonNull(insertTableHandle, "insertTableHandle is null"); + this.rowLevelOperationMode = requireNonNull(rowLevelOperationMode, "rowLevelOperationMode is null"); + this.baseTableMetadataLocation = requireNonNull(baseTableMetadataLocation, "baseTableMetadataLocation is null"); + this.copyOnWriteOperationType = copyOnWriteOperationType == null ? CopyOnWriteOperationType.UPDATE_OR_MERGE : copyOnWriteOperationType; } @Override @@ -46,4 +64,22 @@ public IcebergWritableTableHandle getInsertTableHandle() { return insertTableHandle; } + + @JsonProperty + public RowLevelOperationMode getRowLevelOperationMode() + { + return rowLevelOperationMode; + } + + @JsonProperty + public Optional getBaseTableMetadataLocation() + { + return baseTableMetadataLocation; + } + + @JsonProperty + public CopyOnWriteOperationType getCopyOnWriteOperationType() + { + return copyOnWriteOperationType; + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index 942cae8669f3..08173bc19637 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -168,15 +168,18 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.IcebergManifestUtils; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestReader; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; @@ -184,6 +187,7 @@ import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.RewriteManifests; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Snapshot; @@ -212,6 +216,7 @@ import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.expressions.Term; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.metrics.CommitMetricsResult; import org.apache.iceberg.metrics.CommitReport; import org.apache.iceberg.metrics.MetricsReporters; @@ -227,6 +232,8 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.DataFileSet; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.ParallelIterable; import org.apache.iceberg.util.PartitionUtil; @@ -299,6 +306,11 @@ import static io.trino.plugin.iceberg.ExpressionConverter.isConvertibleToIcebergExpression; import static io.trino.plugin.iceberg.ExpressionConverter.toIcebergExpression; import static io.trino.plugin.iceberg.IcebergAnalyzeProperties.getColumnNames; +import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_DATA_SEQUENCE_NUMBER; +import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_FILE_FIRST_ROW_ID; +import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_FILE_FORMAT; +import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_FILE_RECORD_COUNT; +import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_FILE_SIZE; import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_PARTITION_DATA; import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_PARTITION_SPEC_ID; import static io.trino.plugin.iceberg.IcebergColumnHandle.TRINO_MERGE_ROW_ID; @@ -467,9 +479,13 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.DELETE_MODE; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.MERGE_MODE; import static org.apache.iceberg.TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED; import static org.apache.iceberg.TableProperties.METADATA_PREVIOUS_VERSIONS_MAX; import static org.apache.iceberg.TableProperties.MIN_SNAPSHOTS_TO_KEEP; @@ -477,6 +493,9 @@ import static org.apache.iceberg.TableProperties.OBJECT_STORE_ENABLED; import static org.apache.iceberg.TableProperties.ORC_BLOOM_FILTER_COLUMNS; import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.UPDATE_MODE; import static org.apache.iceberg.TableProperties.WRITE_DATA_LOCATION; import static org.apache.iceberg.TableProperties.WRITE_LOCATION_PROVIDER_IMPL; import static org.apache.iceberg.TableUtil.formatVersion; @@ -546,6 +565,8 @@ public class IcebergMetadata private Transaction transaction; private OptionalLong fromSnapshotForRefresh = OptionalLong.empty(); + private List pendingCowRollbackFilePaths = ImmutableList.of(); + private boolean canCleanupPendingCowRollbackFiles; public IcebergMetadata( TypeManager typeManager, @@ -818,7 +839,8 @@ private IcebergTableHandle tableHandleForSnapshot( false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); } private Optional getTablePartitioning(ConnectorSession session, Table icebergTable) @@ -3527,6 +3549,11 @@ public ColumnHandle getMergeRowIdColumnHandle(ConnectorSession session, Connecto .add(NestedField.required(TRINO_MERGE_PARTITION_SPEC_ID, "partition_spec_id", IntegerType.get())) .add(NestedField.required(TRINO_MERGE_PARTITION_DATA, "partition_data", StringType.get())) .add(NestedField.optional(TRINO_MERGE_SOURCE_ROW_ID, "source_row_id", LongType.get())) + .add(NestedField.required(TRINO_MERGE_FILE_FORMAT, "file_format", IntegerType.get())) + .add(NestedField.required(TRINO_MERGE_FILE_SIZE, "file_size", LongType.get())) + .add(NestedField.required(TRINO_MERGE_FILE_RECORD_COUNT, "file_record_count", LongType.get())) + .add(NestedField.required(TRINO_MERGE_DATA_SEQUENCE_NUMBER, "data_sequence_number", LongType.get())) + .add(NestedField.optional(TRINO_MERGE_FILE_FIRST_ROW_ID, "file_first_row_id", LongType.get())) .build()); NestedField field = NestedField.required(TRINO_MERGE_ROW_ID, TRINO_MERGE_ROW_ID_NAME, type); @@ -3536,10 +3563,29 @@ public ColumnHandle getMergeRowIdColumnHandle(ConnectorSession session, Connecto @Override public Optional getUpdateLayout(ConnectorSession session, ConnectorTableHandle tableHandle) { - return getInsertLayout(session, tableHandle) + Optional partitionedLayout = getInsertLayout(session, tableHandle) .flatMap(ConnectorTableLayout::getPartitioning) .map(IcebergPartitioningHandle.class::cast) .map(IcebergPartitioningHandle::forUpdate); + if (partitionedLayout.isPresent()) { + return partitionedLayout; + } + + // For unpartitioned tables in copy-on-write mode, provide an update layout that + // hashes DELETE/UPDATE rows by file path (via IcebergUpdateBucketFunction). + // Without this, the planner uses round-robin distribution which scatters deletions + // for the same file across multiple merge-writer tasks, causing duplicate rewrites. + // Read CoW properties from the table handle (populated at handle creation from table.properties()) + // to avoid an extra catalog.loadTable() call on every DML operation. + IcebergTableHandle table = (IcebergTableHandle) tableHandle; + Map properties = table.getStorageProperties(); + boolean anyCoW = RowLevelOperationMode.COPY_ON_WRITE.modeName().equals(properties.get(DELETE_MODE)) + || RowLevelOperationMode.COPY_ON_WRITE.modeName().equals(properties.get(UPDATE_MODE)) + || RowLevelOperationMode.COPY_ON_WRITE.modeName().equals(properties.get(MERGE_MODE)); + if (anyCoW) { + return Optional.of(new IcebergPartitioningHandle(true, ImmutableList.of(), ImmutableList.of())); + } + return Optional.empty(); } @Override @@ -3553,8 +3599,22 @@ public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorT beginTransaction(icebergTable); + RowLevelOperationMode operationMode = resolveRowLevelOperationMode(icebergTable, updateCaseColumns); + IcebergMergeTableHandle.CopyOnWriteOperationType copyOnWriteOperationType = updateCaseColumns.isEmpty() + ? IcebergMergeTableHandle.CopyOnWriteOperationType.DELETE + : IcebergMergeTableHandle.CopyOnWriteOperationType.UPDATE_OR_MERGE; + + IcebergTableHandle scanHandle = operationMode == RowLevelOperationMode.COPY_ON_WRITE + ? table.forCopyOnWriteScan() + : table; + Optional baseTableMetadataLocation = Optional.empty(); + if (operationMode == RowLevelOperationMode.COPY_ON_WRITE) { + verify(icebergTable instanceof HasTableOperations, "Unexpected table implementation: %s", icebergTable.getClass().getName()); + baseTableMetadataLocation = Optional.of(((HasTableOperations) icebergTable).operations().current().metadataFileLocation()); + } + IcebergWritableTableHandle insertHandle = newWritableTableHandle(table.getSchemaTableName(), icebergTable); - return new IcebergMergeTableHandle(table, insertHandle); + return new IcebergMergeTableHandle(scanHandle, insertHandle, operationMode, baseTableMetadataLocation, copyOnWriteOperationType); } @Override @@ -3562,7 +3622,40 @@ public void finishMerge(ConnectorSession session, ConnectorMergeTableHandle merg { IcebergMergeTableHandle mergeHandle = (IcebergMergeTableHandle) mergeTableHandle; IcebergTableHandle handle = mergeHandle.getTableHandle(); - finishWrite(session, handle, fragments); + if (mergeHandle.getRowLevelOperationMode() == RowLevelOperationMode.COPY_ON_WRITE) { + finishCopyOnWrite(session, handle, fragments, mergeHandle.getCopyOnWriteOperationType()); + } + else { + finishWrite(session, handle, fragments); + } + } + + private static void removeDanglingDeleteFiles(Schema schema, List rewriteTasks, OverwriteFiles overwriteFiles) + { + DeleteFileSet danglingDeletes = DeleteFileSet.create(); + for (CommitTaskData task : rewriteTasks) { + CommitTaskData.RewriteInfo rewriteInfo = task.rewriteInfo().orElseThrow(); + for (CommitTaskData.DanglingDeleteFile danglingFile : rewriteInfo.danglingDeleteFiles()) { + PartitionSpec partitionSpec = PartitionSpecParser.fromJson(schema, danglingFile.partitionSpecJson()); + Optional partitionData = toPartitionData(partitionSpec, schema, danglingFile.partitionDataJson()); + FileMetadata.Builder deleteBuilder = FileMetadata.deleteFileBuilder(partitionSpec) + .withPath(danglingFile.path()) + .withFormat(FileFormat.fromFileName(danglingFile.path())) + .ofPositionDeletes() + .withFileSizeInBytes(danglingFile.fileSizeInBytes()) + .withRecordCount(danglingFile.recordCount()) + .withReferencedDataFile(danglingFile.referencedDataFile()); + if (danglingFile.contentOffset() != null && danglingFile.contentSizeInBytes() != null) { + deleteBuilder.withContentOffset(danglingFile.contentOffset()); + deleteBuilder.withContentSizeInBytes(danglingFile.contentSizeInBytes()); + } + partitionData.ifPresent(deleteBuilder::withPartition); + danglingDeletes.add(deleteBuilder.build()); + } + } + if (!danglingDeletes.isEmpty()) { + overwriteFiles.deleteFiles(DataFileSet.create(), danglingDeletes); + } } private static void verifyTableVersionForUpdate(IcebergTableHandle table) @@ -3580,6 +3673,211 @@ private static void validateNotModifyingOldSnapshot(IcebergTableHandle table, Ta } } + private static RowLevelOperationMode resolveRowLevelOperationMode(Table icebergTable, Map> updateCaseColumns) + { + Map properties = icebergTable.properties(); + if (updateCaseColumns.isEmpty()) { + // DELETE operation + return RowLevelOperationMode.fromName( + properties.getOrDefault(DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName())); + } + // Trino routes both SQL UPDATE and SQL MERGE through beginMerge, so we cannot + // distinguish them from updateCaseColumns alone. Use CoW if either write.update.mode + // or write.merge.mode is set to copy-on-write. This is safe because CoW is always + // correct (just potentially slower than MoR). + if (RowLevelOperationMode.COPY_ON_WRITE.modeName().equals(properties.get(UPDATE_MODE)) + || RowLevelOperationMode.COPY_ON_WRITE.modeName().equals(properties.get(MERGE_MODE))) { + return RowLevelOperationMode.COPY_ON_WRITE; + } + return RowLevelOperationMode.MERGE_ON_READ; + } + + // Trino cannot reliably distinguish SQL UPDATE vs SQL MERGE in beginMerge(). + // For CoW update/merge writes, choose the stricter isolation level so conflict + // detection is never weaker than either table property intends. + static IsolationLevel resolveCopyOnWriteIsolationLevel( + Map tableProperties, + IcebergMergeTableHandle.CopyOnWriteOperationType operationType) + { + return switch (operationType) { + case DELETE -> IsolationLevel.fromName(tableProperties.getOrDefault(DELETE_ISOLATION_LEVEL, DELETE_ISOLATION_LEVEL_DEFAULT)); + case UPDATE_OR_MERGE -> strictestIsolationLevel( + IsolationLevel.fromName(tableProperties.getOrDefault(UPDATE_ISOLATION_LEVEL, UPDATE_ISOLATION_LEVEL_DEFAULT)), + IsolationLevel.fromName(tableProperties.getOrDefault(MERGE_ISOLATION_LEVEL, MERGE_ISOLATION_LEVEL_DEFAULT))); + }; + } + + private static IsolationLevel strictestIsolationLevel(IsolationLevel first, IsolationLevel second) + { + if (first == IsolationLevel.SERIALIZABLE || second == IsolationLevel.SERIALIZABLE) { + return IsolationLevel.SERIALIZABLE; + } + return IsolationLevel.SNAPSHOT; + } + + private void finishCopyOnWrite( + ConnectorSession session, + IcebergTableHandle table, + Collection fragments, + IcebergMergeTableHandle.CopyOnWriteOperationType copyOnWriteOperationType) + { + Table icebergTable = transaction.table(); + + List commitTasks = fragments.stream() + .map(Slice::getInput) + .map(commitTaskCodec::fromJson) + .collect(toImmutableList()); + + if (commitTasks.isEmpty()) { + transaction = null; + clearCopyOnWriteRollbackCleanupState(); + return; + } + + prepareCopyOnWriteRollbackCleanup(commitTasks); + try { + Schema schema = SchemaParser.fromJson(table.getTableSchemaJson()); + + // Rewrite fragments carry RewriteInfo from worker-side CopyOnWriteFileRewriter + List rewriteTasks = new ArrayList<>(); + List insertTasks = new ArrayList<>(); + + for (CommitTaskData task : commitTasks) { + if (task.rewriteInfo().isPresent()) { + rewriteTasks.add(task); + } + else { + insertTasks.add(task); + } + } + + OptionalLong baseSnapshotId = table.getSnapshotId(); + Map sortOrders = icebergTable.sortOrders(); + + if (!rewriteTasks.isEmpty()) { + OverwriteFiles overwriteFiles = transaction.newOverwrite(); + if (baseSnapshotId.isPresent()) { + overwriteFiles.validateFromSnapshot(baseSnapshotId.getAsLong()); + } + + TupleDomain dataColumnPredicate = table.getEnforcedPredicate().filter((column, domain) -> !isMetadataColumnId(column.getId())); + TupleDomain effectivePredicate = dataColumnPredicate.intersect(table.getUnenforcedPredicate()); + if (isFileBasedConflictDetectionEnabled(session)) { + effectivePredicate = effectivePredicate.intersect(extractTupleDomainsFromCommitTasks(table, icebergTable, commitTasks, typeManager)); + } + effectivePredicate = effectivePredicate.filter((_, domain) -> isConvertibleToIcebergExpression(domain)); + if (!effectivePredicate.isAll()) { + overwriteFiles.conflictDetectionFilter(toIcebergExpression(effectivePredicate)); + } + IsolationLevel isolationLevel = resolveCopyOnWriteIsolationLevel(icebergTable.properties(), copyOnWriteOperationType); + if (isolationLevel == IsolationLevel.SERIALIZABLE) { + overwriteFiles.validateNoConflictingData(); + } + overwriteFiles.validateNoConflictingDeletes(); + overwriteFiles.scanManifestsWith(icebergScanExecutor); + + for (CommitTaskData task : rewriteTasks) { + CommitTaskData.RewriteInfo rewriteInfo = task.rewriteInfo().orElseThrow(); + PartitionSpec partitionSpec = PartitionSpecParser.fromJson(schema, task.partitionSpecJson()); + Optional partitionData = toPartitionData(partitionSpec, schema, task.partitionDataJson()); + + DataFiles.Builder oldBuilder = DataFiles.builder(partitionSpec) + .withPath(rewriteInfo.oldFilePath()) + .withFormat(rewriteInfo.oldFileFormat().toIceberg()) + .withFileSizeInBytes(rewriteInfo.oldFileSizeInBytes()) + .withRecordCount(rewriteInfo.oldRecordCount()); + partitionData.ifPresent(oldBuilder::withPartition); + overwriteFiles.deleteFile(oldBuilder.build()); + + // Empty path signals all rows deleted + if (!task.path().isEmpty()) { + DataFiles.Builder newBuilder = DataFiles.builder(partitionSpec) + .withPath(task.path()) + .withFormat(task.fileFormat().toIceberg()) + .withFileSizeInBytes(task.fileSizeInBytes()) + .withMetrics(task.metrics().metrics()) + .withSortOrder(sortOrders.get(task.sortOrderId())); + task.fileSplitOffsets().ifPresent(newBuilder::withSplitOffsets); + partitionData.ifPresent(newBuilder::withPartition); + overwriteFiles.addFile(newBuilder.build()); + } + } + + for (CommitTaskData task : insertTasks) { + overwriteFiles.addFile(buildInsertDataFile(schema, sortOrders, task)); + } + + removeDanglingDeleteFiles(schema, rewriteTasks, overwriteFiles); + + commitUpdate(overwriteFiles, session, "copy-on-write"); + } + + if (!insertTasks.isEmpty() && rewriteTasks.isEmpty()) { + OverwriteFiles overwriteFiles = transaction.newOverwrite(); + if (baseSnapshotId.isPresent()) { + overwriteFiles.validateFromSnapshot(baseSnapshotId.getAsLong()); + } + IsolationLevel isolationLevel = resolveCopyOnWriteIsolationLevel(icebergTable.properties(), copyOnWriteOperationType); + if (isolationLevel == IsolationLevel.SERIALIZABLE) { + overwriteFiles.validateNoConflictingData(); + } + overwriteFiles.validateNoConflictingDeletes(); + overwriteFiles.scanManifestsWith(icebergScanExecutor); + for (CommitTaskData task : insertTasks) { + overwriteFiles.addFile(buildInsertDataFile(schema, sortOrders, task)); + } + commitUpdate(overwriteFiles, session, "copy-on-write-insert"); + } + + long totalOldBytes = rewriteTasks.stream() + .mapToLong(task -> task.rewriteInfo().orElseThrow().oldFileSizeInBytes()) + .sum(); + long totalNewBytes = rewriteTasks.stream() + .filter(task -> !task.path().isEmpty()) + .mapToLong(CommitTaskData::fileSizeInBytes) + .sum(); + long totalInsertBytes = insertTasks.stream() + .mapToLong(CommitTaskData::fileSizeInBytes) + .sum(); + log.info("CoW commit: %d files overwritten (%d bytes -> %d bytes), %d new files inserted (%d bytes), table=%s", + rewriteTasks.size(), totalOldBytes, totalNewBytes, + insertTasks.size(), totalInsertBytes, table.getTableName()); + + commitTransaction(transaction, "copy-on-write"); + clearCopyOnWriteRollbackCleanupState(); + } + catch (RuntimeException e) { + if (hasCommitStateUnknownException(e)) { + canCleanupPendingCowRollbackFiles = false; + } + throw e; + } + } + + private static DataFile buildInsertDataFile(Schema schema, Map sortOrders, CommitTaskData task) + { + PartitionSpec partitionSpec = PartitionSpecParser.fromJson(schema, task.partitionSpecJson()); + Type[] partitionColumnTypes = partitionSpec.fields().stream() + .map(field -> field.transform().getResultType(schema.findType(field.sourceId()))) + .toArray(Type[]::new); + + DataFiles.Builder builder = DataFiles.builder(partitionSpec) + .withPath(task.path()) + .withFormat(task.fileFormat().toIceberg()) + .withFileSizeInBytes(task.fileSizeInBytes()) + .withMetrics(task.metrics().metrics()) + .withSortOrder(sortOrders.get(task.sortOrderId())); + task.fileSplitOffsets().ifPresent(builder::withSplitOffsets); + + if (!partitionSpec.fields().isEmpty()) { + String partitionDataJson = task.partitionDataJson() + .orElseThrow(() -> new VerifyException("No partition data for partitioned table")); + builder.withPartition(PartitionData.fromJson(partitionDataJson, partitionColumnTypes)); + } + + return builder.build(); + } + private void finishWrite(ConnectorSession session, IcebergTableHandle table, Collection fragments) { Table icebergTable = transaction.table(); @@ -3847,7 +4145,61 @@ public void truncateTable(ConnectorSession session, ConnectorTableHandle tableHa public void rollback() { - // TODO: cleanup open transaction + if (transaction == null) { + clearCopyOnWriteRollbackCleanupState(); + return; + } + cleanupPendingCopyOnWriteFiles(transaction.table()); + clearCopyOnWriteRollbackCleanupState(); + } + + private void prepareCopyOnWriteRollbackCleanup(List commitTasks) + { + pendingCowRollbackFilePaths = commitTasks.stream() + .filter(task -> task.content() == FileContent.DATA) + .map(CommitTaskData::path) + .filter(path -> !path.isEmpty()) + .distinct() + .collect(toImmutableList()); + canCleanupPendingCowRollbackFiles = true; + } + + private void cleanupPendingCopyOnWriteFiles(Table icebergTable) + { + if (pendingCowRollbackFilePaths.isEmpty()) { + return; + } + if (!canCleanupPendingCowRollbackFiles) { + log.warn("Skipping CoW rollback cleanup because commit state is unknown; orphan files may remain until remove_orphan_files runs"); + return; + } + + FileIO fileIo = icebergTable.io(); + for (String path : pendingCowRollbackFilePaths) { + try { + fileIo.deleteFile(path); + } + catch (RuntimeException e) { + // Cleanup is best-effort. Never mask the query failure during rollback. + log.warn(e, "Failed to clean up CoW rollback file: %s", path); + } + } + } + + private void clearCopyOnWriteRollbackCleanupState() + { + pendingCowRollbackFilePaths = ImmutableList.of(); + canCleanupPendingCowRollbackFiles = false; + } + + private static boolean hasCommitStateUnknownException(Throwable throwable) + { + for (Throwable current = throwable; current != null; current = current.getCause()) { + if (current instanceof CommitStateUnknownException) { + return true; + } + } + return false; } @Override @@ -3882,7 +4234,8 @@ public Optional> applyLimit(Connect table.isRecordScannedFiles(), table.getMaxScannedFileSize(), table.getConstraintColumns(), - table.getForAnalyze()); + table.getForAnalyze(), + table.isCopyOnWriteScan()); return Optional.of(new LimitApplicationResult<>(table, false, false)); } @@ -3979,7 +4332,8 @@ else if (isMetadataColumnId(columnHandle.getId())) { table.isRecordScannedFiles(), table.getMaxScannedFileSize(), newConstraintColumns, - table.getForAnalyze()), + table.getForAnalyze(), + table.isCopyOnWriteScan()), remainingConstraint.transformKeys(ColumnHandle.class::cast), extractionResult.remainingExpression(), false)); @@ -4193,7 +4547,8 @@ public TableStatistics getTableStatistics(ConnectorSession session, ConnectorTab false, // recordScannedFiles does not affect stats originalHandle.getMaxScannedFileSize(), ImmutableSet.of(), // constraintColumns do not affect stats - Optional.empty()); // forAnalyze does not affect stats + Optional.empty(), // forAnalyze does not affect stats + false); // copyOnWriteScan does not affect stats return getIncrementally( tableStatisticsCache, cacheKey, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java index b39e3c62cd9c..2e358196de4f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java @@ -115,6 +115,8 @@ public void configure(Binder binder) binder.bind(IcebergFileWriterFactory.class).in(Scopes.SINGLETON); newExporter(binder).export(IcebergFileWriterFactory.class).withGeneratedName(); + binder.bind(CopyOnWriteFileRewriter.class).in(Scopes.SINGLETON); + binder.bind(IcebergEnvironmentContext.class).asEagerSingleton(); Multibinder procedures = newSetBinder(binder, Procedure.class); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java index fcafc808ec00..6d2fa015a8a7 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java @@ -17,6 +17,7 @@ import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; import io.trino.plugin.hive.SortingFileWriterConfig; +import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.procedure.IcebergOptimizeHandle; import io.trino.plugin.iceberg.procedure.IcebergTableExecuteHandle; import io.trino.spi.PageIndexerFactory; @@ -45,6 +46,8 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.ExecutorService; import static com.google.common.collect.Maps.transformValues; import static io.trino.plugin.iceberg.IcebergSessionProperties.maxPartitionsPerWriter; @@ -64,6 +67,9 @@ public class IcebergPageSinkProvider private final Optional sortingFileWriterLocalStagingPath; private final TypeManager typeManager; private final PageSorter pageSorter; + private final CopyOnWriteFileRewriter copyOnWriteFileRewriter; + private final ForwardingFileIoFactory fileIoFactory; + private final ExecutorService copyOnWriteRewriteExecutor; @Inject public IcebergPageSinkProvider( @@ -74,7 +80,10 @@ public IcebergPageSinkProvider( SortingFileWriterConfig sortingFileWriterConfig, IcebergConfig icebergConfig, TypeManager typeManager, - PageSorter pageSorter) + PageSorter pageSorter, + CopyOnWriteFileRewriter copyOnWriteFileRewriter, + ForwardingFileIoFactory fileIoFactory, + @ForIcebergCopyOnWriteRewrite ExecutorService copyOnWriteRewriteExecutor) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null"); @@ -85,6 +94,9 @@ public IcebergPageSinkProvider( this.sortingFileWriterLocalStagingPath = icebergConfig.getSortedWritingLocalStagingPath(); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.pageSorter = requireNonNull(pageSorter, "pageSorter is null"); + this.copyOnWriteFileRewriter = requireNonNull(copyOnWriteFileRewriter, "copyOnWriteFileRewriter is null"); + this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); + this.copyOnWriteRewriteExecutor = requireNonNull(copyOnWriteRewriteExecutor, "copyOnWriteRewriteExecutor is null"); } @Override @@ -191,9 +203,9 @@ public ConnectorMergeSink createMergeSink(ConnectorTransactionHandle transaction Schema outputSchema; if (formatVersion >= 3) { - // Persist row IDs for updated rows; $last_updated_sequence_number is synthesized from file sequence number. List columns = new ArrayList<>(schema.columns()); columns.add(MetadataColumns.ROW_ID); + columns.add(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER); outputSchema = new Schema(columns); } else { @@ -201,18 +213,28 @@ public ConnectorMergeSink createMergeSink(ConnectorTransactionHandle transaction } ConnectorPageSink pageSink = createPageSink(session, tableHandle, outputSchema, fileIoProperties); + OptionalLong snapshotId = merge.getTableHandle().getSnapshotId(); + return new IcebergMergeSink( formatVersion, locationProvider, fileWriterFactory, fileSystemFactory.create(session.getIdentity(), fileIoProperties), + fileIoFactory, + copyOnWriteRewriteExecutor, jsonCodec, session, tableHandle.fileFormat(), + fileIoProperties, tableHandle.storageProperties(), schema, partitionsSpecs, pageSink, - schema.columns().size()); + schema.columns().size(), + tableHandle.sortOrderId(), + merge.getRowLevelOperationMode(), + copyOnWriteFileRewriter, + merge.getBaseTableMetadataLocation(), + snapshotId); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index a82f5b0dfe12..bb8046005a5f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -367,6 +367,7 @@ public ConnectorPageSource createPageSource( start, length, fileSize, + fileRecordCount, partitionSpec.specId(), PartitionData.toJson(partitionData), fileFormat, @@ -534,6 +535,7 @@ public ConnectorPageSource openDeleteFile( delete.fileSizeInBytes(), delete.fileSizeInBytes(), 0, + 0, "", IcebergFileFormat.fromIceberg(delete.format()), schemaFromHandles(columns), @@ -553,6 +555,7 @@ private ReaderPageSourceWithRowPositions createDataPageSource( long start, long length, long fileSize, + long fileRecordCount, int partitionSpecId, String partitionData, IcebergFileFormat fileFormat, @@ -570,8 +573,11 @@ private ReaderPageSourceWithRowPositions createDataPageSource( inputFile, start, length, + fileSize, + fileRecordCount, partitionSpecId, partitionData, + fileFormat, fileSchema, dataColumns, predicate, @@ -596,8 +602,10 @@ private ReaderPageSourceWithRowPositions createDataPageSource( start, length, fileSize, + fileRecordCount, partitionSpecId, partitionData, + fileFormat, fileSchema, dataColumns, ParquetReaderOptions.builder(parquetReaderOptions) @@ -621,8 +629,11 @@ private ReaderPageSourceWithRowPositions createDataPageSource( inputFile, start, length, + fileSize, + fileRecordCount, partitionSpecId, partitionData, + fileFormat, fileSchema, nameMapping, partition, @@ -672,8 +683,11 @@ private static ReaderPageSourceWithRowPositions createOrcPageSource( TrinoInputFile inputFile, long start, long length, + long fileSize, + long fileRecordCount, int partitionSpecId, String partitionData, + IcebergFileFormat fileFormat, Schema tableSchema, List columns, TupleDomain effectivePredicate, @@ -762,7 +776,11 @@ else if (column.isMergeRowIdColumn()) { partitionSpecId, utf8Slice(partitionData), fileFirstRowId, - sourceRowIdOrdinal)); + sourceRowIdOrdinal, + fileFormat.ordinal(), + fileSize, + fileRecordCount, + dataSequenceNumber)); } else if (column.isRowPositionColumn()) { appendRowNumberColumn = true; @@ -995,8 +1013,10 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( long start, long length, long fileSize, + long fileRecordCount, int partitionSpecId, String partitionData, + IcebergFileFormat fileFormat, Schema tableSchema, List columns, ParquetReaderOptions options, @@ -1086,7 +1106,11 @@ else if (column.isMergeRowIdColumn()) { partitionSpecId, utf8Slice(partitionData), fileFirstRowId, - sourceRowIdOrdinal)); + sourceRowIdOrdinal, + fileFormat.ordinal(), + fileSize, + fileRecordCount, + dataSequenceNumber)); } else if (column.isRowPositionColumn()) { appendRowNumberColumn = true; @@ -1252,8 +1276,11 @@ private static ReaderPageSourceWithRowPositions createAvroPageSource( TrinoInputFile inputFile, long start, long length, + long fileSize, + long fileRecordCount, int partitionSpecId, String partitionData, + IcebergFileFormat fileFormat, Schema fileSchema, Optional nameMapping, String partition, @@ -1322,7 +1349,11 @@ else if (column.isMergeRowIdColumn()) { partitionSpecId, utf8Slice(partitionData), fileFirstRowId, - sourceRowIdOrdinal)); + sourceRowIdOrdinal, + fileFormat.ordinal(), + fileSize, + fileRecordCount, + dataSequenceNumber)); } else if (column.isRowPositionColumn()) { appendRowNumberColumn = true; @@ -1723,17 +1754,43 @@ public int hashCode() } } - private record MergeRowIdTransform(VariableWidthBlock filePath, IntArrayBlock partitionSpecId, VariableWidthBlock partitionData, OptionalLong fileFirstRowId, Integer sourceRowIdChannel) + private record MergeRowIdTransform( + VariableWidthBlock filePath, + IntArrayBlock partitionSpecId, + VariableWidthBlock partitionData, + OptionalLong fileFirstRowId, + Integer sourceRowIdChannel, + IntArrayBlock fileFormatBlock, + LongArrayBlock fileSizeBlock, + LongArrayBlock fileRecordCountBlock, + LongArrayBlock dataSequenceNumberBlock, + Block fileFirstRowIdBlock) implements Function { - private static Function create(Slice filePath, int partitionSpecId, Slice partitionData, OptionalLong fileFirstRowId, Integer sourceRowIdChannel) + private static Function create( + Slice filePath, + int partitionSpecId, + Slice partitionData, + OptionalLong fileFirstRowId, + Integer sourceRowIdChannel, + int fileFormatOrdinal, + long fileSize, + long fileRecordCount, + long dataSequenceNumber) { return new MergeRowIdTransform( new VariableWidthBlock(1, filePath, new int[] {0, filePath.length()}, Optional.empty()), new IntArrayBlock(1, Optional.empty(), new int[] {partitionSpecId}), new VariableWidthBlock(1, partitionData, new int[] {0, partitionData.length()}, Optional.empty()), fileFirstRowId, - sourceRowIdChannel); + sourceRowIdChannel, + new IntArrayBlock(1, Optional.empty(), new int[] {fileFormatOrdinal}), + new LongArrayBlock(1, Optional.empty(), new long[] {fileSize}), + new LongArrayBlock(1, Optional.empty(), new long[] {fileRecordCount}), + new LongArrayBlock(1, Optional.empty(), new long[] {dataSequenceNumber}), + fileFirstRowId.isPresent() + ? new LongArrayBlock(1, Optional.empty(), new long[] {fileFirstRowId.getAsLong()}) + : BIGINT.createNullBlock()); } @Override @@ -1765,7 +1822,6 @@ else if (storedSourceRowId != null) { sourceRowIdBlock = storedSourceRowId; } else { - // No row IDs available (v2 table or file without row IDs assigned) sourceRowIdBlock = RunLengthEncodedBlock.create(BIGINT.createNullBlock(), positionCount); } @@ -1774,7 +1830,12 @@ else if (storedSourceRowId != null) { rowPosition, RunLengthEncodedBlock.create(partitionSpecId, positionCount), RunLengthEncodedBlock.create(partitionData, positionCount), - sourceRowIdBlock + sourceRowIdBlock, + RunLengthEncodedBlock.create(fileFormatBlock, positionCount), + RunLengthEncodedBlock.create(fileSizeBlock, positionCount), + RunLengthEncodedBlock.create(fileRecordCountBlock, positionCount), + RunLengthEncodedBlock.create(dataSequenceNumberBlock, positionCount), + RunLengthEncodedBlock.create(fileFirstRowIdBlock, positionCount), }; return RowBlock.fromFieldBlocks(positionCount, fields); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 9d81375a524e..42c1d65bf566 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -376,7 +376,13 @@ private synchronized Iterator prepareFileTasksIterator(L outputRowsLowerBound = saturatedAdd(outputRowsLowerBound, wholeFileTask.file().recordCount()); } - if (fileHasNoDeletions && noDataColumnsProjected(wholeFileTask)) { + if (tableHandle.isCopyOnWriteScan()) { + // In copy-on-write mode, each data file must go to exactly one worker + // so the worker can rewrite the file with the complete deletion vector. + // Never split files in this mode. + scanTaskBuilder.add(fileScanTaskWithDomain); + } + else if (fileHasNoDeletions && noDataColumnsProjected(wholeFileTask)) { scanTaskBuilder.add(fileScanTaskWithDomain); } else { @@ -782,8 +788,13 @@ private double getSplitWeight(FileScanTask task) { double dataWeight = (double) task.length() / tableScan.targetSplitSize(); double weight = dataWeight; + + if (tableHandle.isCopyOnWriteScan()) { + // CoW reads the entire file and writes a new file, roughly 2x cost + weight = dataWeight * 2; + } + if (task.deletes().stream().anyMatch(deleteFile -> deleteFile.content() == POSITION_DELETES)) { - // Presence of each data position is looked up in a combined bitmap of deleted positions weight += dataWeight; } @@ -791,7 +802,6 @@ private double getSplitWeight(FileScanTask task) .filter(deleteFile -> deleteFile.content() == EQUALITY_DELETES) .mapToLong(ContentFile::recordCount) .sum(); - // Every row is a separate equality predicate that must be applied to all data rows weight += equalityDeletes * dataWeight; return weight; } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableHandle.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableHandle.java index 813afd0aab26..deadc0a57b0c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableHandle.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableHandle.java @@ -77,6 +77,10 @@ public class IcebergTableHandle // ANALYZE only. Coordinator-only private final Optional forAnalyze; + // Copy-on-write DML scan. Coordinator-only. + // When true, the split source must not split files so each data file goes to exactly one worker. + private final boolean copyOnWriteScan; + @JsonCreator @DoNotCall // For JSON deserialization only public static IcebergTableHandle fromJsonForDeserializationOnly( @@ -116,7 +120,8 @@ public static IcebergTableHandle fromJsonForDeserializationOnly( false, Optional.empty(), ImmutableSet.of(), - Optional.empty()); + Optional.empty(), + false); } public IcebergTableHandle( @@ -139,7 +144,8 @@ public IcebergTableHandle( boolean recordScannedFiles, Optional maxScannedFileSize, Set constraintColumns, - Optional forAnalyze) + Optional forAnalyze, + boolean copyOnWriteScan) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -165,6 +171,7 @@ public IcebergTableHandle( this.maxScannedFileSize = requireNonNull(maxScannedFileSize, "maxScannedFileSize is null"); this.constraintColumns = ImmutableSet.copyOf(requireNonNull(constraintColumns, "constraintColumns is null")); this.forAnalyze = requireNonNull(forAnalyze, "forAnalyze is null"); + this.copyOnWriteScan = copyOnWriteScan; } @JsonProperty @@ -291,6 +298,12 @@ public Optional getForAnalyze() return forAnalyze; } + @JsonIgnore + public boolean isCopyOnWriteScan() + { + return copyOnWriteScan; + } + public SchemaTableName getSchemaTableName() { return new SchemaTableName(schemaName, tableName); @@ -323,7 +336,34 @@ public IcebergTableHandle withProjectedColumns(Set projecte recordScannedFiles, maxScannedFileSize, constraintColumns, - forAnalyze); + forAnalyze, + copyOnWriteScan); + } + + public IcebergTableHandle forCopyOnWriteScan() + { + return new IcebergTableHandle( + schemaName, + tableName, + tableType, + snapshotId, + tableSchemaJson, + specId, + partitionSpecJsons, + formatVersion, + unenforcedPredicate, + enforcedPredicate, + limit, + projectedColumns, + nameMappingJson, + tableLocation, + storageProperties, + tablePartitioning, + recordScannedFiles, + maxScannedFileSize, + constraintColumns, + forAnalyze, + true); } public IcebergTableHandle forAnalyze() @@ -348,7 +388,8 @@ public IcebergTableHandle forAnalyze() recordScannedFiles, maxScannedFileSize, constraintColumns, - Optional.of(true)); + Optional.of(true), + copyOnWriteScan); } public IcebergTableHandle forOptimize(boolean recordScannedFiles, DataSize maxScannedFileSize) @@ -373,7 +414,8 @@ public IcebergTableHandle forOptimize(boolean recordScannedFiles, DataSize maxSc recordScannedFiles, Optional.of(maxScannedFileSize), constraintColumns, - forAnalyze); + forAnalyze, + copyOnWriteScan); } public IcebergTableHandle withTablePartitioning(Optional requiredTablePartitioning) @@ -398,7 +440,8 @@ public IcebergTableHandle withTablePartitioning(Optional deletionVectorBuilders) + { + ExistingDeletes existingDeletes = getExistingDeletesByMetadataOnly(icebergTable, snapshotId, deletionVectorBuilders.keySet()); + + if (existingDeletes.deletionVectors().isEmpty() + && existingDeletes.fileScopedDeletes().isEmpty() + && existingDeletes.partitionScopedDeletes().isEmpty()) { + return; + } + + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), icebergTable.io().properties()); + + mergePreExistingDeletes(session, fileSystem, existingDeletes, deletionVectorBuilders); + } + + @Override + public void mergePreExistingDeletes( + ConnectorSession session, + TrinoFileSystem fileSystem, + FileIO fileIo, + Map specsById, + Snapshot snapshot, + Map deletionVectorBuilders) + { + ExistingDeletes existingDeletes = getExistingDeletesByMetadataOnly(fileIo, specsById, snapshot, deletionVectorBuilders.keySet()); + + if (existingDeletes.deletionVectors().isEmpty() + && existingDeletes.fileScopedDeletes().isEmpty() + && existingDeletes.partitionScopedDeletes().isEmpty()) { + return; + } + + mergePreExistingDeletes(session, fileSystem, existingDeletes, deletionVectorBuilders); + } + + private void mergePreExistingDeletes( + ConnectorSession session, + TrinoFileSystem fileSystem, + ExistingDeletes existingDeletes, + Map deletionVectorBuilders) + { + // merge existing DVs + existingDeletes.deletionVectors().forEach((dataFilePath, deleteFile) -> { + try (TrinoInput input = fileSystem.newInputFile(Location.of(deleteFile.location()), deleteFile.fileSizeInBytes()).newInput()) { + Slice data = input.readFully(deleteFile.contentOffset(), toIntExact(deleteFile.contentSizeInBytes())); + deletionVectorBuilders.get(dataFilePath).deserialize(data); + } + catch (IOException e) { + throw new TrinoException(ICEBERG_BAD_DATA, "Failed to read existing deletion vector: " + deleteFile.location(), e); + } + }); + + // merge file-scoped position deletes + if (!existingDeletes.fileScopedDeletes().isEmpty()) { + existingDeletes.fileScopedDeletes().forEach((dataFilePath, deleteFile) -> { + try (ConnectorPageSource source = openDeleteFilePageSource(session, deleteFile, fileSystem)) { + PositionDeleteReader.readSingleFilePositionDeletes(source, deletionVectorBuilders.get(dataFilePath)::add); + } + catch (IOException e) { + throw new TrinoException(ICEBERG_BAD_DATA, "Failed to read position delete file: " + deleteFile.location(), e); + } + }); + } + + // merge partition-scoped position deletes + for (DeleteFile deleteFile : existingDeletes.partitionScopedDeletes()) { + try (ConnectorPageSource source = openDeleteFilePageSource(session, deleteFile, fileSystem)) { + PositionDeleteReader.readMultiFilePositionDeletes(source, (dataFilePath, position) -> { + DeletionVector.Builder builder = deletionVectorBuilders.get(dataFilePath); + if (builder != null) { + builder.add(position); + } + }); + } + catch (IOException e) { + throw new TrinoException(ICEBERG_BAD_DATA, "Failed to read position delete file: " + deleteFile.location(), e); + } + } + } + private static ExistingDeletes getExistingDeletesByMetadataOnly(Table table, long snapshotId, Set dataFilePaths) + { + return getExistingDeletesByMetadataOnly(table.io(), table.specs(), table.snapshot(snapshotId), dataFilePaths); + } + + private static ExistingDeletes getExistingDeletesByMetadataOnly(FileIO io, Map specsById, Snapshot snapshot, Set dataFilePaths) { Map deletionVectors = new HashMap<>(); Multimap fileScopedDeletes = ArrayListMultimap.create(); List partitionScopedDeletes = new ArrayList<>(); - FileIO io = table.io(); - Map specsById = table.specs(); - for (ManifestFile manifest : table.snapshot(snapshotId).deleteManifests(io)) { + for (ManifestFile manifest : snapshot.deleteManifests(io)) { try (ManifestReader reader = ManifestFiles.readDeleteManifest(manifest, io, specsById)) { for (DeleteFile deleteFile : reader) { if (deleteFile.content() != FileContent.POSITION_DELETES) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeletionVectorWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeletionVectorWriter.java index 165cdf417bd5..3ede5f26024b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeletionVectorWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeletionVectorWriter.java @@ -14,22 +14,43 @@ package io.trino.plugin.iceberg.delete; import io.airlift.slice.Slice; +import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.iceberg.IcebergTableHandle; import io.trino.plugin.iceberg.PartitionData; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileIO; import java.util.List; +import java.util.Map; import java.util.Optional; import static java.util.Objects.requireNonNull; public interface DeletionVectorWriter { - DeletionVectorWriter UNSUPPORTED_DELETION_VECTOR_WRITER = (session, icebergTable, table, deletionVectorInfos, rowDelta) -> { - throw new UnsupportedOperationException("Deletion Vectors are not supported"); + DeletionVectorWriter UNSUPPORTED_DELETION_VECTOR_WRITER = new DeletionVectorWriter() + { + @Override + public void writeDeletionVectors(ConnectorSession session, Table icebergTable, IcebergTableHandle table, List deletionVectorInfos, RowDelta rowDelta) + { + throw new UnsupportedOperationException("Deletion Vectors are not supported"); + } + + @Override + public void mergePreExistingDeletes(ConnectorSession session, Table icebergTable, long snapshotId, Map deletionVectorBuilders) + { + // no-op: pre-existing delete merging not supported when DVs are unsupported + } + + @Override + public void mergePreExistingDeletes(ConnectorSession session, TrinoFileSystem fileSystem, FileIO fileIo, Map specsById, Snapshot snapshot, Map deletionVectorBuilders) + { + // no-op: pre-existing delete merging not supported when DVs are unsupported + } }; void writeDeletionVectors( @@ -39,6 +60,20 @@ void writeDeletionVectors( List deletionVectorInfos, RowDelta rowDelta); + void mergePreExistingDeletes( + ConnectorSession session, + Table icebergTable, + long snapshotId, + Map deletionVectorBuilders); + + void mergePreExistingDeletes( + ConnectorSession session, + TrinoFileSystem fileSystem, + FileIO fileIo, + Map specsById, + Snapshot snapshot, + Map deletionVectorBuilders); + record DeletionVectorInfo(String dataFilePath, Slice serializedDeletionVector, PartitionSpec partitionSpec, Optional partitionData) { public DeletionVectorInfo diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java index 2ce99b1a2a59..a7dee920d4c0 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java @@ -276,7 +276,8 @@ private static IcebergTableHandle getIcebergTableHandle(PartitionSpec partitionS false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); } private static Table createIcebergTable(PartitionSpec partitionSpec) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java index d6d0d0b8ffd1..886260a38151 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java @@ -78,6 +78,7 @@ public void testDefaults() .setSplitManagerThreads(Integer.toString(Math.min(Runtime.getRuntime().availableProcessors() * 2, 32))) .setPlanningThreads(Integer.toString(Runtime.getRuntime().availableProcessors() * 2)) .setFileDeleteThreads(Integer.toString(Runtime.getRuntime().availableProcessors() * 2)) + .setCopyOnWriteRewriteThreads(Integer.toString(Runtime.getRuntime().availableProcessors())) .setAllowedExtraProperties(ImmutableList.of()) .setIncrementalRefreshEnabled(true) .setMetadataCacheEnabled(true) @@ -126,6 +127,7 @@ public void testExplicitPropertyMappings() .put("iceberg.split-manager-threads", "42") .put("iceberg.planning-threads", "42") .put("iceberg.file-delete-threads", "42") + .put("iceberg.copy-on-write-rewrite-threads", "8") .put("iceberg.allowed-extra-properties", "propX,propY") .put("iceberg.incremental-refresh-enabled", "false") .put("iceberg.materialized-views.refresh-max-snapshots-to-expire", "5") @@ -170,6 +172,7 @@ public void testExplicitPropertyMappings() .setSplitManagerThreads("42") .setPlanningThreads("42") .setFileDeleteThreads("42") + .setCopyOnWriteRewriteThreads("8") .setAllowedExtraProperties(ImmutableList.of("propX", "propY")) .setIncrementalRefreshEnabled(false) .setMetadataCacheEnabled(false) diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWrite.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWrite.java new file mode 100644 index 000000000000..59c07e4f7f46 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWrite.java @@ -0,0 +1,3448 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import io.trino.Session; +import io.trino.plugin.tpch.TpchPlugin; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.DistributedQueryRunner; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.apache.iceberg.BaseTable; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; + +import java.nio.file.Path; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Verify.verify; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; +import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; +import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; +import static io.trino.plugin.iceberg.IcebergTestUtils.loadTable; +import static io.trino.plugin.iceberg.util.EqualityDeleteUtils.writeEqualityDeleteForTable; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static io.trino.testing.TestingSession.testSessionBuilder; +import static org.apache.iceberg.FileContent.DATA; +import static org.apache.iceberg.FileContent.EQUALITY_DELETES; +import static org.apache.iceberg.FileContent.POSITION_DELETES; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; + +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) +public class TestIcebergCopyOnWrite + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + Session icebergSession = testSessionBuilder() + .setCatalog(ICEBERG_CATALOG) + .setSchema("tpch") + .build(); + + QueryRunner queryRunner = DistributedQueryRunner.builder(icebergSession).build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Path dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); + verify(dataDirectory.toFile().mkdirs()); + + queryRunner.installPlugin(new TestingIcebergPlugin(dataDirectory)); + queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", Map.of( + "iceberg.catalog.type", "TESTING_FILE_METASTORE", + "hive.metastore.catalog.dir", dataDirectory.toString(), + "fs.hadoop.enabled", "true", + "iceberg.allowed-extra-properties", "*")); + + queryRunner.execute("CREATE SCHEMA tpch"); + + return queryRunner; + } + + private TestTable newCowTable(String prefix, String tableDefinition) + { + return new TestTable(getQueryRunner()::execute, prefix + randomNameSuffix(), tableDefinition); + } + + @Test + public void testDeleteCopyOnWrite() + { + try (TestTable table = newCowTable("test_cow_delete_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Verify initial state: only data files, no delete files + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id(), "VALUES 1"); + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Delete some rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + + // Verify no delete files were created (CoW rewrites data files instead) + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Verify data files were rewritten (count may differ from initial due to rewrite) + long newDataFileCount = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(newDataFileCount).isGreaterThanOrEqualTo(1); + + // Verify data correctness + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + } + + @Test + public void testUpdateCopyOnWrite() + { + try (TestTable table = newCowTable("test_cow_update_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Update some rows + assertUpdate("UPDATE " + tableName + " SET name = 'MODIFIED' WHERE regionkey = 2", 5); + + // Verify no delete files were created + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Verify data correctness + assertQuery("SELECT name FROM " + tableName + " WHERE regionkey = 2", + "VALUES 'MODIFIED', 'MODIFIED', 'MODIFIED', 'MODIFIED', 'MODIFIED'"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'MODIFIED'", "VALUES 5"); + } + } + + @Test + public void testUpdateCopyOnWriteV3PreservesRowLineage() + { + try (TestTable table = newCowTable("test_cow_update_v3_", + "(id integer, v varchar) WITH (format_version = 3, extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b')", 2); + + long beforeUnchangedRowId = (Long) computeScalar("SELECT \"$row_id\" FROM " + tableName + " WHERE id = 1"); + long beforeUpdatedSequenceNumber = (Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + tableName + " WHERE id = 2"); + long beforeUnchangedSequenceNumber = (Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + tableName + " WHERE id = 1"); + + assertUpdate("UPDATE " + tableName + " SET v = 'bb' WHERE id = 2", 1); + + assertThat((Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + tableName + " WHERE id = 2")).isGreaterThan(beforeUpdatedSequenceNumber); + // For unchanged rows moved by CoW rewrite, Iceberg v3 lineage metadata must be preserved. + assertThat((Long) computeScalar("SELECT \"$row_id\" FROM " + tableName + " WHERE id = 1")).isEqualTo(beforeUnchangedRowId); + assertThat((Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + tableName + " WHERE id = 1")).isEqualTo(beforeUnchangedSequenceNumber); + assertQuery("SELECT * FROM " + tableName, "VALUES (1, 'a'), (2, 'bb')"); + assertNoDeleteFiles(tableName); + } + } + + @Test + public void testDeleteCopyOnWritePreservesPreExistingEqualityDeletes() + throws Exception + { + try (TestTable table = newCowTable("test_cow_equality_delete_", + "(id integer, v varchar) WITH (format_version = 2, extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b'), (3, 'c')", 3); + + BaseTable icebergTable = loadBaseTable(tableName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.empty(), + Optional.empty(), + Map.of("id", 1), + Optional.empty()); + + assertQuery("SELECT id, v FROM " + tableName, "VALUES (2, 'b'), (3, 'c')"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + assertQuery("SELECT id, v FROM " + tableName, "VALUES (3, 'c')"); + } + } + + @Test + public void testCowRewriteIgnoresPartitionScopedEqualityDeletesFromOtherPartitions() + throws Exception + { + try (TestTable table = newCowTable("test_cow_partition_eq_delete_isolation_", + "(part integer, id integer, category varchar) " + + "WITH (format_version = 2, partitioning = ARRAY['part'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(0, 1, 'victim'), " + + "(0, 4, 'keep0'), " + + "(1, 2, 'victim'), " + + "(1, 3, 'keep1')", 4); + + BaseTable icebergTable = loadBaseTable(tableName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.of(icebergTable.spec()), + Optional.of(new PartitionData(new Object[] {0})), + Map.of("category", "victim"), + Optional.empty()); + + assertQuery("SELECT id, part, category FROM " + tableName + " ORDER BY id", + "VALUES (2, 1, 'victim'), (3, 1, 'keep1'), (4, 0, 'keep0')"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 3", 1); + + assertQuery("SELECT id, part, category FROM " + tableName + " ORDER BY id", + "VALUES (2, 1, 'victim'), (4, 0, 'keep0')"); + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + EQUALITY_DELETES.id(), "VALUES 1"); + } + } + + @Test + public void testCowRewriteRetainsPartitionScopedEqualityDeletesEvenWhenAllFilesAreRewritten() + throws Exception + { + // Partition-scoped equality deletes are NOT cleaned up by CoW rewrite (only file-scoped + // DVs are). This matches Spark's behavior -- partition-scoped deletes are cleaned up + // during compaction/optimize instead. The data correctness is unaffected because the + // equality delete still applies correctly at read time. + try (TestTable table = newCowTable("test_cow_partition_eq_delete_cleanup_", + "(part integer, id integer, category varchar) " + + "WITH (format_version = 2, partitioning = ARRAY['part'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(0, 1, 'victim'), " + + "(0, 4, 'keep0'), " + + "(1, 2, 'victim'), " + + "(1, 3, 'keep1')", 4); + + BaseTable icebergTable = loadBaseTable(tableName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.of(icebergTable.spec()), + Optional.of(new PartitionData(new Object[] {0})), + Map.of("category", "victim"), + Optional.empty()); + + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + EQUALITY_DELETES.id(), "VALUES 1"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 4", 1); + + assertQuery("SELECT id, part, category FROM " + tableName + " ORDER BY id", + "VALUES (2, 1, 'victim'), (3, 1, 'keep1')"); + // Partition-scoped equality delete is retained (not cleaned up by CoW) + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + EQUALITY_DELETES.id(), "VALUES 1"); + } + } + + @Test + public void testCowRewriteKeepsPartitionScopedEqualityDeletesWhenOtherFilesInPartitionRemain() + throws Exception + { + try (TestTable table = newCowTable("test_cow_partition_eq_delete_partial_cleanup_", + "(part integer, id integer, category varchar) " + + "WITH (format_version = 2, partitioning = ARRAY['part'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (0, 1, 'victim'), (0, 4, 'keep0a')", 2); + assertUpdate("INSERT INTO " + tableName + " VALUES (0, 5, 'victim'), (0, 6, 'keep0b')", 2); + + BaseTable icebergTable = loadBaseTable(tableName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.of(icebergTable.spec()), + Optional.of(new PartitionData(new Object[] {0})), + Map.of("category", "victim"), + Optional.empty()); + + assertQuery("SELECT id, part, category FROM " + tableName + " ORDER BY id", + "VALUES (4, 0, 'keep0a'), (6, 0, 'keep0b')"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 4", 1); + + assertQuery("SELECT id, part, category FROM " + tableName + " ORDER BY id", + "VALUES (6, 0, 'keep0b')"); + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + EQUALITY_DELETES.id(), "VALUES 1"); + } + } + + @Test + public void testMergeCopyOnWrite() + { + try (TestTable target = newCowTable("test_cow_merge_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation"); + TestTable source = newCowTable("test_cow_merge_source_", + "AS SELECT nationkey, 'NEW_' || name AS name, regionkey, comment FROM tpch.tiny.nation WHERE regionkey = 3")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + // MERGE: update matching rows, insert non-matching + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN UPDATE SET name = s.name " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", 5); + + // Verify no delete files were created + assertQuery("SELECT count(*) FROM \"" + targetName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Verify data correctness - the matched rows should have updated names + assertQuery("SELECT count(*) FROM " + targetName + " WHERE name LIKE 'NEW_%'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 25"); + } + } + + @Test + public void testMergeCopyOnWriteV3PreservesRowLineage() + { + try (TestTable target = newCowTable("test_cow_merge_v3_target_", + "(id integer, v varchar) WITH (format_version = 3, extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write']))"); + TestTable source = newCowTable("test_cow_merge_v3_source_", + "AS SELECT * FROM (VALUES (2, 'bb')) AS t(id, v)")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + assertUpdate("INSERT INTO " + targetName + " VALUES (1, 'a'), (2, 'b')", 2); + + long beforeUnchangedRowId = (Long) computeScalar("SELECT \"$row_id\" FROM " + targetName + " WHERE id = 1"); + long beforeMergedSequenceNumber = (Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + targetName + " WHERE id = 2"); + long beforeUnchangedSequenceNumber = (Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + targetName + " WHERE id = 1"); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET v = s.v", 1); + + assertThat((Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + targetName + " WHERE id = 2")).isGreaterThan(beforeMergedSequenceNumber); + // For unchanged rows moved by CoW rewrite, Iceberg v3 lineage metadata must be preserved. + assertThat((Long) computeScalar("SELECT \"$row_id\" FROM " + targetName + " WHERE id = 1")).isEqualTo(beforeUnchangedRowId); + assertThat((Long) computeScalar("SELECT \"$last_updated_sequence_number\" FROM " + targetName + " WHERE id = 1")).isEqualTo(beforeUnchangedSequenceNumber); + assertQuery("SELECT * FROM " + targetName, "VALUES (1, 'a'), (2, 'bb')"); + assertNoDeleteFiles(targetName); + } + } + + @Test + public void testDeleteCopyOnWritePartitionedTable() + { + try (TestTable table = newCowTable("test_cow_delete_partitioned_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Delete rows from one partition + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + + // Verify no delete files + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Verify data correctness + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 0"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + } + + @Test + public void testDeleteAllRowsCopyOnWrite() + { + try (TestTable table = newCowTable("test_cow_delete_all_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey = 1")) { + String tableName = table.getName(); + + // Delete all rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + + // Verify no delete files + assertQuery("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id(), "VALUES 0"); + + // Verify table is empty + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 0"); + } + } + + @Test + public void testMergeOnReadDefault() + { + // Without setting write.delete.mode, the default should be merge-on-read + try (TestTable table = newCowTable("test_mor_default_", + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + + // Default mode should produce delete files (merge-on-read) + long deleteFileCount = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(deleteFileCount).isGreaterThanOrEqualTo(1); + + // Verify data correctness still works + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); + } + } + + @Test + public void testSequentialDmlOperations() + { + // Real-world: a dimension table receiving daily incremental updates + try (TestTable table = newCowTable("test_cow_sequential_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Day 1: delete obsolete countries + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (0, 1, 2)", 3); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertNoDeleteFiles(tableName); + + // Day 2: update names for rebranding + assertUpdate("UPDATE " + tableName + " SET name = upper(name) WHERE regionkey = 3", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertNoDeleteFiles(tableName); + + // Day 3: another batch of deletes + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 17"); + assertNoDeleteFiles(tableName); + + // Day 4: update remaining rows (nationkeys 1,2 were deleted in day 1, so only 3 remain in regionkey=1) + assertUpdate("UPDATE " + tableName + " SET comment = 'updated' WHERE regionkey = 1", 3); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE comment = 'updated'", "VALUES 3"); + assertNoDeleteFiles(tableName); + + // Final integrity check: regionkeys 0,1,2,3 remain (4 deleted) + assertQuery("SELECT count(DISTINCT regionkey) FROM " + tableName, "VALUES 4"); + } + } + + @Test + public void testUpdateWithComplexExpressions() + { + // Real-world: ETL pipeline applying business transformations + try (TestTable table = newCowTable("test_cow_complex_update_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // CASE expression update + assertUpdate("UPDATE " + tableName + " SET name = CASE " + + "WHEN regionkey = 0 THEN 'AFRICA_' || CAST(nationkey AS VARCHAR) " + + "WHEN regionkey = 1 THEN 'AMERICAS_' || CAST(nationkey AS VARCHAR) " + + "ELSE name END " + + "WHERE regionkey IN (0, 1)", 10); + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name LIKE 'AFRICA_%'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name LIKE 'AMERICAS_%'", "VALUES 5"); + // Untouched rows + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey NOT IN (0, 1)", "VALUES 15"); + } + } + + @Test + public void testMergeWithDeleteClause() + { + // Real-world: CDC merge -- update existing, delete terminated, insert new + try (TestTable target = newCowTable("test_cow_merge_del_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation"); + TestTable source = newCowTable("test_cow_merge_del_source_", + "AS SELECT * FROM (VALUES " + + "(0, 'UPDATED_ALGERIA', 0, 'updated', 'U'), " + + "(1, 'DELETED', 1, 'deleted', 'D'), " + + "(99, 'NEW_COUNTRY', 4, 'new record', 'I')) " + + "AS t(nationkey, name, regionkey, comment, operation)")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED AND s.operation = 'D' THEN DELETE " + + "WHEN MATCHED AND s.operation = 'U' THEN UPDATE SET name = s.name, comment = s.comment " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", 3); + + assertNoDeleteFiles(targetName); + // 25 original - 1 deleted + 1 inserted = 25 + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 25"); + assertQuery("SELECT name FROM " + targetName + " WHERE nationkey = 0", "VALUES 'UPDATED_ALGERIA'"); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE nationkey = 1", "VALUES 0"); + assertQuery("SELECT name FROM " + targetName + " WHERE nationkey = 99", "VALUES 'NEW_COUNTRY'"); + } + } + + @Test + public void testDeleteWithSubquery() + { + // Real-world: deleting rows based on a lookup table + try (TestTable table = newCowTable("test_cow_subquery_del_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + + " WHERE nationkey IN (SELECT nationkey FROM tpch.tiny.nation WHERE regionkey = 2)", 5); + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 2", "VALUES 0"); + } + } + + @Test + public void testUpdateAcrossMultiplePartitions() + { + // Real-world: updating a partitioned fact table across partitions + try (TestTable table = newCowTable("test_cow_update_multi_part_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Update touches multiple partitions + assertUpdate("UPDATE " + tableName + " SET comment = 'batch_update' WHERE nationkey < 10", 10); + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE comment = 'batch_update'", "VALUES 10"); + + // Verify partition integrity -- each partition still has correct count + assertQuery("SELECT regionkey, count(*) FROM " + tableName + " GROUP BY regionkey ORDER BY regionkey", + "SELECT regionkey, count(*) FROM nation GROUP BY regionkey ORDER BY regionkey"); + } + } + + @Test + public void testDeleteWithNoMatchingRows() + { + // Edge case: DELETE that matches zero rows should be a no-op + try (TestTable table = newCowTable("test_cow_noop_del_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + long snapshotsBefore = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 999", 0); + + long snapshotsAfter = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertThat(snapshotsAfter).isEqualTo(snapshotsBefore); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + } + } + + @Test + public void testTimeTravelAfterCowDelete() + { + // Real-world: auditing -- read old snapshot after CoW rewrite + try (TestTable table = newCowTable("test_cow_time_travel_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + long snapshotBeforeDelete = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Time travel to before the delete should show all 25 rows + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshotBeforeDelete, "VALUES 25"); + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshotBeforeDelete + " WHERE regionkey = 1", "VALUES 5"); + } + } + + @Test + public void testCowWithNullableColumns() + { + // Real-world: tables with sparse/nullable data + try (TestTable table = newCowTable("test_cow_nullable_", + "(id INTEGER, name VARCHAR, score DOUBLE, category VARCHAR) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'alice', 95.5, 'A'), (2, NULL, NULL, 'B'), " + + "(3, 'charlie', NULL, NULL), (4, NULL, 80.0, 'A'), (5, 'eve', 70.0, 'B')", 5); + + // Delete rows with null names + assertUpdate("DELETE FROM " + tableName + " WHERE name IS NULL", 2); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 3"); + + // Update null scores + assertUpdate("UPDATE " + tableName + " SET score = 0.0 WHERE score IS NULL", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT id, score FROM " + tableName + " WHERE id = 3", "VALUES (3, 0.0e0)"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 3"); + } + } + + @Test + public void testCowMultipleDataFiles() + { + // Real-world: table with data spread across multiple files (multiple inserts) + try (TestTable table = newCowTable("test_cow_multi_file_", + "(id INTEGER, region VARCHAR, amount DOUBLE) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + // Insert in multiple batches to create multiple data files + assertUpdate("INSERT INTO " + tableName + " SELECT nationkey, name, CAST(nationkey * 100 AS DOUBLE) FROM tpch.tiny.nation WHERE regionkey = 0", 5); + assertUpdate("INSERT INTO " + tableName + " SELECT nationkey, name, CAST(nationkey * 100 AS DOUBLE) FROM tpch.tiny.nation WHERE regionkey = 1", 5); + assertUpdate("INSERT INTO " + tableName + " SELECT nationkey, name, CAST(nationkey * 100 AS DOUBLE) FROM tpch.tiny.nation WHERE regionkey = 2", 5); + + long dataFileCount = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(dataFileCount).isGreaterThanOrEqualTo(3); + + // Delete across all files (nationkeys 0,1,2,3 exist across regions 0-2 = 4 rows) + assertUpdate("DELETE FROM " + tableName + " WHERE id < 5", 4); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 11"); + + // Update across remaining files (nationkeys 12,14,15,16,17,18,21,24 = 8 rows with id >= 10) + assertUpdate("UPDATE " + tableName + " SET amount = amount * 2 WHERE id >= 10", 8); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 11"); + } + } + + @Test + public void testCowIdempotentUpdate() + { + // Edge case: update that sets same value should still work correctly + try (TestTable table = newCowTable("test_cow_idempotent_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("UPDATE " + tableName + " SET name = name WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation"); + } + } + + @Test + public void testCowDeleteThenInsert() + { + // Real-world: SCD Type 1 -- delete old dimension records, insert corrected ones + try (TestTable table = newCowTable("test_cow_delete_insert_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + assertUpdate("INSERT INTO " + tableName + " VALUES (100, 'NEW_NATION_1', 0, 'new'), (101, 'NEW_NATION_2', 0, 'new')", 2); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 2"); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 17"); + } + } + + @Test + public void testMergeWithOnlyMatchedDelete() + { + // Real-world: MERGE used purely for conditional deletes + try (TestTable target = newCowTable("test_cow_merge_only_del_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode', 'write.delete.mode'], ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation"); + TestTable source = newCowTable("test_cow_merge_only_del_source_", + "AS SELECT nationkey FROM tpch.tiny.nation WHERE regionkey = 2")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN DELETE", 5); + + assertNoDeleteFiles(targetName); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE regionkey = 2", "VALUES 0"); + } + } + + @Test + public void testDeletePartitionedEntirePartition() + { + // Real-world: dropping an entire partition's data via DELETE (not DROP PARTITION) + try (TestTable table = newCowTable("test_cow_del_whole_part_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Delete two entire partitions + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey IN (0, 1)", 10); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY regionkey", "VALUES 2, 3, 4"); + + // Remaining partitions untouched + assertQuery("SELECT * FROM " + tableName + " WHERE regionkey = 2", + "SELECT * FROM nation WHERE regionkey = 2"); + } + } + + @Test + public void testCowWithLargerDataset() + { + // Real-world: more realistic data volume using tpch.tiny.orders (15000 rows) + try (TestTable table = newCowTable("test_cow_orders_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.orders")) { + String tableName = table.getName(); + + long initialCount = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertThat(initialCount).isEqualTo(15000L); + + // Delete a chunk of orders + long deletedCount = (long) computeScalar("SELECT count(*) FROM " + tableName + " WHERE orderstatus = 'F'"); + assertUpdate("DELETE FROM " + tableName + " WHERE orderstatus = 'F'", deletedCount); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES " + (initialCount - deletedCount)); + + // Update remaining orders + long pendingCount = (long) computeScalar("SELECT count(*) FROM " + tableName + " WHERE orderstatus = 'P'"); + assertUpdate("UPDATE " + tableName + " SET orderpriority = '1-URGENT' WHERE orderstatus = 'P'", pendingCount); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE orderpriority = '1-URGENT' AND orderstatus = 'P'", + "VALUES " + pendingCount); + } + } + + @Test + public void testTimeTravelAfterMultipleCowOperations() + { + // Real-world: audit trail across multiple CoW rewrites + try (TestTable table = newCowTable("test_cow_multi_tt_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + long snapshot0 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + long snapshot1 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + + assertUpdate("UPDATE " + tableName + " SET name = 'X' WHERE regionkey = 1", 5); + long snapshot2 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + + // Current state + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Snapshot 0: all 25 rows + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot0, "VALUES 25"); + + // Snapshot 1: 20 rows, regionkey=0 deleted + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot1, "VALUES 20"); + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot1 + " WHERE regionkey = 0", "VALUES 0"); + + // Snapshot 2: 20 rows, regionkey=1 updated + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot2 + " WHERE name = 'X'", "VALUES 5"); + } + } + + @Test + public void testMergeOnlyInsert() + { + // Edge case: MERGE where nothing matches -- only inserts, no CoW rewrite needed + try (TestTable target = newCowTable("test_cow_merge_insert_only_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey < 3"); + TestTable source = newCowTable("test_cow_merge_insert_only_source_", + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey >= 3")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + long sourceCount = (long) computeScalar("SELECT count(*) FROM " + sourceName); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", sourceCount); + + assertNoDeleteFiles(targetName); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 25"); + } + } + + @Test + public void testCowReplacesOriginalDataFiles() + { + // Verify that after a CoW operation, original data file paths are gone + // and new data file paths are present (files are rewritten, not just marked with deletes) + try (TestTable table = newCowTable("test_cow_file_replace_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Capture file paths before operation + Set filesBefore = getDataFilePaths(tableName); + assertThat(filesBefore).isNotEmpty(); + + // DELETE: should rewrite files, not create delete files + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + Set filesAfterDelete = getDataFilePaths(tableName); + assertNoDeleteFiles(tableName); + // Files were rewritten: set changed and remains non-empty + assertThat(filesAfterDelete).isNotEmpty(); + assertThat(filesAfterDelete).isNotEqualTo(filesBefore); + + // UPDATE: should again rewrite files + Set filesBeforeUpdate = getDataFilePaths(tableName); + assertUpdate("UPDATE " + tableName + " SET comment = 'changed' WHERE regionkey = 0", 5); + Set filesAfterUpdate = getDataFilePaths(tableName); + assertNoDeleteFiles(tableName); + assertThat(filesAfterUpdate).isNotEmpty(); + assertThat(filesAfterUpdate).isNotEqualTo(filesBeforeUpdate); + + // Verify data correctness + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE comment = 'changed'", "VALUES 5"); + } + } + + @Test + public void testCowRewriteKeepsPartitionedFileLayout() + { + try (TestTable table = newCowTable("test_cow_partition_layout_", + "(part integer, id integer, category varchar) " + + "WITH (partitioning = ARRAY['part'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (0, 1, 'a'), (0, 2, 'b'), (1, 3, 'c')", 3); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + + assertThat(getDataFilePaths(tableName)).allMatch(path -> path.contains("part=")); + } + } + + @Test + public void testCowDeletePreservesSortOrderMetadata() + { + try (TestTable table = newCowTable("test_cow_sort_order_", + "(id integer, v varchar) " + + "WITH (sorted_by = ARRAY['v'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b'), (3, 'c')", 3); + + assertQuery("SELECT DISTINCT sort_order_id FROM \"" + tableName + "$files\" WHERE content = " + DATA.id() + " ORDER BY 1", "VALUES 1"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + + assertQuery("SELECT DISTINCT sort_order_id FROM \"" + tableName + "$files\" WHERE content = " + DATA.id() + " ORDER BY 1", "VALUES 1"); + } + } + + @Test + public void testCowMetricsFileCountAfterMultiFileDelete() + { + // Verify that CoW correctly handles multi-file tables: file count should + // reflect the number of files that still have surviving rows + try (TestTable table = newCowTable("test_cow_metrics_file_count_", + "(nationkey bigint, name varchar, regionkey bigint, comment varchar) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + // Insert data in separate statements to create multiple data files + assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.tiny.nation WHERE regionkey = 0", 5); + assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.tiny.nation WHERE regionkey = 1", 5); + assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.tiny.nation WHERE regionkey = 2", 5); + + Set filesBefore = getDataFilePaths(tableName); + assertThat(filesBefore).hasSizeGreaterThanOrEqualTo(3); + + // Delete all rows from one region — should rewrite only the affected file + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertNoDeleteFiles(tableName); + + Set filesAfter = getDataFilePaths(tableName); + // Files changed: the file containing regionkey=1 was removed (all rows deleted) + assertThat(filesAfter).isNotEmpty(); + assertThat(filesAfter).isNotEqualTo(filesBefore); + + // Data correctness + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 10"); + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY 1", "VALUES 0, 2"); + } + } + + @Test + public void testCowMetricsRowCountConsistency() + { + // Verify row counts stay consistent through multiple CoW operations, + // confirming that rewrite metrics (rows deleted, rows surviving) are accurate + try (TestTable table = newCowTable("test_cow_metrics_rows_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Initial: 25 rows + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + + // DELETE 5 rows (regionkey=4) → 20 rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertNoDeleteFiles(tableName); + + // UPDATE 5 rows (regionkey=0) → still 20 rows + assertUpdate("UPDATE " + tableName + " SET comment = 'updated' WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE comment = 'updated'", "VALUES 5"); + assertNoDeleteFiles(tableName); + + // MERGE: delete regionkey=1, update regionkey=2 → 15 rows remain, 5 updated + assertUpdate("MERGE INTO " + tableName + " t USING (VALUES 1, 2) AS s(rk) " + + "ON t.regionkey = s.rk " + + "WHEN MATCHED AND s.rk = 1 THEN DELETE " + + "WHEN MATCHED AND s.rk = 2 THEN UPDATE SET comment = 'merged'", 10); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE comment = 'merged'", "VALUES 5"); + assertNoDeleteFiles(tableName); + + // Verify no delete files after all operations + Set finalFiles = getDataFilePaths(tableName); + assertThat(finalFiles).isNotEmpty(); + + // Verify all data file sizes are positive (no empty files) + computeActual("SELECT file_size_in_bytes FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()) + .getMaterializedRows() + .forEach(row -> assertThat((Long) row.getField(0)).isGreaterThan(0)); + } + } + + // ----------------------------------------------------------------------- + // Source file format: read ORC source, write new file in table default (PARQUET) + // ----------------------------------------------------------------------- + + @Test + public void testCowDeleteOnMixedFormatTable() + { + // Create table with ORC format, insert data, change to PARQUET, then CoW delete + // This verifies the source file format is read from the file path, not table default + String tableName = "test_cow_mixed_format_delete_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format = 'ORC', extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // Verify initial files are ORC + Set orcFiles = getDataFilePaths(tableName); + assertThat(orcFiles).isNotEmpty(); + assertThat(orcFiles).allMatch(path -> path.endsWith(".orc")); + + // Change default format to PARQUET + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES format = 'PARQUET'"); + + // Insert more data in PARQUET format + assertUpdate("INSERT INTO " + tableName + " SELECT * FROM tpch.tiny.nation WHERE regionkey = 0", 5); + + // Verify mixed formats: some ORC, some PARQUET + Set allFiles = getDataFilePaths(tableName); + assertThat(allFiles.stream().filter(p -> p.endsWith(".orc")).count()).isGreaterThanOrEqualTo(1); + assertThat(allFiles.stream().filter(p -> p.endsWith(".parquet")).count()).isGreaterThanOrEqualTo(1); + + // CoW DELETE on rows that exist in ORC files -- this reads ORC source, writes PARQUET + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertNoDeleteFiles(tableName); + + // Verify new rewritten files are PARQUET (table default), not ORC + Set filesAfterDelete = getDataFilePaths(tableName); + // The ORC file that was rewritten should be replaced by a PARQUET file + // Original ORC file had regionkey=1 rows, so it was rewritten + assertThat(filesAfterDelete).isNotEmpty(); + + // Verify data correctness + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 1", "VALUES 0"); + // The duplicated regionkey=0 rows (5 original + 5 inserted) + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 10"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowUpdateOnMixedFormatTable() + { + // Update touching ORC files when table default is PARQUET + String tableName = "test_cow_mixed_format_update_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format = 'ORC', extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // Switch to PARQUET + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES format = 'PARQUET'"); + + // CoW UPDATE on ORC data + assertUpdate("UPDATE " + tableName + " SET name = 'UPDATED' WHERE regionkey = 2", 5); + assertNoDeleteFiles(tableName); + + // Verify correctness + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'UPDATED'", "VALUES 5"); + assertQuery("SELECT * FROM " + tableName + " WHERE regionkey != 2", + "SELECT * FROM nation WHERE regionkey != 2"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowMergeOnMixedFormatTable() + { + // MERGE touching ORC files when table default is PARQUET + String tableName = "test_cow_mixed_format_merge_" + randomNameSuffix(); + String sourceName = "test_cow_mixed_format_merge_src_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format = 'ORC', extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES format = 'PARQUET'"); + + assertUpdate("CREATE TABLE " + sourceName + " AS SELECT nationkey, 'MERGED_' || name AS name FROM tpch.tiny.nation WHERE regionkey = 3", 5); + + assertUpdate("MERGE INTO " + tableName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN UPDATE SET name = s.name", 5); + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name LIKE 'MERGED_%'", "VALUES 5"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + sourceName); + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Pre-existing delete files: MoR → CoW mode switch + // Verifies previously-deleted rows don't reappear after CoW rewrite + // ----------------------------------------------------------------------- + + @Test + public void testCowAfterMorDeleteV2() + { + // v2 table: MoR delete creates position delete files, then switch to CoW + String tableName = "test_cow_after_mor_v2_" + randomNameSuffix(); + try { + // Create table with MoR mode (default) + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete: creates position delete files + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Verify position delete files were created (MoR behavior) + long deleteFileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(deleteFileCount).isGreaterThanOrEqualTo(1); + + // Switch to CoW mode + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])"); + + // CoW DELETE on different rows -- must NOT resurrect regionkey=1 rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 2", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + + // The key assertion: regionkey=1 rows are still gone + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 1", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 2", "VALUES 0"); + + // Remaining regions intact + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY 1", "VALUES 0, 3, 4"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowAfterMorUpdateV2() + { + // v2 table: MoR produces deletes from an UPDATE, then switch to CoW + String tableName = "test_cow_after_mor_update_v2_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR update (internally: delete old + insert new) + assertUpdate("UPDATE " + tableName + " SET name = 'MOR_UPDATED' WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'MOR_UPDATED'", "VALUES 5"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])"); + + // CoW update on overlapping data file + assertUpdate("UPDATE " + tableName + " SET name = 'COW_UPDATED' WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + + // MoR-updated rows still correct + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'MOR_UPDATED'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'COW_UPDATED'", "VALUES 5"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowAfterMorDeleteV3() + { + // v3 table: MoR delete creates deletion vectors (DVs), then switch to CoW + String tableName = "test_cow_after_mor_v3_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 3) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete: creates DVs in v3 + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 3", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Switch to CoW mode + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])"); + + // CoW delete -- must not resurrect regionkey=3 rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + + // Key assertion: regionkey=3 rows still gone + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 3", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 4", "VALUES 0"); + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY 1", "VALUES 0, 1, 2"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowAfterMultipleMorDeletes() + { + // Multiple MoR deletes accumulate delete files, then CoW must honor all of them + String tableName = "test_cow_after_multi_mor_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // Multiple MoR deletes on the same data file + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 0", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 5", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 10", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete -- must honor all prior MoR deletes + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 15", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 21"); + + // All previously deleted rows still gone + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (0, 5, 10, 15)", "VALUES 0"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowMergeAfterMorDelete() + { + // MoR delete followed by CoW MERGE + String tableName = "test_cow_merge_after_mor_" + randomNameSuffix(); + String sourceName = "test_cow_merge_after_mor_src_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])"); + + // Create merge source + assertUpdate("CREATE TABLE " + sourceName + + " AS SELECT nationkey, 'MERGED' AS name, regionkey, 'merged' AS comment FROM tpch.tiny.nation WHERE regionkey = 1", 5); + + // CoW MERGE -- touches same data file, must not resurrect regionkey=0 + assertUpdate("MERGE INTO " + tableName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN UPDATE SET name = s.name", 5); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'MERGED'", "VALUES 5"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + sourceName); + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Conflict detection: OverwriteFiles with snapshot validation + // ----------------------------------------------------------------------- + + @Test + public void testCowWithConcurrentInsert() + { + // Verify CoW DELETE works correctly when concurrent INSERTs happen + // The OverwriteFiles API validates from the base snapshot; + // an INSERT to a different partition should NOT cause a conflict + try (TestTable table = newCowTable("test_cow_concurrent_insert_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Insert additional rows (simulates a concurrent INSERT to a different partition) + assertUpdate("INSERT INTO " + tableName + + " VALUES (100, 'NEW1', 0, 'new1'), (101, 'NEW2', 0, 'new2')", 2); + + // CoW DELETE on a different partition should succeed without conflict + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertNoDeleteFiles(tableName); + + // Verify all data is correct + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 7"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 4", "VALUES 0"); + } + } + + @Test + public void testCowSnapshotIsolation() + { + // Verify that each CoW operation produces a new snapshot with correct metadata + try (TestTable table = newCowTable("test_cow_snapshot_isolation_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + long snapshot0 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + + // CoW DELETE + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + long snapshot1 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + assertThat(snapshot1).isNotEqualTo(snapshot0); + + // Verify the new snapshot's operation is 'overwrite' (from OverwriteFiles API) + String operation1 = (String) computeScalar( + "SELECT operation FROM \"" + tableName + "$snapshots\" WHERE snapshot_id = " + snapshot1); + assertThat(operation1).isEqualTo("overwrite"); + + // CoW UPDATE should remain overwrite-shaped even when updated rows are written as new data files + assertUpdate("UPDATE " + tableName + " SET name = 'UPDATED' WHERE regionkey = 1", 5); + long snapshot2 = (long) computeScalar( + "SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + assertThat(snapshot2).isNotEqualTo(snapshot1); + + String operation2 = (String) computeScalar( + "SELECT operation FROM \"" + tableName + "$snapshots\" WHERE snapshot_id = " + snapshot2); + assertThat(operation2).isEqualTo("overwrite"); + + // Time travel verification: each snapshot is independently correct + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot0, "VALUES 25"); + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot1, "VALUES 20"); + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot2, "VALUES 20"); + assertQuery("SELECT count(*) FROM \"" + tableName + "\" FOR VERSION AS OF " + snapshot2 + " WHERE name = 'UPDATED'", "VALUES 5"); + } + } + + @Test + public void testCowProducesOverwriteSnapshot() + { + // Verify CoW operations produce 'overwrite' snapshots, not 'delete'/'append' + try (TestTable table = newCowTable("test_cow_overwrite_snapshot_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertThat(computeScalar("SELECT operation FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1")) + .isEqualTo("overwrite"); + + assertUpdate("UPDATE " + tableName + " SET name = 'X' WHERE regionkey = 3", 5); + assertThat(computeScalar("SELECT operation FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1")) + .isEqualTo("overwrite"); + long overwriteCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$snapshots\" WHERE operation = 'overwrite'"); + assertThat(overwriteCount).isGreaterThanOrEqualTo(2); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + } + + @Test + public void testCowPartitionedMergeDoesNotProduceDeleteSnapshot() + { + try (TestTable target = newCowTable("test_cow_partitioned_merge_snapshot_target_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation"); + TestTable source = newCowTable("test_cow_partitioned_merge_snapshot_source_", + "AS SELECT nationkey, 'MERGED_' || name AS name, regionkey, comment " + + "FROM tpch.tiny.nation WHERE regionkey = 3 " + + "UNION ALL " + + "SELECT 100, 'INSERTED', 0, 'inserted'")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN UPDATE SET name = s.name, comment = s.comment " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", 6); + + assertNoDeleteFiles(targetName); + assertThat(computeScalar("SELECT count(*) FROM \"" + targetName + "$snapshots\" WHERE operation = 'delete'")) + .isEqualTo(0L); + assertThat(computeScalar("SELECT operation FROM \"" + targetName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1")) + .isEqualTo("overwrite"); + + assertQuery("SELECT count(*) FROM " + targetName + " WHERE name LIKE 'MERGED_%'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE nationkey = 100", "VALUES 1"); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 26"); + } + } + + // ----------------------------------------------------------------------- + // End-to-end scenario: full MoR → CoW migration workflow + // ----------------------------------------------------------------------- + + @Test + public void testFullMorToCowMigration() + { + // Realistic migration: table starts as MoR, accumulates deletes, then migrates to CoW + String tableName = "test_full_mor_to_cow_" + randomNameSuffix(); + try { + // Phase 1: MoR operations + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (0, 1)", 2); + assertUpdate("UPDATE " + tableName + " SET name = 'MOR_EDIT' WHERE nationkey = 5", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 23"); + + // Verify MoR produced delete files + long morDeleteFiles = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(morDeleteFiles).isGreaterThanOrEqualTo(1); + + // Phase 2: Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])"); + + // Phase 3: CoW operations (must not resurrect MoR-deleted rows) + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 10", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (0, 1, 10)", "VALUES 0"); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 5", "VALUES 'MOR_EDIT'"); + + assertUpdate("UPDATE " + tableName + " SET name = 'COW_EDIT' WHERE nationkey = 15", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 15", "VALUES 'COW_EDIT'"); + + // Final integrity + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (0, 1, 10)", "VALUES 0"); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 5", "VALUES 'MOR_EDIT'"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowOnPartitionedTableWithMorDeletesInDifferentPartitions() + { + // MoR deletes in partition A, then CoW delete in partition B + String tableName = "test_cow_partitioned_mor_cow_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2, partitioning = ARRAY['regionkey']) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete in partition regionkey=0 + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete in different partition regionkey=1 + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + + // Neither partition's deleted rows should be present + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey IN (0, 1)", "VALUES 0"); + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY 1", "VALUES 2, 3, 4"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowOnPartitionedTableWithMorDeletesInSamePartition() + { + // MoR deletes in partition A, then CoW delete also in partition A (same data file!) + String tableName = "test_cow_same_partition_mor_cow_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2, partitioning = ARRAY['regionkey']) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete of specific rows in regionkey=1 (2 of 5 rows) + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (1, 2)", 2); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 23"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete of OTHER rows in the same partition regionkey=1 + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 3", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + + // All three deletes honored: nationkeys 1, 2, 3 gone + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (1, 2, 3)", "VALUES 0"); + // Remaining regionkey=1 rows: nationkeys that are in regionkey=1 but NOT 1,2,3 + // Nation data: regionkey=1 has nationkeys 1,2,3,17,24 + assertQuery("SELECT nationkey FROM " + tableName + " WHERE regionkey = 1 ORDER BY 1", "VALUES 17, 24"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowDeleteAllRowsFromFileWithMorDeletes() + { + // MoR deletes some rows, CoW deletes the remaining rows from the same file + // The result should be: file completely removed (no new file emitted) + String tableName = "test_cow_delete_remaining_after_mor_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2, partitioning = ARRAY['regionkey']) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete: remove 3 of 5 rows from regionkey=2 + // regionkey=2 has nationkeys: 8,9,12,18,21 + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (8, 9, 12)", 3); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 2", "VALUES 2"); + + // Switch to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete: remove the remaining 2 rows from regionkey=2 + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 2", 2); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 2", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowV3WithDvsAndMixedFormat() + { + // Combined test: v3 DVs + mixed format + CoW + String tableName = "test_cow_v3_dvs_mixed_" + randomNameSuffix(); + try { + // Create v3 table with ORC + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 3, format = 'ORC') " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete (creates DVs in v3) + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Switch format to PARQUET and mode to CoW + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES format = 'PARQUET'"); + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete: reads ORC source with DVs merged, writes PARQUET + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + + // Both sets of deleted rows still gone + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey IN (0, 1)", "VALUES 0"); + assertQuery("SELECT DISTINCT regionkey FROM " + tableName + " ORDER BY 1", "VALUES 2, 3, 4"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Schema evolution: ADD/DROP columns with CoW operations + // ----------------------------------------------------------------------- + + @Test + public void testCowAfterAddColumn() + { + String tableName = "test_cow_add_column_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey FROM tpch.tiny.nation", 25); + + // Add a new column after data already exists + assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN score DOUBLE"); + + // CoW DELETE after schema evolution -- rewriter must handle the new column (reads as NULL for old data) + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // CoW UPDATE setting the new column + assertUpdate("UPDATE " + tableName + " SET score = 100.0 WHERE regionkey = 1", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE score = 100.0e0", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowAfterDropColumn() + { + String tableName = "test_cow_drop_column_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation", 25); + + // Drop a column + assertUpdate("ALTER TABLE " + tableName + " DROP COLUMN comment"); + + // CoW DELETE after dropping a column -- rewriter reads with evolved schema + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 2", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Verify the dropped column is not in the result + assertQuery("SELECT nationkey, name, regionkey FROM " + tableName + " WHERE regionkey = 1 ORDER BY nationkey", + "SELECT nationkey, name, regionkey FROM nation WHERE regionkey = 1 ORDER BY nationkey"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Nested types: MAP, ARRAY, ROW columns + // ----------------------------------------------------------------------- + + @Test + public void testCowWithNestedTypes() + { + try (TestTable table = newCowTable("test_cow_nested_", + "(id INTEGER, tags ARRAY(VARCHAR), attrs MAP(VARCHAR, VARCHAR), info ROW(x INTEGER, y VARCHAR)) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, ARRAY['a', 'b'], MAP(ARRAY['k1'], ARRAY['v1']), ROW(10, 'x')), " + + "(2, ARRAY['c'], MAP(ARRAY['k2'], ARRAY['v2']), ROW(20, 'y')), " + + "(3, ARRAY['d', 'e', 'f'], MAP(ARRAY['k3'], ARRAY['v3']), ROW(30, 'z'))", 3); + + // CoW DELETE with nested types + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 2"); + + // CoW UPDATE on a table with nested types + assertUpdate("UPDATE " + tableName + " SET tags = ARRAY['updated'] WHERE id = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT tags FROM " + tableName + " WHERE id = 1", "VALUES ARRAY['updated']"); + + // Verify other nested columns survived the rewrite intact + assertQuery("SELECT info.x FROM " + tableName + " WHERE id = 3", "VALUES 30"); + } + } + + // ----------------------------------------------------------------------- + // Partition evolution: change partition spec then CoW + // ----------------------------------------------------------------------- + + @Test + public void testCowAfterPartitionEvolution() + { + String tableName = "test_cow_partition_evolution_" + randomNameSuffix(); + try { + // Create with partitioning by regionkey + assertUpdate("CREATE TABLE " + tableName + + " WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // Insert more data with current partition spec + assertUpdate("INSERT INTO " + tableName + " VALUES (100, 'EXTRA', 0, 'extra')", 1); + + // Change partition spec (evolution) + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES partitioning = ARRAY['nationkey']"); + + // Insert with new partition spec + assertUpdate("INSERT INTO " + tableName + " VALUES (101, 'NEW_SPEC', 1, 'new spec')", 1); + + // CoW DELETE touching data from old partition spec + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 100", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey = 100", "VALUES 0"); + + // Verify data with new partition spec is intact + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 101", "VALUES 'NEW_SPEC'"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 26"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // UPDATE that changes partition column value + // ----------------------------------------------------------------------- + + @Test + public void testCowUpdatePartitionColumn() + { + try (TestTable table = newCowTable("test_cow_update_partition_col_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // UPDATE that moves rows to a different partition + assertUpdate("UPDATE " + tableName + " SET regionkey = 99 WHERE nationkey = 0", 1); + assertNoDeleteFiles(tableName); + + // Row moved from partition 0 to partition 99 + assertQuery("SELECT regionkey FROM " + tableName + " WHERE nationkey = 0", "VALUES 99"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 99", "VALUES 1"); + } + } + + // ----------------------------------------------------------------------- + // DELETE with complex JOIN predicate + // ----------------------------------------------------------------------- + + @Test + public void testCowDeleteWithJoinPredicate() + { + try (TestTable table = newCowTable("test_cow_join_delete_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // DELETE using EXISTS with a correlated subquery + assertUpdate("DELETE FROM " + tableName + + " WHERE EXISTS (SELECT 1 FROM tpch.tiny.region r WHERE r.regionkey = " + tableName + ".regionkey AND r.name = 'AFRICA')", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + // AFRICA is regionkey=0 + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 0"); + } + } + + // ----------------------------------------------------------------------- + // AVRO format (if supported) + // ----------------------------------------------------------------------- + + @Test + public void testCowWithAvroFormat() + { + String tableName = "test_cow_avro_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format = 'AVRO', extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation", 25); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 0"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowWithOrcFormat() + { + String tableName = "test_cow_orc_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format = 'ORC', extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation", 25); + + // CoW DELETE on ORC table + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // CoW UPDATE on ORC table + assertUpdate("UPDATE " + tableName + " SET name = 'ORC_UPDATED' WHERE nationkey = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 1", "VALUES 'ORC_UPDATED'"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Large single-row delete (only 1 row deleted from a large file) + // Verifies CoW works for minimal changes + // ----------------------------------------------------------------------- + + @Test + public void testCowDeleteSingleRow() + { + try (TestTable table = newCowTable("test_cow_single_row_delete_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.orders")) { + String tableName = table.getName(); + + long initialCount = (long) computeScalar("SELECT count(*) FROM " + tableName); + + // Delete exactly one row + assertUpdate("DELETE FROM " + tableName + " WHERE orderkey = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES " + (initialCount - 1)); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE orderkey = 1", "VALUES 0"); + } + } + + // ----------------------------------------------------------------------- + // Multiple DML types in sequence (DELETE, UPDATE, MERGE, INSERT) + // Verifies CoW state doesn't leak between operations + // ----------------------------------------------------------------------- + + @Test + public void testCowMixedOperationTypes() + { + try (TestTable table = newCowTable("test_cow_mixed_ops_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // 1) DELETE + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 0", 1); + assertNoDeleteFiles(tableName); + + // 2) UPDATE + assertUpdate("UPDATE " + tableName + " SET name = 'UPDATED' WHERE nationkey = 5", 1); + assertNoDeleteFiles(tableName); + + // 3) INSERT (not CoW, just append) + assertUpdate("INSERT INTO " + tableName + " VALUES (100, 'NEW', 0, 'new')", 1); + + // 4) MERGE (delete + update + insert in one operation) + assertUpdate("MERGE INTO " + tableName + " t USING (VALUES (5, 'RE-UPDATED'), (200, 'INSERTED')) AS s(nk, nm) " + + "ON t.nationkey = s.nk " + + "WHEN MATCHED THEN UPDATE SET name = s.nm " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nk, s.nm, 0, 'merged')", 2); + assertNoDeleteFiles(tableName); + + // 5) DELETE again + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 100", 1); + assertNoDeleteFiles(tableName); + + // Final state: 25 - 1(del nationkey=0) + 1(insert 100) - 1(del 100) + 1(insert 200) = 25 + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 5", "VALUES 'RE-UPDATED'"); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 200", "VALUES 'INSERTED'"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (0, 100)", "VALUES 0"); + } + } + + // ----------------------------------------------------------------------- + // Verify CoW with WHERE clause that matches no files (predicate pruning) + // ----------------------------------------------------------------------- + + @Test + public void testCowDeletePredPrunesAllFiles() + { + try (TestTable table = newCowTable("test_cow_pred_prune_", + "WITH (partitioning = ARRAY['regionkey'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + Set filesBefore = getDataFilePaths(tableName); + + // Delete with a predicate that matches rows but from only one partition + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 3 AND nationkey > 100", 0); + + // No files should have changed since no rows matched + Set filesAfter = getDataFilePaths(tableName); + assertThat(filesAfter).isEqualTo(filesBefore); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + } + } + + // ----------------------------------------------------------------------- + // Verify split offsets are preserved on CoW rewritten files + // ----------------------------------------------------------------------- + + @Test + public void testCowPreservesSplitOffsets() + { + try (TestTable table = newCowTable("test_cow_split_offsets_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Verify split offsets exist on the original data files + long originalFilesWithOffsets = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id() + " AND split_offsets IS NOT NULL"); + assertThat(originalFilesWithOffsets).isGreaterThan(0); + + // Perform a CoW DELETE that rewrites data files + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertNoDeleteFiles(tableName); + + // Verify rewritten files still have split offsets populated + long rewrittenFilesWithOffsets = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id() + " AND split_offsets IS NOT NULL"); + long totalRewrittenFiles = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(totalRewrittenFiles).isGreaterThan(0); + assertThat(rewrittenFilesWithOffsets).isEqualTo(totalRewrittenFiles); + + // Verify data correctness + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE regionkey != 1"); + } + } + + // ----------------------------------------------------------------------- + // Verify dangling deletion vectors are cleaned up after CoW rewrite + // ----------------------------------------------------------------------- + + @Test + public void testCowCleansDanglingDeletionVectors() + { + try (TestTable table = newCowTable("test_cow_dv_cleanup_", + "WITH (format_version = 3) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Step 1: perform MoR deletes to create deletion vectors + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 0", 1); + long dvCountAfterMor = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(dvCountAfterMor).isGreaterThan(0); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 24"); + + // Step 2: switch to CoW mode and perform another DELETE on the same partition + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES extra_properties = MAP(" + + "ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 1", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 23"); + + // Step 3: the CoW rewrite should have cleaned up the dangling DVs. + // After CoW, no delete files should remain because: + // - the old DVs from step 1 are removed (dangling DV cleanup) + // - CoW produces new data files without delete files + long dvCountAfterCow = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(dvCountAfterCow).isEqualTo(0); + + // Verify data is fully correct + assertQuery("SELECT * FROM " + tableName, "SELECT * FROM nation WHERE nationkey NOT IN (0, 1)"); + } + } + + @Test + public void testCowCleansDanglingDvsMultipleFiles() + { + try (TestTable table = newCowTable("test_cow_dv_multi_", + "WITH (format_version = 3, partitioning = ARRAY['regionkey']) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Create DVs in multiple partitions via MoR deletes + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (0, 5, 10, 15, 20)", 5); + long dvCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(dvCount).isGreaterThan(0); + + // Switch to CoW and delete from the same partitions + assertUpdate("ALTER TABLE " + tableName + " SET PROPERTIES extra_properties = MAP(" + + "ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey IN (1, 6, 11, 16, 21)", 5); + + // All old DVs should be cleaned up, no new delete files from CoW + long dvCountAfter = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(dvCountAfter).isEqualTo(0); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + assertQuery("SELECT * FROM " + tableName, + "SELECT * FROM nation WHERE nationkey NOT IN (0, 1, 5, 6, 10, 11, 15, 16, 20, 21)"); + } + } + + // ----------------------------------------------------------------------- + // Partition transform tests: bucket and truncate + // ----------------------------------------------------------------------- + + @Test + public void testCowWithBucketPartitioning() + { + String tableName = "test_cow_bucket_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " (id BIGINT, name VARCHAR, amount DOUBLE) " + + "WITH (partitioning = ARRAY['bucket(id, 4)'], " + + "extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))"); + + assertUpdate("INSERT INTO " + tableName + + " SELECT nationkey, name, CAST(nationkey * 10 AS DOUBLE) FROM tpch.tiny.nation", 25); + + // CoW DELETE across buckets + assertUpdate("DELETE FROM " + tableName + " WHERE id < 5", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // CoW UPDATE across buckets + assertUpdate("UPDATE " + tableName + " SET amount = 999.0 WHERE id >= 20", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE amount = 999.0e0", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowWithTruncatePartitioning() + { + String tableName = "test_cow_truncate_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " (id BIGINT, category VARCHAR, value DOUBLE) " + + "WITH (partitioning = ARRAY['truncate(category, 2)'], " + + "extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))"); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'ALPHA', 10.0), (2, 'ALPHA_LONG', 20.0), " + + "(3, 'BETA', 30.0), (4, 'BETA_LONG', 40.0), " + + "(5, 'GAMMA', 50.0), (6, 'GAMMA_LONG', 60.0)", 6); + + // DELETE from one truncated partition (AL -> ALPHA and ALPHA_LONG) + assertUpdate("DELETE FROM " + tableName + " WHERE category LIKE 'ALPHA%'", 2); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 4"); + + // UPDATE in another truncated partition + assertUpdate("UPDATE " + tableName + " SET value = 0.0 WHERE category LIKE 'BETA%'", 2); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE value = 0.0e0", "VALUES 2"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 4"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // OPTIMIZE after CoW operations + // ----------------------------------------------------------------------- + + @Test + public void testCowAfterManySnapshotsAndFiles() + { + // Verifies CoW stays correct after accumulating many snapshots and file rewrites + String tableName = "test_cow_many_snapshots_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // Multiple CoW operations to accumulate snapshots + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertUpdate("UPDATE " + tableName + " SET name = 'X' WHERE regionkey = 1", 5); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 8", 1); + + long snapshotCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertThat(snapshotCount).isGreaterThanOrEqualTo(4); + + // Insert more data, then CoW delete again + assertUpdate("INSERT INTO " + tableName + " VALUES (100, 'EXTRA', 0, 'extra')", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 100", 1); + assertNoDeleteFiles(tableName); + + // Final CoW operation after many snapshots still works correctly + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 14"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey = 0", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'X'", "VALUES 5"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Edge cases: UPDATE to NULL, DML on empty table, repeated same-row updates + // ----------------------------------------------------------------------- + + @Test + public void testCowUpdateSetToNull() + { + try (TestTable table = newCowTable("test_cow_set_null_", + "(id INTEGER, name VARCHAR, score DOUBLE) " + + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'alice', 95.0), (2, 'bob', 80.0), (3, 'charlie', 70.0)", 3); + + // UPDATE to set values to NULL + assertUpdate("UPDATE " + tableName + " SET name = NULL, score = NULL WHERE id = 2", 1); + assertNoDeleteFiles(tableName); + + assertQuery("SELECT name, score FROM " + tableName + " WHERE id = 2", "VALUES (NULL, NULL)"); + assertQuery("SELECT name, score FROM " + tableName + " WHERE id = 1", "VALUES ('alice', 95.0e0)"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 3"); + } + } + + @Test + public void testCowDeleteOnEmptyTable() + { + try (TestTable table = newCowTable("test_cow_empty_del_", + "(id INTEGER, name VARCHAR) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + // DELETE on empty table -- should be a no-op, no snapshot produced + long snapshotsBefore = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 0); + long snapshotsAfter = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertThat(snapshotsAfter).isEqualTo(snapshotsBefore); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 0"); + } + } + + @Test + public void testCowUpdateOnEmptyTable() + { + try (TestTable table = newCowTable("test_cow_empty_upd_", + "(id INTEGER, name VARCHAR) " + + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + long snapshotsBefore = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertUpdate("UPDATE " + tableName + " SET name = 'x' WHERE id = 1", 0); + long snapshotsAfter = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertThat(snapshotsAfter).isEqualTo(snapshotsBefore); + } + } + + @Test + public void testCowRepeatedUpdatesToSameRows() + { + try (TestTable table = newCowTable("test_cow_repeated_updates_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Update the same rows 5 times in succession + for (int i = 1; i <= 5; i++) { + assertUpdate("UPDATE " + tableName + " SET name = 'ITER_" + i + "' WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'ITER_" + i + "'", "VALUES 5"); + } + + // Final state: only the last update's value should exist + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'ITER_5'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name LIKE 'ITER_%'", "VALUES 5"); + // Other regions untouched + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey != 0", "VALUES 20"); + } + } + + @Test + public void testCowUpdateThenDeleteSameRow() + { + try (TestTable table = newCowTable("test_cow_update_then_delete_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // UPDATE a row + assertUpdate("UPDATE " + tableName + " SET name = 'DOOMED' WHERE nationkey = 5", 1); + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 5", "VALUES 'DOOMED'"); + assertNoDeleteFiles(tableName); + + // Now DELETE that same row + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 5", 1); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey = 5", "VALUES 0"); + assertNoDeleteFiles(tableName); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 24"); + } + } + + // ----------------------------------------------------------------------- + // Type preservation: DECIMAL, DATE, TIMESTAMP through CoW rewrite + // ----------------------------------------------------------------------- + + @Test + public void testCowPreservesDecimalDateTimestampTypes() + { + try (TestTable table = newCowTable("test_cow_types_", + "(id INTEGER, amount DECIMAL(18, 6), event_date DATE, event_ts TIMESTAMP(6)) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, CAST(123456.789012 AS DECIMAL(18,6)), DATE '2024-01-15', TIMESTAMP '2024-01-15 10:30:00.123456'), " + + "(2, CAST(999999.999999 AS DECIMAL(18,6)), DATE '2024-06-30', TIMESTAMP '2024-06-30 23:59:59.999999'), " + + "(3, CAST(0.000001 AS DECIMAL(18,6)), DATE '2024-12-31', TIMESTAMP '2024-12-31 00:00:00.000001'), " + + "(4, CAST(-100.500000 AS DECIMAL(18,6)), DATE '2023-01-01', TIMESTAMP '2023-01-01 12:00:00.000000')", 4); + + // DELETE one row, triggering CoW rewrite + assertUpdate("DELETE FROM " + tableName + " WHERE id = 4", 1); + assertNoDeleteFiles(tableName); + + // Verify types are preserved exactly after rewrite using computeScalar + assertThat(computeScalar("SELECT CAST(amount AS VARCHAR) FROM " + tableName + " WHERE id = 1")) + .isEqualTo("123456.789012"); + assertThat(computeScalar("SELECT CAST(amount AS VARCHAR) FROM " + tableName + " WHERE id = 2")) + .isEqualTo("999999.999999"); + assertThat(computeScalar("SELECT CAST(amount AS VARCHAR) FROM " + tableName + " WHERE id = 3")) + .isEqualTo("0.000001"); + assertThat(computeScalar("SELECT CAST(event_date AS VARCHAR) FROM " + tableName + " WHERE id = 1")) + .isEqualTo("2024-01-15"); + assertThat(computeScalar("SELECT CAST(event_date AS VARCHAR) FROM " + tableName + " WHERE id = 2")) + .isEqualTo("2024-06-30"); + + // UPDATE a row and verify types still preserved + assertUpdate("UPDATE " + tableName + " SET amount = CAST(555.555555 AS DECIMAL(18,6)) WHERE id = 1", 1); + assertNoDeleteFiles(tableName); + assertThat(computeScalar("SELECT CAST(amount AS VARCHAR) FROM " + tableName + " WHERE id = 1")) + .isEqualTo("555.555555"); + assertThat(computeScalar("SELECT CAST(event_ts AS VARCHAR) FROM " + tableName + " WHERE id = 2")) + .isEqualTo("2024-06-30 23:59:59.999999"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 3"); + } + } + + // ----------------------------------------------------------------------- + // Scalability: many data files, large number of chained operations + // ----------------------------------------------------------------------- + + @Test + public void testCowWithManyDataFiles() + { + String tableName = "test_cow_many_files_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " (id BIGINT, region VARCHAR, value DOUBLE) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))"); + + // Create 10 separate data files via separate inserts + for (int i = 0; i < 10; i++) { + int baseId = i * 10; + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(" + (baseId + 1) + ", 'R" + (i % 3) + "', " + (baseId + 1) + ".0), " + + "(" + (baseId + 2) + ", 'R" + (i % 3) + "', " + (baseId + 2) + ".0), " + + "(" + (baseId + 3) + ", 'R" + (i % 3) + "', " + (baseId + 3) + ".0)", 3); + } + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 30"); + long fileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(fileCount).isGreaterThanOrEqualTo(10); + + // CoW DELETE across multiple files (ids 1-10 span multiple files) + assertUpdate("DELETE FROM " + tableName + " WHERE id <= 10", 3); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 27"); + + // CoW UPDATE across multiple files + assertUpdate("UPDATE " + tableName + " SET value = -1.0 WHERE region = 'R0'", 9); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE value = -1.0e0", "VALUES 9"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 27"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testCowDataIntegrityThroughChainedOperations() + { + // Stress test: many chained CoW operations with full data integrity verification + try (TestTable table = newCowTable("test_cow_integrity_", + "(id BIGINT, category VARCHAR, value BIGINT) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + // Insert 50 rows + StringBuilder insertValues = new StringBuilder(); + for (int i = 1; i <= 50; i++) { + if (i > 1) { + insertValues.append(", "); + } + insertValues.append("(").append(i).append(", 'CAT_").append(i % 5).append("', ").append(i * 100).append(")"); + } + assertUpdate("INSERT INTO " + tableName + " VALUES " + insertValues, 50); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 50"); + + // Chain of operations -- counts verified against nation data model + // CAT_0: 5,10,15,20,25,30,35,40,45,50; CAT_1: 1,6,11,16,21,26,31,36,41,46 + // CAT_2: 2,7,12,17,22,27,32,37,42,47; CAT_3: 3,8,13,18,23,28,33,38,43,48 + // CAT_4: 4,9,14,19,24,29,34,39,44,49 + assertUpdate("DELETE FROM " + tableName + " WHERE id <= 5", 5); // 45 rows; removed 1(C1),2(C2),3(C3),4(C4),5(C0) + assertUpdate("UPDATE " + tableName + " SET value = 0 WHERE category = 'CAT_0'", 9); // 45 rows; CAT_0 left: 10,15,20,25,30,35,40,45,50 + assertUpdate("DELETE FROM " + tableName + " WHERE id IN (10, 15, 20)", 3); // 42 rows + assertUpdate("INSERT INTO " + tableName + " VALUES (100, 'CAT_NEW', 9999)", 1); // 43 rows + assertUpdate("UPDATE " + tableName + " SET category = 'CAT_X' WHERE id > 40 AND id <= 50", 10); // 43 rows + assertUpdate("DELETE FROM " + tableName + " WHERE id = 100", 1); // 42 rows + assertUpdate("INSERT INTO " + tableName + " VALUES (200, 'CAT_Y', 8888), (201, 'CAT_Y', 7777)", 2); // 44 rows + assertUpdate("UPDATE " + tableName + " SET value = value + 1 WHERE category = 'CAT_X'", 10); // 44 rows + // CAT_1 remaining: 6,11,16,21,26,31,36 (41,46 were changed to CAT_X in step 5) = 7 rows + assertUpdate("DELETE FROM " + tableName + " WHERE category = 'CAT_1'", 7); // 37 rows + assertUpdate("UPDATE " + tableName + " SET value = -1 WHERE id IN (200, 201)", 2); // 37 rows + + // Final data integrity checks + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 37"); + assertNoDeleteFiles(tableName); + + // Verify no duplicate IDs exist + assertQuery("SELECT count(DISTINCT id) FROM " + tableName, "VALUES 37"); + + // Verify specific values + assertQuery("SELECT count(*) FROM " + tableName + " WHERE category = 'CAT_1'", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE category = 'CAT_X'", "VALUES 10"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE category = 'CAT_Y'", "VALUES 2"); + assertQuery("SELECT value FROM " + tableName + " WHERE id = 200", "VALUES -1"); + } + } + + // ----------------------------------------------------------------------- + // MERGE with all three clause types simultaneously + // ----------------------------------------------------------------------- + + @Test + public void testCowMergeAllThreeClauseTypes() + { + try (TestTable target = newCowTable("test_cow_merge_all_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT nationkey, name, regionkey, comment FROM tpch.tiny.nation"); + TestTable source = newCowTable("test_cow_merge_all_source_", + "AS SELECT * FROM (VALUES " + + "(0, 'UPDATED_0', 0, 'updated comment', 'U'), " + + "(1, 'DELETE_ME', 1, 'to delete', 'D'), " + + "(2, 'DELETE_ME_TOO', 1, 'also delete', 'D'), " + + "(98, 'NEW_98', 4, 'brand new 98', 'I'), " + + "(99, 'NEW_99', 4, 'brand new 99', 'I')) " + + "AS t(nationkey, name, regionkey, comment, op)")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + // MERGE with UPDATE + DELETE + INSERT simultaneously + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN MATCHED AND s.op = 'U' THEN UPDATE SET name = s.name, comment = s.comment " + + "WHEN MATCHED AND s.op = 'D' THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", 5); + + assertNoDeleteFiles(targetName); + // 25 - 2 deleted + 2 inserted = 25 + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 25"); + assertQuery("SELECT name FROM " + targetName + " WHERE nationkey = 0", "VALUES 'UPDATED_0'"); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE nationkey IN (1, 2)", "VALUES 0"); + assertQuery("SELECT name FROM " + targetName + " WHERE nationkey = 98", "VALUES 'NEW_98'"); + assertQuery("SELECT name FROM " + targetName + " WHERE nationkey = 99", "VALUES 'NEW_99'"); + + // Verify no duplicate IDs + assertQuery("SELECT count(DISTINCT nationkey) FROM " + targetName, "VALUES 25"); + } + } + + // ----------------------------------------------------------------------- + // Format version explicit tests: v2 and v3 direct CoW (no MoR migration) + // ----------------------------------------------------------------------- + + @Test + public void testCowDirectOnV3Table() + { + // Create v3 table in CoW mode from the start -- verify no DVs are created + String tableName = "test_cow_direct_v3_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 3, " + + "extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // CoW DELETE should NOT produce DVs on a v3 table when CoW mode is active + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // CoW UPDATE + assertUpdate("UPDATE " + tableName + " SET name = 'V3_COW' WHERE regionkey = 1", 5); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'V3_COW'", "VALUES 5"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 20"); + + // Verify snapshot operations are 'overwrite' (not 'delete' which would indicate DV) + long overwriteSnapshots = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$snapshots\" WHERE operation = 'overwrite'"); + assertThat(overwriteSnapshots).isGreaterThanOrEqualTo(1); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // P0 Severity: Data integrity -- no duplication, no loss, no corruption + // ----------------------------------------------------------------------- + + @Test + public void testCowNoDataDuplicationAfterMultipleOperations() + { + // Verify that CoW rewrites never duplicate rows, even through many operations + try (TestTable table = newCowTable("test_cow_no_dup_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Perform many operations that touch overlapping data + assertUpdate("UPDATE " + tableName + " SET comment = 'round1' WHERE regionkey = 0", 5); + assertUpdate("UPDATE " + tableName + " SET comment = 'round2' WHERE regionkey = 0", 5); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 0", 1); + assertUpdate("INSERT INTO " + tableName + " VALUES (0, 'RESURRECTED', 0, 'reinserted')", 1); + assertUpdate("UPDATE " + tableName + " SET comment = 'round3' WHERE nationkey = 0", 1); + + // CRITICAL: verify no duplicates exist + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + long distinctCount = (long) computeScalar("SELECT count(DISTINCT nationkey) FROM " + tableName); + long totalCount = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertThat(distinctCount).isEqualTo(totalCount); + + // Verify specific values + assertQuery("SELECT comment FROM " + tableName + " WHERE nationkey = 0", "VALUES 'round3'"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey = 0", "VALUES 1"); + } + } + + @Test + public void testCowNoDataLossOnPartialDelete() + { + // Verify that deleting from one file doesn't affect rows in other files + try (TestTable table = newCowTable("test_cow_no_loss_", + "(id BIGINT, data VARCHAR) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + // Insert in separate batches to create separate data files + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'file1_row1'), (2, 'file1_row2')", 2); + assertUpdate("INSERT INTO " + tableName + " VALUES (3, 'file2_row1'), (4, 'file2_row2')", 2); + assertUpdate("INSERT INTO " + tableName + " VALUES (5, 'file3_row1'), (6, 'file3_row2')", 2); + + // Delete from one file only + assertUpdate("DELETE FROM " + tableName + " WHERE id IN (1, 2)", 2); + assertNoDeleteFiles(tableName); + + // Verify other files' data is completely intact + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 4"); + assertQuery("SELECT data FROM " + tableName + " WHERE id = 3", "VALUES 'file2_row1'"); + assertQuery("SELECT data FROM " + tableName + " WHERE id = 4", "VALUES 'file2_row2'"); + assertQuery("SELECT data FROM " + tableName + " WHERE id = 5", "VALUES 'file3_row1'"); + assertQuery("SELECT data FROM " + tableName + " WHERE id = 6", "VALUES 'file3_row2'"); + } + } + + @Test + public void testCowFileCountReflectsOperations() + { + // P1: Verify file counts are consistent -- no orphan files in metadata + try (TestTable table = newCowTable("test_cow_file_count_", + "(id BIGINT, region BIGINT) " + + "WITH (partitioning = ARRAY['region'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 0), (2, 0), (3, 0)", 3); + assertUpdate("INSERT INTO " + tableName + " VALUES (4, 1), (5, 1), (6, 1)", 3); + assertUpdate("INSERT INTO " + tableName + " VALUES (7, 2), (8, 2), (9, 2)", 3); + + // Delete all rows from region=1 (entire file should be removed) + assertUpdate("DELETE FROM " + tableName + " WHERE region = 1", 3); + assertNoDeleteFiles(tableName); + + // Verify file count: region 0 and 2 should have files, region 1 should not + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 6"); + long dataFileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + // Should have exactly 2 data files (one per remaining region) + // Region 0 was not touched (original file), Region 2 was not touched + // Region 1 file was removed (all rows deleted) + assertThat(dataFileCount).isEqualTo(2); + + // Partial delete from another region (file should be rewritten, not removed) + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 5"); + + long finalFileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(finalFileCount).isEqualTo(2); + } + } + + @Test + public void testCowRewrittenFilesSmallerAfterDelete() + { + // P2: Verify that rewritten files are smaller when rows are deleted + try (TestTable table = newCowTable("test_cow_file_size_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.orders")) { + String tableName = table.getName(); + + long originalTotalSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + + // Delete a significant chunk of data + long deletedCount = (long) computeScalar("SELECT count(*) FROM " + tableName + " WHERE orderstatus = 'F'"); + assertUpdate("DELETE FROM " + tableName + " WHERE orderstatus = 'F'", deletedCount); + assertNoDeleteFiles(tableName); + + long newTotalSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + + // Rewritten files should be smaller (less data) + assertThat(newTotalSize).isLessThan(originalTotalSize); + + // All file sizes should be positive (no zero-byte files) + computeActual("SELECT file_size_in_bytes FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()) + .getMaterializedRows() + .forEach(row -> assertThat((Long) row.getField(0)).isGreaterThan(0)); + } + } + + @Test + public void testCowV2HandlesPreExistingPositionDeletes() + { + // v2 CoW must remain correct when legacy file-scoped position deletes already exist. + String tableName = "test_cow_v2_skip_dv_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // MoR delete on v2 produces position delete files (not DVs) + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + long deleteFileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(deleteFileCount).isGreaterThanOrEqualTo(1); + + // Switch to CoW mode + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])"); + + // CoW delete should preserve correctness for v2 data with pre-existing deletes. + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 1", 5); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE regionkey IN (0, 1)", "VALUES 0"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // CoW with large update affecting most rows (near-complete rewrite) + // ----------------------------------------------------------------------- + + @Test + public void testCowUpdateAlmostAllRows() + { + try (TestTable table = newCowTable("test_cow_update_almost_all_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // UPDATE 24 out of 25 rows (leave only nationkey=0) + assertUpdate("UPDATE " + tableName + " SET name = 'BULK_UPDATE' WHERE nationkey > 0", 24); + assertNoDeleteFiles(tableName); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name = 'BULK_UPDATE'", "VALUES 24"); + // The one untouched row + assertQuery("SELECT name FROM " + tableName + " WHERE nationkey = 0", + "SELECT name FROM nation WHERE nationkey = 0"); + } + } + + // ----------------------------------------------------------------------- + // SCD Type 2: soft-close old version, insert new version + // ----------------------------------------------------------------------- + + @Test + public void testCowSlowlyChangingDimensionType2() + { + try (TestTable table = newCowTable("test_cow_scd2_", + "(customer_id BIGINT, name VARCHAR, email VARCHAR, is_current BOOLEAN, effective_date DATE, end_date DATE) " + + "WITH (partitioning = ARRAY['is_current'], " + + "extra_properties = MAP(ARRAY['write.update.mode', 'write.delete.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'Alice', 'alice@old.com', true, DATE '2023-01-01', NULL), " + + "(2, 'Bob', 'bob@co.com', true, DATE '2023-01-01', NULL), " + + "(3, 'Charlie', 'charlie@co.com', true, DATE '2023-06-01', NULL)", 3); + + // SCD2: Alice changed email -- soft-close old row, insert new version + assertUpdate("UPDATE " + tableName + + " SET is_current = false, end_date = DATE '2024-03-15'" + + " WHERE customer_id = 1 AND is_current = true", 1); + assertNoDeleteFiles(tableName); + assertUpdate("INSERT INTO " + tableName + + " VALUES (1, 'Alice', 'alice@new.com', true, DATE '2024-03-15', NULL)", 1); + + assertQuery("SELECT count(*) FROM " + tableName + " WHERE customer_id = 1", "VALUES 2"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE customer_id = 1 AND is_current = true", "VALUES 1"); + assertQuery("SELECT email FROM " + tableName + " WHERE customer_id = 1 AND is_current = true", + "VALUES 'alice@new.com'"); + assertQuery("SELECT email FROM " + tableName + " WHERE customer_id = 1 AND is_current = false", + "VALUES 'alice@old.com'"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 4"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE is_current = true", "VALUES 3"); + } + } + + // ----------------------------------------------------------------------- + // GDPR right-to-be-forgotten: delete user across all partitions + // ----------------------------------------------------------------------- + + @Test + public void testCowGdprDeleteAcrossAllPartitions() + { + try (TestTable table = newCowTable("test_cow_gdpr_", + "(user_id BIGINT, event_type VARCHAR, region VARCHAR, data VARCHAR) " + + "WITH (partitioning = ARRAY['region'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'login', 'US', 'data1'), (1, 'purchase', 'EU', 'data2'), (1, 'click', 'APAC', 'data3'), " + + "(2, 'login', 'US', 'data4'), (2, 'purchase', 'EU', 'data5'), " + + "(3, 'login', 'US', 'data6'), (3, 'click', 'APAC', 'data7')", 7); + + assertUpdate("DELETE FROM " + tableName + " WHERE user_id = 1", 3); + assertNoDeleteFiles(tableName); + + assertQuery("SELECT count(*) FROM " + tableName + " WHERE user_id = 1", "VALUES 0"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 4"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE user_id = 2", "VALUES 2"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE user_id = 3", "VALUES 2"); + assertQuery("SELECT count(DISTINCT region) FROM " + tableName, "VALUES 3"); + } + } + + // ----------------------------------------------------------------------- + // Dimension full refresh: DELETE all rows + re-populate + // ----------------------------------------------------------------------- + + @Test + public void testCowDimensionFullRefresh() + { + try (TestTable table = newCowTable("test_cow_full_refresh_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + + // DELETE with no WHERE clause + assertUpdate("DELETE FROM " + tableName, 25); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 0"); + + // Re-populate + assertUpdate("INSERT INTO " + tableName + + " SELECT nationkey, 'REFRESHED_' || name, regionkey, 'fresh' FROM tpch.tiny.nation", 25); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 25"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE name LIKE 'REFRESHED_%'", "VALUES 25"); + assertQuery("SELECT count(DISTINCT nationkey) FROM " + tableName, "VALUES 25"); + } + } + + // ----------------------------------------------------------------------- + // Late-arriving fact update with date partitions + // ----------------------------------------------------------------------- + + @Test + public void testCowLateArrivingFactUpdate() + { + try (TestTable table = newCowTable("test_cow_late_fact_", + "(order_id BIGINT, order_date DATE, amount DOUBLE, status VARCHAR) " + + "WITH (partitioning = ARRAY['order_date'], " + + "extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, DATE '2024-01-15', 100.0, 'SHIPPED'), " + + "(2, DATE '2024-01-15', 200.0, 'SHIPPED'), " + + "(3, DATE '2024-02-20', 300.0, 'SHIPPED'), " + + "(4, DATE '2024-02-20', 400.0, 'PENDING'), " + + "(5, DATE '2024-03-10', 500.0, 'SHIPPED')", 5); + + // Late-arriving correction to old partition + assertUpdate("UPDATE " + tableName + + " SET amount = 250.0, status = 'CORRECTED'" + + " WHERE order_date = DATE '2024-01-15' AND order_id = 2", 1); + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 5"); + assertQuery("SELECT amount FROM " + tableName + " WHERE order_id = 2", "VALUES 250.0e0"); + assertQuery("SELECT status FROM " + tableName + " WHERE order_id = 2", "VALUES 'CORRECTED'"); + // Other partitions untouched + assertQuery("SELECT amount FROM " + tableName + " WHERE order_id = 3", "VALUES 300.0e0"); + assertQuery("SELECT amount FROM " + tableName + " WHERE order_id = 5", "VALUES 500.0e0"); + } + } + + // ----------------------------------------------------------------------- + // Incremental load with deduplication via conditional MERGE + // ----------------------------------------------------------------------- + + @Test + public void testCowIncrementalLoadWithDeduplication() + { + try (TestTable table = newCowTable("test_cow_dedup_target_", + "(id BIGINT, name VARCHAR, updated_at DATE, value DOUBLE) " + + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write']))"); + TestTable staging = newCowTable("test_cow_dedup_staging_", + "AS SELECT * FROM (VALUES " + + "(1, 'Alice_v2', DATE '2024-03-01', 200.0), " + + "(2, 'Bob_v1', DATE '2024-01-01', 100.0), " + + "(4, 'Diana', DATE '2024-03-01', 400.0)) " + + "AS t(id, name, updated_at, value)")) { + String targetName = table.getName(); + String stagingName = staging.getName(); + + assertUpdate("INSERT INTO " + targetName + " VALUES " + + "(1, 'Alice_v1', DATE '2024-01-01', 100.0), " + + "(2, 'Bob_v1', DATE '2024-02-01', 150.0), " + + "(3, 'Charlie', DATE '2024-01-15', 300.0)", 3); + + // Only overwrite if staging row is newer + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + stagingName + " s ON t.id = s.id " + + "WHEN MATCHED AND s.updated_at > t.updated_at " + + " THEN UPDATE SET name = s.name, updated_at = s.updated_at, value = s.value " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.id, s.name, s.updated_at, s.value)", 2); + + assertNoDeleteFiles(targetName); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 4"); + // Alice updated (staging is newer) + assertQuery("SELECT name FROM " + targetName + " WHERE id = 1", "VALUES 'Alice_v2'"); + // Bob NOT updated (staging is older) + assertQuery("SELECT value FROM " + targetName + " WHERE id = 2", "VALUES 150.0e0"); + // Diana inserted + assertQuery("SELECT name FROM " + targetName + " WHERE id = 4", "VALUES 'Diana'"); + } + } + + // ----------------------------------------------------------------------- + // Upsert MERGE with larger source + // ----------------------------------------------------------------------- + + @Test + public void testCowMergeUpsertWithLargeSource() + { + try (TestTable target = newCowTable("test_cow_upsert_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT orderkey, orderstatus, totalprice, orderpriority, comment " + + "FROM tpch.tiny.orders WHERE orderkey <= 5000"); + TestTable source = newCowTable("test_cow_upsert_source_", + "AS SELECT orderkey, 'X' AS orderstatus, totalprice + 1.0 AS totalprice, " + + "orderpriority, 'upserted' AS comment " + + "FROM tpch.tiny.orders")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + long sourceCount = (long) computeScalar("SELECT count(*) FROM " + sourceName); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s ON t.orderkey = s.orderkey " + + "WHEN MATCHED THEN UPDATE SET orderstatus = s.orderstatus, comment = s.comment " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.orderkey, s.orderstatus, " + + "s.totalprice, s.orderpriority, s.comment)", sourceCount); + + assertNoDeleteFiles(targetName); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES " + sourceCount); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE comment = 'upserted'", "VALUES " + sourceCount); + assertQuery("SELECT count(DISTINCT orderkey) FROM " + targetName, "VALUES " + sourceCount); + } + } + + // ----------------------------------------------------------------------- + // MERGE with duplicate source keys -- should fail + // ----------------------------------------------------------------------- + + @Test + public void testCowMergeWithDuplicateSourceKeys() + { + try (TestTable target = newCowTable("test_cow_merge_dup_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey = 0"); + TestTable source = newCowTable("test_cow_merge_dup_source_", + "AS SELECT * FROM (VALUES (0, 'DUP_A'), (0, 'DUP_B'), (99, 'NEW')) " + + "AS t(nationkey, name)")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + assertQueryFails("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s ON t.nationkey = s.nationkey " + + "WHEN MATCHED THEN UPDATE SET name = s.name " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, 0, 'new')", + ".*One MERGE target table row matched more than one source row.*"); + + // Target data unchanged after failed MERGE + assertQuery("SELECT count(*) FROM " + targetName, "VALUES 5"); + assertQuery("SELECT * FROM " + targetName, "SELECT * FROM nation WHERE regionkey = 0"); + } + } + + // ----------------------------------------------------------------------- + // Wide table (50+ columns) + // ----------------------------------------------------------------------- + + @Test + public void testCowWithWideTable() + { + StringBuilder columns = new StringBuilder("(id BIGINT"); + StringBuilder insertRow1 = new StringBuilder("(1"); + StringBuilder insertRow2 = new StringBuilder("(2"); + for (int i = 1; i <= 50; i++) { + columns.append(", col_").append(i).append(" VARCHAR"); + insertRow1.append(", 'v").append(i).append("'"); + insertRow2.append(", 'w").append(i).append("'"); + } + columns.append(")"); + insertRow1.append(")"); + insertRow2.append(")"); + + try (TestTable table = newCowTable("test_cow_wide_", + columns + " WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + insertRow1 + ", " + insertRow2, 2); + + // UPDATE one column out of 51 + assertUpdate("UPDATE " + tableName + " SET col_25 = 'MODIFIED' WHERE id = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT col_25 FROM " + tableName + " WHERE id = 1", "VALUES 'MODIFIED'"); + assertQuery("SELECT col_1 FROM " + tableName + " WHERE id = 1", "VALUES 'v1'"); + assertQuery("SELECT col_50 FROM " + tableName + " WHERE id = 1", "VALUES 'v50'"); + + // DELETE + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 1"); + } + } + + // ----------------------------------------------------------------------- + // CoW-to-MoR-to-CoW alternation + // ----------------------------------------------------------------------- + + @Test + public void testCowToMorAndBackAlternation() + { + String tableName = "test_cow_mor_alternation_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " WITH (format_version = 2, " + + "extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation", 25); + + // CoW delete + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 0", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 24"); + + // Switch to MoR + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['merge-on-read', 'merge-on-read'])"); + + // MoR delete (creates position delete files) + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 5", 1); + long deleteFiles = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + POSITION_DELETES.id()); + assertThat(deleteFiles).isGreaterThanOrEqualTo(1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 23"); + + // Switch back to CoW + assertUpdate("ALTER TABLE " + tableName + + " SET PROPERTIES extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])"); + + // CoW delete must honor both prior deletes + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 10", 1); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 22"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE nationkey IN (0, 5, 10)", "VALUES 0"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // SUM/AVG aggregation integrity after CoW operations + // ----------------------------------------------------------------------- + + @Test + public void testCowAggregationIntegrityAfterOperations() + { + try (TestTable table = newCowTable("test_cow_agg_integrity_", + "(id BIGINT, amount DOUBLE, quantity INTEGER) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + + " SELECT nationkey, CAST(nationkey * 100 AS DOUBLE), CAST(nationkey AS INTEGER) FROM tpch.tiny.nation", 25); + + // Capture baseline SUM + double sumBefore = (double) computeScalar("SELECT SUM(amount) FROM " + tableName); + double deletedSum = (double) computeScalar("SELECT SUM(amount) FROM " + tableName + " WHERE id < 5"); + + // DELETE some rows + assertUpdate("DELETE FROM " + tableName + " WHERE id < 5", 5); + assertNoDeleteFiles(tableName); + + // Verify SUM decreased by exactly the deleted amount + double sumAfterDelete = (double) computeScalar("SELECT SUM(amount) FROM " + tableName); + assertThat(sumAfterDelete).isEqualTo(sumBefore - deletedSum); + + // UPDATE: double all amounts + long remaining = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertUpdate("UPDATE " + tableName + " SET amount = amount * 2", remaining); + assertNoDeleteFiles(tableName); + + // Verify SUM doubled + double sumAfterUpdate = (double) computeScalar("SELECT SUM(amount) FROM " + tableName); + assertThat(sumAfterUpdate).isEqualTo(sumAfterDelete * 2); + } + } + + // ----------------------------------------------------------------------- + // Anti-join verification: no phantom or missing rows + // ----------------------------------------------------------------------- + + @Test + public void testCowAntiJoinVerificationNoPhantomRows() + { + try (TestTable table = newCowTable("test_cow_phantom_check_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + assertUpdate("UPDATE " + tableName + " SET name = 'CHANGED' WHERE regionkey = 1", 5); + assertUpdate("DELETE FROM " + tableName + " WHERE nationkey = 8", 1); + + // Verify deleted rows do not exist (no phantom resurrections) + assertQuery("SELECT count(*) FROM " + tableName + + " WHERE nationkey IN (SELECT nationkey FROM tpch.tiny.nation WHERE regionkey = 0)" + + " OR nationkey = 8", "VALUES 0"); + + // Verify all expected rows still exist (no missing rows) + assertQuery( + "SELECT count(*) FROM tpch.tiny.nation n " + + "WHERE n.regionkey NOT IN (0) AND n.nationkey != 8 " + + "AND NOT EXISTS (SELECT 1 FROM " + tableName + " t WHERE t.nationkey = n.nationkey)", + "VALUES 0"); + } + } + + // ----------------------------------------------------------------------- + // UPDATE on partition transform column (bucket) + // ----------------------------------------------------------------------- + + @Test + public void testCowUpdateColumnUsedInBucketTransform() + { + String tableName = "test_cow_update_bucket_col_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " (id BIGINT, name VARCHAR, value DOUBLE) " + + "WITH (partitioning = ARRAY['bucket(id, 4)'], " + + "extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))"); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'a', 10.0), (2, 'b', 20.0), (3, 'c', 30.0), (4, 'd', 40.0), (5, 'e', 50.0)", 5); + + // UPDATE the bucketed column itself -- row moves to a different bucket + assertUpdate("UPDATE " + tableName + " SET id = 100 WHERE id = 1", 1); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 5"); + assertQuery("SELECT name FROM " + tableName + " WHERE id = 100", "VALUES 'a'"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE id = 1", "VALUES 0"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ----------------------------------------------------------------------- + // Many small files (50+) -- scalability test + // ----------------------------------------------------------------------- + + @Test + public void testCowWithManySmallFiles() + { + String tableName = "test_cow_small_files_" + randomNameSuffix(); + try { + assertUpdate("CREATE TABLE " + tableName + + " (id BIGINT, data VARCHAR) " + + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))"); + + // Create 30 separate data files + for (int i = 0; i < 30; i++) { + assertUpdate("INSERT INTO " + tableName + " VALUES (" + i + ", 'row_" + i + "')", 1); + } + + long fileCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(fileCount).isGreaterThanOrEqualTo(30); + + // CoW DELETE spanning many files + assertUpdate("DELETE FROM " + tableName + " WHERE id < 15", 15); + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15"); + assertQuery("SELECT count(DISTINCT id) FROM " + tableName, "VALUES 15"); + } + finally { + assertUpdate("DROP TABLE IF EXISTS " + tableName); + } + } + + // ======================================================================= + // PERFORMANCE MEASUREMENT TESTS + // ======================================================================= + + @Test + public void testCowPerformanceDeleteOnOrders() + { + // Performance baseline: measure CoW DELETE on tpch.tiny.orders (15000 rows) + try (TestTable table = newCowTable("test_cow_perf_del_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.orders")) { + String tableName = table.getName(); + + long initialCount = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertThat(initialCount).isEqualTo(15000L); + + long initialSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + + long deletedCount = (long) computeScalar( + "SELECT count(*) FROM " + tableName + " WHERE orderstatus = 'F'"); + + long startNanos = System.nanoTime(); + assertUpdate("DELETE FROM " + tableName + " WHERE orderstatus = 'F'", deletedCount); + long elapsedMs = (System.nanoTime() - startNanos) / 1_000_000; + + assertNoDeleteFiles(tableName); + long remainingCount = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertThat(remainingCount).isEqualTo(initialCount - deletedCount); + + long newSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + + // Verify rewritten files are smaller (less data) + assertThat(newSize).isLessThan(initialSize); + // All file sizes are positive + computeActual("SELECT file_size_in_bytes FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()) + .getMaterializedRows() + .forEach(row -> assertThat((Long) row.getField(0)).isGreaterThan(0)); + + // Log performance metrics for manual review + long snapshotCount = (long) computeScalar("SELECT count(*) FROM \"" + tableName + "$snapshots\""); + assertThat(snapshotCount).isGreaterThanOrEqualTo(2); + assertThat(elapsedMs).isLessThan(60_000L); + } + } + + @Test + public void testCowPerformanceUpdateOnOrders() + { + // Performance baseline: measure CoW UPDATE on tpch.tiny.orders (15000 rows) + try (TestTable table = newCowTable("test_cow_perf_upd_", + "WITH (extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.orders")) { + String tableName = table.getName(); + + long updateCount = (long) computeScalar( + "SELECT count(*) FROM " + tableName + " WHERE orderstatus = 'O'"); + + long startNanos = System.nanoTime(); + assertUpdate("UPDATE " + tableName + " SET orderpriority = '1-URGENT' WHERE orderstatus = 'O'", updateCount); + long elapsedMs = (System.nanoTime() - startNanos) / 1_000_000; + + assertNoDeleteFiles(tableName); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 15000"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE orderpriority = '1-URGENT' AND orderstatus = 'O'", + "VALUES " + updateCount); + + assertThat(elapsedMs).isLessThan(60_000L); + } + } + + @Test + public void testCowPerformanceMergeOnOrders() + { + // Performance baseline: measure CoW MERGE with tpch.tiny.orders + try (TestTable target = newCowTable("test_cow_perf_merge_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT orderkey, orderstatus, orderpriority, comment FROM tpch.tiny.orders WHERE orderkey <= 10000"); + TestTable source = newCowTable("test_cow_perf_merge_source_", + "AS SELECT orderkey, 'MERGED' AS orderstatus, orderpriority, 'merged' AS comment " + + "FROM tpch.tiny.orders WHERE orderkey > 5000 AND orderkey <= 12000")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + long sourceCount = (long) computeScalar("SELECT count(*) FROM " + sourceName); + + long startNanos = System.nanoTime(); + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s ON t.orderkey = s.orderkey " + + "WHEN MATCHED THEN UPDATE SET orderstatus = s.orderstatus, comment = s.comment " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.orderkey, s.orderstatus, s.orderpriority, s.comment)", + sourceCount); + long elapsedMs = (System.nanoTime() - startNanos) / 1_000_000; + + assertNoDeleteFiles(targetName); + long totalRows = (long) computeScalar("SELECT count(*) FROM " + targetName); + long distinctKeys = (long) computeScalar("SELECT count(DISTINCT orderkey) FROM " + targetName); + assertThat(distinctKeys).isEqualTo(totalRows); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE comment = 'merged'", "VALUES " + sourceCount); + + assertThat(elapsedMs).isLessThan(60_000L); + } + } + + // ======================================================================= + // OBSERVABILITY / METRICS TESTS + // ======================================================================= + + @Test + public void testCowSnapshotSummaryMetrics() + { + // Verify Iceberg snapshot summary contains expected metrics after CoW operations + try (TestTable table = newCowTable("test_cow_snapshot_metrics_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Initial INSERT snapshot + @SuppressWarnings("unchecked") + Map insertSummary = (Map) computeScalar( + "SELECT summary FROM \"" + tableName + "$snapshots\" ORDER BY committed_at ASC LIMIT 1"); + assertThat(insertSummary).containsKey("added-records"); + assertThat(insertSummary).containsKey("added-data-files"); + + // CoW DELETE + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + + // Check the overwrite snapshot has expected metrics + @SuppressWarnings("unchecked") + Map cowSummary = (Map) computeScalar( + "SELECT summary FROM \"" + tableName + "$snapshots\" WHERE operation = 'overwrite' ORDER BY committed_at DESC LIMIT 1"); + assertThat(cowSummary).containsKey("added-data-files"); + assertThat(cowSummary).containsKey("deleted-data-files"); + assertThat(cowSummary).containsKey("total-records"); + int deletedDataFiles = Integer.parseInt(cowSummary.get("deleted-data-files")); + assertThat(deletedDataFiles).isGreaterThanOrEqualTo(1); + long totalRecords = Long.parseLong(cowSummary.get("total-records")); + assertThat(totalRecords).isEqualTo(20L); + } + } + + @Test + public void testCowFileSizeMetricsInSystemTable() + { + // Verify $files system table accurately reflects CoW rewrite results + try (TestTable table = newCowTable("test_cow_file_metrics_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode', 'write.update.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + long initialTotalSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + + // DELETE half the rows + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey IN (0, 1)", 10); + + // All data files should have positive size + computeActual("SELECT file_size_in_bytes FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()) + .getMaterializedRows() + .forEach(row -> assertThat((Long) row.getField(0)).isGreaterThan(0)); + + // No delete files + assertNoDeleteFiles(tableName); + + // record_count in $files should match actual row counts + long filesRecordSum = (long) computeScalar( + "SELECT sum(record_count) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + long actualRows = (long) computeScalar("SELECT count(*) FROM " + tableName); + assertThat(filesRecordSum).isEqualTo(actualRows); + assertThat(actualRows).isEqualTo(15L); + + // Total size should decrease (fewer rows) + long newTotalSize = (long) computeScalar( + "SELECT sum(file_size_in_bytes) FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()); + assertThat(newTotalSize).isLessThan(initialTotalSize); + } + } + + @Test + public void testCowManifestMetrics() + { + // Verify $manifests system table tracks CoW file additions and deletions + try (TestTable table = newCowTable("test_cow_manifest_metrics_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 0", 5); + + // The latest manifest should show file additions from CoW + long addedFiles = (long) computeScalar( + "SELECT sum(added_data_files_count) FROM \"" + tableName + "$manifests\""); + assertThat(addedFiles).isGreaterThanOrEqualTo(1); + + // Verify no delete file manifests (CoW doesn't produce delete files) + long deleteManifests = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$manifests\" WHERE content = 1"); + assertThat(deleteManifests).isEqualTo(0); + } + } + + @Test + public void testCowWrittenBytesIncludesRewrites() + { + // Verify that EXPLAIN ANALYZE shows non-zero physical written bytes for CoW + try (TestTable table = newCowTable("test_cow_written_bytes_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // Run EXPLAIN ANALYZE to capture physical I/O stats + String explainOutput = (String) computeScalar("EXPLAIN ANALYZE DELETE FROM " + tableName + " WHERE regionkey = 4"); + + // Should report physical input bytes (reads from original files) + assertThat(explainOutput).contains("Physical input:"); + // Should report output bytes (CoW rewrites + any inserts) + assertThat(explainOutput).contains("Output:"); + } + } + + @Test + public void testCowPartitionMetrics() + { + // Verify $partitions reflects CoW changes per partition + try (TestTable table = newCowTable("test_cow_partition_metrics_", + "(id BIGINT, region VARCHAR, value DOUBLE) " + + "WITH (partitioning = ARRAY['region'], " + + "extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'US', 100.0), (2, 'US', 200.0), (3, 'EU', 300.0), " + + "(4, 'EU', 400.0), (5, 'APAC', 500.0)", 5); + + // Delete all rows from one partition + assertUpdate("DELETE FROM " + tableName + " WHERE region = 'US'", 2); + assertNoDeleteFiles(tableName); + + // $partitions should reflect the change + long partitionCount = (long) computeScalar( + "SELECT count(*) FROM \"" + tableName + "$partitions\""); + // EU and APAC remain; US may still appear with 0 rows depending on Iceberg version + assertThat(partitionCount).isGreaterThanOrEqualTo(2); + + // Verify row counts per remaining partition + assertQuery("SELECT count(*) FROM " + tableName + " WHERE region = 'EU'", "VALUES 2"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE region = 'APAC'", "VALUES 1"); + assertQuery("SELECT count(*) FROM " + tableName, "VALUES 3"); + } + } + + @Test + public void testCowSplitSourceMetricsInExplainAnalyze() + { + // Verify EXPLAIN ANALYZE VERBOSE exposes scan metrics relevant to CoW + try (TestTable table = newCowTable("test_cow_split_metrics_", + "WITH (extra_properties = MAP(ARRAY['write.delete.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation")) { + String tableName = table.getName(); + + // First do a CoW delete to create the post-rewrite table state + assertUpdate("DELETE FROM " + tableName + " WHERE regionkey = 4", 5); + assertNoDeleteFiles(tableName); + + // Now run EXPLAIN ANALYZE on a SELECT to see scan metrics + String explainOutput = (String) computeScalar( + "EXPLAIN ANALYZE VERBOSE SELECT count(*) FROM " + tableName); + + // Should contain scan planning metrics from IcebergSplitSource.getMetrics() + assertThat(explainOutput).contains("dataFiles"); + } + } + + @Test + public void testCowInsertOnlyMergeUsesOverwriteCommit() + { + // When a CoW MERGE produces only inserts (no rows matched for delete/update), + // the commit must still go through OverwriteFiles for conflict detection, not AppendFiles. + try (TestTable target = newCowTable("test_cow_insert_only_merge_overwrite_target_", + "WITH (extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write'])) " + + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey < 2"); + TestTable source = newCowTable("test_cow_insert_only_merge_overwrite_source_", + "AS SELECT * FROM tpch.tiny.nation WHERE regionkey >= 4")) { + String targetName = target.getName(); + String sourceName = source.getName(); + + long beforeSnapshotCount = (long) computeScalar( + "SELECT count(*) FROM \"" + targetName + "$snapshots\""); + + long sourceCount = (long) computeScalar("SELECT count(*) FROM " + sourceName); + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.nationkey = s.nationkey " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.nationkey, s.name, s.regionkey, s.comment)", sourceCount); + + assertNoDeleteFiles(targetName); + assertQuery("SELECT count(*) FROM " + targetName, "VALUES " + (10 + sourceCount)); + + // The MERGE should have produced exactly one new snapshot + long afterSnapshotCount = (long) computeScalar( + "SELECT count(*) FROM \"" + targetName + "$snapshots\""); + assertThat(afterSnapshotCount).isEqualTo(beforeSnapshotCount + 1); + + // The snapshot must be an 'overwrite' (conflict-detecting), NOT an 'append' + String lastOp = (String) computeScalar( + "SELECT operation FROM \"" + targetName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1"); + assertThat(lastOp).isEqualTo("overwrite"); + } + } + + @Test + public void testCowUpdatePreservesPreExistingEqualityDeletes() + throws Exception + { + // Verify that when a table has pre-existing equality deletes and a CoW UPDATE + // is applied, the equality-deleted rows remain deleted (not resurrected) and + // the update is correctly applied only to surviving rows. + try (TestTable table = newCowTable("test_cow_eq_delete_update_", + "(id integer, v varchar) WITH (format_version = 2, " + + "extra_properties = MAP(ARRAY['write.update.mode'], ARRAY['copy-on-write']))")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 4); + + BaseTable icebergTable = loadBaseTable(tableName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.empty(), + Optional.empty(), + Map.of("id", 2), + Optional.empty()); + + assertQuery("SELECT id, v FROM " + tableName + " ORDER BY id", "VALUES (1, 'a'), (3, 'c'), (4, 'd')"); + + // UPDATE that touches the same data file. Row id=2 must stay deleted. + assertUpdate("UPDATE " + tableName + " SET v = 'cc' WHERE id = 3", 1); + + assertQuery("SELECT id, v FROM " + tableName + " ORDER BY id", + "VALUES (1, 'a'), (3, 'cc'), (4, 'd')"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE id = 2", "VALUES 0"); + } + } + + @Test + public void testCowMergePreservesPreExistingEqualityDeletes() + throws Exception + { + // End-to-end: equality delete + CoW MERGE (update+insert). The equality-deleted + // rows must not reappear, and the MERGE must correctly update/insert rows. + try (TestTable target = newCowTable("test_cow_eq_delete_merge_target_", + "(id integer, v varchar) WITH (format_version = 2, " + + "extra_properties = MAP(ARRAY['write.merge.mode'], ARRAY['copy-on-write']))"); + TestTable source = newCowTable("test_cow_eq_delete_merge_source_", + "(id integer, v varchar)")) { + String targetName = target.getName(); + String sourceName = source.getName(); + assertUpdate("INSERT INTO " + sourceName + " VALUES (3, 'cc'), (5, 'e')", 2); + assertUpdate("INSERT INTO " + targetName + " VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", 4); + + BaseTable icebergTable = loadBaseTable(targetName); + writeEqualityDeleteForTable( + icebergTable, + getFileSystemFactory(getQueryRunner()), + Optional.empty(), + Optional.empty(), + Map.of("id", 2), + Optional.empty()); + + assertQuery("SELECT id, v FROM " + targetName + " ORDER BY id", "VALUES (1, 'a'), (3, 'c'), (4, 'd')"); + + assertUpdate("MERGE INTO " + targetName + " t " + + "USING " + sourceName + " s " + + "ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET v = s.v " + + "WHEN NOT MATCHED THEN INSERT VALUES (s.id, s.v)", 2); + + // id=2 must still be deleted (not resurrected by merge) + // id=3 must be updated to 'cc' + // id=5 must be inserted + assertQuery("SELECT id, v FROM " + targetName + " ORDER BY id", + "VALUES (1, 'a'), (3, 'cc'), (4, 'd'), (5, 'e')"); + assertQuery("SELECT count(*) FROM " + targetName + " WHERE id = 2", "VALUES 0"); + } + } + + private Set getDataFilePaths(String tableName) + { + return computeActual("SELECT file_path FROM \"" + tableName + "$files\" WHERE content = " + DATA.id()) + .getMaterializedRows().stream() + .map(row -> (String) row.getField(0)) + .collect(toImmutableSet()); + } + + private BaseTable loadBaseTable(String tableName) + { + return loadTable( + tableName, + getHiveMetastore(getQueryRunner()), + getFileSystemFactory(getQueryRunner()), + ICEBERG_CATALOG, + "tpch"); + } + + private void assertNoDeleteFiles(String tableName) + { + assertQuery( + "SELECT count(*) FROM \"" + tableName + "$files\" WHERE content IN (" + POSITION_DELETES.id() + ", " + EQUALITY_DELETES.id() + ")", + "VALUES 0"); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteAvroConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteAvroConnectorTest.java new file mode 100644 index 000000000000..ce916e2e01ea --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteAvroConnectorTest.java @@ -0,0 +1,92 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.iceberg.TestIcebergCopyOnWriteParquetConnectorTest.enableCopyOnWrite; + +/** + * Runs the full {@link BaseIcebergConnectorTest} DML suite with copy-on-write mode + * enabled on all pre-loaded TPCH tables (format version 2, Avro). + */ +class TestIcebergCopyOnWriteAvroConnectorTest + extends BaseIcebergAvroConnectorTest +{ + TestIcebergCopyOnWriteAvroConnectorTest() + { + super(2); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + QueryRunner queryRunner = createQueryRunnerBuilder().build(); + enableCopyOnWrite(queryRunner); + return queryRunner; + } + + @Override + protected IcebergQueryRunner.Builder createQueryRunnerBuilder() + { + return IcebergQueryRunner.builder() + .setIcebergProperties(ImmutableMap.builder() + .put("iceberg.file-format", format.name()) + .put("iceberg.format-version", String.valueOf(formatVersion)) + .put("iceberg.allowed-extra-properties", "*") + .put("iceberg.writer-sort-buffer-size", "1MB") + .buildOrThrow()) + .setInitialTables(REQUIRED_TPCH_TABLES); + } + + @Test + @Override + public void testIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys (sorted_by, write.format.default, comment, extra_properties) are always blocked. + assertQueryFails( + "CREATE TABLE test_create_table_with_illegal_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id']))", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "CREATE TABLE test_create_table_as_with_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['extra_properties'], ARRAY['some_value'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [extra_properties]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['write.format.default'], ARRAY['ORC'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [write.format.default]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [comment]"); + } + + @Test + @Override + public void testSetIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys are always blocked. + try (TestTable table = newTrinoTable("test_set_illegal_table_properties", "(x int)")) { + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id'])", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])", + "\\QIllegal keys in extra_properties: [comment]"); + } + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteIsolationLevelResolution.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteIsolationLevelResolution.java new file mode 100644 index 000000000000..1516bef81622 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteIsolationLevelResolution.java @@ -0,0 +1,93 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import org.apache.iceberg.IsolationLevel; +import org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.trino.plugin.iceberg.IcebergMergeTableHandle.CopyOnWriteOperationType.DELETE; +import static io.trino.plugin.iceberg.IcebergMergeTableHandle.CopyOnWriteOperationType.UPDATE_OR_MERGE; +import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL_DEFAULT; +import static org.assertj.core.api.Assertions.assertThat; + +public class TestIcebergCopyOnWriteIsolationLevelResolution +{ + @Test + public void testDeleteUsesDeleteIsolationLevel() + { + Map properties = Map.of( + DELETE_ISOLATION_LEVEL, "snapshot", + UPDATE_ISOLATION_LEVEL, "serializable", + MERGE_ISOLATION_LEVEL, "serializable"); + + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, DELETE)) + .isEqualTo(IsolationLevel.SNAPSHOT); + } + + @Test + public void testUpdateOrMergeUsesStricterIsolationLevel() + { + Map properties = Map.of( + UPDATE_ISOLATION_LEVEL, "snapshot", + MERGE_ISOLATION_LEVEL, "serializable"); + + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, UPDATE_OR_MERGE)) + .isEqualTo(IsolationLevel.SERIALIZABLE); + } + + @Test + public void testUpdateOrMergeSnapshotWhenBothSnapshot() + { + Map properties = Map.of( + UPDATE_ISOLATION_LEVEL, "snapshot", + MERGE_ISOLATION_LEVEL, "snapshot"); + + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, UPDATE_OR_MERGE)) + .isEqualTo(IsolationLevel.SNAPSHOT); + } + + @Test + public void testUpdateOrMergeUsesStricterIsolationLevelInEitherDirection() + { + Map properties = Map.of( + UPDATE_ISOLATION_LEVEL, "serializable", + MERGE_ISOLATION_LEVEL, "snapshot"); + + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, UPDATE_OR_MERGE)) + .isEqualTo(IsolationLevel.SERIALIZABLE); + } + + @Test + public void testDefaultsWhenIsolationPropertiesAreNotSet() + { + Map properties = Map.of(); + IsolationLevel updateDefault = IsolationLevel.fromName(UPDATE_ISOLATION_LEVEL_DEFAULT); + IsolationLevel mergeDefault = IsolationLevel.fromName(MERGE_ISOLATION_LEVEL_DEFAULT); + IsolationLevel expectedUpdateOrMergeDefault = (updateDefault == IsolationLevel.SERIALIZABLE || mergeDefault == IsolationLevel.SERIALIZABLE) + ? IsolationLevel.SERIALIZABLE + : IsolationLevel.SNAPSHOT; + + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, DELETE)) + .isEqualTo(IsolationLevel.fromName(DELETE_ISOLATION_LEVEL_DEFAULT)); + assertThat(IcebergMetadata.resolveCopyOnWriteIsolationLevel(properties, UPDATE_OR_MERGE)) + .isEqualTo(expectedUpdateOrMergeDefault); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteParquetConnectorTest.java new file mode 100644 index 000000000000..3ac21a797204 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteParquetConnectorTest.java @@ -0,0 +1,101 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import io.trino.tpch.TpchTable; +import org.junit.jupiter.api.Test; + +/** + * Runs the full {@link BaseIcebergConnectorTest} DML suite with copy-on-write mode + * enabled on all pre-loaded TPCH tables (format version 2, Parquet). + */ +class TestIcebergCopyOnWriteParquetConnectorTest + extends BaseIcebergParquetConnectorTest +{ + TestIcebergCopyOnWriteParquetConnectorTest() + { + super(2); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + QueryRunner queryRunner = createQueryRunnerBuilder().build(); + enableCopyOnWrite(queryRunner); + return queryRunner; + } + + @Override + protected IcebergQueryRunner.Builder createQueryRunnerBuilder() + { + return IcebergQueryRunner.builder() + .setIcebergProperties(ImmutableMap.builder() + .put("iceberg.file-format", format.name()) + .put("iceberg.format-version", String.valueOf(formatVersion)) + .put("iceberg.allowed-extra-properties", "*") + .put("iceberg.writer-sort-buffer-size", "1MB") + .buildOrThrow()) + .setInitialTables(REQUIRED_TPCH_TABLES); + } + + static void enableCopyOnWrite(QueryRunner queryRunner) + { + for (TpchTable table : REQUIRED_TPCH_TABLES) { + queryRunner.execute("ALTER TABLE " + table.getTableName() + + " SET PROPERTIES extra_properties = MAP(" + + "ARRAY['write.delete.mode', 'write.update.mode', 'write.merge.mode'], " + + "ARRAY['copy-on-write', 'copy-on-write', 'copy-on-write'])"); + } + } + + @Test + @Override + public void testIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys (sorted_by, write.format.default, comment, extra_properties) are always blocked. + assertQueryFails( + "CREATE TABLE test_create_table_with_illegal_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id']))", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "CREATE TABLE test_create_table_as_with_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['extra_properties'], ARRAY['some_value'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [extra_properties]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['write.format.default'], ARRAY['ORC'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [write.format.default]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [comment]"); + } + + @Test + @Override + public void testSetIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys are always blocked. + try (TestTable table = newTrinoTable("test_set_illegal_table_properties", "(x int)")) { + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id'])", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])", + "\\QIllegal keys in extra_properties: [comment]"); + } + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3AvroConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3AvroConnectorTest.java new file mode 100644 index 000000000000..030a623509b9 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3AvroConnectorTest.java @@ -0,0 +1,92 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.iceberg.TestIcebergCopyOnWriteParquetConnectorTest.enableCopyOnWrite; + +/** + * Runs the full {@link BaseIcebergConnectorTest} DML suite with copy-on-write mode + * enabled on all pre-loaded TPCH tables (format version 3, Avro). + */ +class TestIcebergCopyOnWriteV3AvroConnectorTest + extends BaseIcebergAvroConnectorTest +{ + TestIcebergCopyOnWriteV3AvroConnectorTest() + { + super(3); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + QueryRunner queryRunner = createQueryRunnerBuilder().build(); + enableCopyOnWrite(queryRunner); + return queryRunner; + } + + @Override + protected IcebergQueryRunner.Builder createQueryRunnerBuilder() + { + return IcebergQueryRunner.builder() + .setIcebergProperties(ImmutableMap.builder() + .put("iceberg.file-format", format.name()) + .put("iceberg.format-version", String.valueOf(formatVersion)) + .put("iceberg.allowed-extra-properties", "*") + .put("iceberg.writer-sort-buffer-size", "1MB") + .buildOrThrow()) + .setInitialTables(REQUIRED_TPCH_TABLES); + } + + @Test + @Override + public void testIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys (sorted_by, write.format.default, comment, extra_properties) are always blocked. + assertQueryFails( + "CREATE TABLE test_create_table_with_illegal_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id']))", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "CREATE TABLE test_create_table_as_with_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['extra_properties'], ARRAY['some_value'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [extra_properties]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['write.format.default'], ARRAY['ORC'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [write.format.default]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [comment]"); + } + + @Test + @Override + public void testSetIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys are always blocked. + try (TestTable table = newTrinoTable("test_set_illegal_table_properties", "(x int)")) { + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id'])", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])", + "\\QIllegal keys in extra_properties: [comment]"); + } + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3ParquetConnectorTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3ParquetConnectorTest.java new file mode 100644 index 000000000000..d9f19786d750 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergCopyOnWriteV3ParquetConnectorTest.java @@ -0,0 +1,93 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableMap; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.junit.jupiter.api.Test; + +import static io.trino.plugin.iceberg.TestIcebergCopyOnWriteParquetConnectorTest.enableCopyOnWrite; + +/** + * Runs the full {@link BaseIcebergConnectorTest} DML suite with copy-on-write mode + * enabled on all pre-loaded TPCH tables (format version 3, Parquet). V3 exercises + * deletion vector handling and {@code removeDanglingDeleteFiles} cleanup. + */ +class TestIcebergCopyOnWriteV3ParquetConnectorTest + extends BaseIcebergParquetConnectorTest +{ + TestIcebergCopyOnWriteV3ParquetConnectorTest() + { + super(3); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + QueryRunner queryRunner = createQueryRunnerBuilder().build(); + enableCopyOnWrite(queryRunner); + return queryRunner; + } + + @Override + protected IcebergQueryRunner.Builder createQueryRunnerBuilder() + { + return IcebergQueryRunner.builder() + .setIcebergProperties(ImmutableMap.builder() + .put("iceberg.file-format", format.name()) + .put("iceberg.format-version", String.valueOf(formatVersion)) + .put("iceberg.allowed-extra-properties", "*") + .put("iceberg.writer-sort-buffer-size", "1MB") + .buildOrThrow()) + .setInitialTables(REQUIRED_TPCH_TABLES); + } + + @Test + @Override + public void testIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys (sorted_by, write.format.default, comment, extra_properties) are always blocked. + assertQueryFails( + "CREATE TABLE test_create_table_with_illegal_extra_properties (c1 integer) WITH (extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id']))", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "CREATE TABLE test_create_table_as_with_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['extra_properties'], ARRAY['some_value'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [extra_properties]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['write.format.default'], ARRAY['ORC'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [write.format.default]"); + assertQueryFails( + "CREATE TABLE test_create_table_with_as_illegal_extra_properties WITH (extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])) AS SELECT 1 as c1", + "\\QIllegal keys in extra_properties: [comment]"); + } + + @Test + @Override + public void testSetIllegalExtraPropertyKey() + { + // With allowed-extra-properties=*, arbitrary keys are accepted. + // Only Trino-managed keys are always blocked. + try (TestTable table = newTrinoTable("test_set_illegal_table_properties", "(x int)")) { + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['sorted_by'], ARRAY['id'])", + "\\QIllegal keys in extra_properties: [sorted_by]"); + assertQueryFails( + "ALTER TABLE " + table.getName() + " SET PROPERTIES extra_properties = MAP(ARRAY['comment'], ARRAY['some comment'])", + "\\QIllegal keys in extra_properties: [comment]"); + } + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 2b2cbbd07132..34f0dd2fc55d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -177,7 +177,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)), + Optional.of(false), + false), transaction); TupleDomain splitPruningPredicate = TupleDomain.withColumnDomains( @@ -238,7 +239,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)), + Optional.of(false), + false), transaction); try (ConnectorPageSource emptyPageSource = createTestingPageSource(transaction, icebergConfig, split, tableHandle, ImmutableList.of(keyColumnHandle, dataColumnHandle), getDynamicFilter(splitPruningPredicate))) { @@ -349,7 +351,8 @@ public void testDynamicSplitPruningWithExplicitPartitionFilter() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)), + Optional.of(false), + false), transaction); // Simulate situations where the dynamic filter (e.g.: while performing a JOIN with another table) reduces considerably @@ -511,7 +514,8 @@ public void testDynamicSplitPruningWithExplicitPartitionFilterPartitionEvolution false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)), + Optional.of(false), + false), transaction); // Simulate situations where the dynamic filter (e.g.: while performing a JOIN with another table) reduces considerably diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 2271d22eb42a..0bef0143e550 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -74,6 +74,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Maps.transformValues; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; @@ -437,6 +438,73 @@ private IcebergSplit generateSplit(Table nationTable, IcebergTableHandle tableHa } } + @Test + public void testCopyOnWriteScanProducesOneSplitPerFile() + throws Exception + { + SchemaTableName schemaTableName = new SchemaTableName("tpch", "nation"); + Table nationTable = catalog.loadTable(SESSION, schemaTableName); + + // Create a CoW-mode handle (copyOnWriteScan=true) + IcebergTableHandle cowHandle = new IcebergTableHandle( + schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + TableType.DATA, + OptionalLong.empty(), + SchemaParser.toJson(nationTable.schema()), + nationTable.spec() == null ? OptionalInt.empty() : OptionalInt.of(nationTable.spec().specId()), + transformValues(nationTable.specs(), PartitionSpecParser::toJson), + 1, + TupleDomain.all(), + TupleDomain.all(), + OptionalLong.empty(), + ImmutableSet.of(), + Optional.empty(), + nationTable.location(), + nationTable.properties(), + Optional.empty(), + false, + Optional.empty(), + ImmutableSet.of(), + Optional.of(false), + true); + + assertThat(cowHandle.isCopyOnWriteScan()).isTrue(); + + try (IcebergSplitSource splitSource = new IcebergSplitSource( + new DefaultIcebergFileSystemFactory(fileSystemFactory), + SESSION, + cowHandle, + nationTable, + nationTable.newScan(), + Optional.empty(), + DynamicFilter.EMPTY, + new Duration(0, SECONDS), + alwaysTrue(), + TESTING_TYPE_MANAGER, + false, + 0, + new DefaultCachingHostAddressProvider(), + new InMemoryMetricsReporter(), + newDirectExecutorService())) { + ImmutableList.Builder builder = ImmutableList.builder(); + while (!splitSource.isFinished()) { + splitSource.getNextBatch(100).get() + .getSplits() + .stream() + .map(IcebergSplit.class::cast) + .forEach(builder::add); + } + List splits = builder.build(); + + // In CoW mode, each data file must become exactly one split -- never split + Set filePaths = splits.stream() + .map(IcebergSplit::getPath) + .collect(toImmutableSet()); + assertThat(splits).hasSize(filePaths.size()); + } + } + private static IcebergTableHandle createTableHandle(SchemaTableName schemaTableName, Table nationTable, TupleDomain unenforcedPredicate) { return new IcebergTableHandle( @@ -459,6 +527,7 @@ private static IcebergTableHandle createTableHandle(SchemaTableName schemaTableN false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java index 37152a009bcf..38240d39803b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java @@ -95,6 +95,29 @@ public abstract class BaseTrinoCatalogTest protected abstract TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) throws IOException; + protected static ConnectorMetadata createTestIcebergMetadata(TrinoCatalog catalog) + { + return new IcebergMetadata( + PLANNER_CONTEXT.getTypeManager(), + jsonCodec(CommitTaskData.class), + catalog, + (_, _) -> { + throw new UnsupportedOperationException(); + }, + TABLE_STATISTICS_READER, + new TableStatisticsWriter(new NodeVersion("test-version")), + UNSUPPORTED_DELETION_VECTOR_WRITER, + Optional.empty(), + false, + _ -> false, + newDirectExecutorService(), + directExecutor(), + newDirectExecutorService(), + newDirectExecutorService(), + 0, + ZERO); + } + protected Map defaultNamespaceProperties(String newNamespaceName) { return ImmutableMap.of(); @@ -140,25 +163,7 @@ public void testNonLowercaseNamespace() .contains(schema); // Test with IcebergMetadata, should the ConnectorMetadata implementation behavior depend on that class - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (connectorIdentity, fileIoProperties) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.schemaExists(SESSION, namespace)).as("icebergMetadata.schemaExists(namespace)") .isFalse(); assertThat(icebergMetadata.schemaExists(SESSION, schema)).as("icebergMetadata.schemaExists(schema)") @@ -181,25 +186,7 @@ public void testSchemaWithInvalidProperties() TrinoCatalog catalog = createTrinoCatalog(false); createNamespaceWithProperties(catalog, namespace, ImmutableMap.of("invalid_property", "test-value")); try { - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (_, _) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.getSchemaProperties(SESSION, namespace)) .doesNotContainKey("invalid_property"); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java index 3c0c4c25ea12..bc7674b779b3 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java @@ -19,13 +19,9 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.metastore.TableInfo; import io.trino.plugin.hive.metastore.glue.GlueMetastoreStats; -import io.trino.plugin.iceberg.CommitTaskData; import io.trino.plugin.iceberg.IcebergConfig; -import io.trino.plugin.iceberg.IcebergMetadata; -import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; -import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.ConnectorMetadata; @@ -51,19 +47,13 @@ import java.util.Optional; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.airlift.json.JsonCodec.jsonCodec; -import static io.airlift.units.Duration.ZERO; import static io.trino.hdfs.HdfsTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; import static io.trino.plugin.iceberg.IcebergSchemaProperties.LOCATION_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FORMAT_VERSION_PROPERTY; import static io.trino.plugin.iceberg.IcebergTestUtils.FILE_IO_FACTORY; -import static io.trino.plugin.iceberg.IcebergTestUtils.TABLE_STATISTICS_READER; -import static io.trino.plugin.iceberg.delete.DeletionVectorWriter.UNSUPPORTED_DELETION_VECTOR_WRITER; import static io.trino.spi.type.IntegerType.INTEGER; -import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.util.Locale.ENGLISH; @@ -154,25 +144,7 @@ public void testNonLowercaseGlueDatabase() .contains(trinoSchemaName); // Test with IcebergMetadata, should the ConnectorMetadata implementation behavior depend on that class - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (connectorIdentity, fileIoProperties) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.schemaExists(SESSION, databaseName)).as("icebergMetadata.schemaExists(databaseName)") .isFalse(); assertThat(icebergMetadata.schemaExists(SESSION, trinoSchemaName)).as("icebergMetadata.schemaExists(trinoSchemaName)") diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java index 2395b293cba6..3ac2d3d9b866 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java @@ -16,13 +16,9 @@ import com.google.common.collect.ImmutableMap; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; -import io.trino.plugin.iceberg.CommitTaskData; -import io.trino.plugin.iceberg.IcebergMetadata; -import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.containers.NessieContainer; -import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.SchemaTableName; @@ -43,18 +39,10 @@ import java.net.URI; import java.nio.file.Path; import java.util.Map; -import java.util.Optional; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.airlift.json.JsonCodec.jsonCodec; -import static io.airlift.units.Duration.ZERO; import static io.trino.hdfs.HdfsTestUtils.HDFS_ENVIRONMENT; import static io.trino.hdfs.HdfsTestUtils.HDFS_FILE_SYSTEM_STATS; import static io.trino.plugin.iceberg.IcebergTestUtils.FILE_IO_FACTORY; -import static io.trino.plugin.iceberg.IcebergTestUtils.TABLE_STATISTICS_READER; -import static io.trino.plugin.iceberg.delete.DeletionVectorWriter.UNSUPPORTED_DELETION_VECTOR_WRITER; -import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.nio.file.Files.createTempDirectory; @@ -197,25 +185,7 @@ public void testNonLowercaseNamespace() .contains(namespace); // Test with IcebergMetadata, should the ConnectorMetadata implementation behavior depend on that class - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (connectorIdentity, fileIoProperties) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.schemaExists(SESSION, namespace)).as("icebergMetadata.schemaExists(namespace)") .isTrue(); assertThat(icebergMetadata.schemaExists(SESSION, schema)).as("icebergMetadata.schemaExists(schema)") diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestTrinoRestCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestTrinoRestCatalog.java index 80733c48af8b..6922cccc83b1 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestTrinoRestCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/rest/TestTrinoRestCatalog.java @@ -16,14 +16,10 @@ import com.google.common.collect.ImmutableMap; import io.trino.cache.EvictableCacheBuilder; import io.trino.metastore.TableInfo; -import io.trino.plugin.iceberg.CommitTaskData; import io.trino.plugin.iceberg.DefaultIcebergFileSystemFactory; -import io.trino.plugin.iceberg.IcebergMetadata; -import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.rest.IcebergRestCatalogConfig.Security; -import io.trino.spi.NodeVersion; import io.trino.spi.TrinoException; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; @@ -37,20 +33,12 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Map; -import java.util.Optional; import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.airlift.json.JsonCodec.jsonCodec; -import static io.airlift.units.Duration.ZERO; import static io.trino.hdfs.HdfsTestUtils.HDFS_FILE_SYSTEM_FACTORY; import static io.trino.metastore.TableInfo.ExtendedRelationType.OTHER_VIEW; -import static io.trino.plugin.iceberg.IcebergTestUtils.TABLE_STATISTICS_READER; import static io.trino.plugin.iceberg.catalog.rest.IcebergRestCatalogConfig.SessionType.NONE; import static io.trino.plugin.iceberg.catalog.rest.RestCatalogTestUtils.backendCatalog; -import static io.trino.plugin.iceberg.delete.DeletionVectorWriter.UNSUPPORTED_DELETION_VECTOR_WRITER; -import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.util.Locale.ENGLISH; @@ -132,25 +120,7 @@ public void testNonLowercaseNamespace() .contains(namespace); // Test with IcebergMetadata, should the ConnectorMetadata implementation behavior depend on that class - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (connectorIdentity, fileIoProperties) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.schemaExists(SESSION, namespace)).as("icebergMetadata.schemaExists(namespace)") .isTrue(); assertThat(icebergMetadata.schemaExists(SESSION, schema)).as("icebergMetadata.schemaExists(schema)") diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java index 1c9f8be58acb..981c13794493 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/optimizer/TestConnectorPushdownRulesWithIceberg.java @@ -177,7 +177,8 @@ public void testProjectionPushdown() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); TableHandle table = new TableHandle(catalogHandle, icebergTable, new HiveTransactionHandle(false)); IcebergColumnHandle fullColumn = partialColumn.getBaseColumn(); @@ -262,7 +263,8 @@ public void testPredicatePushdown() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); TableHandle table = new TableHandle(catalogHandle, icebergTable, new HiveTransactionHandle(false)); IcebergColumnHandle column = IcebergColumnHandle.optional(primitiveColumnIdentity(1, "a")).columnType(INTEGER).build(); @@ -314,7 +316,8 @@ public void testColumnPruningProjectionPushdown() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); TableHandle table = new TableHandle(catalogHandle, icebergTable, new HiveTransactionHandle(false)); IcebergColumnHandle columnA = IcebergColumnHandle.optional(primitiveColumnIdentity(0, "a")).columnType(INTEGER).build(); @@ -376,7 +379,8 @@ public void testPushdownWithDuplicateExpressions() false, Optional.empty(), ImmutableSet.of(), - Optional.of(false)); + Optional.of(false), + false); TableHandle table = new TableHandle(catalogHandle, icebergTable, new HiveTransactionHandle(false)); IcebergColumnHandle bigintColumn = IcebergColumnHandle.optional(primitiveColumnIdentity(1, "just_bigint")).columnType(BIGINT).build(); diff --git a/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java b/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java index 6bc50bf0d5c2..52e6dfe51875 100644 --- a/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java +++ b/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java @@ -22,9 +22,6 @@ import io.trino.filesystem.s3.S3FileSystemStats; import io.trino.metastore.TableInfo; import io.trino.plugin.iceberg.ColumnIdentity; -import io.trino.plugin.iceberg.CommitTaskData; -import io.trino.plugin.iceberg.IcebergMetadata; -import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; @@ -32,7 +29,6 @@ import io.trino.plugin.iceberg.catalog.snowflake.SnowflakeIcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.snowflake.TestingSnowflakeServer; import io.trino.plugin.iceberg.catalog.snowflake.TrinoSnowflakeCatalog; -import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; import io.trino.spi.connector.ConnectorViewDefinition; @@ -58,12 +54,7 @@ import java.util.Map; import java.util.Optional; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; -import static io.airlift.json.JsonCodec.jsonCodec; -import static io.airlift.units.Duration.ZERO; import static io.trino.plugin.iceberg.IcebergTestUtils.FILE_IO_FACTORY; -import static io.trino.plugin.iceberg.IcebergTestUtils.TABLE_STATISTICS_READER; import static io.trino.plugin.iceberg.catalog.snowflake.TestIcebergSnowflakeCatalogConnectorSmokeTest.S3_ACCESS_KEY; import static io.trino.plugin.iceberg.catalog.snowflake.TestIcebergSnowflakeCatalogConnectorSmokeTest.S3_REGION; import static io.trino.plugin.iceberg.catalog.snowflake.TestIcebergSnowflakeCatalogConnectorSmokeTest.S3_SECRET_KEY; @@ -75,8 +66,6 @@ import static io.trino.plugin.iceberg.catalog.snowflake.TestingSnowflakeServer.SNOWFLAKE_TEST_DATABASE; import static io.trino.plugin.iceberg.catalog.snowflake.TestingSnowflakeServer.SNOWFLAKE_USER; import static io.trino.plugin.iceberg.catalog.snowflake.TestingSnowflakeServer.TableType.ICEBERG; -import static io.trino.plugin.iceberg.delete.DeletionVectorWriter.UNSUPPORTED_DELETION_VECTOR_WRITER; -import static io.trino.sql.planner.TestingPlannerContext.PLANNER_CONTEXT; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.util.Locale.ENGLISH; @@ -225,25 +214,7 @@ public void testNonLowercaseNamespace() .contains(namespace); // Test with IcebergMetadata, should the ConnectorMetadata implementation behavior depend on that class - ConnectorMetadata icebergMetadata = new IcebergMetadata( - PLANNER_CONTEXT.getTypeManager(), - jsonCodec(CommitTaskData.class), - catalog, - (connectorIdentity, fileIOProperties) -> { - throw new UnsupportedOperationException(); - }, - TABLE_STATISTICS_READER, - new TableStatisticsWriter(new NodeVersion("test-version")), - UNSUPPORTED_DELETION_VECTOR_WRITER, - Optional.empty(), - false, - _ -> false, - newDirectExecutorService(), - directExecutor(), - newDirectExecutorService(), - newDirectExecutorService(), - 0, - ZERO); + ConnectorMetadata icebergMetadata = createTestIcebergMetadata(catalog); assertThat(icebergMetadata.schemaExists(SESSION, namespace)).as("icebergMetadata.schemaExists(namespace)") .isTrue(); assertThat(icebergMetadata.schemaExists(SESSION, schema)).as("icebergMetadata.schemaExists(schema)") diff --git a/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java b/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java index 487dd690bbb4..26a7df9941cf 100644 --- a/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java +++ b/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java @@ -21,6 +21,7 @@ import io.trino.metastore.RawHiveMetastoreFactory; import io.trino.plugin.hive.metastore.MetastoreTypeConfig; import io.trino.plugin.iceberg.CommitTaskData; +import io.trino.plugin.iceberg.CopyOnWriteFileRewriter; import io.trino.plugin.iceberg.DefaultIcebergFileSystemFactory; import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergExecutorModule; @@ -68,6 +69,7 @@ protected void setup(Binder binder) binder.bind(DeletionVectorWriter.class).to(DefaultDeletionVectorWriter.class).in(Scopes.SINGLETON); binder.bind(IcebergMetadataFactory.class).in(Scopes.SINGLETON); binder.bind(IcebergFileWriterFactory.class).in(Scopes.SINGLETON); + binder.bind(CopyOnWriteFileRewriter.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsReader.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsWriter.class).in(Scopes.SINGLETON); binder.bind(IcebergFileSystemFactory.class).to(DefaultIcebergFileSystemFactory.class).in(Scopes.SINGLETON);