-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Add copy-on-write mode for Iceberg row-level DELETE, UPDATE, and MERGE #28958
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
kaveti
wants to merge
1
commit into
trinodb:master
Choose a base branch
from
kaveti:iceberg-copy-on-write
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
264 changes: 264 additions & 0 deletions
264
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CopyOnWriteFileRewriter.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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. | ||
| * | ||
| * <p>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<DataFile> 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> partitionData, | ||
| List<DeleteFile> preExistingDeletes, | ||
| long sourceDataSequenceNumber, | ||
| OptionalLong sourceFileFirstRowId, | ||
| IcebergFileFormat sourceFileFormat, | ||
| IcebergFileFormat writeFileFormat, | ||
| MetricsConfig metricsConfig, | ||
| Map<String, String> fileIoProperties, | ||
| Map<String, String> tableProperties, | ||
| LocationProvider locationProvider, | ||
| Optional<NameMapping> 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<DataFile> 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 : "<removed>", | ||
| 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> partitionData, | ||
| List<DeleteFile> preExistingDeletes, | ||
| long sourceDataSequenceNumber, | ||
| OptionalLong sourceFileFirstRowId, | ||
| IcebergFileFormat fileFormat, | ||
| Map<String, String> fileIoProperties, | ||
| Optional<NameMapping> nameMapping) | ||
| { | ||
| List<IcebergColumnHandle> dataColumns = schema.columns().stream() | ||
| .map(column -> getColumnHandle(column, typeManager)) | ||
| .collect(toImmutableList()); | ||
| List<IcebergColumnHandle> 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); | ||
|
coderabbitai[bot] marked this conversation as resolved.
|
||
| } | ||
| return originalRecordCount; | ||
| } | ||
| } | ||
30 changes: 30 additions & 0 deletions
30
plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/ForIcebergCopyOnWriteRewrite.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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 {} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.