diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/CommitTaskData.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/CommitTaskData.java index f8a625a450f3b..cb471ca37c497 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/CommitTaskData.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/CommitTaskData.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Optional; +import java.util.OptionalLong; import static java.util.Objects.requireNonNull; @@ -30,6 +31,9 @@ public class CommitTaskData private final FileFormat fileFormat; private final Optional referencedDataFile; private final FileContent content; + private final OptionalLong contentOffset; + private final OptionalLong contentSizeInBytes; + private final OptionalLong recordCount; @JsonCreator public CommitTaskData( @@ -40,7 +44,10 @@ public CommitTaskData( @JsonProperty("partitionDataJson") Optional partitionDataJson, @JsonProperty("fileFormat") FileFormat fileFormat, @JsonProperty("referencedDataFile") String referencedDataFile, - @JsonProperty("content") FileContent content) + @JsonProperty("content") FileContent content, + @JsonProperty("contentOffset") OptionalLong contentOffset, + @JsonProperty("contentSizeInBytes") OptionalLong contentSizeInBytes, + @JsonProperty("recordCount") OptionalLong recordCount) { this.path = requireNonNull(path, "path is null"); this.fileSizeInBytes = fileSizeInBytes; @@ -50,6 +57,24 @@ public CommitTaskData( this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); this.referencedDataFile = Optional.ofNullable(referencedDataFile); this.content = requireNonNull(content, "content is null"); + this.contentOffset = contentOffset != null ? contentOffset : OptionalLong.empty(); + this.contentSizeInBytes = contentSizeInBytes != null ? contentSizeInBytes : OptionalLong.empty(); + this.recordCount = recordCount != null ? recordCount : OptionalLong.empty(); + } + + public CommitTaskData( + String path, + long fileSizeInBytes, + MetricsWrapper metrics, + int partitionSpecId, + Optional partitionDataJson, + FileFormat fileFormat, + String referencedDataFile, + FileContent content) + { + this(path, fileSizeInBytes, metrics, partitionSpecId, partitionDataJson, + fileFormat, referencedDataFile, content, + OptionalLong.empty(), OptionalLong.empty(), OptionalLong.empty()); } @JsonProperty @@ -99,4 +124,22 @@ public FileContent getContent() { return content; } + + @JsonProperty + public OptionalLong getContentOffset() + { + return contentOffset; + } + + @JsonProperty + public OptionalLong getContentSizeInBytes() + { + return contentSizeInBytes; + } + + @JsonProperty + public OptionalLong getRecordCount() + { + return recordCount; + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/ExpressionConverter.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/ExpressionConverter.java index 285a9cf863b5f..fd47dddb43b84 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/ExpressionConverter.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/ExpressionConverter.java @@ -201,7 +201,16 @@ private static Object getIcebergLiteralValue(Type type, Marker marker) return toIntExact(((Long) marker.getValue())); } - if (type instanceof TimestampType || type instanceof TimeType) { + if (type instanceof TimestampType) { + TimestampType tsType = (TimestampType) type; + long value = (Long) marker.getValue(); + if (tsType.getPrecision() == MILLISECONDS) { + return MILLISECONDS.toMicros(value); + } + return value; + } + + if (type instanceof TimeType) { return MILLISECONDS.toMicros((Long) marker.getValue()); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/FileFormat.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/FileFormat.java index df4700bc8db80..e2d94f71b6b33 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/FileFormat.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/FileFormat.java @@ -26,7 +26,8 @@ public enum FileFormat ORC("orc", true), PARQUET("parquet", true), AVRO("avro", true), - METADATA("metadata.json", false); + METADATA("metadata.json", false), + PUFFIN("puffin", false); private final String ext; private final boolean splittable; @@ -61,6 +62,9 @@ public static FileFormat fromIcebergFileFormat(org.apache.iceberg.FileFormat for case METADATA: prestoFileFormat = METADATA; break; + case PUFFIN: + prestoFileFormat = PUFFIN; + break; default: throw new PrestoException(NOT_SUPPORTED, "Unsupported file format: " + format); } @@ -81,6 +85,12 @@ public org.apache.iceberg.FileFormat toIceberg() case AVRO: fileFormat = org.apache.iceberg.FileFormat.AVRO; break; + case METADATA: + fileFormat = org.apache.iceberg.FileFormat.METADATA; + break; + case PUFFIN: + fileFormat = org.apache.iceberg.FileFormat.PUFFIN; + break; default: throw new PrestoException(NOT_SUPPORTED, "Unsupported file format: " + this); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java index 9d1c79c3e029b..a4e893767665c 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java @@ -405,12 +405,9 @@ protected static void validateTableForPresto(BaseTable table, Optional tab schema = metadata.schema(); } - // Reject schema default values (initial-default / write-default) - for (Types.NestedField field : schema.columns()) { - if (field.initialDefault() != null || field.writeDefault() != null) { - throw new PrestoException(NOT_SUPPORTED, "Iceberg v3 column default values are not supported"); - } - } + // Iceberg v3 column default values (initial-default / write-default) are supported. + // The Iceberg library handles applying defaults when reading files that were written + // before a column with a default was added via schema evolution. // Reject Iceberg table encryption if (!metadata.encryptionKeys().isEmpty() || snapshot.keyId() != null || metadata.properties().containsKey("encryption.key-id")) { @@ -1524,8 +1521,23 @@ public Optional finishDeleteWithOutput(ConnectorSession .ofPositionDeletes() .withPath(task.getPath()) .withFileSizeInBytes(task.getFileSizeInBytes()) - .withFormat(FileFormat.fromString(task.getFileFormat().name())) - .withMetrics(task.getMetrics().metrics()); + .withFormat(FileFormat.fromString(task.getFileFormat().name())); + + if (task.getFileFormat() == com.facebook.presto.iceberg.FileFormat.PUFFIN) { + builder.withRecordCount(task.getRecordCount().orElseThrow(() -> + new VerifyException("recordCount required for deletion vector"))); + builder.withContentOffset(task.getContentOffset().orElseThrow(() -> + new VerifyException("contentOffset required for deletion vector"))); + builder.withContentSizeInBytes(task.getContentSizeInBytes().orElseThrow(() -> + new VerifyException("contentSizeInBytes required for deletion vector"))); + } + else { + builder.withMetrics(task.getMetrics().metrics()); + } + + if (task.getReferencedDataFile().isPresent()) { + builder.withReferencedDataFile(task.getReferencedDataFile().get()); + } if (!spec.fields().isEmpty()) { String partitionDataJson = task.getPartitionDataJson() diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java index 1d438fdad92af..10991d1f01047 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java @@ -49,6 +49,7 @@ import com.facebook.presto.iceberg.procedure.RegisterTableProcedure; import com.facebook.presto.iceberg.procedure.RemoveOrphanFiles; import com.facebook.presto.iceberg.procedure.RewriteDataFilesProcedure; +import com.facebook.presto.iceberg.procedure.RewriteDeleteFilesProcedure; import com.facebook.presto.iceberg.procedure.RewriteManifestsProcedure; import com.facebook.presto.iceberg.procedure.RollbackToSnapshotProcedure; import com.facebook.presto.iceberg.procedure.RollbackToTimestampProcedure; @@ -195,6 +196,7 @@ protected void setup(Binder binder) procedures.addBinding().toProvider(StatisticsFileCacheInvalidationProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(ManifestFileCacheInvalidationProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(RewriteDataFilesProcedure.class).in(Scopes.SINGLETON); + procedures.addBinding().toProvider(RewriteDeleteFilesProcedure.class).in(Scopes.SINGLETON); procedures.addBinding().toProvider(RewriteManifestsProcedure.class).in(Scopes.SINGLETON); // for orc diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConnector.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConnector.java index 86ad7fc192bea..137f1cc38d8c9 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConnector.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConnector.java @@ -16,6 +16,7 @@ import com.facebook.airlift.bootstrap.LifeCycleManager; import com.facebook.presto.hive.HiveTransactionHandle; import com.facebook.presto.iceberg.function.IcebergBucketFunction; +import com.facebook.presto.iceberg.function.VariantFunctions; import com.facebook.presto.iceberg.function.changelog.ApplyChangelogFunction; import com.facebook.presto.iceberg.transaction.IcebergTransactionManager; import com.facebook.presto.iceberg.transaction.IcebergTransactionMetadata; @@ -256,6 +257,7 @@ public Set> getSystemFunctions() .add(ApplyChangelogFunction.class) .add(IcebergBucketFunction.class) .add(IcebergBucketFunction.Bucket.class) + .add(VariantFunctions.class) .build(); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java index 2b6bc0b6e9f2f..58318ea314c98 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergPageSourceProvider.java @@ -45,6 +45,7 @@ import com.facebook.presto.iceberg.delete.DeleteFile; import com.facebook.presto.iceberg.delete.DeleteFilter; import com.facebook.presto.iceberg.delete.IcebergDeletePageSink; +import com.facebook.presto.iceberg.delete.IcebergDeletionVectorPageSink; import com.facebook.presto.iceberg.delete.PositionDeleteFilter; import com.facebook.presto.iceberg.delete.RowPredicate; import com.facebook.presto.memory.context.AggregatedMemoryContext; @@ -70,6 +71,7 @@ import com.facebook.presto.parquet.predicate.Predicate; import com.facebook.presto.parquet.reader.ParquetReader; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorPageSink; import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; @@ -95,7 +97,11 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.puffin.BlobMetadata; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinReader; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; @@ -113,6 +119,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -863,17 +870,33 @@ else if (subColumn.getId() == MERGE_PARTITION_DATA.getId()) { verify(storageProperties.isPresent(), "storageProperties are null"); LocationProvider locationProvider = getLocationProvider(table.getSchemaTableName(), outputPath.get(), storageProperties.get()); - Supplier deleteSinkSupplier = () -> new IcebergDeletePageSink( - partitionSpec, - split.getPartitionDataJson(), - locationProvider, - fileWriterFactory, - hdfsEnvironment, - hdfsContext, - jsonCodec, - session, - split.getPath(), - split.getFileFormat()); + int tableFormatVersion = Integer.parseInt( + storageProperties.get().getOrDefault("format-version", "2")); + Supplier deleteSinkSupplier; + if (tableFormatVersion >= 3) { + deleteSinkSupplier = () -> new IcebergDeletionVectorPageSink( + partitionSpec, + split.getPartitionDataJson(), + locationProvider, + hdfsEnvironment, + hdfsContext, + jsonCodec, + session, + split.getPath()); + } + else { + deleteSinkSupplier = () -> new IcebergDeletePageSink( + partitionSpec, + split.getPartitionDataJson(), + locationProvider, + fileWriterFactory, + hdfsEnvironment, + hdfsContext, + jsonCodec, + session, + split.getPath(), + split.getFileFormat()); + } boolean storeDeleteFilePath = icebergColumns.contains(DELETE_FILE_PATH_COLUMN_HANDLE); Supplier> deleteFilters = memoize(() -> { // If equality deletes are optimized into a join they don't need to be applied here @@ -980,30 +1003,35 @@ private List readDeletes( for (DeleteFile delete : deleteFiles) { if (delete.content() == POSITION_DELETES) { - if (startRowPosition.isPresent()) { - byte[] lowerBoundBytes = delete.getLowerBounds().get(DELETE_FILE_POS.fieldId()); - Optional positionLowerBound = Optional.ofNullable(lowerBoundBytes) - .map(bytes -> Conversions.fromByteBuffer(DELETE_FILE_POS.type(), ByteBuffer.wrap(bytes))); - - byte[] upperBoundBytes = delete.getUpperBounds().get(DELETE_FILE_POS.fieldId()); - Optional positionUpperBound = Optional.ofNullable(upperBoundBytes) - .map(bytes -> Conversions.fromByteBuffer(DELETE_FILE_POS.type(), ByteBuffer.wrap(bytes))); - - if ((positionLowerBound.isPresent() && positionLowerBound.get() > endRowPosition.get()) || - (positionUpperBound.isPresent() && positionUpperBound.get() < startRowPosition.get())) { - continue; - } + if (delete.format() == FileFormat.PUFFIN) { + readDeletionVector(session, delete, deletedRows); } + else { + if (startRowPosition.isPresent()) { + byte[] lowerBoundBytes = delete.getLowerBounds().get(DELETE_FILE_POS.fieldId()); + Optional positionLowerBound = Optional.ofNullable(lowerBoundBytes) + .map(bytes -> Conversions.fromByteBuffer(DELETE_FILE_POS.type(), ByteBuffer.wrap(bytes))); + + byte[] upperBoundBytes = delete.getUpperBounds().get(DELETE_FILE_POS.fieldId()); + Optional positionUpperBound = Optional.ofNullable(upperBoundBytes) + .map(bytes -> Conversions.fromByteBuffer(DELETE_FILE_POS.type(), ByteBuffer.wrap(bytes))); + + if ((positionLowerBound.isPresent() && positionLowerBound.get() > endRowPosition.get()) || + (positionUpperBound.isPresent() && positionUpperBound.get() < startRowPosition.get())) { + continue; + } + } - try (ConnectorPageSource pageSource = openDeletes(session, delete, deleteColumns, deleteDomain)) { - readPositionDeletes(pageSource, targetPath, deletedRows); - } - catch (IOException e) { - throw new PrestoException(ICEBERG_CANNOT_OPEN_SPLIT, format("Cannot open Iceberg delete file: %s", delete.path()), e); + try (ConnectorPageSource pageSource = openDeletes(session, delete, deleteColumns, deleteDomain)) { + readPositionDeletes(pageSource, targetPath, deletedRows); + } + catch (IOException e) { + throw new PrestoException(ICEBERG_CANNOT_OPEN_SPLIT, format("Cannot open Iceberg delete file: %s", delete.path()), e); + } } if (storeDeleteFilePath) { filters.add(new PositionDeleteFilter(deletedRows, delete.path())); - deletedRows = new Roaring64Bitmap(); // Reset the deleted rows for the next file + deletedRows = new Roaring64Bitmap(); } } else if (delete.content() == EQUALITY_DELETES) { @@ -1032,6 +1060,75 @@ else if (delete.content() == EQUALITY_DELETES) { return filters; } + private void readDeletionVector( + ConnectorSession session, + DeleteFile delete, + LongBitmapDataProvider deletedRows) + { + HdfsContext hdfsContext = new HdfsContext(session); + InputFile inputFile = new HdfsInputFile(new Path(delete.path()), hdfsEnvironment, hdfsContext); + try (PuffinReader reader = hdfsEnvironment.doAs(session.getUser(), () -> Puffin.read(inputFile).build())) { + List blobMetadataList = reader.fileMetadata().blobs(); + if (blobMetadataList.isEmpty()) { + return; + } + for (org.apache.iceberg.util.Pair pair : reader.readAll(blobMetadataList)) { + deserializeDeletionVector(pair.second(), deletedRows); + } + } + catch (IOException e) { + throw new PrestoException(ICEBERG_CANNOT_OPEN_SPLIT, format("Cannot open Iceberg deletion vector file: %s", delete.path()), e); + } + } + + private static void deserializeDeletionVector(ByteBuffer buffer, LongBitmapDataProvider deletedRows) + { + byte[] bytes = new byte[buffer.remaining()]; + buffer.get(bytes); + ByteBuffer buf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + + int cookie = buf.getInt(); + boolean isRunContainer = (cookie & 0xFFFF) == 12347; + int numContainers; + if (isRunContainer) { + numContainers = (cookie >>> 16) + 1; + int runBitmapBytes = (numContainers + 7) / 8; + buf.position(buf.position() + runBitmapBytes); + } + else if ((cookie & 0xFFFF) == 12346) { + numContainers = (cookie >>> 16) + 1; + } + else { + return; + } + + int[] keys = new int[numContainers]; + int[] cardinalities = new int[numContainers]; + for (int i = 0; i < numContainers; i++) { + keys[i] = Short.toUnsignedInt(buf.getShort()); + cardinalities[i] = Short.toUnsignedInt(buf.getShort()) + 1; + } + + for (int i = 0; i < numContainers; i++) { + long highBits = ((long) keys[i]) << 16; + if (cardinalities[i] <= 4096) { + for (int j = 0; j < cardinalities[i]; j++) { + deletedRows.addLong(highBits | Short.toUnsignedInt(buf.getShort())); + } + } + else { + for (int wordIdx = 0; wordIdx < 1024; wordIdx++) { + long word = buf.getLong(); + while (word != 0) { + int bit = Long.numberOfTrailingZeros(word); + deletedRows.addLong(highBits | (wordIdx * 64 + bit)); + word &= word - 1; + } + } + } + } + } + private ConnectorPageSource openDeletes( ConnectorSession session, DeleteFile delete, diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitSource.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitSource.java index 9d4cd1a615636..98ee9f2693450 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitSource.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSplitSource.java @@ -18,7 +18,6 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSplit; import com.facebook.presto.spi.ConnectorSplitSource; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SplitWeight; import com.facebook.presto.spi.connector.ConnectorPartitionHandle; import com.facebook.presto.spi.schedule.NodeSelectionStrategy; @@ -47,7 +46,6 @@ import static com.facebook.presto.iceberg.IcebergUtil.getTargetSplitSize; import static com.facebook.presto.iceberg.IcebergUtil.metadataColumnsMatchPredicates; import static com.facebook.presto.iceberg.IcebergUtil.partitionDataFromStructLike; -import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.Iterators.limit; import static java.util.Objects.requireNonNull; @@ -126,13 +124,6 @@ private ConnectorSplit toIcebergSplit(FileScanTask task) PartitionSpec spec = task.spec(); Optional partitionData = partitionDataFromStructLike(spec, task.file().partition()); - // Validate no PUFFIN deletion vectors (Iceberg v3 feature not yet supported) - for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { - if (deleteFile.format() == org.apache.iceberg.FileFormat.PUFFIN) { - throw new PrestoException(NOT_SUPPORTED, "Iceberg deletion vectors (PUFFIN format) are not supported"); - } - } - // TODO: We should leverage residual expression and convert that to TupleDomain. // The predicate here is used by readers for predicate push down at reader level, // so when we do not use residual expression, we are just wasting CPU cycles diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUpdateablePageSource.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUpdateablePageSource.java index 8a0bbdd1b16e8..044b5a7a045ba 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUpdateablePageSource.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUpdateablePageSource.java @@ -21,8 +21,8 @@ import com.facebook.presto.common.block.RunLengthEncodedBlock; import com.facebook.presto.hive.HivePartitionKey; import com.facebook.presto.iceberg.delete.DeleteFilter; -import com.facebook.presto.iceberg.delete.IcebergDeletePageSink; import com.facebook.presto.iceberg.delete.RowPredicate; +import com.facebook.presto.spi.ConnectorPageSink; import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.UpdatablePageSource; @@ -71,8 +71,8 @@ public class IcebergUpdateablePageSource implements UpdatablePageSource { private final ConnectorPageSource delegate; - private final Supplier deleteSinkSupplier; - private IcebergDeletePageSink positionDeleteSink; + private final Supplier deleteSinkSupplier; + private ConnectorPageSink positionDeleteSink; private final Supplier> deletePredicate; private final Supplier> deleteFilters; @@ -107,7 +107,7 @@ public IcebergUpdateablePageSource( ConnectorPageSource delegate, // represents the columns output by the delegate page source List delegateColumns, - Supplier deleteSinkSupplier, + Supplier deleteSinkSupplier, Supplier> deletePredicate, Supplier> deleteFilters, Supplier updatedRowPageSinkSupplier, @@ -295,7 +295,7 @@ public void updateRows(Page page, List columnValueAndRowIdChannels) public CompletableFuture> finish() { return Optional.ofNullable(positionDeleteSink) - .map(IcebergDeletePageSink::finish) + .map(ConnectorPageSink::finish) .orElseGet(() -> completedFuture(ImmutableList.of())) .thenCombine( Optional.ofNullable(updatedRowPageSink).map(IcebergPageSink::finish) diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java index e06553e66bc65..1bb55d74c3004 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java @@ -221,7 +221,7 @@ public final class IcebergUtil { private static final Logger log = Logger.get(IcebergUtil.class); public static final int MIN_FORMAT_VERSION_FOR_DELETE = 2; - public static final int MAX_FORMAT_VERSION_FOR_ROW_LEVEL_OPERATIONS = 2; + public static final int MAX_FORMAT_VERSION_FOR_ROW_LEVEL_OPERATIONS = 3; public static final int MAX_SUPPORTED_FORMAT_VERSION = 3; public static final long DOUBLE_POSITIVE_ZERO = 0x0000000000000000L; @@ -550,7 +550,16 @@ public static Map createIcebergViewProperties(ConnectorSession s public static Optional> tryGetProperties(Table table) { try { - return Optional.ofNullable(table.properties()); + Map properties = table.properties(); + if (properties != null && table instanceof BaseTable) { + int formatVersion = ((BaseTable) table).operations().current().formatVersion(); + if (!properties.containsKey("format-version")) { + Map enhanced = new HashMap<>(properties); + enhanced.put("format-version", String.valueOf(formatVersion)); + return Optional.of(enhanced); + } + } + return Optional.ofNullable(properties); } catch (TableNotFoundException e) { log.warn(String.format("Unable to fetch properties for table %s: %s", table.name(), e.getMessage())); @@ -779,7 +788,10 @@ public static Domain createDomainFromIcebergPartitionValue( case TIME: case TIMESTAMP: return singleValue(prestoType, MICROSECONDS.toMillis((Long) value)); + case TIMESTAMP_NANO: + return singleValue(prestoType, Math.floorDiv((Long) value, 1000L)); case STRING: + case VARIANT: return singleValue(prestoType, utf8Slice(value.toString())); case FLOAT: return singleValue(prestoType, (long) floatToRawIntBits((Float) value)); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionData.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionData.java index 015972ac949a5..c6b0907d24f75 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionData.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionData.java @@ -150,6 +150,7 @@ public static Object getValue(JsonNode partitionValue, Type type) return partitionValue.asInt(); case LONG: case TIMESTAMP: + case TIMESTAMP_NANO: case TIME: return partitionValue.asLong(); case FLOAT: @@ -175,6 +176,7 @@ public static Object getValue(JsonNode partitionValue, Type type) } return partitionValue.doubleValue(); case STRING: + case VARIANT: return partitionValue.asText(); case FIXED: case BINARY: diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionTable.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionTable.java index 08a5887bed551..84c250a7f442c 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionTable.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/PartitionTable.java @@ -287,7 +287,7 @@ private Object convert(Object value, Type type) if (value == null) { return null; } - if (type instanceof Types.StringType) { + if (type instanceof Types.StringType || type.isVariantType()) { return value.toString(); } if (type instanceof Types.BinaryType) { @@ -303,6 +303,9 @@ private Object convert(Object value, Type type) return MICROSECONDS.toMillis((long) value); } } + if (type instanceof Types.TimestampNanoType) { + return Math.floorDiv((long) value, 1000L); + } if (type instanceof Types.TimeType) { return MICROSECONDS.toMillis((long) value); } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TypeConverter.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TypeConverter.java index ca8db778ae87c..ac164cafecb3f 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TypeConverter.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/TypeConverter.java @@ -124,6 +124,12 @@ public static Type toPrestoType(org.apache.iceberg.types.Type type, TypeManager return TIMESTAMP_WITH_TIME_ZONE; } return TimestampType.TIMESTAMP; + case TIMESTAMP_NANO: + Types.TimestampNanoType tsNanoType = (Types.TimestampNanoType) type.asPrimitiveType(); + if (tsNanoType.shouldAdjustToUTC()) { + return TIMESTAMP_WITH_TIME_ZONE; + } + return TimestampType.TIMESTAMP_MICROSECONDS; case STRING: return VarcharType.createUnboundedVarcharType(); case UUID: @@ -141,6 +147,8 @@ public static Type toPrestoType(org.apache.iceberg.types.Type type, TypeManager return RowType.from(fields.stream() .map(field -> new RowType.Field(Optional.of(field.name()), toPrestoType(field.type(), typeManager))) .collect(toImmutableList())); + case VARIANT: + return VarcharType.createUnboundedVarcharType(); default: throw new UnsupportedOperationException(format("Cannot convert from Iceberg type '%s' (%s) to Presto type", type, type.typeId())); } @@ -402,8 +410,10 @@ private static List toOrcType(int nextFieldTypeIndex, org.apache.iceber case DATE: return ImmutableList.of(new OrcType(OrcType.OrcTypeKind.DATE, ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), Optional.empty(), attributes)); case TIMESTAMP: + case TIMESTAMP_NANO: return ImmutableList.of(new OrcType(OrcType.OrcTypeKind.TIMESTAMP, ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), Optional.empty(), attributes)); case STRING: + case VARIANT: return ImmutableList.of(new OrcType(OrcType.OrcTypeKind.STRING, ImmutableList.of(), ImmutableList.of(), Optional.empty(), Optional.empty(), Optional.empty(), attributes)); case UUID: case FIXED: diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/VariantBinaryCodec.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/VariantBinaryCodec.java new file mode 100644 index 0000000000000..383df5a74302c --- /dev/null +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/VariantBinaryCodec.java @@ -0,0 +1,783 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.StringWriter; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * Encoder/decoder for the Apache Variant binary format as used by Iceberg V3. + * + *

The Variant binary format encodes semi-structured (JSON-like) data in a compact + * binary representation with two components: + *

    + *
  • Metadata: A dictionary of field names (keys) used in objects
  • + *
  • Value: The encoded data using type-tagged values
  • + *
+ * + *

This codec supports encoding JSON strings to Variant binary and decoding + * Variant binary back to JSON strings. It implements the Apache Variant spec + * (version 1) covering: + *

    + *
  • Primitives: null, boolean, int8/16/32/64, float, double, string
  • + *
  • Short strings (0-63 bytes, inlined in header)
  • + *
  • Objects (key-value maps with metadata dictionary references)
  • + *
  • Arrays (ordered value lists)
  • + *
+ * + * @see Iceberg V3 Variant Spec + */ +public final class VariantBinaryCodec +{ + private static final JsonFactory JSON_FACTORY = new JsonFactory(); + + // Basic type codes (bits 7-6 of header byte) + static final int BASIC_TYPE_PRIMITIVE = 0; + static final int BASIC_TYPE_SHORT_STRING = 1; + static final int BASIC_TYPE_OBJECT = 2; + static final int BASIC_TYPE_ARRAY = 3; + + // Primitive type_info values (bits 5-0 when basic_type=0) + static final int PRIMITIVE_NULL = 0; + static final int PRIMITIVE_TRUE = 1; + static final int PRIMITIVE_FALSE = 2; + static final int PRIMITIVE_INT8 = 5; + static final int PRIMITIVE_INT16 = 6; + static final int PRIMITIVE_INT32 = 7; + static final int PRIMITIVE_INT64 = 8; + static final int PRIMITIVE_FLOAT = 9; + static final int PRIMITIVE_DOUBLE = 10; + static final int PRIMITIVE_STRING = 19; + + // Metadata format version + static final int METADATA_VERSION = 1; + + // Maximum short string length (6 bits = 63) + static final int MAX_SHORT_STRING_LENGTH = 63; + + private VariantBinaryCodec() {} + + /** + * Holds the two components of a Variant binary encoding. + */ + public static final class VariantBinary + { + private final byte[] metadata; + private final byte[] value; + + public VariantBinary(byte[] metadata, byte[] value) + { + this.metadata = metadata; + this.value = value; + } + + public byte[] getMetadata() + { + return metadata; + } + + public byte[] getValue() + { + return value; + } + } + + /** + * Encodes a JSON string into Variant binary format. + * + * @param json a valid JSON string + * @return the Variant binary encoding (metadata + value) + * @throws IllegalArgumentException if the JSON is malformed + */ + public static VariantBinary fromJson(String json) + { + try { + MetadataBuilder metadataBuilder = new MetadataBuilder(); + + // First pass: collect all object keys into the metadata dictionary + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + collectKeys(parser, metadataBuilder); + } + + // Build the metadata dictionary + byte[] metadata = metadataBuilder.build(); + Map keyIndex = metadataBuilder.getKeyIndex(); + + // Second pass: encode the value + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + parser.nextToken(); + byte[] value = encodeValue(parser, keyIndex); + return new VariantBinary(metadata, value); + } + } + catch (IOException e) { + throw new UncheckedIOException("Failed to encode JSON to Variant binary: " + json, e); + } + } + + /** + * Decodes Variant binary (metadata + value) back to a JSON string. + * + * @param metadata the metadata dictionary bytes + * @param value the encoded value bytes + * @return the JSON string representation + */ + public static String toJson(byte[] metadata, byte[] value) + { + try { + String[] dictionary = decodeMetadata(metadata); + StringWriter writer = new StringWriter(); + try (JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + decodeValue(value, 0, dictionary, gen); + } + return writer.toString(); + } + catch (IOException e) { + throw new UncheckedIOException("Failed to decode Variant binary to JSON", e); + } + } + + // ---- Metadata encoding/decoding ---- + + /** + * Builds the metadata dictionary (sorted key names with byte offsets). + */ + static final class MetadataBuilder + { + private final TreeMap keys = new TreeMap<>(); + + void addKey(String key) + { + if (!keys.containsKey(key)) { + keys.put(key, keys.size()); + } + } + + Map getKeyIndex() + { + Map index = new LinkedHashMap<>(); + int i = 0; + for (String key : keys.keySet()) { + index.put(key, i++); + } + return index; + } + + byte[] build() + { + List keyBytes = new ArrayList<>(); + for (String key : keys.keySet()) { + keyBytes.add(key.getBytes(StandardCharsets.UTF_8)); + } + + int numKeys = keyBytes.size(); + + // Calculate total key data size + int keyDataSize = 0; + for (byte[] kb : keyBytes) { + keyDataSize += kb.length; + } + + // Metadata format: + // [1 byte] version + // [4 bytes] numKeys (uint32 LE) + // [4 bytes * numKeys] byte offsets to each key + // [keyDataSize bytes] concatenated key strings + int totalSize = 1 + 4 + (4 * numKeys) + keyDataSize; + ByteBuffer buf = ByteBuffer.allocate(totalSize); + buf.order(ByteOrder.LITTLE_ENDIAN); + + buf.put((byte) METADATA_VERSION); + buf.putInt(numKeys); + + // Write offsets + int offset = 0; + for (byte[] kb : keyBytes) { + buf.putInt(offset); + offset += kb.length; + } + + // Write key strings + for (byte[] kb : keyBytes) { + buf.put(kb); + } + + return buf.array(); + } + } + + /** + * Decodes the metadata dictionary from binary. + */ + static String[] decodeMetadata(byte[] metadata) + { + if (metadata == null || metadata.length == 0) { + return new String[0]; + } + + ByteBuffer buf = ByteBuffer.wrap(metadata); + buf.order(ByteOrder.LITTLE_ENDIAN); + + int version = buf.get() & 0xFF; + if (version != METADATA_VERSION) { + throw new IllegalArgumentException("Unsupported Variant metadata version: " + version); + } + + int numKeys = buf.getInt(); + if (numKeys == 0) { + return new String[0]; + } + + int[] offsets = new int[numKeys]; + for (int i = 0; i < numKeys; i++) { + offsets[i] = buf.getInt(); + } + + int keyDataStart = buf.position(); + int keyDataEnd = metadata.length; + + String[] keys = new String[numKeys]; + for (int i = 0; i < numKeys; i++) { + int start = keyDataStart + offsets[i]; + int end = (i + 1 < numKeys) ? keyDataStart + offsets[i + 1] : keyDataEnd; + keys[i] = new String(metadata, start, end - start, StandardCharsets.UTF_8); + } + + return keys; + } + + // ---- Value encoding ---- + + private static void collectKeys(JsonParser parser, MetadataBuilder metadataBuilder) throws IOException + { + while (parser.nextToken() != null) { + if (parser.currentToken() == JsonToken.FIELD_NAME) { + metadataBuilder.addKey(parser.getCurrentName()); + } + } + } + + private static byte[] encodeValue(JsonParser parser, Map keyIndex) throws IOException + { + JsonToken token = parser.currentToken(); + if (token == null) { + return encodePrimitive(PRIMITIVE_NULL); + } + + switch (token) { + case VALUE_NULL: + return encodePrimitive(PRIMITIVE_NULL); + case VALUE_TRUE: + return encodePrimitive(PRIMITIVE_TRUE); + case VALUE_FALSE: + return encodePrimitive(PRIMITIVE_FALSE); + case VALUE_NUMBER_INT: + return encodeInteger(parser.getLongValue()); + case VALUE_NUMBER_FLOAT: + return encodeDouble(parser.getDoubleValue()); + case VALUE_STRING: + return encodeString(parser.getText()); + case START_OBJECT: + return encodeObject(parser, keyIndex); + case START_ARRAY: + return encodeArray(parser, keyIndex); + default: + throw new IllegalArgumentException("Unexpected JSON token: " + token); + } + } + + private static byte[] encodePrimitive(int typeInfo) + { + return new byte[] {makeHeader(BASIC_TYPE_PRIMITIVE, typeInfo)}; + } + + private static byte[] encodeInteger(long value) + { + if (value >= Byte.MIN_VALUE && value <= Byte.MAX_VALUE) { + return new byte[] {makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_INT8), (byte) value}; + } + if (value >= Short.MIN_VALUE && value <= Short.MAX_VALUE) { + ByteBuffer buf = ByteBuffer.allocate(3); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.put(makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_INT16)); + buf.putShort((short) value); + return buf.array(); + } + if (value >= Integer.MIN_VALUE && value <= Integer.MAX_VALUE) { + ByteBuffer buf = ByteBuffer.allocate(5); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.put(makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_INT32)); + buf.putInt((int) value); + return buf.array(); + } + ByteBuffer buf = ByteBuffer.allocate(9); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.put(makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_INT64)); + buf.putLong(value); + return buf.array(); + } + + private static byte[] encodeDouble(double value) + { + ByteBuffer buf = ByteBuffer.allocate(9); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.put(makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_DOUBLE)); + buf.putDouble(value); + return buf.array(); + } + + private static byte[] encodeString(String value) + { + byte[] bytes = value.getBytes(StandardCharsets.UTF_8); + if (bytes.length <= MAX_SHORT_STRING_LENGTH) { + byte[] result = new byte[1 + bytes.length]; + result[0] = makeHeader(BASIC_TYPE_SHORT_STRING, bytes.length); + System.arraycopy(bytes, 0, result, 1, bytes.length); + return result; + } + + ByteBuffer buf = ByteBuffer.allocate(1 + 4 + bytes.length); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.put(makeHeader(BASIC_TYPE_PRIMITIVE, PRIMITIVE_STRING)); + buf.putInt(bytes.length); + buf.put(bytes); + return buf.array(); + } + + private static byte[] encodeObject(JsonParser parser, Map keyIndex) throws IOException + { + List fieldKeyIds = new ArrayList<>(); + List fieldValues = new ArrayList<>(); + + while (parser.nextToken() != JsonToken.END_OBJECT) { + String fieldName = parser.getCurrentName(); + parser.nextToken(); + + Integer keyId = keyIndex.get(fieldName); + if (keyId == null) { + throw new IllegalStateException("Key not found in metadata dictionary: " + fieldName); + } + + fieldKeyIds.add(keyId); + fieldValues.add(encodeValue(parser, keyIndex)); + } + + int numFields = fieldKeyIds.size(); + + // Determine offset size needed (1, 2, or 4 bytes) + int totalValueSize = 0; + for (byte[] fv : fieldValues) { + totalValueSize += fv.length; + } + + int offsetSize = getOffsetSize(totalValueSize); + int offsetSizeBits = offsetSizeToBits(offsetSize); + + // Object binary format: + // [1 byte] header (basic_type=2, type_info encodes offset size + field_id size) + // [4 bytes] numFields (uint32 LE) + // [field_id_size * numFields] field key IDs + // [offsetSize * numFields] offsets to field values (relative to start of value data) + // [totalValueSize bytes] concatenated field values + int fieldIdSize = getFieldIdSize(keyIndex.size()); + int fieldIdSizeBits = offsetSizeToBits(fieldIdSize); + + // type_info encodes: bits 0-1 = value_offset_size_minus_1, bits 2-3 = field_id_size_minus_1 + int typeInfo = (offsetSizeBits & 0x03) | ((fieldIdSizeBits & 0x03) << 2); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + out.write(makeHeader(BASIC_TYPE_OBJECT, typeInfo)); + writeLittleEndianInt(out, numFields); + + // Write field key IDs + for (int keyId : fieldKeyIds) { + writeLittleEndianN(out, keyId, fieldIdSize); + } + + // Write field value offsets + int offset = 0; + for (byte[] fv : fieldValues) { + writeLittleEndianN(out, offset, offsetSize); + offset += fv.length; + } + + // Write field values + for (byte[] fv : fieldValues) { + out.write(fv); + } + + return out.toByteArray(); + } + + private static byte[] encodeArray(JsonParser parser, Map keyIndex) throws IOException + { + List elements = new ArrayList<>(); + + while (parser.nextToken() != JsonToken.END_ARRAY) { + elements.add(encodeValue(parser, keyIndex)); + } + + int numElements = elements.size(); + + int totalValueSize = 0; + for (byte[] el : elements) { + totalValueSize += el.length; + } + + int offsetSize = getOffsetSize(totalValueSize); + int offsetSizeBits = offsetSizeToBits(offsetSize); + + // type_info encodes: bits 0-1 = offset_size_minus_1 + int typeInfo = offsetSizeBits & 0x03; + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + out.write(makeHeader(BASIC_TYPE_ARRAY, typeInfo)); + writeLittleEndianInt(out, numElements); + + // Write element offsets + int offset = 0; + for (byte[] el : elements) { + writeLittleEndianN(out, offset, offsetSize); + offset += el.length; + } + + // Write element values + for (byte[] el : elements) { + out.write(el); + } + + return out.toByteArray(); + } + + // ---- Value decoding ---- + + private static void decodeValue(byte[] data, int pos, String[] dictionary, JsonGenerator gen) throws IOException + { + if (pos >= data.length) { + gen.writeNull(); + return; + } + + int header = data[pos] & 0xFF; + int basicType = header >> 6; + int typeInfo = header & 0x3F; + + switch (basicType) { + case BASIC_TYPE_PRIMITIVE: + decodePrimitive(data, pos, typeInfo, gen); + break; + case BASIC_TYPE_SHORT_STRING: + decodeShortString(data, pos, typeInfo, gen); + break; + case BASIC_TYPE_OBJECT: + decodeObject(data, pos, typeInfo, dictionary, gen); + break; + case BASIC_TYPE_ARRAY: + decodeArray(data, pos, typeInfo, dictionary, gen); + break; + default: + throw new IllegalArgumentException("Unknown Variant basic type: " + basicType); + } + } + + private static void decodePrimitive(byte[] data, int pos, int typeInfo, JsonGenerator gen) throws IOException + { + ByteBuffer buf = ByteBuffer.wrap(data); + buf.order(ByteOrder.LITTLE_ENDIAN); + + switch (typeInfo) { + case PRIMITIVE_NULL: + gen.writeNull(); + break; + case PRIMITIVE_TRUE: + gen.writeBoolean(true); + break; + case PRIMITIVE_FALSE: + gen.writeBoolean(false); + break; + case PRIMITIVE_INT8: + gen.writeNumber(data[pos + 1]); + break; + case PRIMITIVE_INT16: + buf.position(pos + 1); + gen.writeNumber(buf.getShort()); + break; + case PRIMITIVE_INT32: + buf.position(pos + 1); + gen.writeNumber(buf.getInt()); + break; + case PRIMITIVE_INT64: + buf.position(pos + 1); + gen.writeNumber(buf.getLong()); + break; + case PRIMITIVE_FLOAT: + buf.position(pos + 1); + gen.writeNumber(buf.getFloat()); + break; + case PRIMITIVE_DOUBLE: + buf.position(pos + 1); + gen.writeNumber(buf.getDouble()); + break; + case PRIMITIVE_STRING: { + buf.position(pos + 1); + int len = buf.getInt(); + String str = new String(data, pos + 5, len, StandardCharsets.UTF_8); + gen.writeString(str); + break; + } + default: + throw new IllegalArgumentException("Unknown Variant primitive type_info: " + typeInfo); + } + } + + private static void decodeShortString(byte[] data, int pos, int typeInfo, JsonGenerator gen) throws IOException + { + int length = typeInfo; + String str = new String(data, pos + 1, length, StandardCharsets.UTF_8); + gen.writeString(str); + } + + private static void decodeObject(byte[] data, int pos, int typeInfo, String[] dictionary, JsonGenerator gen) throws IOException + { + int offsetSize = (typeInfo & 0x03) + 1; + int fieldIdSize = ((typeInfo >> 2) & 0x03) + 1; + + ByteBuffer buf = ByteBuffer.wrap(data); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.position(pos + 1); + + int numFields = buf.getInt(); + + int[] keyIds = new int[numFields]; + for (int i = 0; i < numFields; i++) { + keyIds[i] = readLittleEndianN(data, buf.position(), fieldIdSize); + buf.position(buf.position() + fieldIdSize); + } + + int[] offsets = new int[numFields]; + for (int i = 0; i < numFields; i++) { + offsets[i] = readLittleEndianN(data, buf.position(), offsetSize); + buf.position(buf.position() + offsetSize); + } + + int valueDataStart = buf.position(); + + gen.writeStartObject(); + for (int i = 0; i < numFields; i++) { + String key = dictionary[keyIds[i]]; + gen.writeFieldName(key); + decodeValue(data, valueDataStart + offsets[i], dictionary, gen); + } + gen.writeEndObject(); + } + + private static void decodeArray(byte[] data, int pos, int typeInfo, String[] dictionary, JsonGenerator gen) throws IOException + { + int offsetSize = (typeInfo & 0x03) + 1; + + ByteBuffer buf = ByteBuffer.wrap(data); + buf.order(ByteOrder.LITTLE_ENDIAN); + buf.position(pos + 1); + + int numElements = buf.getInt(); + + int[] offsets = new int[numElements]; + for (int i = 0; i < numElements; i++) { + offsets[i] = readLittleEndianN(data, buf.position(), offsetSize); + buf.position(buf.position() + offsetSize); + } + + int valueDataStart = buf.position(); + + gen.writeStartArray(); + for (int i = 0; i < numElements; i++) { + decodeValue(data, valueDataStart + offsets[i], dictionary, gen); + } + gen.writeEndArray(); + } + + // ---- Helper methods ---- + + static byte makeHeader(int basicType, int typeInfo) + { + return (byte) ((basicType << 6) | (typeInfo & 0x3F)); + } + + private static int getOffsetSize(int maxOffset) + { + if (maxOffset <= 0xFF) { + return 1; + } + if (maxOffset <= 0xFFFF) { + return 2; + } + return 4; + } + + private static int getFieldIdSize(int numKeys) + { + if (numKeys <= 0xFF) { + return 1; + } + if (numKeys <= 0xFFFF) { + return 2; + } + return 4; + } + + private static int offsetSizeToBits(int offsetSize) + { + switch (offsetSize) { + case 1: return 0; + case 2: return 1; + case 4: return 3; + default: throw new IllegalArgumentException("Invalid offset size: " + offsetSize); + } + } + + private static void writeLittleEndianInt(ByteArrayOutputStream out, int value) + { + out.write(value & 0xFF); + out.write((value >> 8) & 0xFF); + out.write((value >> 16) & 0xFF); + out.write((value >> 24) & 0xFF); + } + + private static void writeLittleEndianN(ByteArrayOutputStream out, int value, int size) + { + for (int i = 0; i < size; i++) { + out.write((value >> (i * 8)) & 0xFF); + } + } + + private static int readLittleEndianN(byte[] data, int pos, int size) + { + int value = 0; + for (int i = 0; i < size; i++) { + value |= (data[pos + i] & 0xFF) << (i * 8); + } + return value; + } + + // ---- Phase 2: Binary format detection and auto-decode ---- + + /** + * Checks if the given metadata and value byte arrays form a valid Variant binary encoding. + * Validates the metadata version byte and value header basic type. + * + * @param metadata the metadata dictionary bytes + * @param value the encoded value bytes + * @return true if the data is valid Variant binary format + */ + public static boolean isVariantBinary(byte[] metadata, byte[] value) + { + if (metadata == null || metadata.length < 5 || value == null || value.length == 0) { + return false; + } + int version = metadata[0] & 0xFF; + if (version != METADATA_VERSION) { + return false; + } + int header = value[0] & 0xFF; + int basicType = header >> 6; + return basicType >= BASIC_TYPE_PRIMITIVE && basicType <= BASIC_TYPE_ARRAY; + } + + /** + * Returns the Variant type name from a binary value header byte. + * Used for type introspection of Variant binary data. + * + * @param value the encoded value bytes + * @return type name: "null", "boolean", "integer", "float", "double", "string", "object", "array" + */ + public static String getValueTypeName(byte[] value) + { + if (value == null || value.length == 0) { + return "null"; + } + + int header = value[0] & 0xFF; + int basicType = header >> 6; + int typeInfo = header & 0x3F; + + switch (basicType) { + case BASIC_TYPE_PRIMITIVE: + switch (typeInfo) { + case PRIMITIVE_NULL: return "null"; + case PRIMITIVE_TRUE: + case PRIMITIVE_FALSE: return "boolean"; + case PRIMITIVE_INT8: + case PRIMITIVE_INT16: + case PRIMITIVE_INT32: + case PRIMITIVE_INT64: + return "integer"; + case PRIMITIVE_FLOAT: return "float"; + case PRIMITIVE_DOUBLE: return "double"; + case PRIMITIVE_STRING: return "string"; + default: return "unknown"; + } + case BASIC_TYPE_SHORT_STRING: return "string"; + case BASIC_TYPE_OBJECT: return "object"; + case BASIC_TYPE_ARRAY: return "array"; + default: return "unknown"; + } + } + + /** + * Attempts to decode raw bytes as Variant data, handling both JSON text and binary format. + * If the data starts with a valid JSON character ({, [, ", t, f, n, digit, -), + * it's treated as UTF-8 JSON text. Otherwise, it's treated as binary Variant value + * with empty metadata (suitable for primitives and strings). + * + *

For full binary Variant decoding with metadata dictionary support, + * use {@link #toJson(byte[], byte[])} directly with separate metadata and value arrays. + * + * @param data raw bytes that may be JSON or binary Variant + * @return JSON string representation + */ + public static String decodeVariantAuto(byte[] data) + { + if (data == null || data.length == 0) { + return "null"; + } + byte first = data[0]; + if (first == '{' || first == '[' || first == '"' || first == 't' || + first == 'f' || first == 'n' || (first >= '0' && first <= '9') || first == '-' || first == ' ') { + return new String(data, StandardCharsets.UTF_8); + } + // Try binary Variant decode with empty metadata + try { + byte[] emptyMetadata = new MetadataBuilder().build(); + return toJson(emptyMetadata, data); + } + catch (Exception e) { + return new String(data, StandardCharsets.UTF_8); + } + } +} diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/DeleteFile.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/DeleteFile.java index 16b4943000651..691d073f1a90a 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/DeleteFile.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/DeleteFile.java @@ -42,6 +42,7 @@ public final class DeleteFile private final List equalityFieldIds; private final Map lowerBounds; private final Map upperBounds; + private final long dataSequenceNumber; public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) { @@ -50,6 +51,8 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) Map upperBounds = firstNonNull(deleteFile.upperBounds(), ImmutableMap.of()) .entrySet().stream().collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().array().clone())); + long dataSequenceNumber = deleteFile.dataSequenceNumber() != null ? deleteFile.dataSequenceNumber() : 0L; + return new DeleteFile( fromIcebergFileContent(deleteFile.content()), deleteFile.path().toString(), @@ -58,7 +61,8 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) deleteFile.fileSizeInBytes(), Optional.ofNullable(deleteFile.equalityFieldIds()).orElseGet(ImmutableList::of), lowerBounds, - upperBounds); + upperBounds, + dataSequenceNumber); } @JsonCreator @@ -70,7 +74,8 @@ public DeleteFile( @JsonProperty("fileSizeInBytes") long fileSizeInBytes, @JsonProperty("equalityFieldIds") List equalityFieldIds, @JsonProperty("lowerBounds") Map lowerBounds, - @JsonProperty("upperBounds") Map upperBounds) + @JsonProperty("upperBounds") Map upperBounds, + @JsonProperty("dataSequenceNumber") long dataSequenceNumber) { this.content = requireNonNull(content, "content is null"); this.path = requireNonNull(path, "path is null"); @@ -80,6 +85,7 @@ public DeleteFile( this.equalityFieldIds = ImmutableList.copyOf(requireNonNull(equalityFieldIds, "equalityFieldIds is null")); this.lowerBounds = ImmutableMap.copyOf(requireNonNull(lowerBounds, "lowerBounds is null")); this.upperBounds = ImmutableMap.copyOf(requireNonNull(upperBounds, "upperBounds is null")); + this.dataSequenceNumber = dataSequenceNumber; } @JsonProperty @@ -130,12 +136,19 @@ public Map getUpperBounds() return upperBounds; } + @JsonProperty + public long getDataSequenceNumber() + { + return dataSequenceNumber; + } + @Override public String toString() { return toStringHelper(this) .addValue(path) .add("records", recordCount) + .add("dataSequenceNumber", dataSequenceNumber) .toString(); } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletionVectorPageSink.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletionVectorPageSink.java new file mode 100644 index 0000000000000..8f66ebba56ef2 --- /dev/null +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/delete/IcebergDeletionVectorPageSink.java @@ -0,0 +1,218 @@ +/* + * 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 com.facebook.presto.iceberg.delete; + +import com.facebook.airlift.json.JsonCodec; +import com.facebook.presto.common.Page; +import com.facebook.presto.common.block.Block; +import com.facebook.presto.common.type.BigintType; +import com.facebook.presto.hive.HdfsContext; +import com.facebook.presto.hive.HdfsEnvironment; +import com.facebook.presto.iceberg.CommitTaskData; +import com.facebook.presto.iceberg.FileFormat; +import com.facebook.presto.iceberg.HdfsOutputFile; +import com.facebook.presto.iceberg.MetricsWrapper; +import com.facebook.presto.iceberg.PartitionData; +import com.facebook.presto.spi.ConnectorPageSink; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.PrestoException; +import com.google.common.collect.ImmutableList; +import io.airlift.slice.Slice; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; + +import static com.facebook.presto.iceberg.FileContent.POSITION_DELETES; +import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_BAD_DATA; +import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_WRITER_CLOSE_ERROR; +import static com.facebook.presto.iceberg.IcebergUtil.partitionDataFromJson; +import static io.airlift.slice.Slices.wrappedBuffer; +import static java.util.Objects.requireNonNull; +import static java.util.UUID.randomUUID; +import static java.util.concurrent.CompletableFuture.completedFuture; + +public class IcebergDeletionVectorPageSink + implements ConnectorPageSink +{ + private static final int SERIAL_COOKIE_NO_RUNCONTAINER = 12346; + + private final PartitionSpec partitionSpec; + private final Optional partitionData; + private final HdfsEnvironment hdfsEnvironment; + private final HdfsContext hdfsContext; + private final JsonCodec jsonCodec; + private final ConnectorSession session; + private final String dataFile; + private final LocationProvider locationProvider; + + private final List collectedPositions = new ArrayList<>(); + + public IcebergDeletionVectorPageSink( + PartitionSpec partitionSpec, + Optional partitionDataAsJson, + LocationProvider locationProvider, + HdfsEnvironment hdfsEnvironment, + HdfsContext hdfsContext, + JsonCodec jsonCodec, + ConnectorSession session, + String dataFile) + { + this.partitionSpec = requireNonNull(partitionSpec, "partitionSpec is null"); + this.partitionData = partitionDataFromJson(partitionSpec, partitionDataAsJson); + this.locationProvider = requireNonNull(locationProvider, "locationProvider is null"); + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.hdfsContext = requireNonNull(hdfsContext, "hdfsContext is null"); + this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null"); + this.session = requireNonNull(session, "session is null"); + this.dataFile = requireNonNull(dataFile, "dataFile is null"); + } + + @Override + public long getCompletedBytes() + { + return 0; + } + + @Override + public long getSystemMemoryUsage() + { + return collectedPositions.size() * (long) Integer.BYTES; + } + + @Override + public long getValidationCpuNanos() + { + return 0; + } + + @Override + public CompletableFuture appendPage(Page page) + { + if (page.getChannelCount() != 1) { + throw new PrestoException(ICEBERG_BAD_DATA, + "Expecting Page with one channel but got " + page.getChannelCount()); + } + + Block block = page.getBlock(0); + for (int i = 0; i < block.getPositionCount(); i++) { + long position = BigintType.BIGINT.getLong(block, i); + collectedPositions.add((int) position); + } + return NOT_BLOCKED; + } + + @Override + public CompletableFuture> finish() + { + if (collectedPositions.isEmpty()) { + return completedFuture(ImmutableList.of()); + } + + Collections.sort(collectedPositions); + + byte[] roaringBitmapBytes = serializeRoaringBitmap(collectedPositions); + + String fileName = "dv-" + randomUUID() + ".puffin"; + Path puffinPath = partitionData + .map(partition -> new Path(locationProvider.newDataLocation(partitionSpec, partition, fileName))) + .orElseGet(() -> new Path(locationProvider.newDataLocation(fileName))); + + OutputFile outputFile = new HdfsOutputFile(puffinPath, hdfsEnvironment, hdfsContext); + + long puffinFileSize; + long blobOffset; + long blobLength; + + try { + PuffinWriter writer = hdfsEnvironment.doAs(session.getUser(), () -> + Puffin.write(outputFile).createdBy("presto").build()); + try { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + 0, + 0, + ByteBuffer.wrap(roaringBitmapBytes))); + hdfsEnvironment.doAs(session.getUser(), () -> { + writer.finish(); + return null; + }); + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + finally { + hdfsEnvironment.doAs(session.getUser(), () -> { + writer.close(); + return null; + }); + } + } + catch (IOException e) { + throw new PrestoException(ICEBERG_WRITER_CLOSE_ERROR, "Failed to write deletion vector puffin file", e); + } + + CommitTaskData task = new CommitTaskData( + puffinPath.toString(), + puffinFileSize, + new MetricsWrapper(new Metrics(collectedPositions.size(), null, null, null, null)), + partitionSpec.specId(), + partitionData.map(PartitionData::toJson), + FileFormat.PUFFIN, + dataFile, + POSITION_DELETES, + OptionalLong.of(blobOffset), + OptionalLong.of(blobLength), + OptionalLong.of(collectedPositions.size())); + + return completedFuture(ImmutableList.of(wrappedBuffer(jsonCodec.toJsonBytes(task)))); + } + + @Override + public void abort() + { + // Nothing to clean up since we write the Puffin file atomically in finish() + } + + private static byte[] serializeRoaringBitmap(List sortedPositions) + { + int numPositions = sortedPositions.size(); + int dataSize = 4 + 4 + numPositions * 2; + ByteBuffer buffer = ByteBuffer.allocate(dataSize); + buffer.order(ByteOrder.LITTLE_ENDIAN); + buffer.putInt(SERIAL_COOKIE_NO_RUNCONTAINER); + buffer.putShort((short) 0); + buffer.putShort((short) (numPositions - 1)); + for (int pos : sortedPositions) { + buffer.putShort((short) (pos & 0xFFFF)); + } + return buffer.array(); + } +} diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/function/VariantFunctions.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/function/VariantFunctions.java new file mode 100644 index 0000000000000..82775581f40c5 --- /dev/null +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/function/VariantFunctions.java @@ -0,0 +1,456 @@ +/* + * 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 com.facebook.presto.iceberg.function; + +import com.facebook.presto.common.type.StandardTypes; +import com.facebook.presto.iceberg.VariantBinaryCodec; +import com.facebook.presto.iceberg.VariantBinaryCodec.VariantBinary; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlNullable; +import com.facebook.presto.spi.function.SqlType; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.List; + +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; + +/** + * SQL scalar functions for working with Iceberg V3 Variant data. + * + *

Variant data in Presto is stored as VARCHAR (JSON strings). These functions + * provide field extraction (with dot-path and array indexing), validation, + * normalization, type introspection, key enumeration, binary round-trip, + * and explicit cast capabilities for Variant values. + * + *

Functions are registered via {@code IcebergConnector.getSystemFunctions()} + * and accessed as {@code iceberg.system.(...)}. + * + *

Phase 2: Binary Interoperability

+ *

{@code parse_variant} and {@code variant_binary_roundtrip} exercise the + * {@link VariantBinaryCodec} which implements the Apache Variant binary spec (v1). + * Full Parquet read/write path integration (transparent binary decode/encode in + * {@code IcebergPageSourceProvider} / {@code IcebergPageSink}) is documented as + * a future enhancement — the codec is ready; the page source wiring requires + * detecting VARIANT columns at the Parquet schema level. + * + *

Phase 4: Predicate Pushdown

+ *

{@code IS NULL} / {@code IS NOT NULL} predicates on VARIANT columns already + * work through the VARCHAR type mapping. Pushdown of {@code variant_get(col, 'field') = 'value'} + * would require an optimizer rule to rewrite the expression into a domain constraint, + * which is tracked as future work. + */ +public final class VariantFunctions +{ + private static final JsonFactory JSON_FACTORY = new JsonFactory(); + + private VariantFunctions() {} + + // ---- Phase 3: Enhanced variant_get with dot-path and array indexing ---- + + /** + * Extracts a value from a Variant (JSON) by path expression. + * Supports dot-notation for nested objects and bracket notation for arrays. + * + *

Path syntax: + *

    + *
  • {@code 'name'} — top-level field
  • + *
  • {@code 'address.city'} — nested field via dot-notation
  • + *
  • {@code 'items[0]'} — array element by index
  • + *
  • {@code 'users[0].name'} — combined path
  • + *
+ * + *

Returns NULL if the path doesn't exist, the input is invalid JSON, + * or a path segment references a non-existent field/index. + * For complex values (objects/arrays), returns the JSON string representation. + * + *

Usage: {@code variant_get('{"users":[{"name":"Alice"}]}', 'users[0].name')} → {@code 'Alice'} + */ + @ScalarFunction("variant_get") + @SqlNullable + @SqlType(StandardTypes.VARCHAR) + public static Slice variantGet( + @SqlType(StandardTypes.VARCHAR) Slice variant, + @SqlType(StandardTypes.VARCHAR) Slice path) + { + if (variant == null || path == null) { + return null; + } + + String json = variant.toStringUtf8(); + String pathStr = path.toStringUtf8(); + List segments = parsePath(pathStr); + + try { + String current = json; + for (PathSegment segment : segments) { + if (current == null) { + return null; + } + if (segment.isArrayIndex) { + current = extractArrayElement(current, segment.arrayIndex); + } + else { + current = extractObjectField(current, segment.fieldName); + } + } + return current != null ? Slices.utf8Slice(current) : null; + } + catch (IOException e) { + return null; + } + } + + // ---- Phase 3: variant_keys ---- + + /** + * Returns the top-level keys of a Variant JSON object as a JSON array. + * Returns NULL if the input is not a JSON object. + * + *

Usage: {@code variant_keys('{"name":"Alice","age":30}')} → {@code '["name","age"]'} + */ + @ScalarFunction("variant_keys") + @SqlNullable + @SqlType(StandardTypes.VARCHAR) + public static Slice variantKeys(@SqlType(StandardTypes.VARCHAR) Slice variant) + { + if (variant == null) { + return null; + } + + String json = variant.toStringUtf8(); + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + if (parser.nextToken() != JsonToken.START_OBJECT) { + return null; + } + + StringWriter writer = new StringWriter(); + try (JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + gen.writeStartArray(); + while (parser.nextToken() != JsonToken.END_OBJECT) { + gen.writeString(parser.getCurrentName()); + parser.nextToken(); + parser.skipChildren(); + } + gen.writeEndArray(); + } + return Slices.utf8Slice(writer.toString()); + } + catch (IOException e) { + return null; + } + } + + // ---- Phase 3: variant_type ---- + + /** + * Returns the JSON type of a Variant value as a string. + * Possible return values: "object", "array", "string", "number", "boolean", "null". + * Returns NULL if the input cannot be parsed. + * + *

Usage: {@code variant_type('{"a":1}')} → {@code 'object'} + */ + @ScalarFunction("variant_type") + @SqlNullable + @SqlType(StandardTypes.VARCHAR) + public static Slice variantType(@SqlType(StandardTypes.VARCHAR) Slice variant) + { + if (variant == null) { + return null; + } + + String json = variant.toStringUtf8(); + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + JsonToken token = parser.nextToken(); + if (token == null) { + return null; + } + switch (token) { + case START_OBJECT: return Slices.utf8Slice("object"); + case START_ARRAY: return Slices.utf8Slice("array"); + case VALUE_STRING: return Slices.utf8Slice("string"); + case VALUE_NUMBER_INT: + case VALUE_NUMBER_FLOAT: + return Slices.utf8Slice("number"); + case VALUE_TRUE: + case VALUE_FALSE: + return Slices.utf8Slice("boolean"); + case VALUE_NULL: return Slices.utf8Slice("null"); + default: return null; + } + } + catch (IOException e) { + return null; + } + } + + // ---- Phase 5: to_variant (explicit cast) ---- + + /** + * Validates a JSON string and returns it as a Variant value. + * This is the explicit cast function from VARCHAR to VARIANT. + * Throws an error if the input is not valid JSON. + * + *

Since VARIANT is represented as VARCHAR in Presto, this function serves + * as the explicit validation boundary — it guarantees the output is well-formed JSON. + * + *

Usage: {@code to_variant('{"name":"Alice"}')} → {@code '{"name":"Alice"}'} + */ + @ScalarFunction("to_variant") + @SqlType(StandardTypes.VARCHAR) + public static Slice toVariant(@SqlType(StandardTypes.VARCHAR) Slice json) + { + String input = json.toStringUtf8(); + try { + StringWriter writer = new StringWriter(); + try (JsonParser parser = JSON_FACTORY.createParser(input); + JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + JsonToken token = parser.nextToken(); + if (token == null) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Empty input is not valid Variant JSON"); + } + gen.copyCurrentStructure(parser); + if (parser.nextToken() != null) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, + "Trailing content after JSON value"); + } + } + return Slices.utf8Slice(writer.toString()); + } + catch (PrestoException e) { + throw e; + } + catch (IOException e) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, + "Invalid JSON for Variant: " + e.getMessage(), e); + } + } + + // ---- Phase 2: parse_variant (binary codec validation) ---- + + /** + * Parses and validates a JSON string as a Variant value by encoding it + * to Variant binary format (Apache Iceberg V3 spec) and decoding back. + * Returns the normalized (compact) JSON representation. + * Throws if the input is not valid JSON. + * + *

This exercises the full binary codec round-trip, validating that + * the data can be represented in Variant binary format for interoperability + * with other engines (Spark, Trino). + * + *

Usage: {@code parse_variant('{"name":"Alice"}')} → {@code '{"name":"Alice"}'} + */ + @ScalarFunction("parse_variant") + @SqlType(StandardTypes.VARCHAR) + public static Slice parseVariant(@SqlType(StandardTypes.VARCHAR) Slice json) + { + String input = json.toStringUtf8(); + try { + VariantBinary binary = VariantBinaryCodec.fromJson(input); + String normalized = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + return Slices.utf8Slice(normalized); + } + catch (Exception e) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, + "Invalid JSON for Variant: " + e.getMessage(), e); + } + } + + // ---- Phase 2: variant_to_json ---- + + /** + * Converts a Variant value to its normalized JSON string representation. + * Normalizes the JSON through Jackson round-trip (compact form). + * + *

Usage: {@code variant_to_json(variant_column)} → {@code '{"name":"Alice"}'} + */ + @ScalarFunction("variant_to_json") + @SqlType(StandardTypes.VARCHAR) + public static Slice variantToJson(@SqlType(StandardTypes.VARCHAR) Slice variant) + { + String input = variant.toStringUtf8(); + try { + StringWriter writer = new StringWriter(); + try (JsonParser parser = JSON_FACTORY.createParser(input); + JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + parser.nextToken(); + gen.copyCurrentStructure(parser); + } + return Slices.utf8Slice(writer.toString()); + } + catch (IOException e) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, + "Invalid Variant JSON: " + e.getMessage(), e); + } + } + + // ---- Phase 2: variant_binary_roundtrip ---- + + /** + * Encodes a JSON string into Variant binary format (Apache Iceberg V3 spec) + * and decodes it back to JSON. Validates binary round-trip fidelity. + * Useful for testing binary interoperability with other engines (Spark, Trino). + * + *

Usage: {@code variant_binary_roundtrip('{"a":1}')} → {@code '{"a":1}'} + */ + @ScalarFunction("variant_binary_roundtrip") + @SqlType(StandardTypes.VARCHAR) + public static Slice variantBinaryRoundtrip(@SqlType(StandardTypes.VARCHAR) Slice json) + { + String input = json.toStringUtf8(); + try { + VariantBinary binary = VariantBinaryCodec.fromJson(input); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + return Slices.utf8Slice(decoded); + } + catch (Exception e) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, + "Failed Variant binary round-trip: " + e.getMessage(), e); + } + } + + // ---- Path parsing and JSON navigation helpers ---- + + private static final class PathSegment + { + final String fieldName; + final int arrayIndex; + final boolean isArrayIndex; + + PathSegment(String fieldName) + { + this.fieldName = fieldName; + this.arrayIndex = -1; + this.isArrayIndex = false; + } + + PathSegment(int arrayIndex) + { + this.fieldName = null; + this.arrayIndex = arrayIndex; + this.isArrayIndex = true; + } + } + + static List parsePath(String path) + { + List segments = new ArrayList<>(); + StringBuilder current = new StringBuilder(); + + for (int i = 0; i < path.length(); i++) { + char c = path.charAt(i); + if (c == '.') { + if (current.length() > 0) { + segments.add(new PathSegment(current.toString())); + current.setLength(0); + } + } + else if (c == '[') { + if (current.length() > 0) { + segments.add(new PathSegment(current.toString())); + current.setLength(0); + } + int end = path.indexOf(']', i); + if (end == -1) { + segments.add(new PathSegment(path.substring(i))); + return segments; + } + String indexStr = path.substring(i + 1, end); + try { + segments.add(new PathSegment(Integer.parseInt(indexStr))); + } + catch (NumberFormatException e) { + segments.add(new PathSegment(indexStr)); + } + i = end; + } + else { + current.append(c); + } + } + + if (current.length() > 0) { + segments.add(new PathSegment(current.toString())); + } + return segments; + } + + private static String extractObjectField(String json, String fieldName) throws IOException + { + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + if (parser.nextToken() != JsonToken.START_OBJECT) { + return null; + } + + while (parser.nextToken() != JsonToken.END_OBJECT) { + String currentField = parser.getCurrentName(); + JsonToken valueToken = parser.nextToken(); + + if (fieldName.equals(currentField)) { + if (valueToken == JsonToken.VALUE_NULL) { + return "null"; + } + if (valueToken == JsonToken.START_OBJECT || valueToken == JsonToken.START_ARRAY) { + StringWriter writer = new StringWriter(); + try (JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + gen.copyCurrentStructure(parser); + } + return writer.toString(); + } + return parser.getText(); + } + parser.skipChildren(); + } + } + return null; + } + + private static String extractArrayElement(String json, int index) throws IOException + { + try (JsonParser parser = JSON_FACTORY.createParser(json)) { + if (parser.nextToken() != JsonToken.START_ARRAY) { + return null; + } + + int currentIndex = 0; + while (parser.nextToken() != JsonToken.END_ARRAY) { + if (currentIndex == index) { + JsonToken token = parser.currentToken(); + if (token == JsonToken.VALUE_NULL) { + return "null"; + } + if (token == JsonToken.START_OBJECT || token == JsonToken.START_ARRAY) { + StringWriter writer = new StringWriter(); + try (JsonGenerator gen = JSON_FACTORY.createGenerator(writer)) { + gen.copyCurrentStructure(parser); + } + return writer.toString(); + } + return parser.getText(); + } + parser.skipChildren(); + currentIndex++; + } + } + return null; + } +} diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/RewriteDeleteFilesProcedure.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/RewriteDeleteFilesProcedure.java new file mode 100644 index 0000000000000..c2c1d01f2cf73 --- /dev/null +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/procedure/RewriteDeleteFilesProcedure.java @@ -0,0 +1,356 @@ +/* + * 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 com.facebook.presto.iceberg.procedure; + +import com.facebook.presto.iceberg.IcebergAbstractMetadata; +import com.facebook.presto.iceberg.IcebergMetadataFactory; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.classloader.ThreadContextClassLoader; +import com.facebook.presto.spi.procedure.Procedure; +import com.facebook.presto.spi.procedure.Procedure.Argument; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import jakarta.inject.Inject; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.BlobMetadata; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinReader; +import org.apache.iceberg.puffin.PuffinWriter; + +import javax.inject.Provider; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.lang.invoke.MethodHandle; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static com.facebook.presto.common.block.MethodHandleUtil.methodHandle; +import static com.facebook.presto.common.type.StandardTypes.VARCHAR; +import static com.facebook.presto.iceberg.IcebergUtil.getIcebergTable; +import static java.util.Objects.requireNonNull; + +/** + * Procedure to compact deletion vectors (DVs) on V3 Iceberg tables. + * + * When multiple DELETE operations target rows in the same data file, each produces + * a separate DV (Puffin file). This procedure merges all DVs per data file into + * a single consolidated DV, reducing metadata overhead and improving read performance. + * + * Usage: CALL iceberg.system.rewrite_delete_files('schema', 'table') + */ +public class RewriteDeleteFilesProcedure + implements Provider +{ + private static final MethodHandle REWRITE_DELETE_FILES = methodHandle( + RewriteDeleteFilesProcedure.class, + "rewriteDeleteFiles", + ConnectorSession.class, + String.class, + String.class); + + private final IcebergMetadataFactory metadataFactory; + + @Inject + public RewriteDeleteFilesProcedure(IcebergMetadataFactory metadataFactory) + { + this.metadataFactory = requireNonNull(metadataFactory, "metadataFactory is null"); + } + + @Override + public Procedure get() + { + return new Procedure( + "system", + "rewrite_delete_files", + ImmutableList.of( + new Argument("schema", VARCHAR), + new Argument("table_name", VARCHAR)), + REWRITE_DELETE_FILES.bindTo(this)); + } + + public void rewriteDeleteFiles(ConnectorSession clientSession, String schemaName, String tableName) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(getClass().getClassLoader())) { + SchemaTableName schemaTableName = new SchemaTableName(schemaName, tableName); + IcebergAbstractMetadata metadata = (IcebergAbstractMetadata) metadataFactory.create(); + Table icebergTable = getIcebergTable(metadata, clientSession, schemaTableName); + + int formatVersion = ((BaseTable) icebergTable).operations().current().formatVersion(); + if (formatVersion < 3) { + return; + } + + // Group delete files by their referenced data file + Map> dvsByDataFile = new HashMap<>(); + Set allDeleteFiles = new HashSet<>(); + + try (CloseableIterable tasks = icebergTable.newScan().planFiles()) { + CloseableIterator iterator = tasks.iterator(); + while (iterator.hasNext()) { + FileScanTask task = iterator.next(); + String dataFilePath = task.file().path().toString(); + for (DeleteFile deleteFile : task.deletes()) { + if (deleteFile.format() == FileFormat.PUFFIN && + deleteFile.content() == FileContent.POSITION_DELETES) { + dvsByDataFile.computeIfAbsent(dataFilePath, k -> new ArrayList<>()).add(deleteFile); + allDeleteFiles.add(deleteFile); + } + } + } + iterator.close(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + + // Find data files with multiple DVs that need compaction + Set filesToRemove = new HashSet<>(); + Set filesToAdd = new HashSet<>(); + + for (Map.Entry> entry : dvsByDataFile.entrySet()) { + List dvs = entry.getValue(); + if (dvs.size() <= 1) { + continue; + } + + String dataFilePath = entry.getKey(); + + // Merge roaring bitmaps from all DVs for this data file + Set mergedPositions = new HashSet<>(); + for (DeleteFile dv : dvs) { + readDeletionVectorPositions(icebergTable, dv, mergedPositions); + filesToRemove.add(dv); + } + + // Write consolidated DV + DeleteFile mergedDv = writeMergedDeletionVector( + icebergTable, + dvs.get(0), + dataFilePath, + mergedPositions); + filesToAdd.add(mergedDv); + } + + if (filesToRemove.isEmpty()) { + metadata.commit(); + return; + } + + // Commit the rewrite: remove old DVs, add merged DVs + RewriteFiles rewriteFiles = icebergTable.newRewrite() + .rewriteFiles(ImmutableSet.of(), filesToRemove, ImmutableSet.of(), filesToAdd); + rewriteFiles.commit(); + metadata.commit(); + } + } + + private void readDeletionVectorPositions(Table table, DeleteFile dv, Set positions) + { + InputFile inputFile = table.io().newInputFile(dv.path().toString()); + try (PuffinReader reader = Puffin.read(inputFile).build()) { + List blobMetadataList = reader.fileMetadata().blobs(); + if (blobMetadataList.isEmpty()) { + return; + } + for (org.apache.iceberg.puffin.Pair pair : reader.readAll(blobMetadataList)) { + ByteBuffer blobData = pair.second(); + deserializeRoaringBitmap(blobData, positions); + } + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private DeleteFile writeMergedDeletionVector( + Table table, + DeleteFile templateDv, + String dataFilePath, + Set mergedPositions) + { + List sortedPositions = new ArrayList<>(mergedPositions); + sortedPositions.sort(Integer::compareTo); + byte[] roaringBytes = serializeRoaringBitmap(sortedPositions); + + String fileName = "dv-" + UUID.randomUUID() + ".puffin"; + String dvPath = table.location() + "/data/" + fileName; + OutputFile outputFile = table.io().newOutputFile(dvPath); + + long puffinFileSize; + long blobOffset; + long blobLength; + try { + PuffinWriter writer = Puffin.write(outputFile).createdBy("presto").build(); + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + 0, + 0, + ByteBuffer.wrap(roaringBytes))); + writer.finish(); + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + writer.close(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + + return FileMetadata.deleteFileBuilder(table.specs().get(templateDv.specId())) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(sortedPositions.size()) + .withContentSizeInBytes(blobLength) + .withContentOffset(blobOffset) + .withReferencedDataFile(dataFilePath) + .build(); + } + + private static void deserializeRoaringBitmap(ByteBuffer buffer, Set positions) + { + byte[] bytes = new byte[buffer.remaining()]; + buffer.get(bytes); + java.nio.ByteBuffer buf = java.nio.ByteBuffer.wrap(bytes).order(java.nio.ByteOrder.LITTLE_ENDIAN); + + int cookie = buf.getInt(); + boolean isRunContainer = (cookie & 0xFFFF) == 12347; + int numContainers; + if (isRunContainer) { + numContainers = (cookie >>> 16) + 1; + // skip run bitmap + int runBitmapBytes = (numContainers + 7) / 8; + buf.position(buf.position() + runBitmapBytes); + } + else if ((cookie & 0xFFFF) == 12346) { + numContainers = (cookie >>> 16) + 1; + } + else { + return; + } + + int[] keys = new int[numContainers]; + int[] cardinalities = new int[numContainers]; + for (int i = 0; i < numContainers; i++) { + keys[i] = Short.toUnsignedInt(buf.getShort()); + cardinalities[i] = Short.toUnsignedInt(buf.getShort()) + 1; + } + + for (int i = 0; i < numContainers; i++) { + int highBits = keys[i] << 16; + if (cardinalities[i] <= 4096) { + // Array container + for (int j = 0; j < cardinalities[i]; j++) { + positions.add(highBits | Short.toUnsignedInt(buf.getShort())); + } + } + else { + // Bitmap container + for (int wordIdx = 0; wordIdx < 1024; wordIdx++) { + long word = buf.getLong(); + while (word != 0) { + int bit = Long.numberOfTrailingZeros(word); + positions.add(highBits | (wordIdx * 64 + bit)); + word &= word - 1; + } + } + } + } + } + + static byte[] serializeRoaringBitmap(List sortedPositions) + { + // Group positions into containers (each container covers 2^16 values) + Map> containers = new HashMap<>(); + for (int pos : sortedPositions) { + int key = pos >>> 16; + int low = pos & 0xFFFF; + containers.computeIfAbsent(key, k -> new ArrayList<>()).add(low); + } + + List sortedKeys = new ArrayList<>(containers.keySet()); + sortedKeys.sort(Integer::compareTo); + + // Calculate size + int numContainers = sortedKeys.size(); + // Cookie (4 bytes) + key-cardinality pairs (4 bytes each) + int headerSize = 4 + numContainers * 4; + int dataSize = 0; + for (int key : sortedKeys) { + int card = containers.get(key).size(); + if (card <= 4096) { + dataSize += card * 2; // array container + } + else { + dataSize += 1024 * 8; // bitmap container + } + } + + java.nio.ByteBuffer buf = java.nio.ByteBuffer.allocate(headerSize + dataSize) + .order(java.nio.ByteOrder.LITTLE_ENDIAN); + + // Cookie: SERIAL_COOKIE_NO_RUNCONTAINER (12346) | (numContainers - 1) << 16 + buf.putInt(12346 | ((numContainers - 1) << 16)); + + // Key-cardinality pairs + for (int key : sortedKeys) { + buf.putShort((short) key); + buf.putShort((short) (containers.get(key).size() - 1)); + } + + // Container data + for (int key : sortedKeys) { + List values = containers.get(key); + values.sort(Integer::compareTo); + if (values.size() <= 4096) { + for (int val : values) { + buf.putShort((short) val); + } + } + else { + long[] bitmap = new long[1024]; + for (int val : values) { + bitmap[val >>> 6] |= 1L << (val & 63); + } + for (long word : bitmap) { + buf.putLong(word); + } + } + } + + return buf.array(); + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergV3.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergV3.java index fb28aee470d42..de9de3e3fd095 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergV3.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergV3.java @@ -15,6 +15,7 @@ import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; @@ -30,12 +31,20 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.types.Types; import org.testng.annotations.Test; import java.io.File; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.file.Path; import java.util.Map; import java.util.OptionalInt; +import java.util.UUID; import static com.facebook.presto.iceberg.CatalogType.HADOOP; import static com.facebook.presto.iceberg.FileFormat.PARQUET; @@ -44,6 +53,8 @@ import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class TestIcebergV3 extends AbstractTestQueryFramework @@ -137,7 +148,7 @@ public void testInsertIntoV3Table() } @Test - public void testDeleteOnV3TableNotSupported() + public void testDeleteOnV3Table() { String tableName = "test_v3_delete"; try { @@ -147,8 +158,31 @@ public void testDeleteOnV3TableNotSupported() + " VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0)", 3); assertQuery("SELECT * FROM " + tableName + " ORDER BY id", "VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0)"); - assertThatThrownBy(() -> getQueryRunner().execute("DELETE FROM " + tableName + " WHERE id = 1")) - .hasMessageContaining("Iceberg table updates for format version 3 are not supported yet"); + + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0), (3, 'Charlie', 300.0)"); + + Table table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + + // Verify DV metadata: the delete should have produced a PUFFIN-format deletion vector + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { + assertEquals(deleteFile.format(), FileFormat.PUFFIN); + assertTrue(deleteFile.path().toString().endsWith(".puffin"), + "Deletion vector file should have .puffin extension"); + assertTrue(deleteFile.fileSizeInBytes() > 0, + "Deletion vector file size should be positive"); + } + } + } + + // Delete more rows + assertUpdate("DELETE FROM " + tableName + " WHERE id = 3", 1); + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0)"); } finally { dropTable(tableName); @@ -207,7 +241,7 @@ public void testMetadataDeleteOnV3PartitionedTable() } @Test - public void testUpdateOnV3TableNotSupported() + public void testUpdateOnV3Table() { String tableName = "test_v3_update"; try { @@ -218,9 +252,10 @@ public void testUpdateOnV3TableNotSupported() 3); assertQuery("SELECT * FROM " + tableName + " ORDER BY id", "VALUES (1, 'Alice', 'active', 85.5), (2, 'Bob', 'active', 92.0), (3, 'Charlie', 'inactive', 78.3)"); - assertThatThrownBy(() -> getQueryRunner() - .execute("UPDATE " + tableName + " SET status = 'updated', score = 95.0 WHERE id = 1")) - .hasMessageContaining("Iceberg table updates for format version 3 are not supported yet"); + + assertUpdate("UPDATE " + tableName + " SET status = 'updated', score = 95.0 WHERE id = 1", 1); + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 'updated', 95.0), (2, 'Bob', 'active', 92.0), (3, 'Charlie', 'inactive', 78.3)"); } finally { dropTable(tableName); @@ -228,7 +263,7 @@ public void testUpdateOnV3TableNotSupported() } @Test - public void testMergeOnV3TableNotSupported() + public void testMergeOnV3Table() { String tableName = "test_v3_merge_target"; String sourceTable = "test_v3_merge_source"; @@ -242,11 +277,14 @@ public void testMergeOnV3TableNotSupported() assertQuery("SELECT * FROM " + tableName + " ORDER BY id", "VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0)"); assertQuery("SELECT * FROM " + sourceTable + " ORDER BY id", "VALUES (1, 'Alice Updated', 150.0), (3, 'Charlie', 300.0)"); - assertThatThrownBy(() -> getQueryRunner().execute( + + getQueryRunner().execute( "MERGE INTO " + tableName + " t USING " + sourceTable + " s ON t.id = s.id " + "WHEN MATCHED THEN UPDATE SET name = s.name, value = s.value " + - "WHEN NOT MATCHED THEN INSERT (id, name, value) VALUES (s.id, s.name, s.value)")) - .hasMessageContaining("Iceberg table updates for format version 3 are not supported yet"); + "WHEN NOT MATCHED THEN INSERT (id, name, value) VALUES (s.id, s.name, s.value)"); + + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice Updated', 150.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0)"); } finally { dropTable(tableName); @@ -279,10 +317,10 @@ public void testOptimizeOnV3Table() } @Test - public void testPuffinDeletionVectorsNotSupported() + public void testPuffinDeletionVectorsAccepted() throws Exception { - String tableName = "test_puffin_deletion_vectors_not_supported"; + String tableName = "test_puffin_deletion_vectors_accepted"; try { assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two')", 2); @@ -309,7 +347,20 @@ public void testPuffinDeletionVectorsNotSupported() .commit(); } - assertQueryFails("SELECT * FROM " + tableName, "Iceberg deletion vectors.*PUFFIN.*not supported"); + // The PUFFIN delete file is now accepted by the split source (no longer + // throws NOT_SUPPORTED). The query will fail downstream because the fake + // .puffin file doesn't exist on disk, but the important thing is that the + // coordinator no longer rejects it at split enumeration time. + try { + computeActual("SELECT * FROM " + tableName); + } + catch (RuntimeException e) { + // Verify the error is NOT the old "PUFFIN not supported" rejection. + // Other failures (e.g., fake .puffin file not on disk) are acceptable. + assertFalse( + e.getMessage().contains("Iceberg deletion vectors") && e.getMessage().contains("not supported"), + "PUFFIN deletion vectors should be accepted, not rejected: " + e.getMessage()); + } } finally { dropTable(tableName); @@ -466,6 +517,635 @@ private File getCatalogDirectory() return catalogDirectory.toFile(); } + @Test + public void testDeletionVectorEndToEnd() + throws Exception + { + String tableName = "test_dv_end_to_end"; + try { + // Step 1: Create V3 table and insert data + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two'), (3, 'three'), (4, 'four'), (5, 'five')", 5); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 5"); + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (1, 'one'), (2, 'two'), (3, 'three'), (4, 'four'), (5, 'five')"); + + Table table = loadTable(tableName); + + // Step 2: Write a real Puffin file with a valid roaring bitmap deletion vector. + // The roaring bitmap uses the portable "no-run" format (cookie = 12346). + // We mark row positions 1 and 3 (0-indexed) as deleted — these correspond + // to the rows (2, 'two') and (4, 'four') in insertion order. + byte[] roaringBitmapBytes = serializeRoaringBitmapNoRun(new int[] {1, 3}); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + String dataFilePath = task.file().path().toString(); + + // Write the roaring bitmap as a blob inside a Puffin file + String dvPath = table.location() + "/data/dv-" + UUID.randomUUID() + ".puffin"; + OutputFile outputFile = table.io().newOutputFile(dvPath); + + long blobOffset; + long blobLength; + long puffinFileSize; + + try (PuffinWriter writer = Puffin.write(outputFile) + .createdBy("presto-test") + .build()) { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + ByteBuffer.wrap(roaringBitmapBytes))); + writer.finish(); + + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + + // Step 3: Attach the Puffin DV file to the table using Iceberg API + DeleteFile puffinDeleteFile = FileMetadata.deleteFileBuilder(task.spec()) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(2) + .withContentOffset(blobOffset) + .withContentSizeInBytes(blobLength) + .withReferencedDataFile(dataFilePath) + .build(); + + table.newRowDelta() + .addDeletes(puffinDeleteFile) + .commit(); + } + + // Step 4: Verify coordinator-side metadata is correct. + // Reload the table and verify the DV file was committed with correct metadata. + table = loadTable(tableName); + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + java.util.List deletes = task.deletes(); + assertFalse(deletes.isEmpty(), "Table should have deletion vector files"); + + org.apache.iceberg.DeleteFile dvFile = deletes.get(0); + assertEquals(dvFile.format(), FileFormat.PUFFIN, "Delete file should be PUFFIN format"); + assertEquals(dvFile.recordCount(), 2, "Delete file should have 2 deleted records"); + assertTrue(dvFile.fileSizeInBytes() > 0, "PUFFIN file size should be positive"); + } + + // Step 5: Verify the coordinator can enumerate splits without error. + // The query will attempt to read data. On a Java worker, the actual DV + // reading is not implemented (that's in Velox's DeletionVectorReader), + // so we verify the coordinator path succeeds by running a SELECT. + // The PUFFIN delete file will either be silently ignored by the Java + // page source (returning all 5 rows) or cause a non-DV-rejection error. + try { + computeActual("SELECT * FROM " + tableName); + } + catch (RuntimeException e) { + // The Java page source may fail trying to read the PUFFIN file as + // positional deletes (since it doesn't have a DV reader). That's expected. + // The important assertion is that the error is NOT the old + // "PUFFIN not supported" rejection from the coordinator. + assertFalse( + e.getMessage().contains("Iceberg deletion vectors") && e.getMessage().contains("not supported"), + "Coordinator should not reject PUFFIN deletion vectors: " + e.getMessage()); + + // Also verify it's not a file-not-found error (the Puffin file exists) + assertFalse( + e.getMessage().contains("FileNotFoundException"), + "PUFFIN file should exist on disk: " + e.getMessage()); + } + } + finally { + dropTable(tableName); + } + } + + /** + * Serializes a roaring bitmap in the portable "no-run" format. + * Standard format: cookie = (numContainers - 1) << 16 | 12346 as a single int32, + * followed by container headers (key + cardinality-1, 2 bytes each), + * then container data (sorted uint16 values). + * Only supports positions within a single container (all < 65536). + */ + private static byte[] serializeRoaringBitmapNoRun(int[] positions) + { + // Cookie with embedded numContainers (4 bytes) + // + 1 container key-cardinality pair (4 bytes) + // + sorted uint16 values (2 bytes each) + int numPositions = positions.length; + int dataSize = 4 + 4 + numPositions * 2; + ByteBuffer buffer = ByteBuffer.allocate(dataSize); + buffer.order(ByteOrder.LITTLE_ENDIAN); + + // Cookie: (numContainers - 1) << 16 | SERIAL_COOKIE_NO_RUNCONTAINER + // For 1 container: (1 - 1) << 16 | 12346 = 12346 + buffer.putInt(12346); + // Container key (high 16 bits): 0, cardinality - 1 + buffer.putShort((short) 0); + buffer.putShort((short) (numPositions - 1)); + // Container data: sorted uint16 values (low 16 bits of each position) + java.util.Arrays.sort(positions); + for (int pos : positions) { + buffer.putShort((short) (pos & 0xFFFF)); + } + + return buffer.array(); + } + + @Test + public void testDeletionVectorDeletesAllRows() + throws Exception + { + String tableName = "test_dv_deletes_all_rows"; + try { + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two'), (3, 'three')", 3); + + Table table = loadTable(tableName); + + // Write a DV that deletes all 3 rows (positions 0, 1, 2). + byte[] roaringBitmapBytes = serializeRoaringBitmapNoRun(new int[] {0, 1, 2}); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + String dataFilePath = task.file().path().toString(); + + String dvPath = table.location() + "/data/dv-all-" + UUID.randomUUID() + ".puffin"; + OutputFile outputFile = table.io().newOutputFile(dvPath); + + long blobOffset; + long blobLength; + long puffinFileSize; + + try (PuffinWriter writer = Puffin.write(outputFile) + .createdBy("presto-test") + .build()) { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + ByteBuffer.wrap(roaringBitmapBytes))); + writer.finish(); + + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + + DeleteFile puffinDeleteFile = FileMetadata.deleteFileBuilder(task.spec()) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(3) + .withContentOffset(blobOffset) + .withContentSizeInBytes(blobLength) + .withReferencedDataFile(dataFilePath) + .build(); + + table.newRowDelta() + .addDeletes(puffinDeleteFile) + .commit(); + } + + // Verify the coordinator can enumerate splits. On Java workers the DV + // reader isn't implemented, so the query may either succeed (returning + // all rows because the Java page source ignores the DV) or fail with a + // non-rejection error. The key assertion is that it doesn't throw + // "PUFFIN not supported". + try { + computeActual("SELECT * FROM " + tableName); + } + catch (RuntimeException e) { + assertFalse( + e.getMessage().contains("Iceberg deletion vectors") && e.getMessage().contains("not supported"), + "Coordinator should not reject PUFFIN deletion vectors: " + e.getMessage()); + } + } + finally { + dropTable(tableName); + } + } + + @Test + public void testDeletionVectorOnMultipleDataFiles() + throws Exception + { + String tableName = "test_dv_multiple_data_files"; + try { + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + // Two separate inserts create two separate data files. + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two'), (3, 'three')", 3); + assertUpdate("INSERT INTO " + tableName + " VALUES (4, 'four'), (5, 'five'), (6, 'six')", 3); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 6"); + + Table table = loadTable(tableName); + + // Attach a DV only to the first data file (positions 0 and 2 → rows 1 + // and 3 from the first insert). The second data file has no deletes. + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask firstTask = tasks.iterator().next(); + String firstDataFilePath = firstTask.file().path().toString(); + + byte[] roaringBitmapBytes = serializeRoaringBitmapNoRun(new int[] {0, 2}); + String dvPath = table.location() + "/data/dv-partial-" + UUID.randomUUID() + ".puffin"; + OutputFile outputFile = table.io().newOutputFile(dvPath); + + long blobOffset; + long blobLength; + long puffinFileSize; + + try (PuffinWriter writer = Puffin.write(outputFile) + .createdBy("presto-test") + .build()) { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + ByteBuffer.wrap(roaringBitmapBytes))); + writer.finish(); + + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + + DeleteFile puffinDeleteFile = FileMetadata.deleteFileBuilder(firstTask.spec()) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(2) + .withContentOffset(blobOffset) + .withContentSizeInBytes(blobLength) + .withReferencedDataFile(firstDataFilePath) + .build(); + + table.newRowDelta() + .addDeletes(puffinDeleteFile) + .commit(); + } + + // Verify coordinator metadata: only the first file's task should have deletes. + table = loadTable(tableName); + int tasksWithDeletes = 0; + int tasksWithoutDeletes = 0; + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + if (task.deletes().isEmpty()) { + tasksWithoutDeletes++; + } + else { + tasksWithDeletes++; + assertEquals(task.deletes().size(), 1, "First data file should have exactly 1 DV"); + assertEquals(task.deletes().get(0).format(), FileFormat.PUFFIN); + } + } + } + assertEquals(tasksWithDeletes, 1, "Exactly one data file should have a DV"); + assertEquals(tasksWithoutDeletes, 1, "Exactly one data file should have no deletes"); + + // Run a query — coordinator should enumerate splits without error. + try { + computeActual("SELECT * FROM " + tableName); + } + catch (RuntimeException e) { + assertFalse( + e.getMessage().contains("Iceberg deletion vectors") && e.getMessage().contains("not supported"), + "Coordinator should not reject PUFFIN deletion vectors: " + e.getMessage()); + } + } + finally { + dropTable(tableName); + } + } + + @Test + public void testV3SchemaEvolution() + { + String tableName = "test_v3_schema_evolution"; + try { + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two')", 2); + + // Add a new column via Iceberg API + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("score", org.apache.iceberg.types.Types.DoubleType.get()) + .commit(); + + // New inserts include the new column + assertUpdate("INSERT INTO " + tableName + " VALUES (3, 'three', 99.5)", 1); + + // Verify all rows are readable (old rows have NULL for the new column) + assertQuery("SELECT id, value FROM " + tableName + " ORDER BY id", + "VALUES (1, 'one'), (2, 'two'), (3, 'three')"); + assertQuery("SELECT id, score FROM " + tableName + " WHERE score IS NOT NULL", + "VALUES (3, 99.5)"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE score IS NULL", "SELECT 2"); + + // Rename a column + table = loadTable(tableName); + table.updateSchema() + .renameColumn("value", "label") + .commit(); + + // Verify reads still work after rename + assertQuery("SELECT id, label FROM " + tableName + " ORDER BY id", + "VALUES (1, 'one'), (2, 'two'), (3, 'three')"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testV3MultipleSnapshotsWithDV() + throws Exception + { + String tableName = "test_v3_multi_snapshot_dv"; + try { + // Snapshot 1: initial data + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two'), (3, 'three')", 3); + + Table table = loadTable(tableName); + long snapshot1Id = table.currentSnapshot().snapshotId(); + + // Snapshot 2: attach a DV deleting row at position 1 (row id=2, 'two') + byte[] roaringBitmapBytes = serializeRoaringBitmapNoRun(new int[] {1}); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + String dataFilePath = task.file().path().toString(); + + String dvPath = table.location() + "/data/dv-snap-" + UUID.randomUUID() + ".puffin"; + OutputFile outputFile = table.io().newOutputFile(dvPath); + + long blobOffset; + long blobLength; + long puffinFileSize; + + try (PuffinWriter writer = Puffin.write(outputFile) + .createdBy("presto-test") + .build()) { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + ByteBuffer.wrap(roaringBitmapBytes))); + writer.finish(); + + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + + DeleteFile puffinDeleteFile = FileMetadata.deleteFileBuilder(task.spec()) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(1) + .withContentOffset(blobOffset) + .withContentSizeInBytes(blobLength) + .withReferencedDataFile(dataFilePath) + .build(); + + table.newRowDelta() + .addDeletes(puffinDeleteFile) + .commit(); + } + + // Snapshot 3: more data added after the DV + assertUpdate("INSERT INTO " + tableName + " VALUES (4, 'four'), (5, 'five')", 2); + + // Verify the table now has 3 snapshots + table = loadTable(tableName); + int snapshotCount = 0; + for (org.apache.iceberg.Snapshot snapshot : table.snapshots()) { + snapshotCount++; + } + assertTrue(snapshotCount >= 3, "Table should have at least 3 snapshots, got: " + snapshotCount); + + // Verify coordinator can enumerate all splits (including those with DVs + // and those from the post-DV insert). + try (CloseableIterable tasks = table.newScan().planFiles()) { + int totalFiles = 0; + int filesWithDeletes = 0; + for (FileScanTask task : tasks) { + totalFiles++; + if (!task.deletes().isEmpty()) { + filesWithDeletes++; + } + } + assertEquals(totalFiles, 2, "Should have 2 data files (one from each insert)"); + assertEquals(filesWithDeletes, 1, "Only the first data file should have DV deletes"); + } + + // Run a query to verify coordinator enumeration succeeds. + try { + computeActual("SELECT * FROM " + tableName); + } + catch (RuntimeException e) { + assertFalse( + e.getMessage().contains("Iceberg deletion vectors") && e.getMessage().contains("not supported"), + "Coordinator should not reject PUFFIN deletion vectors: " + e.getMessage()); + } + } + finally { + dropTable(tableName); + } + } + + @Test + public void testV3DeletionVectorMetadataFields() + throws Exception + { + String tableName = "test_dv_metadata_fields"; + try { + assertUpdate("CREATE TABLE " + tableName + " (id integer, value varchar) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two')", 2); + + Table table = loadTable(tableName); + + byte[] roaringBitmapBytes = serializeRoaringBitmapNoRun(new int[] {0}); + String dvPath = table.location() + "/data/dv-meta-" + UUID.randomUUID() + ".puffin"; + + long blobOffset; + long blobLength; + long puffinFileSize; + + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + String dataFilePath = task.file().path().toString(); + + OutputFile outputFile = table.io().newOutputFile(dvPath); + + try (PuffinWriter writer = Puffin.write(outputFile) + .createdBy("presto-test") + .build()) { + writer.add(new Blob( + "deletion-vector-v2", + ImmutableList.of(), + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + ByteBuffer.wrap(roaringBitmapBytes))); + writer.finish(); + + puffinFileSize = writer.fileSize(); + blobOffset = writer.writtenBlobsMetadata().get(0).offset(); + blobLength = writer.writtenBlobsMetadata().get(0).length(); + } + + DeleteFile puffinDeleteFile = FileMetadata.deleteFileBuilder(task.spec()) + .ofPositionDeletes() + .withPath(dvPath) + .withFileSizeInBytes(puffinFileSize) + .withFormat(FileFormat.PUFFIN) + .withRecordCount(1) + .withContentOffset(blobOffset) + .withContentSizeInBytes(blobLength) + .withReferencedDataFile(dataFilePath) + .build(); + + table.newRowDelta() + .addDeletes(puffinDeleteFile) + .commit(); + } + + // Verify the committed DV file has correct metadata fields. + table = loadTable(tableName); + try (CloseableIterable tasks = table.newScan().planFiles()) { + FileScanTask task = tasks.iterator().next(); + java.util.List deletes = task.deletes(); + assertFalse(deletes.isEmpty(), "Should have deletion vector files"); + + org.apache.iceberg.DeleteFile dvFile = deletes.get(0); + assertEquals(dvFile.format(), FileFormat.PUFFIN, "Format should be PUFFIN"); + assertEquals(dvFile.recordCount(), 1, "Record count should match deleted positions"); + assertTrue(dvFile.fileSizeInBytes() > 0, "File size must be positive"); + + // Verify the DV file path ends with .puffin as expected. + assertTrue(dvFile.path().toString().endsWith(".puffin"), "DV file should be a .puffin file"); + } + } + finally { + dropTable(tableName); + } + } + + @Test + public void testV3WriteReadRoundTrip() + throws Exception + { + String tableName = "test_v3_write_read_round_trip"; + try { + // Step 1: Create V3 table and insert initial data + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, name VARCHAR, value DOUBLE) WITH (\"format-version\" = '3', \"write.delete.mode\" = 'merge-on-read')"); + assertUpdate("INSERT INTO " + tableName + + " VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)", 5); + + // Step 2: Verify initial data via read path + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 5"); + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)"); + + // Step 3: First DELETE via write path (produces DV #1) + assertUpdate("DELETE FROM " + tableName + " WHERE id IN (1, 3)", 2); + + // Step 4: Verify read path filters DV #1 correctly + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)"); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 3"); + + // Step 5: Cross-validate DV #1 metadata via Iceberg API + Table table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + + int dvCount = 0; + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { + dvCount++; + assertEquals(deleteFile.format(), FileFormat.PUFFIN, + "Presto-written DV must use PUFFIN format"); + assertTrue(deleteFile.path().toString().endsWith(".puffin"), + "DV file path must end with .puffin"); + assertTrue(deleteFile.fileSizeInBytes() > 0, + "DV file size must be positive"); + assertTrue(deleteFile.contentOffset() >= 0, + "DV content offset must be non-negative"); + assertTrue(deleteFile.contentSizeInBytes() > 0, + "DV content size must be positive"); + assertTrue(deleteFile.recordCount() > 0, + "DV record count must be positive"); + } + } + } + assertTrue(dvCount > 0, "Should have at least one deletion vector after DELETE"); + + // Step 6: Insert more data (creates a new data file alongside existing ones) + assertUpdate("INSERT INTO " + tableName + + " VALUES (6, 'Frank', 600.0), (7, 'Grace', 700.0)", 2); + + // Step 7: Verify read path handles mixed state: old data with DVs + new data + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0), (6, 'Frank', 600.0), (7, 'Grace', 700.0)"); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 5"); + + // Step 8: Second DELETE via write path (produces DV #2, targeting new and old data) + assertUpdate("DELETE FROM " + tableName + " WHERE id IN (2, 7)", 2); + + // Step 9: Verify cumulative read path correctness with two rounds of DVs + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (4, 'Dave', 400.0), (5, 'Eve', 500.0), (6, 'Frank', 600.0)"); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 3"); + + // Step 10: Cross-validate cumulative DV metadata via Iceberg API + table = loadTable(tableName); + int totalDvs = 0; + int totalDataFiles = 0; + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + totalDataFiles++; + for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { + totalDvs++; + assertEquals(deleteFile.format(), FileFormat.PUFFIN, + "All DVs must use PUFFIN format"); + assertTrue(deleteFile.recordCount() > 0, + "Each DV must have positive record count"); + } + } + } + assertTrue(totalDvs > 0, "Should have deletion vectors after two rounds of DELETE"); + assertTrue(totalDataFiles > 0, "Should have data files remaining"); + + // Step 11: Verify aggregation works correctly over DV-filtered data + assertQuery("SELECT SUM(value) FROM " + tableName, "SELECT 1500.0"); + assertQuery("SELECT MIN(id), MAX(id) FROM " + tableName, "VALUES (4, 6)"); + + // Step 12: Verify predicates work correctly with DVs + assertQuery("SELECT * FROM " + tableName + " WHERE value > 450.0 ORDER BY id", + "VALUES (5, 'Eve', 500.0), (6, 'Frank', 600.0)"); + assertQuery("SELECT * FROM " + tableName + " WHERE name LIKE '%a%' ORDER BY id", + "VALUES (4, 'Dave', 400.0), (6, 'Frank', 600.0)"); + } + finally { + dropTable(tableName); + } + } + private void dropTableViaIceberg(String tableName) { Catalog catalog = CatalogUtil.loadCatalog( @@ -474,4 +1154,463 @@ private void dropTableViaIceberg(String tableName) catalog.dropTable( TableIdentifier.of(TEST_SCHEMA, tableName), true); } + + @Test + public void testRewriteDeleteFilesProcedure() + throws Exception + { + String tableName = "test_rewrite_delete_files"; + try { + // Step 1: Create V3 table and insert data + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, name VARCHAR, value DOUBLE) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + + " VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Carol', 300.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)", 5); + + // Step 2: Perform multiple deletes to create multiple DVs per data file + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE id = 3", 1); + + // Step 3: Verify we have multiple delete files before compaction + Table table = loadTable(tableName); + int dvCountBefore = 0; + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + dvCountBefore += task.deletes().size(); + } + } + assertTrue(dvCountBefore >= 2, "Should have at least 2 DVs before compaction, got: " + dvCountBefore); + + // Step 4: Verify data is correct before compaction + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)"); + + // Step 5: Run DV compaction + assertQuerySucceeds(format("CALL system.rewrite_delete_files('%s', '%s')", TEST_SCHEMA, tableName)); + + // Step 6: Verify data is still correct after compaction + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob', 200.0), (4, 'Dave', 400.0), (5, 'Eve', 500.0)"); + + // Step 7: Verify DVs were compacted (fewer or equal DVs) + table.refresh(); + int dvCountAfter = 0; + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + for (DeleteFile dv : task.deletes()) { + dvCountAfter++; + assertEquals(dv.format(), FileFormat.PUFFIN, "Compacted DV must use PUFFIN format"); + } + } + } + assertTrue(dvCountAfter <= dvCountBefore, + "DV count after compaction (" + dvCountAfter + ") should be <= before (" + dvCountBefore + ")"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testRewriteDeleteFilesOnV2Table() + { + String tableName = "test_rewrite_delete_files_v2"; + try { + // V2 tables should be a no-op (no DVs to compact) + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, value VARCHAR) WITH (\"format-version\" = '2', delete_mode = 'merge-on-read')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'one'), (2, 'two'), (3, 'three')", 3); + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + + assertQuerySucceeds(format("CALL system.rewrite_delete_files('%s', '%s')", TEST_SCHEMA, tableName)); + + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (2, 'two'), (3, 'three')"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testV3DefaultValues() + throws Exception + { + String tableName = "test_v3_default_values"; + try { + // Step 1: Create V3 table and insert initial data + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, name VARCHAR) WITH (\"format-version\" = '3')"); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'Alice'), (2, 'Bob')", 2); + + // Step 2: Add column with default value via Iceberg API + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("score", org.apache.iceberg.types.Types.DoubleType.get()) + .setDefaultValue("score", 99.0) + .commit(); + + // Step 3: Verify we can read old data — the new column should have default value + assertQuery("SELECT id, name FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice'), (2, 'Bob')"); + + // Step 4: Insert new data with the new column + assertUpdate("INSERT INTO " + tableName + " VALUES (3, 'Carol', 300.0)", 1); + + // Step 5: Verify new data reads correctly + assertQuery("SELECT id, name, score FROM " + tableName + " WHERE id = 3", + "VALUES (3, 'Carol', 300.0)"); + + // Step 6: Verify old rows get default value (99.0) from Iceberg schema evolution + assertQuery("SELECT id, name, score FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 99.0), (2, 'Bob', 99.0), (3, 'Carol', 300.0)"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testMultiArgumentPartitionTransforms() + { + String tableName = "test_v3_multi_arg_transforms"; + try { + // Create V3 table with bucket(4, id) partitioning + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, name VARCHAR, value DOUBLE)" + + " WITH (\"format-version\" = '3', partitioning = ARRAY['bucket(id, 4)'])"); + + // Verify table was created with correct partition spec + Table table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + assertEquals(table.spec().fields().size(), 1); + assertEquals(table.spec().fields().get(0).transform().toString(), "bucket[4]"); + + // Insert data — should distribute across buckets + assertUpdate("INSERT INTO " + tableName + + " VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0), (4, 'Diana', 400.0)", 4); + + // Verify data reads correctly + assertQuery("SELECT * FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 100.0), (2, 'Bob', 200.0), (3, 'Charlie', 300.0), (4, 'Diana', 400.0)"); + + // Verify partition pruning works — query with equality predicate + assertQuery("SELECT name, value FROM " + tableName + " WHERE id = 2", + "VALUES ('Bob', 200.0)"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testTruncatePartitionTransform() + { + String tableName = "test_v3_truncate_transform"; + try { + // Create V3 table with truncate(10, value) partitioning on a varchar column + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, category VARCHAR, amount DOUBLE)" + + " WITH (\"format-version\" = '3', partitioning = ARRAY['truncate(category, 3)'])"); + + Table table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + assertEquals(table.spec().fields().size(), 1); + assertEquals(table.spec().fields().get(0).transform().toString(), "truncate[3]"); + + // Insert data with varying category prefixes + assertUpdate("INSERT INTO " + tableName + + " VALUES (1, 'food_pizza', 15.0), (2, 'food_burger', 12.0)," + + " (3, 'drink_coffee', 5.0), (4, 'drink_tea', 3.0)", 4); + + // Verify data reads correctly + assertQuery("SELECT id, category, amount FROM " + tableName + " ORDER BY id", + "VALUES (1, 'food_pizza', 15.0), (2, 'food_burger', 12.0)," + + " (3, 'drink_coffee', 5.0), (4, 'drink_tea', 3.0)"); + + // Verify we can filter + assertQuery("SELECT id FROM " + tableName + " WHERE category = 'food_pizza'", + "VALUES 1"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testNanosecondTimestampSchema() + { + String tableName = "test_v3_timestamp_nano"; + try { + // Create V3 table with Presto + assertUpdate("CREATE TABLE " + tableName + " (id INTEGER) WITH (\"format-version\" = '3')"); + + // Add nanosecond timestamp columns via Iceberg API + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("ts_nano", Types.TimestampNanoType.withoutZone()) + .addColumn("ts_nano_tz", Types.TimestampNanoType.withZone()) + .commit(); + + // Verify Presto can read the schema with nanosecond columns + // ts_nano maps to timestamp microseconds, ts_nano_tz maps to timestamp with time zone + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 0"); + + // Insert data through Presto — the nanosecond columns accept null values + assertUpdate("INSERT INTO " + tableName + " (id) VALUES (1)", 1); + assertQuery("SELECT id FROM " + tableName, "VALUES 1"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testVariantColumnSchema() + { + String tableName = "test_v3_variant"; + try { + // Create V3 table with Presto + assertUpdate("CREATE TABLE " + tableName + " (id INTEGER) WITH (\"format-version\" = '3')"); + + // Add variant column via Iceberg API + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("data", Types.VariantType.get()) + .commit(); + + // Verify Presto can read the schema with the variant column + // Variant maps to VARCHAR in Presto + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 0"); + + // Insert data — the variant column accepts null values + assertUpdate("INSERT INTO " + tableName + " (id) VALUES (1)", 1); + assertQuery("SELECT id FROM " + tableName, "VALUES 1"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testVariantTypeEndToEnd() + { + String tableName = "test_v3_variant_e2e"; + try { + // Step 1: Create V3 table and add variant columns via Iceberg schema evolution + assertUpdate("CREATE TABLE " + tableName + " (id INTEGER, name VARCHAR) WITH (\"format-version\" = '3')"); + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("metadata", Types.VariantType.get()) + .commit(); + + // Step 2: Verify empty table with variant column is queryable + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 0"); + + // Step 3: Insert data — variant column receives NULLs + assertUpdate("INSERT INTO " + tableName + " (id, name) VALUES (1, 'Alice'), (2, 'Bob'), (3, 'Charlie')", 3); + + // Step 4: Verify full row reads including NULL variant values + assertQuery("SELECT id, name, metadata FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', NULL), (2, 'Bob', NULL), (3, 'Charlie', NULL)"); + + // Step 5: Test IS NULL predicate on variant column + assertQuery("SELECT count(*) FROM " + tableName + " WHERE metadata IS NULL", "SELECT 3"); + + // Step 6: Test filtering on non-variant columns with variant columns in projection + assertQuery("SELECT id, name, metadata FROM " + tableName + " WHERE id > 1 ORDER BY id", + "VALUES (2, 'Bob', NULL), (3, 'Charlie', NULL)"); + + // Step 7: Test aggregation with variant columns in the table + assertQuery("SELECT count(*), min(id), max(id) FROM " + tableName, "VALUES (3, 1, 3)"); + assertQuery("SELECT name, count(*) FROM " + tableName + " GROUP BY name ORDER BY name", + "VALUES ('Alice', 1), ('Bob', 1), ('Charlie', 1)"); + + // Step 8: DELETE rows from a table with variant columns + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 2"); + assertQuery("SELECT id, name FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice'), (3, 'Charlie')"); + + // Step 9: Insert more data after deletion + assertUpdate("INSERT INTO " + tableName + " (id, name) VALUES (4, 'Diana'), (5, 'Eve')", 2); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 4"); + + // Step 10: Verify mixed snapshots (pre-delete and post-delete) read correctly + assertQuery("SELECT id, name FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice'), (3, 'Charlie'), (4, 'Diana'), (5, 'Eve')"); + + // Step 11: Further schema evolution — add another variant column alongside the first + table = loadTable(tableName); + table.updateSchema() + .addColumn("tags", Types.VariantType.get()) + .commit(); + + // Step 12: Verify reads still work with two variant columns + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 4"); + assertQuery("SELECT id, name FROM " + tableName + " WHERE id = 1", + "VALUES (1, 'Alice')"); + + // Step 13: Insert with both variant columns NULL + assertUpdate("INSERT INTO " + tableName + " (id, name) VALUES (6, 'Frank')", 1); + assertQuery("SELECT id, metadata, tags FROM " + tableName + " WHERE id = 6", + "VALUES (6, NULL, NULL)"); + + // Step 14: Verify V3 format preserved through all operations + table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testVariantColumnWithPartitioning() + { + String tableName = "test_v3_variant_partitioned"; + try { + // Create V3 partitioned table with variant column + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, category VARCHAR) WITH (\"format-version\" = '3', partitioning = ARRAY['category'])"); + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("data", Types.VariantType.get()) + .commit(); + + // Insert data into multiple partitions + assertUpdate("INSERT INTO " + tableName + " (id, category) VALUES (1, 'A'), (2, 'A'), (3, 'B'), (4, 'C')", 4); + + // Verify partition pruning works with variant column present + assertQuery("SELECT id FROM " + tableName + " WHERE category = 'A' ORDER BY id", + "VALUES 1, 2"); + assertQuery("SELECT id FROM " + tableName + " WHERE category = 'B'", + "VALUES 3"); + + // Verify cross-partition aggregation + assertQuery("SELECT category, count(*) FROM " + tableName + " GROUP BY category ORDER BY category", + "VALUES ('A', 2), ('B', 1), ('C', 1)"); + + // Delete within a partition + assertUpdate("DELETE FROM " + tableName + " WHERE category = 'A'", 2); + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 2"); + assertQuery("SELECT id FROM " + tableName + " ORDER BY id", + "VALUES 3, 4"); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testVariantJsonDataRoundTrip() + { + String tableName = "test_v3_variant_json_data"; + try { + // Step 1: Create V3 table and add variant column via Iceberg API + assertUpdate("CREATE TABLE " + tableName + " (id INTEGER, name VARCHAR) WITH (\"format-version\" = '3')"); + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("metadata", Types.VariantType.get()) + .commit(); + + // Step 2: Insert rows with actual JSON string data into the variant column. + // Since VARIANT maps to VARCHAR in Presto, JSON strings are written as-is. + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(1, 'Alice', '{\"age\":30,\"city\":\"NYC\"}'), " + + "(2, 'Bob', '{\"age\":25}'), " + + "(3, 'Charlie', NULL)", 3); + + // Step 3: Verify round-trip — JSON strings survive write → Parquet → read + assertQuery("SELECT id, name, metadata FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', '{\"age\":30,\"city\":\"NYC\"}'), " + + "(2, 'Bob', '{\"age\":25}'), " + + "(3, 'Charlie', NULL)"); + + // Step 4: Test filtering on non-variant columns with variant data present + assertQuery("SELECT metadata FROM " + tableName + " WHERE id = 1", + "VALUES ('{\"age\":30,\"city\":\"NYC\"}')"); + + // Step 5: Test IS NULL / IS NOT NULL on variant column with actual data + assertQuery("SELECT count(*) FROM " + tableName + " WHERE metadata IS NOT NULL", "SELECT 2"); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE metadata IS NULL", "SELECT 1"); + + // Step 6: Insert rows with different JSON value types (number, string, boolean) + assertUpdate("INSERT INTO " + tableName + " VALUES " + + "(4, 'Diana', '42'), " + + "(5, 'Eve', '\"simple string\"'), " + + "(6, 'Frank', 'true')", 3); + + // Step 7: Verify all rows + assertQuery("SELECT count(*) FROM " + tableName, "SELECT 6"); + assertQuery("SELECT metadata FROM " + tableName + " WHERE id = 4", "VALUES ('42')"); + assertQuery("SELECT metadata FROM " + tableName + " WHERE id = 6", "VALUES ('true')"); + + // Step 8: Delete rows with variant data + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertQuery("SELECT count(*) FROM " + tableName + " WHERE metadata IS NOT NULL", "SELECT 4"); + + // Step 9: Verify remaining data + assertQuery("SELECT id, name FROM " + tableName + " ORDER BY id", + "VALUES (2, 'Bob'), (3, 'Charlie'), (4, 'Diana'), (5, 'Eve'), (6, 'Frank')"); + + // Step 10: Verify V3 format preserved + table = loadTable(tableName); + assertEquals(((BaseTable) table).operations().current().formatVersion(), 3); + } + finally { + dropTable(tableName); + } + } + + @Test + public void testVariantColumnWithDeleteAndUpdate() + { + String tableName = "test_v3_variant_dml"; + try { + // Create V3 table with merge-on-read delete mode and variant column + assertUpdate("CREATE TABLE " + tableName + + " (id INTEGER, name VARCHAR, score DOUBLE)" + + " WITH (\"format-version\" = '3', \"write.delete.mode\" = 'merge-on-read', \"write.update.mode\" = 'merge-on-read')"); + Table table = loadTable(tableName); + table.updateSchema() + .addColumn("extra", Types.VariantType.get()) + .commit(); + + // Insert data + assertUpdate("INSERT INTO " + tableName + " (id, name, score) VALUES " + + "(1, 'Alice', 85.5), (2, 'Bob', 92.0), (3, 'Charlie', 78.3), (4, 'Diana', 95.0)", 4); + + // Verify initial data + assertQuery("SELECT id, name, score FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 85.5), (2, 'Bob', 92.0), (3, 'Charlie', 78.3), (4, 'Diana', 95.0)"); + + // Row-level DELETE (produces deletion vector) + assertUpdate("DELETE FROM " + tableName + " WHERE id = 2", 1); + assertQuery("SELECT id, name FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice'), (3, 'Charlie'), (4, 'Diana')"); + + // Verify DV metadata is PUFFIN format + table = loadTable(tableName); + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask task : tasks) { + for (org.apache.iceberg.DeleteFile deleteFile : task.deletes()) { + assertEquals(deleteFile.format(), FileFormat.PUFFIN); + } + } + } + + // UPDATE on table with variant column + assertUpdate("UPDATE " + tableName + " SET score = 99.9 WHERE id = 1", 1); + assertQuery("SELECT id, name, score FROM " + tableName + " WHERE id = 1", + "VALUES (1, 'Alice', 99.9)"); + + // Verify final state + assertQuery("SELECT id, name, score FROM " + tableName + " ORDER BY id", + "VALUES (1, 'Alice', 99.9), (3, 'Charlie', 78.3), (4, 'Diana', 95.0)"); + } + finally { + dropTable(tableName); + } + } } diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantBinaryCodec.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantBinaryCodec.java new file mode 100644 index 0000000000000..f8dea3365ffa6 --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantBinaryCodec.java @@ -0,0 +1,510 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.facebook.presto.iceberg.VariantBinaryCodec.VariantBinary; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +public class TestVariantBinaryCodec +{ + @Test + public void testNullValue() + { + String json = "null"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertNotNull(binary.getMetadata()); + assertNotNull(binary.getValue()); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testBooleanTrue() + { + String json = "true"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testBooleanFalse() + { + String json = "false"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testSmallInteger() + { + String json = "42"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testNegativeInteger() + { + String json = "-100"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testLargeInteger() + { + String json = "2147483648"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testZero() + { + String json = "0"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testInt16Range() + { + // Value that requires int16 (> 127) + String json = "1000"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testInt32Range() + { + // Value that requires int32 (> 32767) + String json = "100000"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testInt64Range() + { + // Value that requires int64 (> 2^31 - 1) + String json = "9999999999"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testDouble() + { + String json = "3.14"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testNegativeDouble() + { + String json = "-2.718"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testShortString() + { + String json = "\"hello\""; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testEmptyString() + { + String json = "\"\""; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testLongString() + { + // String longer than 63 bytes (exceeds short string limit) + StringBuilder sb = new StringBuilder("\""); + for (int i = 0; i < 100; i++) { + sb.append('a'); + } + sb.append("\""); + String json = sb.toString(); + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testUnicodeString() + { + String json = "\"café ☕\""; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testSimpleObject() + { + String json = "{\"name\":\"Alice\",\"age\":30}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + // Object keys are sorted in the metadata dictionary, so the output + // should have keys in sorted order + assertNotNull(decoded); + // Verify it round-trips (keys may be reordered due to sorted dictionary) + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + String decoded2 = VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()); + assertEquals(decoded2, decoded); + } + + @Test + public void testEmptyObject() + { + String json = "{}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testNestedObject() + { + String json = "{\"user\":{\"name\":\"Bob\",\"score\":95}}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + // Verify double round-trip stability + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + @Test + public void testSimpleArray() + { + String json = "[1,2,3]"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testEmptyArray() + { + String json = "[]"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testMixedArray() + { + String json = "[1,\"two\",true,null,3.14]"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testNestedArray() + { + String json = "[[1,2],[3,4]]"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + assertEquals(VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()), json); + } + + @Test + public void testComplexDocument() + { + String json = "{\"name\":\"Alice\",\"scores\":[95,87,92],\"active\":true,\"address\":null}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + // Verify double round-trip stability + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + @Test + public void testDeeplyNested() + { + String json = "{\"a\":{\"b\":{\"c\":{\"d\":\"deep\"}}}}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + @Test + public void testArrayOfObjects() + { + String json = "[{\"id\":1,\"name\":\"a\"},{\"id\":2,\"name\":\"b\"}]"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + @Test + public void testMetadataDictionary() + { + // Verify that the metadata dictionary is built correctly + String json = "{\"z_key\":1,\"a_key\":2}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + // Metadata dictionary should have keys sorted alphabetically + String[] keys = VariantBinaryCodec.decodeMetadata(binary.getMetadata()); + assertEquals(keys.length, 2); + assertEquals(keys[0], "a_key"); + assertEquals(keys[1], "z_key"); + } + + @Test + public void testEmptyMetadataForPrimitives() + { + // Primitive values should have an empty metadata dictionary + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + String[] keys = VariantBinaryCodec.decodeMetadata(binary.getMetadata()); + assertEquals(keys.length, 0); + } + + @Test + public void testHeaderEncoding() + { + // Verify header byte construction + assertEquals(VariantBinaryCodec.makeHeader(VariantBinaryCodec.BASIC_TYPE_PRIMITIVE, VariantBinaryCodec.PRIMITIVE_NULL), (byte) 0x00); + assertEquals(VariantBinaryCodec.makeHeader(VariantBinaryCodec.BASIC_TYPE_PRIMITIVE, VariantBinaryCodec.PRIMITIVE_TRUE), (byte) 0x01); + assertEquals(VariantBinaryCodec.makeHeader(VariantBinaryCodec.BASIC_TYPE_SHORT_STRING, 5), (byte) 0x45); // 01_000101 + assertEquals(VariantBinaryCodec.makeHeader(VariantBinaryCodec.BASIC_TYPE_OBJECT, 0), (byte) 0x80); // 10_000000 + assertEquals(VariantBinaryCodec.makeHeader(VariantBinaryCodec.BASIC_TYPE_ARRAY, 0), (byte) 0xC0); // 11_000000 + } + + @Test + public void testStringWithSpecialChars() + { + String json = "{\"key\":\"value with \\\"quotes\\\" and \\\\backslash\"}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + @Test + public void testObjectWithMixedValues() + { + String json = "{\"bool\":false,\"int\":42,\"float\":1.5,\"null\":null,\"str\":\"hello\"}"; + VariantBinary binary = VariantBinaryCodec.fromJson(json); + String decoded = VariantBinaryCodec.toJson(binary.getMetadata(), binary.getValue()); + assertNotNull(decoded); + VariantBinary binary2 = VariantBinaryCodec.fromJson(decoded); + assertEquals(VariantBinaryCodec.toJson(binary2.getMetadata(), binary2.getValue()), decoded); + } + + // ---- Phase 2: isVariantBinary tests ---- + + @Test + public void testIsVariantBinaryValidObject() + { + VariantBinary binary = VariantBinaryCodec.fromJson("{\"a\":1}"); + assertTrue(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), binary.getValue())); + } + + @Test + public void testIsVariantBinaryValidPrimitive() + { + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + assertTrue(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), binary.getValue())); + } + + @Test + public void testIsVariantBinaryValidArray() + { + VariantBinary binary = VariantBinaryCodec.fromJson("[1,2,3]"); + assertTrue(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), binary.getValue())); + } + + @Test + public void testIsVariantBinaryValidString() + { + VariantBinary binary = VariantBinaryCodec.fromJson("\"hello\""); + assertTrue(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), binary.getValue())); + } + + @Test + public void testIsVariantBinaryNullMetadata() + { + assertFalse(VariantBinaryCodec.isVariantBinary(null, new byte[] {0})); + } + + @Test + public void testIsVariantBinaryNullValue() + { + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + assertFalse(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), null)); + } + + @Test + public void testIsVariantBinaryEmptyValue() + { + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + assertFalse(VariantBinaryCodec.isVariantBinary(binary.getMetadata(), new byte[0])); + } + + @Test + public void testIsVariantBinaryShortMetadata() + { + assertFalse(VariantBinaryCodec.isVariantBinary(new byte[] {1, 0}, new byte[] {0})); + } + + // ---- Phase 2: getValueTypeName tests ---- + + @Test + public void testGetValueTypeNameNull() + { + VariantBinary binary = VariantBinaryCodec.fromJson("null"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "null"); + } + + @Test + public void testGetValueTypeNameTrue() + { + VariantBinary binary = VariantBinaryCodec.fromJson("true"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "boolean"); + } + + @Test + public void testGetValueTypeNameFalse() + { + VariantBinary binary = VariantBinaryCodec.fromJson("false"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "boolean"); + } + + @Test + public void testGetValueTypeNameInteger() + { + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "integer"); + } + + @Test + public void testGetValueTypeNameDouble() + { + VariantBinary binary = VariantBinaryCodec.fromJson("3.14"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "double"); + } + + @Test + public void testGetValueTypeNameShortString() + { + VariantBinary binary = VariantBinaryCodec.fromJson("\"hello\""); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "string"); + } + + @Test + public void testGetValueTypeNameObject() + { + VariantBinary binary = VariantBinaryCodec.fromJson("{\"a\":1}"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "object"); + } + + @Test + public void testGetValueTypeNameArray() + { + VariantBinary binary = VariantBinaryCodec.fromJson("[1,2]"); + assertEquals(VariantBinaryCodec.getValueTypeName(binary.getValue()), "array"); + } + + @Test + public void testGetValueTypeNameEmptyValue() + { + assertEquals(VariantBinaryCodec.getValueTypeName(new byte[0]), "null"); + } + + @Test + public void testGetValueTypeNameNullValue() + { + assertEquals(VariantBinaryCodec.getValueTypeName(null), "null"); + } + + // ---- Phase 2: decodeVariantAuto tests ---- + + @Test + public void testDecodeVariantAutoJsonObject() + { + byte[] data = "{\"a\":1}".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "{\"a\":1}"); + } + + @Test + public void testDecodeVariantAutoJsonArray() + { + byte[] data = "[1,2,3]".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "[1,2,3]"); + } + + @Test + public void testDecodeVariantAutoJsonString() + { + byte[] data = "\"hello\"".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "\"hello\""); + } + + @Test + public void testDecodeVariantAutoJsonNumber() + { + byte[] data = "42".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "42"); + } + + @Test + public void testDecodeVariantAutoJsonBoolean() + { + byte[] data = "true".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "true"); + } + + @Test + public void testDecodeVariantAutoJsonNull() + { + byte[] data = "null".getBytes(java.nio.charset.StandardCharsets.UTF_8); + assertEquals(VariantBinaryCodec.decodeVariantAuto(data), "null"); + } + + @Test + public void testDecodeVariantAutoEmpty() + { + assertEquals(VariantBinaryCodec.decodeVariantAuto(new byte[0]), "null"); + } + + @Test + public void testDecodeVariantAutoNull() + { + assertEquals(VariantBinaryCodec.decodeVariantAuto(null), "null"); + } + + @Test + public void testDecodeVariantAutoBinaryPrimitive() + { + VariantBinary binary = VariantBinaryCodec.fromJson("42"); + String decoded = VariantBinaryCodec.decodeVariantAuto(binary.getValue()); + assertEquals(decoded, "42"); + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantFunctions.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantFunctions.java new file mode 100644 index 0000000000000..4db8930fb623b --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestVariantFunctions.java @@ -0,0 +1,562 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.facebook.presto.common.CatalogSchemaName; +import com.facebook.presto.iceberg.function.VariantFunctions; +import com.facebook.presto.metadata.FunctionExtractor; +import com.facebook.presto.operator.scalar.AbstractTestFunctions; +import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.analyzer.FunctionsConfig; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.SessionTestUtils.TEST_SESSION; +import static com.facebook.presto.common.type.VarcharType.VARCHAR; + +public class TestVariantFunctions + extends AbstractTestFunctions +{ + private static final String CATALOG_SCHEMA = "iceberg.system"; + + public TestVariantFunctions() + { + super(TEST_SESSION, new FeaturesConfig(), new FunctionsConfig(), false); + } + + @BeforeClass + public void registerFunction() + { + ImmutableList.Builder> functions = ImmutableList.builder(); + functions.add(VariantFunctions.class); + functionAssertions.addConnectorFunctions(FunctionExtractor.extractFunctions(functions.build(), + new CatalogSchemaName("iceberg", "system")), "iceberg"); + } + + // ---- variant_get: simple field extraction ---- + + @Test + public void testVariantGetStringField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"name\":\"Alice\",\"age\":30}', 'name')", + VARCHAR, + "Alice"); + } + + @Test + public void testVariantGetNumberField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"name\":\"Alice\",\"age\":30}', 'age')", + VARCHAR, + "30"); + } + + @Test + public void testVariantGetBooleanField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"active\":true}', 'active')", + VARCHAR, + "true"); + } + + @Test + public void testVariantGetNestedObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"address\":{\"city\":\"NYC\"}}', 'address')", + VARCHAR, + "{\"city\":\"NYC\"}"); + } + + @Test + public void testVariantGetNestedArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"items\":[1,2,3]}', 'items')", + VARCHAR, + "[1,2,3]"); + } + + @Test + public void testVariantGetMissingField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"name\":\"Alice\"}', 'missing')", + VARCHAR, + null); + } + + @Test + public void testVariantGetNonObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('\"just a string\"', 'field')", + VARCHAR, + null); + } + + @Test + public void testVariantGetNullField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"key\":null}', 'key')", + VARCHAR, + "null"); + } + + // ---- variant_get: dot-path navigation ---- + + @Test + public void testVariantGetDotPath() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"address\":{\"city\":\"NYC\"}}', 'address.city')", + VARCHAR, + "NYC"); + } + + @Test + public void testVariantGetDotPathDeep() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"a\":{\"b\":{\"c\":\"deep\"}}}', 'a.b.c')", + VARCHAR, + "deep"); + } + + @Test + public void testVariantGetDotPathMissing() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"address\":{\"city\":\"NYC\"}}', 'address.zip')", + VARCHAR, + null); + } + + @Test + public void testVariantGetDotPathNestedObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"a\":{\"b\":{\"c\":1}}}', 'a.b')", + VARCHAR, + "{\"c\":1}"); + } + + // ---- variant_get: array indexing ---- + + @Test + public void testVariantGetArrayIndex() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('[10,20,30]', '[0]')", + VARCHAR, + "10"); + } + + @Test + public void testVariantGetArrayIndexLast() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('[10,20,30]', '[2]')", + VARCHAR, + "30"); + } + + @Test + public void testVariantGetArrayOutOfBounds() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('[10,20,30]', '[5]')", + VARCHAR, + null); + } + + @Test + public void testVariantGetArrayOfObjects() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('[{\"id\":1},{\"id\":2}]', '[1]')", + VARCHAR, + "{\"id\":2}"); + } + + // ---- variant_get: combined dot-path + array indexing ---- + + @Test + public void testVariantGetFieldThenArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"items\":[1,2,3]}', 'items[1]')", + VARCHAR, + "2"); + } + + @Test + public void testVariantGetArrayThenField() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"users\":[{\"name\":\"Alice\"},{\"name\":\"Bob\"}]}', 'users[0].name')", + VARCHAR, + "Alice"); + } + + @Test + public void testVariantGetComplexPath() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_get('{\"data\":{\"rows\":[{\"v\":99}]}}', 'data.rows[0].v')", + VARCHAR, + "99"); + } + + // ---- variant_keys ---- + + @Test + public void testVariantKeysSimple() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_keys('{\"name\":\"Alice\",\"age\":30}')", + VARCHAR, + "[\"name\",\"age\"]"); + } + + @Test + public void testVariantKeysEmpty() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_keys('{}')", + VARCHAR, + "[]"); + } + + @Test + public void testVariantKeysNonObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_keys('[1,2,3]')", + VARCHAR, + null); + } + + @Test + public void testVariantKeysScalar() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_keys('42')", + VARCHAR, + null); + } + + @Test + public void testVariantKeysNested() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_keys('{\"a\":{\"b\":1},\"c\":[1]}')", + VARCHAR, + "[\"a\",\"c\"]"); + } + + // ---- variant_type ---- + + @Test + public void testVariantTypeObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('{\"a\":1}')", + VARCHAR, + "object"); + } + + @Test + public void testVariantTypeArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('[1,2]')", + VARCHAR, + "array"); + } + + @Test + public void testVariantTypeString() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('\"hello\"')", + VARCHAR, + "string"); + } + + @Test + public void testVariantTypeNumber() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('42')", + VARCHAR, + "number"); + } + + @Test + public void testVariantTypeFloat() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('3.14')", + VARCHAR, + "number"); + } + + @Test + public void testVariantTypeBoolean() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('true')", + VARCHAR, + "boolean"); + } + + @Test + public void testVariantTypeNull() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_type('null')", + VARCHAR, + "null"); + } + + // ---- to_variant (Phase 5: CAST) ---- + + @Test + public void testToVariantObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('{\"name\":\"Alice\"}')", + VARCHAR, + "{\"name\":\"Alice\"}"); + } + + @Test + public void testToVariantArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('[1,2,3]')", + VARCHAR, + "[1,2,3]"); + } + + @Test + public void testToVariantScalar() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('42')", + VARCHAR, + "42"); + } + + @Test + public void testToVariantBoolean() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('true')", + VARCHAR, + "true"); + } + + @Test + public void testToVariantNull() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('null')", + VARCHAR, + "null"); + } + + @Test(expectedExceptions = RuntimeException.class) + public void testToVariantInvalid() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('not valid json')", + VARCHAR, + null); + } + + @Test(expectedExceptions = RuntimeException.class) + public void testToVariantTrailingContent() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".to_variant('{\"a\":1} extra')", + VARCHAR, + null); + } + + // ---- parse_variant (binary codec round-trip) ---- + + @Test + public void testParseVariantSimpleObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('{\"a\":1}')", + VARCHAR, + "{\"a\":1}"); + } + + @Test + public void testParseVariantArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('[1,2,3]')", + VARCHAR, + "[1,2,3]"); + } + + @Test + public void testParseVariantString() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('\"hello\"')", + VARCHAR, + "\"hello\""); + } + + @Test + public void testParseVariantNumber() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('42')", + VARCHAR, + "42"); + } + + @Test + public void testParseVariantBoolean() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('true')", + VARCHAR, + "true"); + } + + @Test + public void testParseVariantNull() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('null')", + VARCHAR, + "null"); + } + + @Test + public void testParseVariantNestedObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".parse_variant('{\"a\":{\"b\":1},\"c\":[true,false]}')", + VARCHAR, + "{\"a\":{\"b\":1},\"c\":[true,false]}"); + } + + // ---- variant_to_json ---- + + @Test + public void testVariantToJsonObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_to_json('{\"name\":\"Alice\"}')", + VARCHAR, + "{\"name\":\"Alice\"}"); + } + + @Test + public void testVariantToJsonArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_to_json('[1,2,3]')", + VARCHAR, + "[1,2,3]"); + } + + @Test + public void testVariantToJsonScalar() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_to_json('42')", + VARCHAR, + "42"); + } + + // ---- variant_binary_roundtrip ---- + + @Test + public void testVariantBinaryRoundtripObject() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('{\"a\":1,\"b\":\"hello\"}')", + VARCHAR, + "{\"a\":1,\"b\":\"hello\"}"); + } + + @Test + public void testVariantBinaryRoundtripArray() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('[1,true,\"text\",null]')", + VARCHAR, + "[1,true,\"text\",null]"); + } + + @Test + public void testVariantBinaryRoundtripNested() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('{\"outer\":{\"inner\":[1,2]}}')", + VARCHAR, + "{\"outer\":{\"inner\":[1,2]}}"); + } + + @Test + public void testVariantBinaryRoundtripScalar() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('42')", + VARCHAR, + "42"); + } + + @Test + public void testVariantBinaryRoundtripString() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('\"hello world\"')", + VARCHAR, + "\"hello world\""); + } + + @Test + public void testVariantBinaryRoundtripBoolean() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('true')", + VARCHAR, + "true"); + } + + @Test + public void testVariantBinaryRoundtripNull() + { + functionAssertions.assertFunction( + CATALOG_SCHEMA + ".variant_binary_roundtrip('null')", + VARCHAR, + "null"); + } +} diff --git a/presto-native-execution/presto_cpp/main/connectors/IcebergPrestoToVeloxConnector.cpp b/presto-native-execution/presto_cpp/main/connectors/IcebergPrestoToVeloxConnector.cpp index 707739048c721..2f0317901bc52 100644 --- a/presto-native-execution/presto_cpp/main/connectors/IcebergPrestoToVeloxConnector.cpp +++ b/presto-native-execution/presto_cpp/main/connectors/IcebergPrestoToVeloxConnector.cpp @@ -30,6 +30,8 @@ velox::connector::hive::iceberg::FileContent toVeloxFileContent( return velox::connector::hive::iceberg::FileContent::kData; } else if (content == protocol::iceberg::FileContent::POSITION_DELETES) { return velox::connector::hive::iceberg::FileContent::kPositionalDeletes; + } else if (content == protocol::iceberg::FileContent::EQUALITY_DELETES) { + return velox::connector::hive::iceberg::FileContent::kEqualityDeletes; } VELOX_UNSUPPORTED("Unsupported file content: {}", fmt::underlying(content)); } @@ -40,6 +42,14 @@ velox::dwio::common::FileFormat toVeloxFileFormat( return velox::dwio::common::FileFormat::ORC; } else if (format == protocol::iceberg::FileFormat::PARQUET) { return velox::dwio::common::FileFormat::PARQUET; + } else if (format == protocol::iceberg::FileFormat::PUFFIN) { + // PUFFIN is used for Iceberg V3 deletion vectors. The DeletionVectorReader + // reads raw binary from the file and does not use the DWRF/Parquet reader, + // so we map PUFFIN to DWRF as a placeholder — the format value is not + // actually used by the reader. This mapping is only safe for deletion + // vector files; if PUFFIN is encountered for other file content types, + // the DV routing logic in toHiveIcebergSplit() must reclassify it first. + return velox::dwio::common::FileFormat::DWRF; } VELOX_UNSUPPORTED("Unsupported file format: {}", fmt::underlying(format)); } @@ -171,11 +181,14 @@ IcebergPrestoToVeloxConnector::toVeloxSplit( const protocol::ConnectorId& catalogId, const protocol::ConnectorSplit* connectorSplit, const protocol::SplitContext* splitContext) const { - auto icebergSplit = + const auto* icebergSplit = dynamic_cast(connectorSplit); VELOX_CHECK_NOT_NULL( icebergSplit, "Unexpected split type {}", connectorSplit->_type); + const int64_t dataSequenceNumber = + icebergSplit->dataSequenceNumber; // NOLINT(facebook-bugprone-unchecked-pointer-access) + std::unordered_map> partitionKeys; for (const auto& entry : icebergSplit->partitionKeys) { partitionKeys.emplace( @@ -191,28 +204,42 @@ IcebergPrestoToVeloxConnector::toVeloxSplit( std::vector deletes; deletes.reserve(icebergSplit->deletes.size()); for (const auto& deleteFile : icebergSplit->deletes) { - std::unordered_map lowerBounds( + const std::unordered_map lowerBounds( deleteFile.lowerBounds.begin(), deleteFile.lowerBounds.end()); - std::unordered_map upperBounds( + const std::unordered_map upperBounds( deleteFile.upperBounds.begin(), deleteFile.upperBounds.end()); - velox::connector::hive::iceberg::IcebergDeleteFile icebergDeleteFile( - toVeloxFileContent(deleteFile.content), + // Iceberg V3 deletion vectors arrive from the coordinator as + // POSITION_DELETES with PUFFIN format. Reclassify them as + // kDeletionVector so that IcebergSplitReader routes them to + // DeletionVectorReader instead of PositionalDeleteFileReader. + velox::connector::hive::iceberg::FileContent veloxContent = + toVeloxFileContent(deleteFile.content); + if (veloxContent == + velox::connector::hive::iceberg::FileContent::kPositionalDeletes && + deleteFile.format == protocol::iceberg::FileFormat::PUFFIN) { + veloxContent = + velox::connector::hive::iceberg::FileContent::kDeletionVector; + } + + const velox::connector::hive::iceberg::IcebergDeleteFile icebergDeleteFile( + veloxContent, deleteFile.path, toVeloxFileFormat(deleteFile.format), deleteFile.recordCount, deleteFile.fileSizeInBytes, std::vector(deleteFile.equalityFieldIds), lowerBounds, - upperBounds); + upperBounds, + deleteFile.dataSequenceNumber); deletes.emplace_back(icebergDeleteFile); } + std::unordered_map infoColumns = { - {"$data_sequence_number", - std::to_string(icebergSplit->dataSequenceNumber)}, + {"$data_sequence_number", std::to_string(dataSequenceNumber)}, {"$path", icebergSplit->path}}; return std::make_unique( @@ -227,7 +254,9 @@ IcebergPrestoToVeloxConnector::toVeloxSplit( nullptr, splitContext->cacheable, deletes, - infoColumns); + infoColumns, + std::nullopt, + dataSequenceNumber); } std::unique_ptr diff --git a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp index ec74e80c58192..0a5a82eaea408 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp @@ -306,7 +306,8 @@ static const std::pair FileFormat_enum_table[] = {FileFormat::ORC, "ORC"}, {FileFormat::PARQUET, "PARQUET"}, {FileFormat::AVRO, "AVRO"}, - {FileFormat::METADATA, "METADATA"}}; + {FileFormat::METADATA, "METADATA"}, + {FileFormat::PUFFIN, "PUFFIN"}}; void to_json(json& j, const FileFormat& e) { static_assert(std::is_enum::value, "FileFormat must be an enum!"); const auto* it = std::find_if( @@ -371,6 +372,13 @@ void to_json(json& j, const DeleteFile& p) { "DeleteFile", "Map", "upperBounds"); + to_json_key( + j, + "dataSequenceNumber", + p.dataSequenceNumber, + "DeleteFile", + "int64_t", + "dataSequenceNumber"); } void from_json(const json& j, DeleteFile& p) { @@ -408,6 +416,13 @@ void from_json(const json& j, DeleteFile& p) { "DeleteFile", "Map", "upperBounds"); + from_json_key( + j, + "dataSequenceNumber", + p.dataSequenceNumber, + "DeleteFile", + "int64_t", + "dataSequenceNumber"); } } // namespace facebook::presto::protocol::iceberg namespace facebook::presto::protocol::iceberg { diff --git a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h index b09cd4903a5bf..6d1cfd204992c 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h +++ b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h @@ -78,12 +78,16 @@ void to_json(json& j, const ChangelogSplitInfo& p); void from_json(const json& j, ChangelogSplitInfo& p); } // namespace facebook::presto::protocol::iceberg namespace facebook::presto::protocol::iceberg { -enum class FileContent { DATA, POSITION_DELETES, EQUALITY_DELETES }; +enum class FileContent { + DATA, + POSITION_DELETES, + EQUALITY_DELETES, +}; extern void to_json(json& j, const FileContent& e); extern void from_json(const json& j, FileContent& e); } // namespace facebook::presto::protocol::iceberg namespace facebook::presto::protocol::iceberg { -enum class FileFormat { ORC, PARQUET, AVRO, METADATA }; +enum class FileFormat { ORC, PARQUET, AVRO, METADATA, PUFFIN }; extern void to_json(json& j, const FileFormat& e); extern void from_json(const json& j, FileFormat& e); } // namespace facebook::presto::protocol::iceberg @@ -97,6 +101,7 @@ struct DeleteFile { List equalityFieldIds = {}; Map lowerBounds = {}; Map upperBounds = {}; + int64_t dataSequenceNumber = {}; }; void to_json(json& j, const DeleteFile& p); void from_json(const json& j, DeleteFile& p); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/TableHandle.java b/presto-spi/src/main/java/com/facebook/presto/spi/TableHandle.java index a38055fbaacf2..9d3f64be97647 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/TableHandle.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/TableHandle.java @@ -55,7 +55,7 @@ public TableHandle( ConnectorTransactionHandle transaction, ConnectorTableLayoutHandle connectorTableLayout) { - this(connectorId, connectorHandle, transaction, Optional.of(connectorTableLayout), Optional.empty()); + this(connectorId, connectorHandle, transaction, Optional.ofNullable(connectorTableLayout), Optional.empty()); } public TableHandle(