You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "nastra (via GitHub)" <gi...@apache.org> on 2023/02/14 11:28:55 UTC

[GitHub] [iceberg] nastra commented on a diff in pull request #6799: Core: Use avro compression properties from table properties when writing manifests and manifest lists

nastra commented on code in PR #6799:
URL: https://github.com/apache/iceberg/pull/6799#discussion_r1105656897


##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -162,6 +165,19 @@ public class TableTestBase {
 
   static final FileIO FILE_IO = new TestTables.LocalFileIO();
 
+  static final Map<String, String> CODEC_METADATA_MAPPING =
+      ImmutableMap.<String, String>builder()
+          .put("uncompressed", "null")

Review Comment:
   nit: as mentioned on https://github.com/apache/iceberg/pull/5893 I still think this should use the enums from `AvroCodec` as it's not clear that those are actually avro codecs. I'm curious what others think here



##########
core/src/main/java/org/apache/iceberg/ManifestListWriter.java:
##########
@@ -31,14 +31,18 @@
 abstract class ManifestListWriter implements FileAppender<ManifestFile> {
   private final FileAppender<ManifestFile> writer;
 
-  private ManifestListWriter(OutputFile file, Map<String, String> meta) {
-    this.writer = newAppender(file, meta);
+  private ManifestListWriter(
+      OutputFile file,
+      Map<String, String> meta,
+      String compressionCodec,
+      Integer compressionLevel) {
+    this.writer = newAppender(file, meta, compressionCodec, compressionLevel);
   }
 
   protected abstract ManifestFile prepare(ManifestFile manifest);
 
   protected abstract FileAppender<ManifestFile> newAppender(
-      OutputFile file, Map<String, String> meta);
+      OutputFile file, Map<String, String> meta, String compressionCodec, Integer compressionLevel);

Review Comment:
   this is what's causing the RevAPI breaks. I think it would be better to have 
   ```
   protected FileAppender<ManifestFile> newAppender(
         OutputFile file,
         Map<String, String> meta,
         String compressionCodec,
         Integer compressionLevel) {
       return newAppender(file, meta, null, null);
     }
   ```
   which calls the original `newAppender(..)` method with either nulls or default values for compression codec & level



##########
versions.props:
##########
@@ -48,3 +48,4 @@ com.esotericsoftware:kryo = 4.0.2
 org.eclipse.jetty:* = 9.4.43.v20210629
 org.testcontainers:* = 1.17.5
 io.delta:delta-core_* = 2.2.0
+com.github.luben:zstd-jni = 1.5.2-3

Review Comment:
   looks like there's a newer version available ([v1.5.4-1](https://github.com/luben/zstd-jni/releases/tag/v1.5.4-1)), should we switch to that?
   



##########
core/src/test/java/org/apache/iceberg/TestManifestWriter.java:
##########
@@ -234,4 +251,24 @@ private DataFile newFile(long recordCount, StructLike partition) {
     }
     return builder.build();
   }
+
+  <F extends ContentFile<F>> void validateManifestCompressionCodec(
+      CheckedFunction<String, ManifestFile> createManifestFunc,
+      CheckedFunction<ManifestFile, ManifestReader<F>> manifestReaderFunc)
+      throws IOException {
+    for (Map.Entry<String, String> entry : CODEC_METADATA_MAPPING.entrySet()) {
+      String codec = entry.getKey();
+      String expectedCodecValue = entry.getValue();
+
+      ManifestFile manifest = createManifestFunc.apply(codec);
+
+      try (ManifestReader<F> reader = manifestReaderFunc.apply(manifest)) {
+        Map<String, String> metadata = reader.metadata();
+        Assert.assertEquals(

Review Comment:
   can be simplified to `Assertions.assertThat(reader.getMetadata()).containsEntry(AVRO_CODEC_KEY, expectedCodecValue);`



##########
core/src/test/java/org/apache/iceberg/TestManifestListWriter.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.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.assumeTrue("delete files are only written for format version > 1", formatVersion > 1);
+    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<String, InputFile> createManifestListFunc) throws IOException {
+    for (Map.Entry<String, String> entry : CODEC_METADATA_MAPPING.entrySet()) {
+      String codec = entry.getKey();
+      String expectedCodecValue = entry.getValue();
+
+      InputFile manifestList = createManifestListFunc.apply(codec);
+      try (AvroIterable<ManifestFile> reader = ManifestLists.manifestFileIterable(manifestList)) {
+        Map<String, String> metadata = reader.getMetadata();
+        Assert.assertEquals(

Review Comment:
   can be simplified to `Assertions.assertThat(reader.getMetadata()).containsEntry(AVRO_CODEC_KEY, expectedCodecValue);`



##########
.palantir/revapi.yml:
##########
@@ -261,6 +261,16 @@ acceptedBreaks:
         \ T) throws java.io.IOException, com.fasterxml.jackson.core.JacksonException\
         \ @ org.apache.iceberg.rest.RESTSerializers.UpdateRequirementDeserializer"
       justification: "False positive - JacksonException is a subclass of IOException"
+    - code: "java.method.numberOfParametersChanged"
+      old: "method org.apache.iceberg.io.FileAppender<org.apache.iceberg.ManifestEntry<F>>\
+        \ org.apache.iceberg.ManifestWriter<F extends org.apache.iceberg.ContentFile<F\
+        \ extends org.apache.iceberg.ContentFile<F>>>::newAppender(org.apache.iceberg.PartitionSpec,\
+        \ org.apache.iceberg.io.OutputFile)"
+      new: "method org.apache.iceberg.io.FileAppender<org.apache.iceberg.ManifestEntry<F>>\
+        \ org.apache.iceberg.ManifestWriter<F extends org.apache.iceberg.ContentFile<F\
+        \ extends org.apache.iceberg.ContentFile<F>>>::newAppender(org.apache.iceberg.PartitionSpec,\
+        \ org.apache.iceberg.io.OutputFile, java.lang.String, java.lang.Integer)"

Review Comment:
   rather than having to break the API I think it would be better to define a new `newAppender(..)` method that by default calls the original one with null/default params. See also my comment further below.
   
   Once this is done, you shouldn't have any diff on `revapi.yml` anymore.



##########
core/src/test/java/org/apache/iceberg/TableTestBase.java:
##########
@@ -162,6 +165,19 @@ public class TableTestBase {
 
   static final FileIO FILE_IO = new TestTables.LocalFileIO();
 
+  static final Map<String, String> CODEC_METADATA_MAPPING =
+      ImmutableMap.<String, String>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;

Review Comment:
   agreed, there doesn't seem too much value in having this in a constant field



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org