From bdf9d24be6c5815a6e0e3f8ec88760c17303f7e4 Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Thu, 29 Sep 2022 16:48:11 -0700 Subject: [PATCH 1/5] Respect avro compression properties for manifest files --- build.gradle | 2 + .../apache/iceberg/BaseRewriteManifests.java | 8 +- .../java/org/apache/iceberg/FastAppend.java | 3 +- .../org/apache/iceberg/ManifestFiles.java | 80 ++++++++++++++++--- .../apache/iceberg/ManifestListWriter.java | 34 +++++--- .../org/apache/iceberg/ManifestLists.java | 9 ++- .../org/apache/iceberg/ManifestReader.java | 6 ++ .../org/apache/iceberg/ManifestWriter.java | 31 ++++--- .../iceberg/MergingSnapshotProducer.java | 3 +- .../org/apache/iceberg/SnapshotProducer.java | 15 +++- .../org/apache/iceberg/TableTestBase.java | 13 +-- .../iceberg/TestManifestListVersions.java | 18 +++-- .../apache/iceberg/TestManifestWriter.java | 2 +- .../iceberg/TestManifestWriterVersions.java | 77 ++++++++++++++++-- .../org/apache/iceberg/TestSnapshotJson.java | 3 +- .../org/apache/iceberg/TestTransaction.java | 3 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 20 +++-- .../flink/sink/IcebergFilesCommitter.java | 5 +- .../flink/TestManifestFileSerialization.java | 2 +- .../iceberg/flink/sink/TestFlinkManifest.java | 12 ++- .../apache/iceberg/spark/SparkTableUtil.java | 13 ++- .../actions/RewriteManifestsSparkAction.java | 24 ++++-- .../TestManifestFileSerialization.java | 2 +- .../source/TestForwardCompatibility.java | 3 +- versions.props | 2 + 25 files changed, 303 insertions(+), 87 deletions(-) diff --git a/build.gradle b/build.gradle index 78bcea88804d..ee8e1b930cab 100644 --- a/build.gradle +++ b/build.gradle @@ -286,6 +286,8 @@ project(':iceberg-core') { testImplementation "org.xerial:sqlite-jdbc" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation "com.esotericsoftware:kryo" + testImplementation "org.xerial.snappy:snappy-java" + testImplementation "com.github.luben:zstd-jni" } } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 816bc0c8a7ec..5a6341cfbe54 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -164,7 +164,13 @@ private ManifestFile copyManifest(ManifestFile manifest) { InputFile toCopy = ops.io().newInputFile(manifest.path()); OutputFile newFile = newManifestOutput(); return ManifestFiles.copyRewriteManifest( - current.formatVersion(), toCopy, specsById, newFile, snapshotId(), summaryBuilder); + current.formatVersion(), + toCopy, + specsById, + newFile, + snapshotId(), + summaryBuilder, + current.properties()); } @Override diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index f3955e15f6ce..d2d3532bfd29 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -137,7 +137,8 @@ private ManifestFile copyManifest(ManifestFile manifest) { current.specsById(), newManifestPath, snapshotId(), - summaryBuilder); + summaryBuilder, + current.properties()); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 85e268d43378..03f782908fa9 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -141,7 +141,23 @@ public static ManifestReader read( * @return a manifest writer */ public static ManifestWriter write(PartitionSpec spec, OutputFile outputFile) { - return write(1, spec, outputFile, null); + return write(spec, outputFile, ImmutableMap.of()); + } + + /** + * Create a new {@link ManifestWriter}. + * + *

Manifests created by this writer have all entry snapshot IDs set to null. All entries will + * inherit the snapshot ID that will be assigned to the manifest on commit. + * + * @param spec {@link PartitionSpec} used to produce {@link DataFile} partition tuples + * @param outputFile the destination file location + * @param config the configuration used by the Manifest writer + * @return a manifest writer + */ + public static ManifestWriter write( + PartitionSpec spec, OutputFile outputFile, Map config) { + return write(1, spec, outputFile, null, config); } /** @@ -155,11 +171,30 @@ public static ManifestWriter write(PartitionSpec spec, OutputFile outp */ public static ManifestWriter write( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { + return write(formatVersion, spec, outputFile, snapshotId, ImmutableMap.of()); + } + + /** + * Create a new {@link ManifestWriter} for the given format version. + * + * @param formatVersion a target format version + * @param spec a {@link PartitionSpec} + * @param outputFile an {@link OutputFile} where the manifest will be written + * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID + * @param config the configuration used by the Manifest writer + * @return a manifest writer + */ + public static ManifestWriter write( + int formatVersion, + PartitionSpec spec, + OutputFile outputFile, + Long snapshotId, + Map config) { switch (formatVersion) { case 1: - return new ManifestWriter.V1Writer(spec, outputFile, snapshotId); + return new ManifestWriter.V1Writer(spec, outputFile, snapshotId, config); case 2: - return new ManifestWriter.V2Writer(spec, outputFile, snapshotId); + return new ManifestWriter.V2Writer(spec, outputFile, snapshotId, config); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -195,11 +230,30 @@ public static ManifestReader readDeleteManifest( */ public static ManifestWriter writeDeleteManifest( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { + return writeDeleteManifest(formatVersion, spec, outputFile, snapshotId, ImmutableMap.of()); + } + + /** + * Create a new {@link ManifestWriter} for the given format version. + * + * @param formatVersion a target format version + * @param spec a {@link PartitionSpec} + * @param outputFile an {@link OutputFile} where the manifest will be written + * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID + * @param config the configuration used by the Manifest writer + * @return a manifest writer + */ + public static ManifestWriter writeDeleteManifest( + int formatVersion, + PartitionSpec spec, + OutputFile outputFile, + Long snapshotId, + Map config) { switch (formatVersion) { case 1: throw new IllegalArgumentException("Cannot write delete files in a v1 table"); case 2: - return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId); + return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId, config); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -252,7 +306,8 @@ static ManifestFile copyAppendManifest( Map specsById, OutputFile outputFile, long snapshotId, - SnapshotSummary.Builder summaryBuilder) { + SnapshotSummary.Builder summaryBuilder, + Map config) { // use metadata that will add the current snapshot's ID for the rewrite InheritableMetadata inheritableMetadata = InheritableMetadataFactory.forCopy(snapshotId); try (ManifestReader reader = @@ -263,7 +318,8 @@ static ManifestFile copyAppendManifest( outputFile, snapshotId, summaryBuilder, - ManifestEntry.Status.ADDED); + ManifestEntry.Status.ADDED, + config); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", toCopy.location()); } @@ -275,7 +331,8 @@ static ManifestFile copyRewriteManifest( Map specsById, OutputFile outputFile, long snapshotId, - SnapshotSummary.Builder summaryBuilder) { + SnapshotSummary.Builder summaryBuilder, + Map config) { // for a rewritten manifest all snapshot ids should be set. use empty metadata to throw an // exception if it is not InheritableMetadata inheritableMetadata = InheritableMetadataFactory.empty(); @@ -287,7 +344,8 @@ static ManifestFile copyRewriteManifest( outputFile, snapshotId, summaryBuilder, - ManifestEntry.Status.EXISTING); + ManifestEntry.Status.EXISTING, + config); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", toCopy.location()); } @@ -300,8 +358,10 @@ private static ManifestFile copyManifestInternal( OutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder, - ManifestEntry.Status allowedEntryStatus) { - ManifestWriter writer = write(formatVersion, reader.spec(), outputFile, snapshotId); + ManifestEntry.Status allowedEntryStatus, + Map config) { + ManifestWriter writer = + write(formatVersion, reader.spec(), outputFile, snapshotId, config); boolean threw = true; try { for (ManifestEntry entry : reader.entries()) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index 3f7f20d4df6c..e76c8f3adac0 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -31,14 +31,15 @@ abstract class ManifestListWriter implements FileAppender { private final FileAppender writer; - private ManifestListWriter(OutputFile file, Map meta) { - this.writer = newAppender(file, meta); + private ManifestListWriter( + OutputFile file, Map meta, Map config) { + this.writer = newAppender(file, meta, ImmutableMap.copyOf(config)); } protected abstract ManifestFile prepare(ManifestFile manifest); protected abstract FileAppender newAppender( - OutputFile file, Map meta); + OutputFile file, Map meta, Map config); @Override public void add(ManifestFile manifest) { @@ -73,14 +74,20 @@ public long length() { static class V2Writer extends ManifestListWriter { private final V2Metadata.IndexedManifestFile wrapper; - V2Writer(OutputFile snapshotFile, long snapshotId, Long parentSnapshotId, long sequenceNumber) { + V2Writer( + OutputFile snapshotFile, + long snapshotId, + Long parentSnapshotId, + long sequenceNumber, + Map config) { super( snapshotFile, ImmutableMap.of( "snapshot-id", String.valueOf(snapshotId), "parent-snapshot-id", String.valueOf(parentSnapshotId), "sequence-number", String.valueOf(sequenceNumber), - "format-version", "2")); + "format-version", "2"), + config); this.wrapper = new V2Metadata.IndexedManifestFile(snapshotId, sequenceNumber); } @@ -90,12 +97,14 @@ protected ManifestFile prepare(ManifestFile manifest) { } @Override - protected FileAppender newAppender(OutputFile file, Map meta) { + protected FileAppender newAppender( + OutputFile file, Map meta, Map config) { try { return Avro.write(file) .schema(V2Metadata.MANIFEST_LIST_SCHEMA) .named("manifest_file") .meta(meta) + .setAll(config) .overwrite() .build(); @@ -108,13 +117,18 @@ protected FileAppender newAppender(OutputFile file, Map config) { super( snapshotFile, ImmutableMap.of( "snapshot-id", String.valueOf(snapshotId), "parent-snapshot-id", String.valueOf(parentSnapshotId), - "format-version", "1")); + "format-version", "1"), + config); } @Override @@ -126,12 +140,14 @@ protected ManifestFile prepare(ManifestFile manifest) { } @Override - protected FileAppender newAppender(OutputFile file, Map meta) { + protected FileAppender newAppender( + OutputFile file, Map meta, Map config) { try { return Avro.write(file) .schema(V1Metadata.MANIFEST_LIST_SCHEMA) .named("manifest_file") .meta(meta) + .setAll(config) .overwrite() .build(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index c7b3e5fee5a9..19d173b321b0 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; @@ -55,17 +56,19 @@ static ManifestListWriter write( OutputFile manifestListFile, long snapshotId, Long parentSnapshotId, - long sequenceNumber) { + long sequenceNumber, + Map config) { switch (formatVersion) { case 1: Preconditions.checkArgument( sequenceNumber == TableMetadata.INITIAL_SEQUENCE_NUMBER, "Invalid sequence number for v1 manifest list: %s", sequenceNumber); - return new ManifestListWriter.V1Writer(manifestListFile, snapshotId, parentSnapshotId); + return new ManifestListWriter.V1Writer( + manifestListFile, snapshotId, parentSnapshotId, config); case 2: return new ManifestListWriter.V2Writer( - manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); + manifestListFile, snapshotId, parentSnapshotId, sequenceNumber, config); } throw new UnsupportedOperationException( "Cannot write manifest list for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index 5f47de534562..3fc2ce4db6a8 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -38,6 +38,7 @@ import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -265,6 +266,11 @@ CloseableIterable> liveEntries() { entry -> entry != null && entry.status() != ManifestEntry.Status.DELETED); } + @VisibleForTesting + Map metadata() { + return metadata; + } + /** @return an Iterator of DataFile. Makes defensive copies of files before returning */ @Override public CloseableIterator iterator() { diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 7de0cb7be561..7d0d05535ccc 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -19,11 +19,13 @@ package org.apache.iceberg; import java.io.IOException; +import java.util.Map; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; /** * Writer for manifest files. @@ -52,10 +54,11 @@ public abstract class ManifestWriter> implements FileAp private long deletedRows = 0L; private Long minSequenceNumber = null; - private ManifestWriter(PartitionSpec spec, OutputFile file, Long snapshotId) { + private ManifestWriter( + PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { this.file = file; this.specId = spec.specId(); - this.writer = newAppender(spec, file); + this.writer = newAppender(spec, file, ImmutableMap.copyOf(config)); this.snapshotId = snapshotId; this.reused = new GenericManifestEntry<>(spec.partitionType()); this.stats = new PartitionSummary(spec); @@ -64,7 +67,7 @@ private ManifestWriter(PartitionSpec spec, OutputFile file, Long snapshotId) { protected abstract ManifestEntry prepare(ManifestEntry entry); protected abstract FileAppender> newAppender( - PartitionSpec spec, OutputFile outputFile); + PartitionSpec spec, OutputFile outputFile, Map config); protected ManifestContent content() { return ManifestContent.DATA; @@ -202,8 +205,8 @@ public void close() throws IOException { static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; - V2Writer(PartitionSpec spec, OutputFile file, Long snapshotId) { - super(spec, file, snapshotId); + V2Writer(PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { + super(spec, file, snapshotId, config); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -214,7 +217,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file) { + PartitionSpec spec, OutputFile file, Map config) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { return Avro.write(file) @@ -225,6 +228,7 @@ protected FileAppender> newAppender( .meta("partition-spec-id", String.valueOf(spec.specId())) .meta("format-version", "2") .meta("content", "data") + .setAll(config) .overwrite() .build(); } catch (IOException e) { @@ -236,8 +240,9 @@ protected FileAppender> newAppender( static class V2DeleteWriter extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; - V2DeleteWriter(PartitionSpec spec, OutputFile file, Long snapshotId) { - super(spec, file, snapshotId); + V2DeleteWriter( + PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { + super(spec, file, snapshotId, config); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -248,7 +253,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file) { + PartitionSpec spec, OutputFile file, Map config) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { return Avro.write(file) @@ -259,6 +264,7 @@ protected FileAppender> newAppender( .meta("partition-spec-id", String.valueOf(spec.specId())) .meta("format-version", "2") .meta("content", "deletes") + .setAll(config) .overwrite() .build(); } catch (IOException e) { @@ -275,8 +281,8 @@ protected ManifestContent content() { static class V1Writer extends ManifestWriter { private final V1Metadata.IndexedManifestEntry entryWrapper; - V1Writer(PartitionSpec spec, OutputFile file, Long snapshotId) { - super(spec, file, snapshotId); + V1Writer(PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { + super(spec, file, snapshotId, config); this.entryWrapper = new V1Metadata.IndexedManifestEntry(spec.partitionType()); } @@ -287,7 +293,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file) { + PartitionSpec spec, OutputFile file, Map config) { Schema manifestSchema = V1Metadata.entrySchema(spec.partitionType()); try { return Avro.write(file) @@ -297,6 +303,7 @@ protected FileAppender> newAppender( .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) .meta("partition-spec-id", String.valueOf(spec.specId())) .meta("format-version", "1") + .setAll(config) .overwrite() .build(); } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index b82244f0714f..024bed2956c2 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -264,7 +264,8 @@ private ManifestFile copyManifest(ManifestFile manifest) { current.specsById(), newManifestPath, snapshotId(), - appendedManifestsSummary); + appendedManifestsSummary, + current.properties()); } /** diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index a561abf93518..316833df9e2d 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -207,7 +207,8 @@ public Snapshot apply() { manifestList, snapshotId(), parentSnapshotId, - sequenceNumber)) { + sequenceNumber, + ops.current().properties())) { // keep track of the manifest lists created manifestLists.add(manifestList.location()); @@ -460,12 +461,20 @@ protected OutputFile newManifestOutput() { protected ManifestWriter newManifestWriter(PartitionSpec spec) { return ManifestFiles.write( - ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); + ops.current().formatVersion(), + spec, + newManifestOutput(), + snapshotId(), + ops.current().properties()); } protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) { return ManifestFiles.writeDeleteManifest( - ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); + ops.current().formatVersion(), + spec, + newManifestOutput(), + snapshotId(), + ops.current().properties()); } protected ManifestReader newManifestReader(ManifestFile manifest) { diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 53516c980fcd..1aa5a8c46826 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -242,7 +242,8 @@ ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOExceptio OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); + ManifestFiles.write( + formatVersion, table.spec(), outputFile, snapshotId, table.properties()); try { for (DataFile file : files) { writer.add(file); @@ -273,12 +274,13 @@ > ManifestFile writeManifest( if (entries[0].file() instanceof DataFile) { writer = (ManifestWriter) - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); + ManifestFiles.write( + formatVersion, table.spec(), outputFile, snapshotId, table.properties()); } else { writer = (ManifestWriter) ManifestFiles.writeDeleteManifest( - formatVersion, table.spec(), outputFile, snapshotId); + formatVersion, table.spec(), outputFile, snapshotId, table.properties()); } try { for (ManifestEntry entry : entries) { @@ -297,7 +299,8 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi org.apache.iceberg.Files.localOutput( FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.writeDeleteManifest(newFormatVersion, SPEC, manifestFile, snapshotId); + ManifestFiles.writeDeleteManifest( + newFormatVersion, SPEC, manifestFile, snapshotId, table.properties()); try { for (DeleteFile deleteFile : deleteFiles) { writer.add(deleteFile); @@ -314,7 +317,7 @@ ManifestFile writeManifestWithName(String name, DataFile... files) throws IOExce OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, null); + ManifestFiles.write(formatVersion, table.spec(), outputFile, null, table.properties()); try { for (DataFile file : files) { writer.add(file); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index fe026a0a5c65..01f7970f3eec 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; +import java.util.Map; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.iceberg.avro.Avro; @@ -33,6 +34,7 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; @@ -103,7 +105,7 @@ public void testV1WriteDeleteManifest() { "Should fail to write a DELETE manifest to v1", IllegalArgumentException.class, "Cannot store delete manifests in a v1 table", - () -> writeManifestList(TEST_DELETE_MANIFEST, 1)); + () -> writeManifestList(TEST_DELETE_MANIFEST, 1, ImmutableMap.of())); } @Test @@ -154,7 +156,7 @@ public void testV2Write() throws IOException { @Test public void testV1ForwardCompatibility() throws IOException { - InputFile manifestList = writeManifestList(TEST_MANIFEST, 1); + InputFile manifestList = writeManifestList(TEST_MANIFEST, 1, ImmutableMap.of()); GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed @@ -182,7 +184,7 @@ public void testV1ForwardCompatibility() throws IOException { public void testV2ForwardCompatibility() throws IOException { // v2 manifest list files can be read by v1 readers, but the sequence numbers and content will // be ignored. - InputFile manifestList = writeManifestList(TEST_MANIFEST, 2); + InputFile manifestList = writeManifestList(TEST_MANIFEST, 2, ImmutableMap.of()); GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed @@ -295,7 +297,7 @@ public void testManifestsPartitionSummary() throws IOException { partitionFieldSummaries, KEY_METADATA); - InputFile manifestList = writeManifestList(manifest, 2); + InputFile manifestList = writeManifestList(manifest, 2, ImmutableMap.of()); List files = ManifestLists.read(manifestList); ManifestFile returnedManifest = Iterables.getOnlyElement(files); @@ -331,7 +333,8 @@ public void testManifestsPartitionSummary() throws IOException { second.upperBound()); } - private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { + private InputFile writeManifestList( + ManifestFile manifest, int formatVersion, Map config) throws IOException { OutputFile manifestList = new InMemoryOutputFile(); try (FileAppender writer = ManifestLists.write( @@ -339,7 +342,8 @@ private InputFile writeManifestList(ManifestFile manifest, int formatVersion) th manifestList, SNAPSHOT_ID, SNAPSHOT_ID - 1, - formatVersion > 1 ? SEQ_NUM : 0)) { + formatVersion > 1 ? SEQ_NUM : 0, + config)) { writer.add(manifest); } return manifestList.toInputFile(); @@ -356,7 +360,7 @@ private GenericData.Record readGeneric(InputFile manifestList, Schema schema) th private ManifestFile writeAndReadManifestList(int formatVersion) throws IOException { List manifests = - ManifestLists.read(writeManifestList(TEST_MANIFEST, formatVersion)); + ManifestLists.read(writeManifestList(TEST_MANIFEST, formatVersion, ImmutableMap.of())); Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 4b7f4d3b8f92..8bbea419de77 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -104,7 +104,7 @@ public void testWriteManifestWithSequenceNumber() throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L); + ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L, table.properties()); writer.add(newFile(10, TestHelpers.Row.of(1)), 1000L); writer.close(); ManifestFile manifest = writer.toManifestFile(); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 9c606f5e7e60..5f6ef0397fad 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; @@ -40,6 +41,7 @@ public class TestManifestWriterVersions { private static final FileIO FILE_IO = new TestTables.LocalFileIO(); + private static final String AVRO_CODEC_KEY = "avro.codec"; private static final Schema SCHEMA = new Schema( @@ -95,6 +97,13 @@ public class TestManifestWriterVersions { SORT_ORDER_ID, null); + private static final Map CODEC_METADATA_MAPPING = + ImmutableMap.builder() + .put("uncompressed", "null") + .put("zstd", "zstandard") + .put("gzip", "deflate") + .build(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); @Test @@ -104,6 +113,12 @@ public void testV1Write() throws IOException { checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA); } + @Test + public void testV1WriteWithConfig() throws IOException { + checkManifestMetadata(config -> writeManifest(DATA_FILE, 1, config), + manifest -> ManifestFiles.read(manifest, FILE_IO)); + } + @Test public void testV1WriteDelete() { AssertHelpers.assertThrows( @@ -130,6 +145,13 @@ public void testV2Write() throws IOException { checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA); } + @Test + public void testV2WriteWithConfig() throws IOException { + checkManifestMetadata( + config -> writeManifest(DATA_FILE, 2, config), + manifest -> ManifestFiles.read(manifest, FILE_IO)); + } + @Test public void testV2WriteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeManifest(2), 2); @@ -149,6 +171,13 @@ public void testV2WriteDelete() throws IOException { readDeleteManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.EQUALITY_DELETES); } + @Test + public void testV2WriteDeleteWithConfig() throws IOException { + checkManifestMetadata( + config -> writeDeleteManifest(2, config), + manifest -> ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)); + } + @Test public void testV2WriteDeleteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeDeleteManifest(2), 2); @@ -257,7 +286,27 @@ void checkRewrittenManifest( Assert.assertEquals("Deleted rows count", (Long) 0L, manifest.deletedRowsCount()); } - private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { + > void checkManifestMetadata( + CheckedFunction, ManifestFile> createManifestFunc, + CheckedFunction> manifestReaderFunc) + throws IOException { + for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { + String codec = entry.getKey(); + String expectedCodecValue = entry.getValue(); + + Map config = ImmutableMap.of(TableProperties.AVRO_COMPRESSION, codec); + ManifestFile manifest = createManifestFunc.apply(config); + + try (ManifestReader manifestReader = manifestReaderFunc.apply(manifest)) { + Map metadata = manifestReader.metadata(); + Assert.assertEquals( + "Codec value must match", expectedCodecValue, metadata.get(AVRO_CODEC_KEY)); + } + } + } + + private InputFile writeManifestList( + ManifestFile manifest, int formatVersion, Map config) throws IOException { OutputFile manifestList = new InMemoryOutputFile(); try (FileAppender writer = ManifestLists.write( @@ -265,7 +314,8 @@ private InputFile writeManifestList(ManifestFile manifest, int formatVersion) th manifestList, SNAPSHOT_ID, SNAPSHOT_ID - 1, - formatVersion > 1 ? SEQUENCE_NUMBER : 0)) { + formatVersion > 1 ? SEQUENCE_NUMBER : 0, + config)) { writer.add(manifest); } return manifestList.toInputFile(); @@ -273,7 +323,8 @@ private InputFile writeManifestList(ManifestFile manifest, int formatVersion) th private ManifestFile writeAndReadManifestList(ManifestFile manifest, int formatVersion) throws IOException { - List manifests = ManifestLists.read(writeManifestList(manifest, formatVersion)); + List manifests = + ManifestLists.read(writeManifestList(manifest, formatVersion, ImmutableMap.of())); Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } @@ -283,7 +334,7 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); + ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, ImmutableMap.of()); try { writer.existing(readManifest(manifest)); } finally { @@ -293,14 +344,15 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) } private ManifestFile writeManifest(int formatVersion) throws IOException { - return writeManifest(DATA_FILE, formatVersion); + return writeManifest(DATA_FILE, formatVersion, ImmutableMap.of()); } - private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOException { + private ManifestFile writeManifest(DataFile file, int formatVersion, Map config) + throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); + ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, config); try { writer.add(file); } finally { @@ -319,10 +371,15 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { + return writeDeleteManifest(formatVersion, ImmutableMap.of()); + } + + private ManifestFile writeDeleteManifest(int formatVersion, Map config) + throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); + ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, config); try { writer.add(DELETE_FILE); } finally { @@ -339,4 +396,8 @@ private ManifestEntry readDeleteManifest(ManifestFile manifest) thro return entries.get(0); } } + + interface CheckedFunction { + R apply(T t) throws IOException; + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index c460336942b4..8e914ab649cf 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -140,7 +140,8 @@ public void testJsonConversionWithManifestList() throws IOException { manifestList.deleteOnExit(); try (ManifestListWriter writer = - ManifestLists.write(1, Files.localOutput(manifestList), id, parentId, 0)) { + ManifestLists.write( + 1, Files.localOutput(manifestList), id, parentId, 0, ImmutableMap.of())) { writer.addAll(manifests); } diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 970a2310d738..c34cadd28691 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -535,7 +535,8 @@ public void testTransactionRetryAndAppendManifests() throws Exception { // create a manifest append OutputFile manifestLocation = Files.localOutput("/tmp/" + UUID.randomUUID().toString() + ".avro"); - ManifestWriter writer = ManifestFiles.write(table.spec(), manifestLocation); + ManifestWriter writer = + ManifestFiles.write(table.spec(), manifestLocation, table.properties()); try { writer.add(FILE_D); } finally { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index 25badc372abf..4eeae2da95cf 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.function.Supplier; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -43,9 +44,13 @@ class FlinkManifestUtil { private FlinkManifestUtil() {} static ManifestFile writeDataFiles( - OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { + OutputFile outputFile, + PartitionSpec spec, + List dataFiles, + Map config) + throws IOException { ManifestWriter writer = - ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); + ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID, config); try (ManifestWriter closeableWriter = writer) { closeableWriter.addAll(dataFiles); @@ -74,7 +79,10 @@ static ManifestOutputFileFactory createOutputFileFactory( } static DeltaManifests writeCompletedFiles( - WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) + WriteResult result, + Supplier outputFileSupplier, + PartitionSpec spec, + Map config) throws IOException { ManifestFile dataManifest = null; @@ -83,7 +91,8 @@ static DeltaManifests writeCompletedFiles( // Write the completed data files into a newly created data manifest file. if (result.dataFiles() != null && result.dataFiles().length > 0) { dataManifest = - writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); + writeDataFiles( + outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles()), config); } // Write the completed delete files into a newly created delete manifest file. @@ -91,7 +100,8 @@ static DeltaManifests writeCompletedFiles( OutputFile deleteManifestFile = outputFileSupplier.get(); ManifestWriter deleteManifestWriter = - ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); + ManifestFiles.writeDeleteManifest( + FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID, config); try (ManifestWriter writer = deleteManifestWriter) { for (DeleteFile deleteFile : result.deleteFiles()) { writer.add(deleteFile); diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 8aa2c0304eb0..bd261fba3246 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -407,7 +407,10 @@ private byte[] writeToManifest(long checkpointId) throws IOException { WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( - result, () -> manifestOutputFileFactory.create(checkpointId), table.spec()); + result, + () -> manifestOutputFileFactory.create(checkpointId), + table.spec(), + table.properties()); return SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, deltaManifests); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 6bd94e9ca61c..05134ef3f8c5 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -156,7 +156,7 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile, ImmutableMap.of()); try { for (DataFile file : files) { writer.add(file); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 27dc665055cc..e37cdbe09b08 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -107,7 +107,8 @@ public void testIO() throws IOException { .addDeleteFiles(posDeleteFiles) .build(), () -> factory.create(curCkpId), - table.spec()); + table.spec(), + table.properties()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io()); Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); @@ -147,7 +148,8 @@ public void testUserProvidedManifestLocation() throws IOException { FlinkManifestUtil.writeCompletedFiles( WriteResult.builder().addDataFiles(dataFiles).build(), () -> factory.create(checkpointId), - table.spec()); + table.spec(), + table.properties()); Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); @@ -187,7 +189,8 @@ public void testVersionedSerializer() throws IOException { .addDeleteFiles(posDeleteFiles) .build(), () -> factory.create(checkpointId), - table.spec()); + table.spec(), + table.properties()); byte[] versionedSerializeData = SimpleVersionedSerialization.writeVersionAndSerialize( @@ -215,7 +218,8 @@ public void testCompatibility() throws IOException { List dataFiles = generateDataFiles(10); ManifestFile manifest = - FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); + FlinkManifestUtil.writeDataFiles( + factory.create(checkpointId), table.spec(), dataFiles, table.properties()); byte[] dataV1 = SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index c38a394c5a25..f21503028eaa 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -367,7 +367,8 @@ private static Iterator buildManifest( SerializableConfiguration conf, PartitionSpec spec, String basePath, - Iterator> fileTuples) { + Iterator> fileTuples, + Map config) { if (fileTuples.hasNext()) { FileIO io = new HadoopFileIO(conf.get()); TaskContext ctx = TaskContext.get(); @@ -376,7 +377,7 @@ private static Iterator buildManifest( Path location = new Path(basePath, suffix); String outputPath = FileFormat.AVRO.addExtension(location.toString()); OutputFile outputFile = io.newOutputFile(outputPath); - ManifestWriter writer = ManifestFiles.write(spec, outputFile); + ManifestWriter writer = ManifestFiles.write(spec, outputFile, config); try (ManifestWriter writerRef = writer) { fileTuples.forEachRemaining(fileTuple -> writerRef.add(fileTuple._2)); @@ -623,7 +624,13 @@ public static void importSparkPartitions( .orderBy(col("_1")) .mapPartitions( (MapPartitionsFunction, ManifestFile>) - fileTuple -> buildManifest(serializableConf, spec, stagingDir, fileTuple), + fileTuple -> + buildManifest( + serializableConf, + spec, + stagingDir, + fileTuple, + targetTable.properties()), Encoders.javaSerialization(ManifestFile.class)) .collectAsList(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 43476d21daed..6ac4902c6a6e 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; @@ -226,7 +227,8 @@ private List writeManifestsForUnpartitionedTable( formatVersion, combinedPartitionType, spec, - sparkType), + sparkType, + table.properties()), manifestEncoder) .collectAsList(); } @@ -256,7 +258,8 @@ private List writeManifestsForPartitionedTable( formatVersion, combinedPartitionType, spec, - sparkType), + sparkType, + table.properties()), manifestEncoder) .collectAsList(); }); @@ -355,7 +358,8 @@ private static ManifestFile writeManifest( int format, Types.StructType combinedPartitionType, PartitionSpec spec, - StructType sparkType) + StructType sparkType, + Map config) throws IOException { String manifestName = "optimized-m-" + UUID.randomUUID(); @@ -367,7 +371,7 @@ private static ManifestFile writeManifest( Types.StructType manifestFileType = DataFile.getType(spec.partitionType()); SparkDataFile wrapper = new SparkDataFile(combinedFileType, manifestFileType, sparkType); - ManifestWriter writer = ManifestFiles.write(format, spec, outputFile, null); + ManifestWriter writer = ManifestFiles.write(format, spec, outputFile, null, config); try { for (int index = startIndex; index < endIndex; index++) { @@ -391,7 +395,8 @@ private static MapPartitionsFunction toManifests( int format, Types.StructType combinedPartitionType, PartitionSpec spec, - StructType sparkType) { + StructType sparkType, + Map config) { return rows -> { List rowsAsList = Lists.newArrayList(rows); @@ -412,7 +417,8 @@ private static MapPartitionsFunction toManifests( format, combinedPartitionType, spec, - sparkType)); + sparkType, + config)); } else { int midIndex = rowsAsList.size() / 2; manifests.add( @@ -425,7 +431,8 @@ private static MapPartitionsFunction toManifests( format, combinedPartitionType, spec, - sparkType)); + sparkType, + config)); manifests.add( writeManifest( rowsAsList, @@ -436,7 +443,8 @@ private static MapPartitionsFunction toManifests( format, combinedPartitionType, spec, - sparkType)); + sparkType, + config)); } return manifests.iterator(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java index a20b2d9f05de..223204cf5337 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -199,7 +199,7 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile, ImmutableMap.of()); try { for (DataFile file : files) { writer.add(file); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 42d9ac6a79ba..a5c958cb503b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -195,7 +195,8 @@ public void testSparkCanReadUnknownTransform() throws IOException { .build(); OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); - ManifestWriter manifestWriter = ManifestFiles.write(FAKE_SPEC, manifestFile); + ManifestWriter manifestWriter = + ManifestFiles.write(FAKE_SPEC, manifestFile, table.properties()); try { manifestWriter.add(file); } finally { diff --git a/versions.props b/versions.props index c24e43a49a85..93f0fe864dc5 100644 --- a/versions.props +++ b/versions.props @@ -44,3 +44,5 @@ org.mock-server:mockserver-netty = 5.13.2 org.mock-server:mockserver-client-java = 5.13.2 com.esotericsoftware:kryo = 4.0.2 org.eclipse.jetty:* = 9.4.43.v20210629 +org.xerial.snappy:snappy-java = 1.1.8.4 +com.github.luben:zstd-jni = 1.5.2-3 \ No newline at end of file From e4d948594d957b15faf117e113915595a21f9bb0 Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Thu, 29 Sep 2022 17:20:54 -0700 Subject: [PATCH 2/5] Fix spotless java check --- .../java/org/apache/iceberg/TestManifestWriterVersions.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 5f6ef0397fad..aa524c56cd7b 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -115,8 +115,9 @@ public void testV1Write() throws IOException { @Test public void testV1WriteWithConfig() throws IOException { - checkManifestMetadata(config -> writeManifest(DATA_FILE, 1, config), - manifest -> ManifestFiles.read(manifest, FILE_IO)); + checkManifestMetadata( + config -> writeManifest(DATA_FILE, 1, config), + manifest -> ManifestFiles.read(manifest, FILE_IO)); } @Test From 4790f817ad8307ab00ef171ae91edafac430968e Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Thu, 29 Sep 2022 17:35:02 -0700 Subject: [PATCH 3/5] Fix stylecheck errors --- .../java/org/apache/iceberg/TestManifestWriterVersions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index aa524c56cd7b..0f3576fec800 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -399,6 +399,6 @@ private ManifestEntry readDeleteManifest(ManifestFile manifest) thro } interface CheckedFunction { - R apply(T t) throws IOException; + R apply(T args) throws IOException; } } From 288458bdab8cf4904e3facec806919df3695293c Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Wed, 5 Oct 2022 03:16:10 -0700 Subject: [PATCH 4/5] Explicitly pass compression codec and compression level instead of generic config map --- .../apache/iceberg/BaseRewriteManifests.java | 3 +- .../java/org/apache/iceberg/FastAppend.java | 3 +- .../org/apache/iceberg/ManifestFiles.java | 69 +++++++--- .../apache/iceberg/ManifestListWriter.java | 70 ++++++---- .../org/apache/iceberg/ManifestLists.java | 39 +++--- .../org/apache/iceberg/ManifestWriter.java | 126 +++++++++++------- .../iceberg/MergingSnapshotProducer.java | 3 +- .../org/apache/iceberg/SnapshotProducer.java | 9 +- .../org/apache/iceberg/TableTestBase.java | 36 ++++- .../iceberg/TestManifestListVersions.java | 18 ++- .../apache/iceberg/TestManifestWriter.java | 8 +- .../iceberg/TestManifestWriterVersions.java | 109 ++++++++++++--- .../org/apache/iceberg/TestSnapshotJson.java | 8 +- .../org/apache/iceberg/TestTransaction.java | 6 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 23 +++- .../flink/sink/IcebergFilesCommitter.java | 4 +- .../flink/TestManifestFileSerialization.java | 4 +- .../iceberg/flink/sink/TestFlinkManifest.java | 16 ++- .../apache/iceberg/spark/SparkTableUtil.java | 9 +- .../actions/RewriteManifestsSparkAction.java | 25 ++-- .../TestManifestFileSerialization.java | 4 +- .../source/TestForwardCompatibility.java | 6 +- 22 files changed, 421 insertions(+), 177 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 5a6341cfbe54..6b481928848f 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -170,7 +170,8 @@ private ManifestFile copyManifest(ManifestFile manifest) { newFile, snapshotId(), summaryBuilder, - current.properties()); + current.properties().get(TableProperties.AVRO_COMPRESSION), + current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } @Override diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index d2d3532bfd29..8c54e6a56f98 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -138,7 +138,8 @@ private ManifestFile copyManifest(ManifestFile manifest) { newManifestPath, snapshotId(), summaryBuilder, - current.properties()); + current.properties().get(TableProperties.AVRO_COMPRESSION), + current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 03f782908fa9..5972bc2df9f0 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -141,7 +141,7 @@ public static ManifestReader read( * @return a manifest writer */ public static ManifestWriter write(PartitionSpec spec, OutputFile outputFile) { - return write(spec, outputFile, ImmutableMap.of()); + return write(spec, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); } /** @@ -152,12 +152,13 @@ public static ManifestWriter write(PartitionSpec spec, OutputFile outp * * @param spec {@link PartitionSpec} used to produce {@link DataFile} partition tuples * @param outputFile the destination file location - * @param config the configuration used by the Manifest writer + * @param compressionCodec compression codec for the manifest file + * @param compressionLevel compression level of the compressionCodec * @return a manifest writer */ public static ManifestWriter write( - PartitionSpec spec, OutputFile outputFile, Map config) { - return write(1, spec, outputFile, null, config); + PartitionSpec spec, OutputFile outputFile, String compressionCodec, String compressionLevel) { + return write(1, spec, outputFile, null, compressionCodec, compressionLevel); } /** @@ -171,7 +172,13 @@ public static ManifestWriter write( */ public static ManifestWriter write( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { - return write(formatVersion, spec, outputFile, snapshotId, ImmutableMap.of()); + return write( + formatVersion, + spec, + outputFile, + snapshotId, + /* compressionCodec */ null, + /* compressionLevel */ null); } /** @@ -181,7 +188,8 @@ public static ManifestWriter write( * @param spec a {@link PartitionSpec} * @param outputFile an {@link OutputFile} where the manifest will be written * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID - * @param config the configuration used by the Manifest writer + * @param compressionCodec compression codec for the manifest file + * @param compressionLevel compression level of the compressionCodec * @return a manifest writer */ public static ManifestWriter write( @@ -189,12 +197,15 @@ public static ManifestWriter write( PartitionSpec spec, OutputFile outputFile, Long snapshotId, - Map config) { + String compressionCodec, + String compressionLevel) { switch (formatVersion) { case 1: - return new ManifestWriter.V1Writer(spec, outputFile, snapshotId, config); + return new ManifestWriter.V1Writer( + spec, outputFile, snapshotId, compressionCodec, compressionLevel); case 2: - return new ManifestWriter.V2Writer(spec, outputFile, snapshotId, config); + return new ManifestWriter.V2Writer( + spec, outputFile, snapshotId, compressionCodec, compressionLevel); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -230,7 +241,13 @@ public static ManifestReader readDeleteManifest( */ public static ManifestWriter writeDeleteManifest( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { - return writeDeleteManifest(formatVersion, spec, outputFile, snapshotId, ImmutableMap.of()); + return writeDeleteManifest( + formatVersion, + spec, + outputFile, + snapshotId, + /* compressionCodec */ null, + /* compressionLevel */ null); } /** @@ -240,7 +257,8 @@ public static ManifestWriter writeDeleteManifest( * @param spec a {@link PartitionSpec} * @param outputFile an {@link OutputFile} where the manifest will be written * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID - * @param config the configuration used by the Manifest writer + * @param compressionCodec compression codec for the manifest file + * @param compressionLevel compression level of the compressionCodec * @return a manifest writer */ public static ManifestWriter writeDeleteManifest( @@ -248,12 +266,14 @@ public static ManifestWriter writeDeleteManifest( PartitionSpec spec, OutputFile outputFile, Long snapshotId, - Map config) { + String compressionCodec, + String compressionLevel) { switch (formatVersion) { case 1: throw new IllegalArgumentException("Cannot write delete files in a v1 table"); case 2: - return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId, config); + return new ManifestWriter.V2DeleteWriter( + spec, outputFile, snapshotId, compressionCodec, compressionLevel); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -307,7 +327,8 @@ static ManifestFile copyAppendManifest( OutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder, - Map config) { + String compressionCodec, + String compressionLevel) { // use metadata that will add the current snapshot's ID for the rewrite InheritableMetadata inheritableMetadata = InheritableMetadataFactory.forCopy(snapshotId); try (ManifestReader reader = @@ -319,7 +340,8 @@ static ManifestFile copyAppendManifest( snapshotId, summaryBuilder, ManifestEntry.Status.ADDED, - config); + compressionCodec, + compressionLevel); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", toCopy.location()); } @@ -332,7 +354,8 @@ static ManifestFile copyRewriteManifest( OutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder, - Map config) { + String compressionCodec, + String compressionLevel) { // for a rewritten manifest all snapshot ids should be set. use empty metadata to throw an // exception if it is not InheritableMetadata inheritableMetadata = InheritableMetadataFactory.empty(); @@ -345,7 +368,8 @@ static ManifestFile copyRewriteManifest( snapshotId, summaryBuilder, ManifestEntry.Status.EXISTING, - config); + compressionCodec, + compressionLevel); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest: %s", toCopy.location()); } @@ -359,9 +383,16 @@ private static ManifestFile copyManifestInternal( long snapshotId, SnapshotSummary.Builder summaryBuilder, ManifestEntry.Status allowedEntryStatus, - Map config) { + String compressionCodec, + String compressionLevel) { ManifestWriter writer = - write(formatVersion, reader.spec(), outputFile, snapshotId, config); + write( + formatVersion, + reader.spec(), + outputFile, + snapshotId, + compressionCodec, + compressionLevel); boolean threw = true; try { for (ManifestEntry entry : reader.entries()) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index e76c8f3adac0..a54cb66229f1 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -32,14 +32,14 @@ abstract class ManifestListWriter implements FileAppender { private final FileAppender writer; private ManifestListWriter( - OutputFile file, Map meta, Map config) { - this.writer = newAppender(file, meta, ImmutableMap.copyOf(config)); + OutputFile file, Map meta, String compressionCodec, String compressionLevel) { + this.writer = newAppender(file, meta, compressionCodec, compressionLevel); } protected abstract ManifestFile prepare(ManifestFile manifest); protected abstract FileAppender newAppender( - OutputFile file, Map meta, Map config); + OutputFile file, Map meta, String compressionCodec, String compressionLevel); @Override public void add(ManifestFile manifest) { @@ -79,7 +79,8 @@ static class V2Writer extends ManifestListWriter { long snapshotId, Long parentSnapshotId, long sequenceNumber, - Map config) { + String compressionCodec, + String compressionLevel) { super( snapshotFile, ImmutableMap.of( @@ -87,7 +88,8 @@ static class V2Writer extends ManifestListWriter { "parent-snapshot-id", String.valueOf(parentSnapshotId), "sequence-number", String.valueOf(sequenceNumber), "format-version", "2"), - config); + compressionCodec, + compressionLevel); this.wrapper = new V2Metadata.IndexedManifestFile(snapshotId, sequenceNumber); } @@ -98,16 +100,24 @@ protected ManifestFile prepare(ManifestFile manifest) { @Override protected FileAppender newAppender( - OutputFile file, Map meta, Map config) { + OutputFile file, + Map meta, + String compressionCodec, + String compressionLevel) { try { - return Avro.write(file) - .schema(V2Metadata.MANIFEST_LIST_SCHEMA) - .named("manifest_file") - .meta(meta) - .setAll(config) - .overwrite() - .build(); - + Avro.WriteBuilder builder = + Avro.write(file) + .schema(V2Metadata.MANIFEST_LIST_SCHEMA) + .named("manifest_file") + .meta(meta) + .overwrite(); + if (compressionCodec != null) { + builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); + } + if (compressionLevel != null) { + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + } + return builder.build(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to create snapshot list writer for path: %s", file); } @@ -121,14 +131,16 @@ static class V1Writer extends ManifestListWriter { OutputFile snapshotFile, long snapshotId, Long parentSnapshotId, - Map config) { + String compressionCodec, + String compressionLevel) { super( snapshotFile, ImmutableMap.of( "snapshot-id", String.valueOf(snapshotId), "parent-snapshot-id", String.valueOf(parentSnapshotId), "format-version", "1"), - config); + compressionCodec, + compressionLevel); } @Override @@ -141,16 +153,24 @@ protected ManifestFile prepare(ManifestFile manifest) { @Override protected FileAppender newAppender( - OutputFile file, Map meta, Map config) { + OutputFile file, + Map meta, + String compressionCodec, + String compressionLevel) { try { - return Avro.write(file) - .schema(V1Metadata.MANIFEST_LIST_SCHEMA) - .named("manifest_file") - .meta(meta) - .setAll(config) - .overwrite() - .build(); - + Avro.WriteBuilder builder = + Avro.write(file) + .schema(V1Metadata.MANIFEST_LIST_SCHEMA) + .named("manifest_file") + .meta(meta) + .overwrite(); + if (compressionCodec != null) { + builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); + } + if (compressionLevel != null) { + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + } + return builder.build(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to create snapshot list writer for path: %s", file); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index 19d173b321b0..afa78a599d16 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -20,12 +20,13 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -33,31 +34,34 @@ class ManifestLists { private ManifestLists() {} static List read(InputFile manifestList) { - try (CloseableIterable files = - Avro.read(manifestList) - .rename("manifest_file", GenericManifestFile.class.getName()) - .rename("partitions", GenericPartitionFieldSummary.class.getName()) - .rename("r508", GenericPartitionFieldSummary.class.getName()) - .classLoader(GenericManifestFile.class.getClassLoader()) - .project(ManifestFile.schema()) - .reuseContainers(false) - .build()) { - + try (CloseableIterable files = manifestFileIterable(manifestList)) { return Lists.newLinkedList(files); - } catch (IOException e) { throw new RuntimeIOException( e, "Cannot read manifest list file: %s", manifestList.location()); } } + @VisibleForTesting + static AvroIterable manifestFileIterable(InputFile manifestList) { + return Avro.read(manifestList) + .rename("manifest_file", GenericManifestFile.class.getName()) + .rename("partitions", GenericPartitionFieldSummary.class.getName()) + .rename("r508", GenericPartitionFieldSummary.class.getName()) + .classLoader(GenericManifestFile.class.getClassLoader()) + .project(ManifestFile.schema()) + .reuseContainers(false) + .build(); + } + static ManifestListWriter write( int formatVersion, OutputFile manifestListFile, long snapshotId, Long parentSnapshotId, long sequenceNumber, - Map config) { + String compressionCodec, + String compressionLevel) { switch (formatVersion) { case 1: Preconditions.checkArgument( @@ -65,10 +69,15 @@ static ManifestListWriter write( "Invalid sequence number for v1 manifest list: %s", sequenceNumber); return new ManifestListWriter.V1Writer( - manifestListFile, snapshotId, parentSnapshotId, config); + manifestListFile, snapshotId, parentSnapshotId, compressionCodec, compressionLevel); case 2: return new ManifestListWriter.V2Writer( - manifestListFile, snapshotId, parentSnapshotId, sequenceNumber, config); + manifestListFile, + snapshotId, + parentSnapshotId, + sequenceNumber, + compressionCodec, + compressionLevel); } throw new UnsupportedOperationException( "Cannot write manifest list for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 7d0d05535ccc..71deca5de44c 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -19,13 +19,11 @@ package org.apache.iceberg; import java.io.IOException; -import java.util.Map; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; /** * Writer for manifest files. @@ -55,10 +53,14 @@ public abstract class ManifestWriter> implements FileAp private Long minSequenceNumber = null; private ManifestWriter( - PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { + PartitionSpec spec, + OutputFile file, + Long snapshotId, + String compressionCodec, + String compressionLevel) { this.file = file; this.specId = spec.specId(); - this.writer = newAppender(spec, file, ImmutableMap.copyOf(config)); + this.writer = newAppender(spec, file, compressionCodec, compressionLevel); this.snapshotId = snapshotId; this.reused = new GenericManifestEntry<>(spec.partitionType()); this.stats = new PartitionSummary(spec); @@ -67,7 +69,7 @@ private ManifestWriter( protected abstract ManifestEntry prepare(ManifestEntry entry); protected abstract FileAppender> newAppender( - PartitionSpec spec, OutputFile outputFile, Map config); + PartitionSpec spec, OutputFile outputFile, String compressionCodec, String compressionLevel); protected ManifestContent content() { return ManifestContent.DATA; @@ -205,8 +207,13 @@ public void close() throws IOException { static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; - V2Writer(PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { - super(spec, file, snapshotId, config); + V2Writer( + PartitionSpec spec, + OutputFile file, + Long snapshotId, + String compressionCodec, + String compressionLevel) { + super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -217,20 +224,26 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, Map config) { + PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { - return Avro.write(file) - .schema(manifestSchema) - .named("manifest_entry") - .meta("schema", SchemaParser.toJson(spec.schema())) - .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) - .meta("partition-spec-id", String.valueOf(spec.specId())) - .meta("format-version", "2") - .meta("content", "data") - .setAll(config) - .overwrite() - .build(); + Avro.WriteBuilder builder = + Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "2") + .meta("content", "data") + .overwrite(); + if (compressionCodec != null) { + builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); + } + if (compressionLevel != null) { + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + } + return builder.build(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); } @@ -241,8 +254,12 @@ static class V2DeleteWriter extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; V2DeleteWriter( - PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { - super(spec, file, snapshotId, config); + PartitionSpec spec, + OutputFile file, + Long snapshotId, + String compressionCodec, + String compressionLevel) { + super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -253,20 +270,26 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, Map config) { + PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { - return Avro.write(file) - .schema(manifestSchema) - .named("manifest_entry") - .meta("schema", SchemaParser.toJson(spec.schema())) - .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) - .meta("partition-spec-id", String.valueOf(spec.specId())) - .meta("format-version", "2") - .meta("content", "deletes") - .setAll(config) - .overwrite() - .build(); + Avro.WriteBuilder builder = + Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "2") + .meta("content", "deletes") + .overwrite(); + if (compressionCodec != null) { + builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); + } + if (compressionLevel != null) { + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + } + return builder.build(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); } @@ -281,8 +304,13 @@ protected ManifestContent content() { static class V1Writer extends ManifestWriter { private final V1Metadata.IndexedManifestEntry entryWrapper; - V1Writer(PartitionSpec spec, OutputFile file, Long snapshotId, Map config) { - super(spec, file, snapshotId, config); + V1Writer( + PartitionSpec spec, + OutputFile file, + Long snapshotId, + String compressionCodec, + String compressionLevel) { + super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V1Metadata.IndexedManifestEntry(spec.partitionType()); } @@ -293,19 +321,25 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, Map config) { + PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { Schema manifestSchema = V1Metadata.entrySchema(spec.partitionType()); try { - return Avro.write(file) - .schema(manifestSchema) - .named("manifest_entry") - .meta("schema", SchemaParser.toJson(spec.schema())) - .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) - .meta("partition-spec-id", String.valueOf(spec.specId())) - .meta("format-version", "1") - .setAll(config) - .overwrite() - .build(); + Avro.WriteBuilder builder = + Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "1") + .overwrite(); + if (compressionCodec != null) { + builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); + } + if (compressionLevel != null) { + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + } + return builder.build(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 024bed2956c2..af1c8ae5d1e3 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -265,7 +265,8 @@ private ManifestFile copyManifest(ManifestFile manifest) { newManifestPath, snapshotId(), appendedManifestsSummary, - current.properties()); + current.properties().get(TableProperties.AVRO_COMPRESSION), + current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } /** diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 316833df9e2d..a28e254747a2 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -208,7 +208,8 @@ public Snapshot apply() { snapshotId(), parentSnapshotId, sequenceNumber, - ops.current().properties())) { + ops.current().properties().get(TableProperties.AVRO_COMPRESSION), + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))) { // keep track of the manifest lists created manifestLists.add(manifestList.location()); @@ -465,7 +466,8 @@ protected ManifestWriter newManifestWriter(PartitionSpec spec) { spec, newManifestOutput(), snapshotId(), - ops.current().properties()); + ops.current().properties().get(TableProperties.AVRO_COMPRESSION), + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) { @@ -474,7 +476,8 @@ protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) spec, newManifestOutput(), snapshotId(), - ops.current().properties()); + ops.current().properties().get(TableProperties.AVRO_COMPRESSION), + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } protected ManifestReader newManifestReader(ManifestFile manifest) { diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 1aa5a8c46826..7e22efe5e681 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -243,7 +243,12 @@ ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOExceptio ManifestWriter writer = ManifestFiles.write( - formatVersion, table.spec(), outputFile, snapshotId, table.properties()); + formatVersion, + table.spec(), + outputFile, + snapshotId, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); try { for (DataFile file : files) { writer.add(file); @@ -275,12 +280,22 @@ > ManifestFile writeManifest( writer = (ManifestWriter) ManifestFiles.write( - formatVersion, table.spec(), outputFile, snapshotId, table.properties()); + formatVersion, + table.spec(), + outputFile, + snapshotId, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } else { writer = (ManifestWriter) ManifestFiles.writeDeleteManifest( - formatVersion, table.spec(), outputFile, snapshotId, table.properties()); + formatVersion, + table.spec(), + outputFile, + snapshotId, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); } try { for (ManifestEntry entry : entries) { @@ -300,7 +315,12 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = ManifestFiles.writeDeleteManifest( - newFormatVersion, SPEC, manifestFile, snapshotId, table.properties()); + newFormatVersion, + SPEC, + manifestFile, + snapshotId, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); try { for (DeleteFile deleteFile : deleteFiles) { writer.add(deleteFile); @@ -317,7 +337,13 @@ ManifestFile writeManifestWithName(String name, DataFile... files) throws IOExce OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, null, table.properties()); + ManifestFiles.write( + formatVersion, + table.spec(), + outputFile, + null, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); try { for (DataFile file : files) { writer.add(file); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index 01f7970f3eec..d794b338bfbb 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -23,7 +23,6 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.List; -import java.util.Map; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.iceberg.avro.Avro; @@ -34,7 +33,6 @@ import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; @@ -105,7 +103,7 @@ public void testV1WriteDeleteManifest() { "Should fail to write a DELETE manifest to v1", IllegalArgumentException.class, "Cannot store delete manifests in a v1 table", - () -> writeManifestList(TEST_DELETE_MANIFEST, 1, ImmutableMap.of())); + () -> writeManifestList(TEST_DELETE_MANIFEST, 1)); } @Test @@ -156,7 +154,7 @@ public void testV2Write() throws IOException { @Test public void testV1ForwardCompatibility() throws IOException { - InputFile manifestList = writeManifestList(TEST_MANIFEST, 1, ImmutableMap.of()); + InputFile manifestList = writeManifestList(TEST_MANIFEST, 1); GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed @@ -184,7 +182,7 @@ public void testV1ForwardCompatibility() throws IOException { public void testV2ForwardCompatibility() throws IOException { // v2 manifest list files can be read by v1 readers, but the sequence numbers and content will // be ignored. - InputFile manifestList = writeManifestList(TEST_MANIFEST, 2, ImmutableMap.of()); + InputFile manifestList = writeManifestList(TEST_MANIFEST, 2); GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed @@ -297,7 +295,7 @@ public void testManifestsPartitionSummary() throws IOException { partitionFieldSummaries, KEY_METADATA); - InputFile manifestList = writeManifestList(manifest, 2, ImmutableMap.of()); + InputFile manifestList = writeManifestList(manifest, 2); List files = ManifestLists.read(manifestList); ManifestFile returnedManifest = Iterables.getOnlyElement(files); @@ -333,8 +331,7 @@ public void testManifestsPartitionSummary() throws IOException { second.upperBound()); } - private InputFile writeManifestList( - ManifestFile manifest, int formatVersion, Map config) throws IOException { + private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { OutputFile manifestList = new InMemoryOutputFile(); try (FileAppender writer = ManifestLists.write( @@ -343,7 +340,8 @@ private InputFile writeManifestList( SNAPSHOT_ID, SNAPSHOT_ID - 1, formatVersion > 1 ? SEQ_NUM : 0, - config)) { + /* compressionCodec */ null, + /* compressionLevel */ null)) { writer.add(manifest); } return manifestList.toInputFile(); @@ -360,7 +358,7 @@ private GenericData.Record readGeneric(InputFile manifestList, Schema schema) th private ManifestFile writeAndReadManifestList(int formatVersion) throws IOException { List manifests = - ManifestLists.read(writeManifestList(TEST_MANIFEST, formatVersion, ImmutableMap.of())); + ManifestLists.read(writeManifestList(TEST_MANIFEST, formatVersion)); Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 8bbea419de77..79046ac7ee72 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -104,7 +104,13 @@ public void testWriteManifestWithSequenceNumber() throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L, table.properties()); + ManifestFiles.write( + formatVersion, + table.spec(), + outputFile, + 1L, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); writer.add(newFile(10, TestHelpers.Row.of(1)), 1000L); writer.close(); ManifestFile manifest = writer.toManifestFile(); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 0f3576fec800..e3e789618b68 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import org.apache.iceberg.avro.AvroIterable; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; @@ -114,12 +115,21 @@ public void testV1Write() throws IOException { } @Test - public void testV1WriteWithConfig() throws IOException { + public void testV1ManifestFileWriteWithCompression() throws IOException { checkManifestMetadata( - config -> writeManifest(DATA_FILE, 1, config), + compressionCodec -> writeManifest(DATA_FILE, 1, compressionCodec), manifest -> ManifestFiles.read(manifest, FILE_IO)); } + @Test + public void testV1ManifestListWriteWithCompression() throws IOException { + checkManifestListMetadata( + compressionCodec -> { + ManifestFile manifest = writeManifest(DATA_FILE, 1, compressionCodec); + return writeManifestList(manifest, 1, compressionCodec); + }); + } + @Test public void testV1WriteDelete() { AssertHelpers.assertThrows( @@ -147,12 +157,21 @@ public void testV2Write() throws IOException { } @Test - public void testV2WriteWithConfig() throws IOException { + public void testV2ManifestFileWriteWithCompression() throws IOException { checkManifestMetadata( - config -> writeManifest(DATA_FILE, 2, config), + compressionCodec -> writeManifest(DATA_FILE, 2, compressionCodec), manifest -> ManifestFiles.read(manifest, FILE_IO)); } + @Test + public void testV2ManifestListWriteWithCompression() throws IOException { + checkManifestListMetadata( + compressionCodec -> { + ManifestFile manifest = writeManifest(DATA_FILE, 2, compressionCodec); + return writeManifestList(manifest, 2, compressionCodec); + }); + } + @Test public void testV2WriteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeManifest(2), 2); @@ -173,12 +192,21 @@ public void testV2WriteDelete() throws IOException { } @Test - public void testV2WriteDeleteWithConfig() throws IOException { + public void testV2DeleteManifestFileWriteWithCompression() throws IOException { checkManifestMetadata( - config -> writeDeleteManifest(2, config), + compressionCodec -> writeDeleteManifest(2, compressionCodec), manifest -> ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)); } + @Test + public void testV2DeleteManifestListWriteWithCompression() throws IOException { + checkManifestListMetadata( + compressionCodec -> { + ManifestFile manifest = writeDeleteManifest(2, compressionCodec); + return writeManifestList(manifest, 2, compressionCodec); + }); + } + @Test public void testV2WriteDeleteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeDeleteManifest(2), 2); @@ -288,26 +316,44 @@ void checkRewrittenManifest( } > void checkManifestMetadata( - CheckedFunction, ManifestFile> createManifestFunc, + CheckedFunction createManifestFunc, CheckedFunction> manifestReaderFunc) throws IOException { for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { String codec = entry.getKey(); String expectedCodecValue = entry.getValue(); - Map config = ImmutableMap.of(TableProperties.AVRO_COMPRESSION, codec); - ManifestFile manifest = createManifestFunc.apply(config); + ManifestFile manifest = createManifestFunc.apply(codec); - try (ManifestReader manifestReader = manifestReaderFunc.apply(manifest)) { - Map metadata = manifestReader.metadata(); + try (ManifestReader reader = manifestReaderFunc.apply(manifest)) { + Map metadata = reader.metadata(); Assert.assertEquals( - "Codec value must match", expectedCodecValue, metadata.get(AVRO_CODEC_KEY)); + "Manifest file codec value must match", + expectedCodecValue, + metadata.get(AVRO_CODEC_KEY)); + } + } + } + + void checkManifestListMetadata(CheckedFunction createManifestListFunc) + throws IOException { + for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { + String codec = entry.getKey(); + String expectedCodecValue = entry.getValue(); + + InputFile manifestList = createManifestListFunc.apply(codec); + try (AvroIterable reader = ManifestLists.manifestFileIterable(manifestList)) { + Map metadata = reader.getMetadata(); + Assert.assertEquals( + "Manifest list codec value must match", + expectedCodecValue, + metadata.get(AVRO_CODEC_KEY)); } } } private InputFile writeManifestList( - ManifestFile manifest, int formatVersion, Map config) throws IOException { + ManifestFile manifest, int formatVersion, String compressionCodec) throws IOException { OutputFile manifestList = new InMemoryOutputFile(); try (FileAppender writer = ManifestLists.write( @@ -316,7 +362,8 @@ private InputFile writeManifestList( SNAPSHOT_ID, SNAPSHOT_ID - 1, formatVersion > 1 ? SEQUENCE_NUMBER : 0, - config)) { + compressionCodec, + /* compressionLevel */ null)) { writer.add(manifest); } return manifestList.toInputFile(); @@ -325,7 +372,7 @@ private InputFile writeManifestList( private ManifestFile writeAndReadManifestList(ManifestFile manifest, int formatVersion) throws IOException { List manifests = - ManifestLists.read(writeManifestList(manifest, formatVersion, ImmutableMap.of())); + ManifestLists.read(writeManifestList(manifest, formatVersion, /* compressionCodec */ null)); Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } @@ -335,7 +382,13 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, ImmutableMap.of()); + ManifestFiles.write( + formatVersion, + SPEC, + manifestFile, + SNAPSHOT_ID, + /* compressionCodec */ null, + /* compressionLevel */ null); try { writer.existing(readManifest(manifest)); } finally { @@ -345,15 +398,21 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) } private ManifestFile writeManifest(int formatVersion) throws IOException { - return writeManifest(DATA_FILE, formatVersion, ImmutableMap.of()); + return writeManifest(DATA_FILE, formatVersion, /* compressionCodec */ null); } - private ManifestFile writeManifest(DataFile file, int formatVersion, Map config) + private ManifestFile writeManifest(DataFile file, int formatVersion, String compressionCodec) throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, config); + ManifestFiles.write( + formatVersion, + SPEC, + manifestFile, + SNAPSHOT_ID, + compressionCodec, + /* compressionLevel */ null); try { writer.add(file); } finally { @@ -372,15 +431,21 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { - return writeDeleteManifest(formatVersion, ImmutableMap.of()); + return writeDeleteManifest(formatVersion, /* compressionCodec */ null); } - private ManifestFile writeDeleteManifest(int formatVersion, Map config) + private ManifestFile writeDeleteManifest(int formatVersion, String compressionCodec) throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID, config); + ManifestFiles.writeDeleteManifest( + formatVersion, + SPEC, + manifestFile, + SNAPSHOT_ID, + compressionCodec, + /* compressionLevel */ null); try { writer.add(DELETE_FILE); } finally { diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 8e914ab649cf..040e55de8722 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -141,7 +141,13 @@ public void testJsonConversionWithManifestList() throws IOException { try (ManifestListWriter writer = ManifestLists.write( - 1, Files.localOutput(manifestList), id, parentId, 0, ImmutableMap.of())) { + 1, + Files.localOutput(manifestList), + id, + parentId, + 0, + /* compressionCodec */ null, + /* compressionLevel */ null)) { writer.addAll(manifests); } diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index c34cadd28691..e2b3d9d9b97b 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -536,7 +536,11 @@ public void testTransactionRetryAndAppendManifests() throws Exception { OutputFile manifestLocation = Files.localOutput("/tmp/" + UUID.randomUUID().toString() + ".avro"); ManifestWriter writer = - ManifestFiles.write(table.spec(), manifestLocation, table.properties()); + ManifestFiles.write( + table.spec(), + manifestLocation, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); try { writer.add(FILE_D); } finally { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index 4eeae2da95cf..fa77b2a59df0 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.List; -import java.util.Map; import java.util.function.Supplier; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -47,10 +46,12 @@ static ManifestFile writeDataFiles( OutputFile outputFile, PartitionSpec spec, List dataFiles, - Map config) + String compressionCodec, + String compressionLevel) throws IOException { ManifestWriter writer = - ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID, config); + ManifestFiles.write( + FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID, compressionCodec, compressionLevel); try (ManifestWriter closeableWriter = writer) { closeableWriter.addAll(dataFiles); @@ -82,7 +83,8 @@ static DeltaManifests writeCompletedFiles( WriteResult result, Supplier outputFileSupplier, PartitionSpec spec, - Map config) + String compressionCodec, + String compressionLevel) throws IOException { ManifestFile dataManifest = null; @@ -92,7 +94,11 @@ static DeltaManifests writeCompletedFiles( if (result.dataFiles() != null && result.dataFiles().length > 0) { dataManifest = writeDataFiles( - outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles()), config); + outputFileSupplier.get(), + spec, + Lists.newArrayList(result.dataFiles()), + compressionCodec, + compressionLevel); } // Write the completed delete files into a newly created delete manifest file. @@ -101,7 +107,12 @@ static DeltaManifests writeCompletedFiles( ManifestWriter deleteManifestWriter = ManifestFiles.writeDeleteManifest( - FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID, config); + FORMAT_V2, + spec, + deleteManifestFile, + DUMMY_SNAPSHOT_ID, + compressionCodec, + compressionLevel); try (ManifestWriter writer = deleteManifestWriter) { for (DeleteFile deleteFile : result.deleteFiles()) { writer.add(deleteFile); diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index bd261fba3246..e92afd644c6f 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -46,6 +46,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -410,7 +411,8 @@ private byte[] writeToManifest(long checkpointId) throws IOException { result, () -> manifestOutputFileFactory.create(checkpointId), table.spec(), - table.properties()); + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); return SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, deltaManifests); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 05134ef3f8c5..2b863aa7b84d 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -156,7 +156,9 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile, ImmutableMap.of()); + ManifestWriter writer = + ManifestFiles.write( + SPEC, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); try { for (DataFile file : files) { writer.add(file); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index e37cdbe09b08..f875afd95f65 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -38,6 +38,7 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -108,7 +109,8 @@ public void testIO() throws IOException { .build(), () -> factory.create(curCkpId), table.spec(), - table.properties()); + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io()); Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); @@ -149,7 +151,8 @@ public void testUserProvidedManifestLocation() throws IOException { WriteResult.builder().addDataFiles(dataFiles).build(), () -> factory.create(checkpointId), table.spec(), - table.properties()); + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); @@ -190,7 +193,8 @@ public void testVersionedSerializer() throws IOException { .build(), () -> factory.create(checkpointId), table.spec(), - table.properties()); + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); byte[] versionedSerializeData = SimpleVersionedSerialization.writeVersionAndSerialize( @@ -219,7 +223,11 @@ public void testCompatibility() throws IOException { List dataFiles = generateDataFiles(10); ManifestFile manifest = FlinkManifestUtil.writeDataFiles( - factory.create(checkpointId), table.spec(), dataFiles, table.properties()); + factory.create(checkpointId), + table.spec(), + dataFiles, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); byte[] dataV1 = SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index f21503028eaa..e164dbc7b400 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -368,7 +368,8 @@ private static Iterator buildManifest( PartitionSpec spec, String basePath, Iterator> fileTuples, - Map config) { + String compressionCodec, + String compressionLevel) { if (fileTuples.hasNext()) { FileIO io = new HadoopFileIO(conf.get()); TaskContext ctx = TaskContext.get(); @@ -377,7 +378,8 @@ private static Iterator buildManifest( Path location = new Path(basePath, suffix); String outputPath = FileFormat.AVRO.addExtension(location.toString()); OutputFile outputFile = io.newOutputFile(outputPath); - ManifestWriter writer = ManifestFiles.write(spec, outputFile, config); + ManifestWriter writer = + ManifestFiles.write(spec, outputFile, compressionCodec, compressionLevel); try (ManifestWriter writerRef = writer) { fileTuples.forEachRemaining(fileTuple -> writerRef.add(fileTuple._2)); @@ -630,7 +632,8 @@ public static void importSparkPartitions( spec, stagingDir, fileTuple, - targetTable.properties()), + targetTable.properties().get(TableProperties.AVRO_COMPRESSION), + targetTable.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), Encoders.javaSerialization(ManifestFile.class)) .collectAsList(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 6ac4902c6a6e..deba69396f03 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; @@ -228,7 +227,8 @@ private List writeManifestsForUnpartitionedTable( combinedPartitionType, spec, sparkType, - table.properties()), + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), manifestEncoder) .collectAsList(); } @@ -259,7 +259,8 @@ private List writeManifestsForPartitionedTable( combinedPartitionType, spec, sparkType, - table.properties()), + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), manifestEncoder) .collectAsList(); }); @@ -359,7 +360,8 @@ private static ManifestFile writeManifest( Types.StructType combinedPartitionType, PartitionSpec spec, StructType sparkType, - Map config) + String compressionCodec, + String compressionLevel) throws IOException { String manifestName = "optimized-m-" + UUID.randomUUID(); @@ -371,7 +373,8 @@ private static ManifestFile writeManifest( Types.StructType manifestFileType = DataFile.getType(spec.partitionType()); SparkDataFile wrapper = new SparkDataFile(combinedFileType, manifestFileType, sparkType); - ManifestWriter writer = ManifestFiles.write(format, spec, outputFile, null, config); + ManifestWriter writer = + ManifestFiles.write(format, spec, outputFile, null, compressionCodec, compressionLevel); try { for (int index = startIndex; index < endIndex; index++) { @@ -396,7 +399,8 @@ private static MapPartitionsFunction toManifests( Types.StructType combinedPartitionType, PartitionSpec spec, StructType sparkType, - Map config) { + String compressionCodec, + String compressionLevel) { return rows -> { List rowsAsList = Lists.newArrayList(rows); @@ -418,7 +422,8 @@ private static MapPartitionsFunction toManifests( combinedPartitionType, spec, sparkType, - config)); + compressionCodec, + compressionLevel)); } else { int midIndex = rowsAsList.size() / 2; manifests.add( @@ -432,7 +437,8 @@ private static MapPartitionsFunction toManifests( combinedPartitionType, spec, sparkType, - config)); + compressionCodec, + compressionLevel)); manifests.add( writeManifest( rowsAsList, @@ -444,7 +450,8 @@ private static MapPartitionsFunction toManifests( combinedPartitionType, spec, sparkType, - config)); + compressionCodec, + compressionLevel)); } return manifests.iterator(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java index 223204cf5337..9dbefa627518 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -199,7 +199,9 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile, ImmutableMap.of()); + ManifestWriter writer = + ManifestFiles.write( + SPEC, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); try { for (DataFile file : files) { writer.add(file); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index a5c958cb503b..5abceeca46aa 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -196,7 +196,11 @@ public void testSparkCanReadUnknownTransform() throws IOException { OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter manifestWriter = - ManifestFiles.write(FAKE_SPEC, manifestFile, table.properties()); + ManifestFiles.write( + FAKE_SPEC, + manifestFile, + table.properties().get(TableProperties.AVRO_COMPRESSION), + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); try { manifestWriter.add(file); } finally { From 93a38663f191c3c429ef8292ccd42ea866b1999e Mon Sep 17 00:00:00 2001 From: Sumeet Gajjar Date: Thu, 6 Oct 2022 18:35:56 -0700 Subject: [PATCH 5/5] Incorporate second round of feedback 1. remove unecessary edits to use compression codec and compression level from tests 2. move ManifestWriter tests to TestManifestWriter 3. move ManifestListWriter tests to TestManifestListWriter 4. remove unwanted `ManifestLists#write` method --- .../apache/iceberg/BaseRewriteManifests.java | 3 +- .../java/org/apache/iceberg/FastAppend.java | 3 +- .../org/apache/iceberg/ManifestFiles.java | 29 +--- .../apache/iceberg/ManifestListWriter.java | 19 ++- .../org/apache/iceberg/ManifestLists.java | 18 ++- .../org/apache/iceberg/ManifestWriter.java | 22 +-- .../iceberg/MergingSnapshotProducer.java | 3 +- .../org/apache/iceberg/SnapshotProducer.java | 10 +- .../org/apache/iceberg/util/NumberUtil.java | 35 +++++ .../org/apache/iceberg/TableTestBase.java | 90 ++++++++--- .../iceberg/TestManifestListVersions.java | 4 +- .../iceberg/TestManifestListWriter.java | 81 ++++++++++ .../apache/iceberg/TestManifestWriter.java | 46 +++++- .../iceberg/TestManifestWriterVersions.java | 143 +----------------- .../org/apache/iceberg/TestSnapshotJson.java | 9 +- .../org/apache/iceberg/TestTransaction.java | 7 +- .../iceberg/flink/sink/FlinkManifestUtil.java | 17 ++- .../flink/sink/IcebergFilesCommitter.java | 4 +- .../flink/TestManifestFileSerialization.java | 4 +- .../iceberg/flink/sink/TestFlinkManifest.java | 20 +-- .../apache/iceberg/spark/SparkTableUtil.java | 10 +- .../actions/RewriteManifestsSparkAction.java | 11 +- .../TestManifestFileSerialization.java | 4 +- .../source/TestForwardCompatibility.java | 7 +- 24 files changed, 328 insertions(+), 271 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/NumberUtil.java create mode 100644 core/src/test/java/org/apache/iceberg/TestManifestListWriter.java diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 6b481928848f..d5c1bb566f30 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -44,6 +44,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.Tasks; @@ -171,7 +172,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { snapshotId(), summaryBuilder, current.properties().get(TableProperties.AVRO_COMPRESSION), - current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger(current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); } @Override diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 8c54e6a56f98..f6befcb81f49 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -33,6 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.NumberUtil; /** * {@link AppendFiles Append} implementation that adds a new manifest file for the write. @@ -139,7 +140,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { snapshotId(), summaryBuilder, current.properties().get(TableProperties.AVRO_COMPRESSION), - current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger(current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 5972bc2df9f0..298dd34788cf 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -141,24 +141,7 @@ public static ManifestReader read( * @return a manifest writer */ public static ManifestWriter write(PartitionSpec spec, OutputFile outputFile) { - return write(spec, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); - } - - /** - * Create a new {@link ManifestWriter}. - * - *

Manifests created by this writer have all entry snapshot IDs set to null. All entries will - * inherit the snapshot ID that will be assigned to the manifest on commit. - * - * @param spec {@link PartitionSpec} used to produce {@link DataFile} partition tuples - * @param outputFile the destination file location - * @param compressionCodec compression codec for the manifest file - * @param compressionLevel compression level of the compressionCodec - * @return a manifest writer - */ - public static ManifestWriter write( - PartitionSpec spec, OutputFile outputFile, String compressionCodec, String compressionLevel) { - return write(1, spec, outputFile, null, compressionCodec, compressionLevel); + return write(1, spec, outputFile, null); } /** @@ -198,7 +181,7 @@ public static ManifestWriter write( OutputFile outputFile, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { switch (formatVersion) { case 1: return new ManifestWriter.V1Writer( @@ -267,7 +250,7 @@ public static ManifestWriter writeDeleteManifest( OutputFile outputFile, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { switch (formatVersion) { case 1: throw new IllegalArgumentException("Cannot write delete files in a v1 table"); @@ -328,7 +311,7 @@ static ManifestFile copyAppendManifest( long snapshotId, SnapshotSummary.Builder summaryBuilder, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { // use metadata that will add the current snapshot's ID for the rewrite InheritableMetadata inheritableMetadata = InheritableMetadataFactory.forCopy(snapshotId); try (ManifestReader reader = @@ -355,7 +338,7 @@ static ManifestFile copyRewriteManifest( long snapshotId, SnapshotSummary.Builder summaryBuilder, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { // for a rewritten manifest all snapshot ids should be set. use empty metadata to throw an // exception if it is not InheritableMetadata inheritableMetadata = InheritableMetadataFactory.empty(); @@ -384,7 +367,7 @@ private static ManifestFile copyManifestInternal( SnapshotSummary.Builder summaryBuilder, ManifestEntry.Status allowedEntryStatus, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { ManifestWriter writer = write( formatVersion, diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index a54cb66229f1..9bf2e41c7af1 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -32,14 +32,17 @@ abstract class ManifestListWriter implements FileAppender { private final FileAppender writer; private ManifestListWriter( - OutputFile file, Map meta, String compressionCodec, String compressionLevel) { + OutputFile file, + Map meta, + String compressionCodec, + Integer compressionLevel) { this.writer = newAppender(file, meta, compressionCodec, compressionLevel); } protected abstract ManifestFile prepare(ManifestFile manifest); protected abstract FileAppender newAppender( - OutputFile file, Map meta, String compressionCodec, String compressionLevel); + OutputFile file, Map meta, String compressionCodec, Integer compressionLevel); @Override public void add(ManifestFile manifest) { @@ -80,7 +83,7 @@ static class V2Writer extends ManifestListWriter { Long parentSnapshotId, long sequenceNumber, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { super( snapshotFile, ImmutableMap.of( @@ -103,7 +106,7 @@ protected FileAppender newAppender( OutputFile file, Map meta, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { try { Avro.WriteBuilder builder = Avro.write(file) @@ -115,7 +118,7 @@ protected FileAppender newAppender( builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); } if (compressionLevel != null) { - builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel.toString()); } return builder.build(); } catch (IOException e) { @@ -132,7 +135,7 @@ static class V1Writer extends ManifestListWriter { long snapshotId, Long parentSnapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { super( snapshotFile, ImmutableMap.of( @@ -156,7 +159,7 @@ protected FileAppender newAppender( OutputFile file, Map meta, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { try { Avro.WriteBuilder builder = Avro.write(file) @@ -168,7 +171,7 @@ protected FileAppender newAppender( builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); } if (compressionLevel != null) { - builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel.toString()); } return builder.build(); } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index afa78a599d16..4ebb46884e21 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -54,6 +54,22 @@ static AvroIterable manifestFileIterable(InputFile manifestList) { .build(); } + static ManifestListWriter write( + int formatVersion, + OutputFile manifestListFile, + long snapshotId, + Long parentSnapshotId, + long sequenceNumber) { + return write( + formatVersion, + manifestListFile, + snapshotId, + parentSnapshotId, + sequenceNumber, + /* compressionCodec */ null, + /* compressionLevel */ null); + } + static ManifestListWriter write( int formatVersion, OutputFile manifestListFile, @@ -61,7 +77,7 @@ static ManifestListWriter write( Long parentSnapshotId, long sequenceNumber, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { switch (formatVersion) { case 1: Preconditions.checkArgument( diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 71deca5de44c..4dd85365344b 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -57,7 +57,7 @@ private ManifestWriter( OutputFile file, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { this.file = file; this.specId = spec.specId(); this.writer = newAppender(spec, file, compressionCodec, compressionLevel); @@ -69,7 +69,7 @@ private ManifestWriter( protected abstract ManifestEntry prepare(ManifestEntry entry); protected abstract FileAppender> newAppender( - PartitionSpec spec, OutputFile outputFile, String compressionCodec, String compressionLevel); + PartitionSpec spec, OutputFile outputFile, String compressionCodec, Integer compressionLevel); protected ManifestContent content() { return ManifestContent.DATA; @@ -212,7 +212,7 @@ static class V2Writer extends ManifestWriter { OutputFile file, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -224,7 +224,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { + PartitionSpec spec, OutputFile file, String compressionCodec, Integer compressionLevel) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { Avro.WriteBuilder builder = @@ -241,7 +241,7 @@ protected FileAppender> newAppender( builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); } if (compressionLevel != null) { - builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel.toString()); } return builder.build(); } catch (IOException e) { @@ -258,7 +258,7 @@ static class V2DeleteWriter extends ManifestWriter { OutputFile file, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -270,7 +270,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { + PartitionSpec spec, OutputFile file, String compressionCodec, Integer compressionLevel) { Schema manifestSchema = V2Metadata.entrySchema(spec.partitionType()); try { Avro.WriteBuilder builder = @@ -287,7 +287,7 @@ protected FileAppender> newAppender( builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); } if (compressionLevel != null) { - builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel.toString()); } return builder.build(); } catch (IOException e) { @@ -309,7 +309,7 @@ static class V1Writer extends ManifestWriter { OutputFile file, Long snapshotId, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { super(spec, file, snapshotId, compressionCodec, compressionLevel); this.entryWrapper = new V1Metadata.IndexedManifestEntry(spec.partitionType()); } @@ -321,7 +321,7 @@ protected ManifestEntry prepare(ManifestEntry entry) { @Override protected FileAppender> newAppender( - PartitionSpec spec, OutputFile file, String compressionCodec, String compressionLevel) { + PartitionSpec spec, OutputFile file, String compressionCodec, Integer compressionLevel) { Schema manifestSchema = V1Metadata.entrySchema(spec.partitionType()); try { Avro.WriteBuilder builder = @@ -337,7 +337,7 @@ protected FileAppender> newAppender( builder.set(TableProperties.AVRO_COMPRESSION, compressionCodec); } if (compressionLevel != null) { - builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel); + builder.set(TableProperties.AVRO_COMPRESSION_LEVEL, compressionLevel.toString()); } return builder.build(); } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index af1c8ae5d1e3..eebe98b63758 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -52,6 +52,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionSet; import org.apache.iceberg.util.SnapshotUtil; @@ -266,7 +267,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { snapshotId(), appendedManifestsSummary, current.properties().get(TableProperties.AVRO_COMPRESSION), - current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger(current.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); } /** diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index a28e254747a2..d550245b9260 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -51,6 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Exceptions; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -209,7 +210,8 @@ public Snapshot apply() { parentSnapshotId, sequenceNumber, ops.current().properties().get(TableProperties.AVRO_COMPRESSION), - ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))) { + NumberUtil.createInteger( + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)))) { // keep track of the manifest lists created manifestLists.add(manifestList.location()); @@ -467,7 +469,8 @@ protected ManifestWriter newManifestWriter(PartitionSpec spec) { newManifestOutput(), snapshotId(), ops.current().properties().get(TableProperties.AVRO_COMPRESSION), - ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger( + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); } protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) { @@ -477,7 +480,8 @@ protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) newManifestOutput(), snapshotId(), ops.current().properties().get(TableProperties.AVRO_COMPRESSION), - ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger( + ops.current().properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); } protected ManifestReader newManifestReader(ManifestFile manifest) { diff --git a/core/src/main/java/org/apache/iceberg/util/NumberUtil.java b/core/src/main/java/org/apache/iceberg/util/NumberUtil.java new file mode 100644 index 000000000000..9c5edd3ec224 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/NumberUtil.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +public class NumberUtil { + + private NumberUtil() {} + + /** + * @param value the string to convert, can be null + * @return parsed integer, returns null if the string is null + */ + public static Integer createInteger(String value) { + if (value == null) { + return null; + } + return Integer.parseInt(value); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 7e22efe5e681..86b02ed6c483 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -28,11 +28,14 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.LongStream; import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -163,6 +166,19 @@ public class TableTestBase { static final FileIO FILE_IO = new TestTables.LocalFileIO(); + static final Map CODEC_METADATA_MAPPING = + ImmutableMap.builder() + .put("uncompressed", "null") + .put("zstd", "zstandard") + .put("gzip", "deflate") + .build(); + + static final String AVRO_CODEC_KEY = "avro.codec"; + + static final long SNAPSHOT_ID = 987134631982734L; + + private static final long SEQUENCE_NUMBER = 34L; + @Rule public TemporaryFolder temp = new TemporaryFolder(); protected File tableDir = null; @@ -237,7 +253,12 @@ ManifestFile writeManifest(DataFile... files) throws IOException { } ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); + return writeManifest(snapshotId, /* compressionCodec */ null, files); + } + + ManifestFile writeManifest(Long snapshotId, String compressionCodec, DataFile... files) + throws IOException { + File manifestFile = temp.newFile(); Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); @@ -247,8 +268,8 @@ ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOExceptio table.spec(), outputFile, snapshotId, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + compressionCodec, + /* compressionLevel */ null); try { for (DataFile file : files) { writer.add(file); @@ -279,23 +300,12 @@ > ManifestFile writeManifest( if (entries[0].file() instanceof DataFile) { writer = (ManifestWriter) - ManifestFiles.write( - formatVersion, - table.spec(), - outputFile, - snapshotId, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); } else { writer = (ManifestWriter) ManifestFiles.writeDeleteManifest( - formatVersion, - table.spec(), - outputFile, - snapshotId, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + formatVersion, table.spec(), outputFile, snapshotId); } try { for (ManifestEntry entry : entries) { @@ -310,6 +320,13 @@ > ManifestFile writeManifest( ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFile... deleteFiles) throws IOException { + return writeDeleteManifest( + newFormatVersion, snapshotId, /* compressionCodec */ null, deleteFiles); + } + + ManifestFile writeDeleteManifest( + int newFormatVersion, Long snapshotId, String compressionCodec, DeleteFile... deleteFiles) + throws IOException { OutputFile manifestFile = org.apache.iceberg.Files.localOutput( FileFormat.AVRO.addExtension(temp.newFile().toString())); @@ -319,8 +336,8 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi SPEC, manifestFile, snapshotId, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + compressionCodec, + /* compressionLevel */ null); try { for (DeleteFile deleteFile : deleteFiles) { writer.add(deleteFile); @@ -331,19 +348,37 @@ ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFi return writer.toManifestFile(); } + InputFile writeManifestList(String compressionCodec, ManifestFile... manifestFiles) + throws IOException { + File manifestListFile = temp.newFile(); + Assert.assertTrue(manifestListFile.delete()); + OutputFile outputFile = + org.apache.iceberg.Files.localOutput( + FileFormat.AVRO.addExtension(manifestListFile.toString())); + + try (FileAppender writer = + ManifestLists.write( + formatVersion, + outputFile, + SNAPSHOT_ID, + SNAPSHOT_ID - 1, + formatVersion > 1 ? SEQUENCE_NUMBER : 0, + compressionCodec, + /* compressionLevel */ null)) { + for (ManifestFile manifestFile : manifestFiles) { + writer.add(manifestFile); + } + } + return outputFile.toInputFile(); + } + ManifestFile writeManifestWithName(String name, DataFile... files) throws IOException { File manifestFile = temp.newFile(name + ".avro"); Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write( - formatVersion, - table.spec(), - outputFile, - null, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + ManifestFiles.write(formatVersion, table.spec(), outputFile, null); try { for (DataFile file : files) { writer.add(file); @@ -648,4 +683,9 @@ void assertEquals(String context, Object expected, Object actual) { protected interface Action { void invoke(); } + + @FunctionalInterface + interface CheckedFunction { + R apply(T args) throws IOException; + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index d794b338bfbb..fe026a0a5c65 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -339,9 +339,7 @@ private InputFile writeManifestList(ManifestFile manifest, int formatVersion) th manifestList, SNAPSHOT_ID, SNAPSHOT_ID - 1, - formatVersion > 1 ? SEQ_NUM : 0, - /* compressionCodec */ null, - /* compressionLevel */ null)) { + formatVersion > 1 ? SEQ_NUM : 0)) { writer.add(manifest); } return manifestList.toInputFile(); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestListWriter.java new file mode 100644 index 000000000000..3f0e3eb93e71 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestManifestListWriter.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.io.InputFile; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestManifestListWriter extends TableTestBase { + + @Parameterized.Parameters(name = "formatVersion = {0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + public TestManifestListWriter(int formatVersion) { + super(formatVersion); + } + + @Test + public void testWriteManifestListWithCompression() throws IOException { + validateManifestListCompressionCodec( + compressionCodec -> { + ManifestFile manifest = writeManifest(SNAPSHOT_ID, compressionCodec, FILE_A); + return writeManifestList(compressionCodec, manifest); + }); + } + + @Test + public void testWriteDeleteManifestListWithCompression() throws IOException { + Assume.assumeThat(formatVersion, Matchers.is(2)); + validateManifestListCompressionCodec( + compressionCodec -> { + ManifestFile manifest = writeManifest(SNAPSHOT_ID, compressionCodec, FILE_A); + ManifestFile deleteManifest = + writeDeleteManifest(formatVersion, SNAPSHOT_ID, compressionCodec, FILE_A_DELETES); + return writeManifestList(compressionCodec, manifest, deleteManifest); + }); + } + + void validateManifestListCompressionCodec( + CheckedFunction createManifestListFunc) throws IOException { + for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { + String codec = entry.getKey(); + String expectedCodecValue = entry.getValue(); + + InputFile manifestList = createManifestListFunc.apply(codec); + try (AvroIterable reader = ManifestLists.manifestFileIterable(manifestList)) { + Map metadata = reader.getMetadata(); + Assert.assertEquals( + "Manifest list codec value must match", + expectedCodecValue, + metadata.get(AVRO_CODEC_KEY)); + } + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 79046ac7ee72..f698542f0f98 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -21,11 +21,13 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.UUID; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; +import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -104,13 +106,7 @@ public void testWriteManifestWithSequenceNumber() throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = - ManifestFiles.write( - formatVersion, - table.spec(), - outputFile, - 1L, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L); writer.add(newFile(10, TestHelpers.Row.of(1)), 1000L); writer.close(); ManifestFile manifest = writer.toManifestFile(); @@ -124,6 +120,22 @@ public void testWriteManifestWithSequenceNumber() throws IOException { } } + @Test + public void testWriteManifestWithCompression() throws IOException { + validateManifestCompressionCodec( + compressionCodec -> writeManifest(SNAPSHOT_ID, compressionCodec, FILE_A), + manifest -> ManifestFiles.read(manifest, FILE_IO)); + } + + @Test + public void testWriteDeleteManifestWithCompression() throws IOException { + Assume.assumeThat(formatVersion, Matchers.is(2)); + validateManifestCompressionCodec( + compressionCodec -> + writeDeleteManifest(formatVersion, SNAPSHOT_ID, compressionCodec, FILE_A_DELETES), + manifest -> ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)); + } + private DataFile newFile(long recordCount) { return newFile(recordCount, null); } @@ -140,4 +152,24 @@ private DataFile newFile(long recordCount, StructLike partition) { } return builder.build(); } + + > void validateManifestCompressionCodec( + CheckedFunction createManifestFunc, + CheckedFunction> manifestReaderFunc) + throws IOException { + for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { + String codec = entry.getKey(); + String expectedCodecValue = entry.getValue(); + + ManifestFile manifest = createManifestFunc.apply(codec); + + try (ManifestReader reader = manifestReaderFunc.apply(manifest)) { + Map metadata = reader.metadata(); + Assert.assertEquals( + "Manifest file codec value must match", + expectedCodecValue, + metadata.get(AVRO_CODEC_KEY)); + } + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index e3e789618b68..9c606f5e7e60 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -22,8 +22,6 @@ import java.io.IOException; import java.util.List; -import java.util.Map; -import org.apache.iceberg.avro.AvroIterable; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileIO; @@ -42,7 +40,6 @@ public class TestManifestWriterVersions { private static final FileIO FILE_IO = new TestTables.LocalFileIO(); - private static final String AVRO_CODEC_KEY = "avro.codec"; private static final Schema SCHEMA = new Schema( @@ -98,13 +95,6 @@ public class TestManifestWriterVersions { SORT_ORDER_ID, null); - private static final Map CODEC_METADATA_MAPPING = - ImmutableMap.builder() - .put("uncompressed", "null") - .put("zstd", "zstandard") - .put("gzip", "deflate") - .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); @Test @@ -114,22 +104,6 @@ public void testV1Write() throws IOException { checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA); } - @Test - public void testV1ManifestFileWriteWithCompression() throws IOException { - checkManifestMetadata( - compressionCodec -> writeManifest(DATA_FILE, 1, compressionCodec), - manifest -> ManifestFiles.read(manifest, FILE_IO)); - } - - @Test - public void testV1ManifestListWriteWithCompression() throws IOException { - checkManifestListMetadata( - compressionCodec -> { - ManifestFile manifest = writeManifest(DATA_FILE, 1, compressionCodec); - return writeManifestList(manifest, 1, compressionCodec); - }); - } - @Test public void testV1WriteDelete() { AssertHelpers.assertThrows( @@ -156,22 +130,6 @@ public void testV2Write() throws IOException { checkEntry(readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.DATA); } - @Test - public void testV2ManifestFileWriteWithCompression() throws IOException { - checkManifestMetadata( - compressionCodec -> writeManifest(DATA_FILE, 2, compressionCodec), - manifest -> ManifestFiles.read(manifest, FILE_IO)); - } - - @Test - public void testV2ManifestListWriteWithCompression() throws IOException { - checkManifestListMetadata( - compressionCodec -> { - ManifestFile manifest = writeManifest(DATA_FILE, 2, compressionCodec); - return writeManifestList(manifest, 2, compressionCodec); - }); - } - @Test public void testV2WriteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeManifest(2), 2); @@ -191,22 +149,6 @@ public void testV2WriteDelete() throws IOException { readDeleteManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, FileContent.EQUALITY_DELETES); } - @Test - public void testV2DeleteManifestFileWriteWithCompression() throws IOException { - checkManifestMetadata( - compressionCodec -> writeDeleteManifest(2, compressionCodec), - manifest -> ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)); - } - - @Test - public void testV2DeleteManifestListWriteWithCompression() throws IOException { - checkManifestListMetadata( - compressionCodec -> { - ManifestFile manifest = writeDeleteManifest(2, compressionCodec); - return writeManifestList(manifest, 2, compressionCodec); - }); - } - @Test public void testV2WriteDeleteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeDeleteManifest(2), 2); @@ -315,45 +257,7 @@ void checkRewrittenManifest( Assert.assertEquals("Deleted rows count", (Long) 0L, manifest.deletedRowsCount()); } - > void checkManifestMetadata( - CheckedFunction createManifestFunc, - CheckedFunction> manifestReaderFunc) - throws IOException { - for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { - String codec = entry.getKey(); - String expectedCodecValue = entry.getValue(); - - ManifestFile manifest = createManifestFunc.apply(codec); - - try (ManifestReader reader = manifestReaderFunc.apply(manifest)) { - Map metadata = reader.metadata(); - Assert.assertEquals( - "Manifest file codec value must match", - expectedCodecValue, - metadata.get(AVRO_CODEC_KEY)); - } - } - } - - void checkManifestListMetadata(CheckedFunction createManifestListFunc) - throws IOException { - for (Map.Entry entry : CODEC_METADATA_MAPPING.entrySet()) { - String codec = entry.getKey(); - String expectedCodecValue = entry.getValue(); - - InputFile manifestList = createManifestListFunc.apply(codec); - try (AvroIterable reader = ManifestLists.manifestFileIterable(manifestList)) { - Map metadata = reader.getMetadata(); - Assert.assertEquals( - "Manifest list codec value must match", - expectedCodecValue, - metadata.get(AVRO_CODEC_KEY)); - } - } - } - - private InputFile writeManifestList( - ManifestFile manifest, int formatVersion, String compressionCodec) throws IOException { + private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { OutputFile manifestList = new InMemoryOutputFile(); try (FileAppender writer = ManifestLists.write( @@ -361,9 +265,7 @@ private InputFile writeManifestList( manifestList, SNAPSHOT_ID, SNAPSHOT_ID - 1, - formatVersion > 1 ? SEQUENCE_NUMBER : 0, - compressionCodec, - /* compressionLevel */ null)) { + formatVersion > 1 ? SEQUENCE_NUMBER : 0)) { writer.add(manifest); } return manifestList.toInputFile(); @@ -371,8 +273,7 @@ private InputFile writeManifestList( private ManifestFile writeAndReadManifestList(ManifestFile manifest, int formatVersion) throws IOException { - List manifests = - ManifestLists.read(writeManifestList(manifest, formatVersion, /* compressionCodec */ null)); + List manifests = ManifestLists.read(writeManifestList(manifest, formatVersion)); Assert.assertEquals("Should contain one manifest", 1, manifests.size()); return manifests.get(0); } @@ -382,13 +283,7 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write( - formatVersion, - SPEC, - manifestFile, - SNAPSHOT_ID, - /* compressionCodec */ null, - /* compressionLevel */ null); + ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { writer.existing(readManifest(manifest)); } finally { @@ -398,21 +293,14 @@ private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) } private ManifestFile writeManifest(int formatVersion) throws IOException { - return writeManifest(DATA_FILE, formatVersion, /* compressionCodec */ null); + return writeManifest(DATA_FILE, formatVersion); } - private ManifestFile writeManifest(DataFile file, int formatVersion, String compressionCodec) - throws IOException { + private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.write( - formatVersion, - SPEC, - manifestFile, - SNAPSHOT_ID, - compressionCodec, - /* compressionLevel */ null); + ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { writer.add(file); } finally { @@ -431,21 +319,10 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { - return writeDeleteManifest(formatVersion, /* compressionCodec */ null); - } - - private ManifestFile writeDeleteManifest(int formatVersion, String compressionCodec) - throws IOException { OutputFile manifestFile = Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); ManifestWriter writer = - ManifestFiles.writeDeleteManifest( - formatVersion, - SPEC, - manifestFile, - SNAPSHOT_ID, - compressionCodec, - /* compressionLevel */ null); + ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { writer.add(DELETE_FILE); } finally { @@ -462,8 +339,4 @@ private ManifestEntry readDeleteManifest(ManifestFile manifest) thro return entries.get(0); } } - - interface CheckedFunction { - R apply(T args) throws IOException; - } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 040e55de8722..c460336942b4 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -140,14 +140,7 @@ public void testJsonConversionWithManifestList() throws IOException { manifestList.deleteOnExit(); try (ManifestListWriter writer = - ManifestLists.write( - 1, - Files.localOutput(manifestList), - id, - parentId, - 0, - /* compressionCodec */ null, - /* compressionLevel */ null)) { + ManifestLists.write(1, Files.localOutput(manifestList), id, parentId, 0)) { writer.addAll(manifests); } diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index e2b3d9d9b97b..970a2310d738 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -535,12 +535,7 @@ public void testTransactionRetryAndAppendManifests() throws Exception { // create a manifest append OutputFile manifestLocation = Files.localOutput("/tmp/" + UUID.randomUUID().toString() + ".avro"); - ManifestWriter writer = - ManifestFiles.write( - table.spec(), - manifestLocation, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + ManifestWriter writer = ManifestFiles.write(table.spec(), manifestLocation); try { writer.add(FILE_D); } finally { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index fa77b2a59df0..1df113c66434 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -42,12 +42,18 @@ class FlinkManifestUtil { private FlinkManifestUtil() {} + static ManifestFile writeDataFiles( + OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { + return writeDataFiles( + outputFile, spec, dataFiles, /* compressionCodec */ null, /* compressionLevel */ null); + } + static ManifestFile writeDataFiles( OutputFile outputFile, PartitionSpec spec, List dataFiles, String compressionCodec, - String compressionLevel) + Integer compressionLevel) throws IOException { ManifestWriter writer = ManifestFiles.write( @@ -79,12 +85,19 @@ static ManifestOutputFileFactory createOutputFileFactory( attemptNumber); } + static DeltaManifests writeCompletedFiles( + WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) + throws IOException { + return writeCompletedFiles( + result, outputFileSupplier, spec, /* compressionCodec */ null, /* compressionLevel */ null); + } + static DeltaManifests writeCompletedFiles( WriteResult result, Supplier outputFileSupplier, PartitionSpec spec, String compressionCodec, - String compressionLevel) + Integer compressionLevel) throws IOException { ManifestFile dataManifest = null; diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index e92afd644c6f..be274d404b2b 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -56,6 +56,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -412,7 +413,8 @@ private byte[] writeToManifest(long checkpointId) throws IOException { () -> manifestOutputFileFactory.create(checkpointId), table.spec(), table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + NumberUtil.createInteger( + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))); return SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, deltaManifests); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 2b863aa7b84d..6bd94e9ca61c 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -156,9 +156,7 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = - ManifestFiles.write( - SPEC, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); try { for (DataFile file : files) { writer.add(file); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index f875afd95f65..27dc665055cc 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -38,7 +38,6 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -108,9 +107,7 @@ public void testIO() throws IOException { .addDeleteFiles(posDeleteFiles) .build(), () -> factory.create(curCkpId), - table.spec(), - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + table.spec()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io()); Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); @@ -150,9 +147,7 @@ public void testUserProvidedManifestLocation() throws IOException { FlinkManifestUtil.writeCompletedFiles( WriteResult.builder().addDataFiles(dataFiles).build(), () -> factory.create(checkpointId), - table.spec(), - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + table.spec()); Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); @@ -192,9 +187,7 @@ public void testVersionedSerializer() throws IOException { .addDeleteFiles(posDeleteFiles) .build(), () -> factory.create(checkpointId), - table.spec(), - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + table.spec()); byte[] versionedSerializeData = SimpleVersionedSerialization.writeVersionAndSerialize( @@ -222,12 +215,7 @@ public void testCompatibility() throws IOException { List dataFiles = generateDataFiles(10); ManifestFile manifest = - FlinkManifestUtil.writeDataFiles( - factory.create(checkpointId), - table.spec(), - dataFiles, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); byte[] dataV1 = SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index e164dbc7b400..2c16d5a1d2d9 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -59,6 +59,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -369,7 +370,7 @@ private static Iterator buildManifest( String basePath, Iterator> fileTuples, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { if (fileTuples.hasNext()) { FileIO io = new HadoopFileIO(conf.get()); TaskContext ctx = TaskContext.get(); @@ -379,7 +380,7 @@ private static Iterator buildManifest( String outputPath = FileFormat.AVRO.addExtension(location.toString()); OutputFile outputFile = io.newOutputFile(outputPath); ManifestWriter writer = - ManifestFiles.write(spec, outputFile, compressionCodec, compressionLevel); + ManifestFiles.write(1, spec, outputFile, null, compressionCodec, compressionLevel); try (ManifestWriter writerRef = writer) { fileTuples.forEachRemaining(fileTuple -> writerRef.add(fileTuple._2)); @@ -633,7 +634,10 @@ public static void importSparkPartitions( stagingDir, fileTuple, targetTable.properties().get(TableProperties.AVRO_COMPRESSION), - targetTable.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), + NumberUtil.createInteger( + targetTable + .properties() + .get(TableProperties.AVRO_COMPRESSION_LEVEL))), Encoders.javaSerialization(ManifestFile.class)) .collectAsList(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index deba69396f03..cb3067be5cce 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -54,6 +54,7 @@ import org.apache.iceberg.spark.SparkDataFile; import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.NumberUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; @@ -228,7 +229,8 @@ private List writeManifestsForUnpartitionedTable( spec, sparkType, table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), + NumberUtil.createInteger( + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))), manifestEncoder) .collectAsList(); } @@ -260,7 +262,8 @@ private List writeManifestsForPartitionedTable( spec, sparkType, table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)), + NumberUtil.createInteger( + table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL))), manifestEncoder) .collectAsList(); }); @@ -361,7 +364,7 @@ private static ManifestFile writeManifest( PartitionSpec spec, StructType sparkType, String compressionCodec, - String compressionLevel) + Integer compressionLevel) throws IOException { String manifestName = "optimized-m-" + UUID.randomUUID(); @@ -400,7 +403,7 @@ private static MapPartitionsFunction toManifests( PartitionSpec spec, StructType sparkType, String compressionCodec, - String compressionLevel) { + Integer compressionLevel) { return rows -> { List rowsAsList = Lists.newArrayList(rows); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java index 9dbefa627518..a20b2d9f05de 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -199,9 +199,7 @@ private ManifestFile writeManifest(DataFile... files) throws IOException { Assert.assertTrue(manifestFile.delete()); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - ManifestWriter writer = - ManifestFiles.write( - SPEC, outputFile, /* compressionCodec */ null, /* compressionLevel */ null); + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); try { for (DataFile file : files) { writer.add(file); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 5abceeca46aa..42d9ac6a79ba 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -195,12 +195,7 @@ public void testSparkCanReadUnknownTransform() throws IOException { .build(); OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); - ManifestWriter manifestWriter = - ManifestFiles.write( - FAKE_SPEC, - manifestFile, - table.properties().get(TableProperties.AVRO_COMPRESSION), - table.properties().get(TableProperties.AVRO_COMPRESSION_LEVEL)); + ManifestWriter manifestWriter = ManifestFiles.write(FAKE_SPEC, manifestFile); try { manifestWriter.add(file); } finally {