You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/12 08:05:54 UTC

[GitHub] [iceberg] findepi opened a new pull request, #4537: Add Indexes and Stats format reader and writer

findepi opened a new pull request, #4537:
URL: https://github.com/apache/iceberg/pull/4537

   Add reader and writer for the Index and Statistics File Format.
   
   Format documentation: https://github.com/apache/iceberg-docs/pull/69


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r848621582


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(MAGIC)).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  // TODO use LZ4 with frames, see https://trinodb.slack.com/archives/CP1MUNEUX/p1649676596198729

Review Comment:
   LZ4 frame support draft in aircompressor: https://github.com/airlift/aircompressor/pull/142



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870912928


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,

Review Comment:
   `blob`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871096930


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);

Review Comment:
   The blob size. We didn't impose any limits on the size of blobs.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870839652


##########
core/src/main/java/org/apache/iceberg/util/JsonUtil.java:
##########
@@ -162,7 +162,10 @@ public static Set<Integer> getIntegerSetOrNull(String property, JsonNode node) {
     if (!node.has(property) || node.get(property).isNull()) {
       return null;
     }
+    return getIntegerSet(property, node);

Review Comment:
   Nit: we would normally include an empty newline between the control flow block and the following statement.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871385087


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);

Review Comment:
   Good catch. I didn't read the javadoc fully.  Anyway, it seems imperfect. What is the best way to write `ByteBuffer` to an `OutputStream`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872367489


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {

Review Comment:
   yes, this is  footer payload size + a constant, but a user doesn't control "footer payload size" on its own
   
   Footer size can be obtained 
   
   - from manually inspecting the file structure, and `Footer` 's size this is
   - from stats writer, as it exposes this information for convenience
   
   I can surely add some javadoc but i don't know what it should contain besides "@param footerSize the size of the file footer defined in the file format specification". Please advise.



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856666133


##########
build.gradle:
##########
@@ -217,6 +217,7 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'

Review Comment:
   Is there an additional `NOTICE` / license file that needs to be added to the releases with the new dependency?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857589239


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");

Review Comment:
   it cannot return an invalid length



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r884339950


##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Map;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class StatsFiles {

Review Comment:
   To match the other formats, I think this should just be `Puffin`, as in `Puffin.write(outFile)compressFooter().compressBlobs("zstd").build()`



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r887885087


##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Map;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class StatsFiles {

Review Comment:
   good point



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889752411


##########
core/src/main/java/org/apache/iceberg/puffin/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.puffin;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String FIELDS = "fields";
+  private static final String OFFSET = "offset";
+  private static final String LENGTH = "length";
+  private static final String COMPRESSION_CODEC = "compression-codec";
+
+  public static String toJson(FileMetadata fileMetadata) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();

Review Comment:
   Minor: usually, we add a flag to turn on pretty printing, but leave it off by default.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893013912


##########
core/src/main/java/org/apache/iceberg/puffin/Puffin.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.puffin;
+
+import java.util.Map;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class Puffin {
+  private Puffin() {
+  }
+
+  public static WriteBuilder write(OutputFile outputFile) {
+    return new WriteBuilder(outputFile);
+  }
+
+  public static class WriteBuilder {
+    private final OutputFile outputFile;
+    private final Map<String, String> properties = Maps.newLinkedHashMap();
+    private boolean compressFooter = false;
+    private PuffinCompressionCodec defaultBlobCompression = PuffinCompressionCodec.NONE;
+
+    private WriteBuilder(OutputFile outputFile) {
+      this.outputFile = outputFile;
+    }
+
+    public WriteBuilder set(String property, String value) {
+      properties.put(property, value);
+      return this;
+    }
+
+    public WriteBuilder setAll(Map<String, String> props) {
+      this.properties.putAll(props);
+      return this;
+    }
+
+    public WriteBuilder compressFooter() {
+      this.compressFooter = true;
+      return this;
+    }
+
+    public WriteBuilder compressBlobs(PuffinCompressionCodec compression) {
+      this.defaultBlobCompression = compression;
+      return this;
+    }
+
+    public PuffinWriter build() {
+      return new PuffinWriter(outputFile, properties, compressFooter, defaultBlobCompression);

Review Comment:
   Could you set `created-by` here?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893353211


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);
+    while (buffer.remaining() > 0) {
+      channel.write(buffer);
+    }
+  }
+
+  public long footerSize() {
+    return footerSize.orElseThrow(() -> new IllegalStateException("Footer not written yet"));
+  }
+
+  public long fileSize() {
+    return fileSize.orElseThrow(() -> new IllegalStateException("File not written yet"));

Review Comment:
   no, because it's written (but not closed) in finish



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r855857942


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   This is from the format spec https://github.com/apache/iceberg-docs/pull/69:
   
   > `Magic` is four bytes 0x50, 0x46, 0x49, 0x53 (short for: Plain Format for
     Indices and Statistics),
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877824612


##########
core/src/main/java/org/apache/iceberg/stats/Blob.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.stats;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public final class Blob {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final ByteBuffer blobData;
+  private final StatsCompressionCodec requestedCompression;
+
+  public Blob(String type, List<Integer> inputFields, ByteBuffer blobData) {
+    this(type, inputFields, blobData, null);
+  }
+
+  public Blob(
+      String type, List<Integer> inputFields, ByteBuffer blobData,
+      @Nullable StatsCompressionCodec requestedCompression) {

Review Comment:
   It is. When not set, the writer's configured default compression is applied, per https://github.com/apache/iceberg/pull/4537#discussion_r871523692



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876702284


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;

Review Comment:
   added



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897820688


##########
core/src/main/java/org/apache/iceberg/puffin/BlobMetadata.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.puffin;
+
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+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.ImmutableMap;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+  private final Map<String, String> properties;
+
+  public BlobMetadata(
+      String type, List<Integer> inputFields, long offset, long length,
+      @Nullable String compressionCodec, Map<String, String> properties) {
+    Preconditions.checkNotNull(type, "type is null");
+    Preconditions.checkNotNull(inputFields, "inputFields is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    this.type = type;
+    this.inputFields = ImmutableList.copyOf(inputFields);
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+    this.properties = ImmutableMap.copyOf(properties);
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public List<Integer> inputFields() {

Review Comment:
   Iceberg field IDs are integers, so the implementation is chosen to be limited to integers.
   
   We can maybe change `fields            | list of JSON long` in the puffin spec to be list of integers.
   
   



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r898266495


##########
core/src/main/java/org/apache/iceberg/io/IOUtil.java:
##########
@@ -46,6 +50,21 @@ public static void readFully(InputStream stream, byte[] bytes, int offset, int l
     }
   }
 
+  /**
+   * Writes a buffer into a stream, making multiple write calls if necessary.
+   */
+  public static void writeFully(OutputStream outputStream, ByteBuffer buffer) throws IOException {
+    if (!buffer.hasRemaining()) {
+      return;
+    }
+    byte[] chunk = new byte[WRITE_CHUNK_SIZE];

Review Comment:
   Rather than allocating every time this is called, can you create a `ThreadLocal` to share this buffer? Alternatively, you could pass the temporary buffer in.



-- 
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


[GitHub] [iceberg] singhpk234 commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
singhpk234 commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r850117735


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import java.util.Set;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String COLUMNS = "columns";
+  private static final String OFFSET = "offset";
+  private static final String LENGTH = "length";
+  private static final String COMPRESSION_CODEC = "compression_codec";
+
+  public static String toJson(FileMetadata fileMetadata) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(fileMetadata, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to write json for: " + fileMetadata, e);
+    }
+  }
+
+  public static FileMetadata fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static FileMetadata fromJson(JsonNode json) {
+    return fileMetadataFromJson(json);
+  }
+
+  static void toJson(FileMetadata fileMetadata, JsonGenerator generator) throws IOException {
+    generator.writeStartObject();
+
+    generator.writeArrayFieldStart(BLOBS);
+    for (BlobMetadata blobMetadata : fileMetadata.blobs()) {
+      toJson(blobMetadata, generator);
+    }
+    generator.writeEndArray();
+
+    generator.writeObjectFieldStart(PROPERTIES);
+    for (Map.Entry<String, String> entry : fileMetadata.properties().entrySet()) {
+      generator.writeStringField(entry.getKey(), entry.getValue());
+    }
+    generator.writeEndObject();
+
+    generator.writeEndObject();

Review Comment:
   we are calling `writeEndObject` twice, is it intentional ? 



##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Objects.requireNonNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Objects.requireNonNull(name, "name is null");
+    Objects.requireNonNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Objects.requireNonNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);
+    @Nullable String codecName = compression.map(StatsCompressionCodec::getCodecName).orElse(null);
+    blobs.add(new BlobMetadata(type, columnsCovered, fileOffset, length, codecName));
+  }
+
+  public void setCompressFooter(boolean compressFooter) {
+    this.compressFooter = compressFooter;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    outputStream.write(StatsFormat.getMagic());
+    headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    writeHeaderIfNeeded();
+    if (finished) {
+      throw new IllegalStateException("Already finished");
+    }

Review Comment:
   can call `checkNotFinished()`



##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static byte[] decompressFooterPayload(byte[] footer, int offset, int length) {
+    return decompress(StatsCompressionCodec.LZ4, footer, offset, length);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static byte[] decompressBlob(StatsCompressionCodec codec, byte[] data, int dataOffset, int dataLength) {
+    return decompress(codec, data, dataOffset, dataLength);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input,  output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static byte[] decompress(StatsCompressionCodec codec, byte[] input, int inputOffset, int inputLength) {
+    switch (codec) {
+      case LZ4: {
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        throw new UnsupportedOperationException("LZ4 is not supported yet");

Review Comment:
   can add a break here instead and let the exp in L#112 take care of throwing UnsupportedException, something done above.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r848085218


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(MAGIC)).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  // TODO use LZ4 with frames, see https://trinodb.slack.com/archives/CP1MUNEUX/p1649676596198729

Review Comment:
   good catch, see https://github.com/apache/iceberg-docs/pull/69#issuecomment-1096264428



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877920158


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported

Review Comment:
   This method was never called for `NONE` codec.
   to make the code clearer, i refactored this a bit



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r884339991


##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;

Review Comment:
   What about moving this to `org.apache.iceberg.puffin` or just `org.apache.puffin`?



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1103732854

   AC 
   @rdblue  @singhpk234 thanks for your review, let me know what else I can change.
   
   
   


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857588271


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");
+        return ByteBuffer.wrap(decompressed);
+      }
+    }
+
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static byte[] swap(byte[] bytes) {

Review Comment:
   renaming to `swapBytes`



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870900050


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =

Review Comment:
   Style: Iceberg adds empty lines between control flow blocks and the following statements.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870895948


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Preconditions.checkNotNull(codecName, "codecName is null");

Review Comment:
   I agree, we can use the `valueOf` with `toUpperCase`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870901601


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);

Review Comment:
   Is there a maximum size that we could copy out in this case?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870903868


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");

Review Comment:
   It seems strange to me to check for null with an `Optional`. We typically don't use `Optional` and just pass `empty` as `null`. Since you're checking anyway, that seems like a reasonable update.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870911052


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.

Review Comment:
   In Iceberg, we typically omit `IOException` and wrap with `UncheckedIOException` because that has to happen in so many places anyway. I think you can remove the exception.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870912076


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();

Review Comment:
   You might consider lazily opening the stream.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1119570722

   (just rebased, no other changes)


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871534646


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);

Review Comment:
   Using a channel is a good idea. You'll just need a `writeFully` method, I think.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872365194


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);

Review Comment:
   This is unclear to me, what should i do here?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877826788


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(0 <= byteNumber && byteNumber < StatsFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;

Review Comment:
   fixed-length binary part of Footer.
   
   better name?



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856392532


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);

Review Comment:
   FOOTER_RESERVED_OFFSET



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856391982


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);

Review Comment:
   FOOTER_FLAG_OFFSET



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856350865


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported

Review Comment:
   Isn't this required for the footers at the moment?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r855857077


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import java.util.Objects;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Objects.requireNonNull(codecName, "codecName is null");

Review Comment:
   from perf perspective, for enum with two elements, i don't know which is actually faster
   from readability perspective, it's probably subjective which is more readable, so which one you'd prefer?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871075100


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   I don't see a benefit of declaring it as a constant field, and i do feel strongly about sharing mutable state that's not supposed to be changed.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871080569


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {

Review Comment:
   Be your way



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1124741049

   Thanks @rdblue for a detailed review!
   
   Applied comments except for API change https://github.com/apache/iceberg/pull/4537#discussion_r870916478 


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870897570


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   While it is mutable, it should be fine to rely on people not modifying it. This isn't public.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870912928


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,

Review Comment:
   `blob`?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870839253


##########
build.gradle:
##########
@@ -217,6 +217,7 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'

Review Comment:
   This is already included because it is pulled in through ORC.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870895687


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {

Review Comment:
   Style: Iceberg doesn't use `get` in method names. It usually signals that there's either a more specific verb, like `load` or `find` or `fetch`, or is just needless filler. In this case, I'd omit it.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889755852


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {
+  @Test
+  public void testWriteIntegerLittleEndian() throws Exception {
+    testWriteIntegerLittleEndian(0, bytes(0, 0, 0, 0));
+    testWriteIntegerLittleEndian(42, bytes(42, 0, 0, 0));
+    testWriteIntegerLittleEndian(Integer.MAX_VALUE - 5, bytes(0xFa, 0xFF, 0xFF, 0x7F));

Review Comment:
   Nit: lower-case `a` should be upper case.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1131333884

   Thanks @alexjo2144 for your review. AC


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876702727


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = 8;
+  static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  public static final String CREATED_BY_PROPERTY = "created-by";
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x42, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);

Review Comment:
   good idea



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877919097


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = 8;
+  static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  public static final String CREATED_BY_PROPERTY = "created-by";
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x42, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    if (codec == StatsCompressionCodec.NONE) {
+      return input.duplicate();
+    }
+    Compressor compressor = createCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor createCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case NONE:
+        break;

Review Comment:
   Yes, it's never called with `NONE`, so `NoOpCompressor` would be a dead code.
   to make the code clearer, i refactored this a bit



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893020362


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.puffin;
+
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.InMemoryOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.NONE;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.readTestResource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class TestPuffinWriter {
+  @Test
+  public void testEmptyFooterCompressed() {
+    InMemoryOutputFile outputFile = new InMemoryOutputFile();
+
+    PuffinWriter writer = Puffin.write(outputFile)
+        .compressFooter()
+        .build();
+    assertThatThrownBy(writer::footerSize)
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessage("Footer not written yet");
+    assertThatThrownBy(writer::finish)

Review Comment:
   Why allow an explicit call to `finish`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893362128


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   per `Channels.newChannel` docs:
   
   >  Closing the channel will in turn cause the stream to be closed.
   
   this is not what we want here. 
   
   is there a better way to write a `ByteBuffer` to an `OutputStream`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893348051


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));

Review Comment:
   fixed!



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893009483


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));

Review Comment:
   Style: it's hard to read whether `.getBytes` is called on the result of `toJson` or `wrap` because of this line wrapping. We generally prefer making it clear by wrapping earlier:
   
   ```java
       ByteBuffer footerJson = ByteBuffer.wrap(
           FileMetadataParser.toJson(fileMetadata, false).getBytes(StandardCharsets.UTF_8));
   ```



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893347177


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();

Review Comment:
   I agree that there is no point in having files with no stats. However, this is a container file format, it would be odd not to be able to create a file that happens to have no blobs.
   
   note that it's here only because i don't write header in the constructor. maybe that would be ok to do.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1098007657

   @rdblue i applied most of the comments (no more `jackson-datatype-jdk8`, json-related annotations, `get` in accessor names). I also switched over from base16-encoded test "fixtures" to test resources.
   
   I switched writing to use `ByteBuffer`, this removes removed data copying upon compression.
   This is as a fixup for now. Please let me know if you want me to use `ByteBuffer` in the reader/writer APIs as well?
   It seems `byte[]` works there well (eg because uncompressed size is known, so no need to re-allocation), so please confirm before I apply further changes.


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856532202


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;

Review Comment:
   That's an empty block comment. That's a what we use in Trino to make formatter happy and to allow addition of new enum options without changing lines of existing options (clean git annotate output).



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856399238


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs

Review Comment:
   Should add the full java doc for this as well not just exception stubs, also nit: we don't do vertical alignment usually



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897821828


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);

Review Comment:
   i think both are fine



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer compress(Compressor compressor, ByteBuffer input) {
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  static ByteBuffer decompress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");
+        return ByteBuffer.wrap(decompressed);
+      }
+    }
+
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);

Review Comment:
   i think both are fine



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r898262307


##########
core/src/main/java/org/apache/iceberg/puffin/BlobMetadata.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.puffin;
+
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+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.ImmutableMap;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+  private final Map<String, String> properties;
+
+  public BlobMetadata(
+      String type, List<Integer> inputFields, long offset, long length,
+      @Nullable String compressionCodec, Map<String, String> properties) {
+    Preconditions.checkNotNull(type, "type is null");
+    Preconditions.checkNotNull(inputFields, "inputFields is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    this.type = type;
+    this.inputFields = ImmutableList.copyOf(inputFields);
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+    this.properties = ImmutableMap.copyOf(properties);
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public List<Integer> inputFields() {

Review Comment:
   Yes, we should update the spec to match the type used by the table spec for field IDs.



-- 
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


[GitHub] [iceberg] rdblue merged pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4537:
URL: https://github.com/apache/iceberg/pull/4537


-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1130222459

   AC


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871520555


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);

Review Comment:
   When you open a stream, you check whether it implements `RangeReadable`. If not, then get the file length and subtract. But there is no need for the extra HEAD request for S3 to get the file length if `readTail` is available.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r884340041


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {

Review Comment:
   `PuffinFormat`? Does this need to be public?



##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+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.io.ByteStreams;
+import org.apache.iceberg.stats.StatsFormat.Flag;
+import org.apache.iceberg.util.Pair;
+
+public class StatsReader implements Closeable {

Review Comment:
   `PuffinReader`? and `PuffinWriter`?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877222291


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+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.io.ByteStreams;
+import org.apache.iceberg.stats.StatsFormat.Flag;
+import org.apache.iceberg.util.Pair;
+
+public class StatsReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = StatsFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public StatsReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;
+    this.input = inputFile.newStream();
+    if (footerSize != null) {
+      Preconditions.checkArgument(0 < footerSize && footerSize <= this.fileSize - MAGIC.length,
+          "Invalid footer size: %s", footerSize);
+      this.knownFooterSize = Math.toIntExact(footerSize);
+    }
+  }
+
+  public FileMetadata fileMetadata() throws IOException {
+    if (knownFileMetadata == null) {
+      int footerSize = footerSize();
+      byte[] footer = readInput(fileSize - footerSize, footerSize);
+
+      checkMagic(footer, StatsFormat.FOOTER_MAGIC_OFFSET);
+      int footerStructOffset = footerSize - StatsFormat.FOOTER_STRUCT_LENGTH;
+      checkMagic(footer, footerStructOffset + StatsFormat.FOOTER_STRUCT_MAGIC_OFFSET);
+
+      StatsCompressionCodec footerCompression = StatsCompressionCodec.NONE;
+      for (Flag flag : decodeFlags(footer, footerStructOffset)) {
+        switch (flag) {
+          case FOOTER_PAYLOAD_COMPRESSED:
+            footerCompression = StatsFormat.FOOTER_COMPRESSION_CODEC;
+            break;
+          default:
+            throw new IllegalStateException("Unsupported flag: " + flag);
+        }
+      }
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(
+          footer,
+          footerStructOffset + StatsFormat.FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET);
+      Preconditions.checkState(footerSize == footerPayloadSize + StatsFormat.FOOTER_STRUCT_LENGTH + 4 /* magic */,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      ByteBuffer footerJson = StatsFormat.decompress(footerCompression, footerPayload);
+      this.knownFileMetadata = parseFileMetadata(footerJson);
+    }
+    return knownFileMetadata;
+  }
+
+  private Set<Flag> decodeFlags(byte[] footer, int footerStructOffset) {
+    EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+    for (int byteNumber = 0; byteNumber < StatsFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int flagByte =
+          Byte.toUnsignedInt(footer[footerStructOffset + StatsFormat.FOOTER_STRUCT_FLAGS_OFFSET + byteNumber]);
+      int bitNumber = 0;
+      while (flagByte != 0) {
+        if ((flagByte & 0x1) != 0) {
+          Flag flag = Flag.fromBit(byteNumber, bitNumber);
+          Preconditions.checkState(flag != null, "Unknown flag byte %s and bit %s set", byteNumber, bitNumber);
+          flags.add(flag);
+        }
+        flagByte = flagByte >> 1;
+        bitNumber++;
+      }
+    }
+    return flags;
+  }
+
+  public Iterable<Pair<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) {
+    if (blobs.isEmpty()) {
+      return ImmutableList.of();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return () -> blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer rawData = ByteBuffer.wrap(bytes);
+            StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+            ByteBuffer data = StatsFormat.decompress(codec, rawData);
+            return Pair.of(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        })
+        .iterator();
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    byte[] read = Arrays.copyOfRange(data, offset, offset + MAGIC.length);
+    if (!Arrays.equals(read, MAGIC)) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s: %s, but got %s",
+          offset, Arrays.toString(MAGIC), Arrays.toString(read)));
+    }
+  }
+
+  private int footerSize() throws IOException {
+    if (knownFooterSize == null) {
+      if (fileSize < StatsFormat.FOOTER_STRUCT_LENGTH) {
+        throw new IllegalStateException(String.format(
+            "Invalid file: file length %s is less tha minimal length of the footer tail %s",
+            fileSize,
+            StatsFormat.FOOTER_STRUCT_LENGTH));
+      }
+      byte[] footerStruct = readInput(fileSize - StatsFormat.FOOTER_STRUCT_LENGTH, StatsFormat.FOOTER_STRUCT_LENGTH);
+      checkMagic(footerStruct, StatsFormat.FOOTER_STRUCT_MAGIC_OFFSET);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(
+          footerStruct, StatsFormat.FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET);
+      knownFooterSize = 4 /* magic */ + footerPayloadSize + StatsFormat.FOOTER_STRUCT_LENGTH;

Review Comment:
   Rather than "magic" I think it is more clear to say "Footer start magic". I'd also make this a constant that comes from the magic bytes so we don't have to find hard-coded instances.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877215366


##########
bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java:
##########
@@ -73,6 +74,7 @@ public class GuavaClasses {
     ImmutableList.class.getName();
     ImmutableMap.class.getName();
     ImmutableSet.class.getName();
+    ImmutableTable.class.getName();

Review Comment:
   We generally avoid adding new Guava classes because we want to decrease dependence on Guava. Is this needed and can we use something else?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877828034


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+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.io.ByteStreams;
+import org.apache.iceberg.stats.StatsFormat.Flag;
+import org.apache.iceberg.util.Pair;
+
+public class StatsReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = StatsFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public StatsReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;
+    this.input = inputFile.newStream();
+    if (footerSize != null) {
+      Preconditions.checkArgument(0 < footerSize && footerSize <= this.fileSize - MAGIC.length,
+          "Invalid footer size: %s", footerSize);
+      this.knownFooterSize = Math.toIntExact(footerSize);
+    }
+  }
+
+  public FileMetadata fileMetadata() throws IOException {
+    if (knownFileMetadata == null) {
+      int footerSize = footerSize();
+      byte[] footer = readInput(fileSize - footerSize, footerSize);
+
+      checkMagic(footer, StatsFormat.FOOTER_MAGIC_OFFSET);
+      int footerStructOffset = footerSize - StatsFormat.FOOTER_STRUCT_LENGTH;
+      checkMagic(footer, footerStructOffset + StatsFormat.FOOTER_STRUCT_MAGIC_OFFSET);
+
+      StatsCompressionCodec footerCompression = StatsCompressionCodec.NONE;
+      for (Flag flag : decodeFlags(footer, footerStructOffset)) {
+        switch (flag) {
+          case FOOTER_PAYLOAD_COMPRESSED:
+            footerCompression = StatsFormat.FOOTER_COMPRESSION_CODEC;
+            break;
+          default:
+            throw new IllegalStateException("Unsupported flag: " + flag);
+        }
+      }
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(
+          footer,
+          footerStructOffset + StatsFormat.FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET);
+      Preconditions.checkState(footerSize == footerPayloadSize + StatsFormat.FOOTER_STRUCT_LENGTH + 4 /* magic */,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      ByteBuffer footerJson = StatsFormat.decompress(footerCompression, footerPayload);
+      this.knownFileMetadata = parseFileMetadata(footerJson);
+    }
+    return knownFileMetadata;
+  }
+
+  private Set<Flag> decodeFlags(byte[] footer, int footerStructOffset) {
+    EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+    for (int byteNumber = 0; byteNumber < StatsFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int flagByte =
+          Byte.toUnsignedInt(footer[footerStructOffset + StatsFormat.FOOTER_STRUCT_FLAGS_OFFSET + byteNumber]);
+      int bitNumber = 0;
+      while (flagByte != 0) {
+        if ((flagByte & 0x1) != 0) {
+          Flag flag = Flag.fromBit(byteNumber, bitNumber);
+          Preconditions.checkState(flag != null, "Unknown flag byte %s and bit %s set", byteNumber, bitNumber);
+          flags.add(flag);
+        }
+        flagByte = flagByte >> 1;
+        bitNumber++;
+      }
+    }
+    return flags;
+  }
+
+  public Iterable<Pair<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) {
+    if (blobs.isEmpty()) {
+      return ImmutableList.of();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return () -> blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer rawData = ByteBuffer.wrap(bytes);
+            StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+            ByteBuffer data = StatsFormat.decompress(codec, rawData);
+            return Pair.of(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        })
+        .iterator();
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    byte[] read = Arrays.copyOfRange(data, offset, offset + MAGIC.length);
+    if (!Arrays.equals(read, MAGIC)) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s: %s, but got %s",
+          offset, Arrays.toString(MAGIC), Arrays.toString(read)));
+    }
+  }
+
+  private int footerSize() throws IOException {
+    if (knownFooterSize == null) {
+      if (fileSize < StatsFormat.FOOTER_STRUCT_LENGTH) {

Review Comment:
   done



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r882153170


##########
bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java:
##########
@@ -73,6 +74,7 @@ public class GuavaClasses {
     ImmutableList.class.getName();
     ImmutableMap.class.getName();
     ImmutableSet.class.getName();
+    ImmutableTable.class.getName();

Review Comment:
   I'll take a look at it, this sounds reasonable.



-- 
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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889642947


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;

Review Comment:
   Seeing as how many of these files are likely going to be small, we might want to consider using `RangeReadable` interface to try to opportunistically read in the whole file with a `readTail` to reduce the number of round-trips to the storage layer.  @findepi this is similar to Trino's `readTail`.
   
   We could just add a default implementation for other FileIO's that mimics the behavior with other calls.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889734195


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;

Review Comment:
   @danielcweeks thanks for the suggestion. please see my earlier response https://github.com/apache/iceberg/pull/4537#discussion_r872407417 .  does it answer the concerns here?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889862322


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> BY_BYTE_AND_BIT = Stream.of(values())

Review Comment:
   awesome idea!



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1148310218

   Now that the format spec https://github.com/apache/iceberg/pull/4944 is merged (thank you @rdblue! )
   i think this one is ready to merge too as it implements the format.


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893008482


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();

Review Comment:
   Minor: Why write the header? Isn't the file empty if there is no header by this point? We would normally avoid creating the file in that case.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893012607


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   I think this needs to be wrapped in try-with-resources because `Channel` is `Closeable`.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1151033920

   Applied comments. @rdblue can you please give it another look?


-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r855566545


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   Question: Does this particular magic bytes correlate with anything in particular or did we just decide on it?



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857083456


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import java.util.Objects;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Objects.requireNonNull(codecName, "codecName is null");

Review Comment:
   For 2 elements actually it’s probably overkill in my opinion. Good point.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857586914


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Preconditions.checkNotNull(codecName, "codecName is null");

Review Comment:
   https://github.com/apache/iceberg-docs/pull/69 mandates the name to be lowercase.
   also, since it's not internal, i wanted to have conversion to and from  file-level name explicit.
   
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857588834


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;

Review Comment:
   good idea



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1122354604

   Superseded by https://github.com/apache/iceberg/pull/4741 


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870884777


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {
+    this.type = Preconditions.checkNotNull(type, "type is null");
+    this.columns = ImmutableSet.copyOf(Preconditions.checkNotNull(columns, "columns is null"));

Review Comment:
   I think columns should probably be a list rather than a set because order matters when computing inputs to blobs. For example, if you have a blob for NDV across two columns you'd need to know how to construct the value tuple that is used as input to the theta sketch.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871076817


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Preconditions.checkNotNull(codecName, "codecName is null");

Review Comment:
   This will be considered a "reference implementation" of the spec. If we're flexible here, we should be flexible in the spec. Otherwise we would add support for situation that's invalid and should never occur, as if mandating it existence.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870905913


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);

Review Comment:
   I think converting the magic number to an int makes this code harder to reason about. Magic is not checked in a tight loop, so I don't think that the optimization of comparing the bytes all at once is worth it. I'd prefer to just check each byte in a loop and avoid needing to validate that this is correct:
   
   ```java
   static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
   ```



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870913379


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {

Review Comment:
   I agree with Kyle. I think this would be cleaner if you added `NONE` and always passed the codec as the enum.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872417968


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {

Review Comment:
   added



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r849437209


##########
build.gradle:
##########
@@ -217,9 +217,11 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'
     implementation 'org.apache.httpcomponents.client5:httpclient5'
     implementation "com.fasterxml.jackson.core:jackson-databind"
     implementation "com.fasterxml.jackson.core:jackson-core"
+    implementation "com.fasterxml.jackson.datatype:jackson-datatype-jdk8"

Review Comment:
   `jackson-datatype-jdk8` no longer used here



-- 
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


[GitHub] [iceberg] rdblue commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1098274889

   @findepi, I would prefer to use `ByteBuffer` everywhere. If the backing buffer was easy to allocate and work with, that's great. But you're not forced to reallocate if it is not.


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897368436


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   If you'd like to avoid using Channel, that's fine with me. I just want to make sure we're not leaking resources and this uses a factory method so we aren't guaranteed to have a channel that doesn't require being closed.
   
   Why don't you write a method to implement `writeFully` from a byte buffer in `IOUtil` and we can just call that instead?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897806547


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   > I just want to make sure we're not leaking resources and this uses a factory method so we aren't guaranteed to have a channel that doesn't require being closed.
   
   right
   
   or, an implementation which is buffering, and we end up not writing (all) the data
   
   > Why don't you write a method to implement writeFully from a byte buffer in IOUtil and we can just call that instead?
   
   will do!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897884116


##########
core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.puffin;
+
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestFileMetadataParser {

Review Comment:
   > I think i am following the established JSON parsing patterns, so it probably is a pre-existing problem
   
   https://github.com/apache/iceberg/pull/5048



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897821399


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");

Review Comment:
   This is enum initialization code. It can fail only if an engineer changes the enum definition itself and then they will be able to quickly realize what the problem is. 



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897834674


##########
core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.puffin;
+
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestFileMetadataParser {

Review Comment:
   good idea.
   
   this let me realize there is a problem with trailing contentn
   
   i.e. this should fail:
   ```java
   FileMetadataParser.fromJson("{\"blobs\": []} {}")
   ```
   
   but it does not.
   
   I think i am following the established JSON parsing patterns, so it probably is a pre-existing problem
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872407417


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);

Review Comment:
   I don't do additional HEAD request, because file size and footer size are known. (https://github.com/apache/iceberg-docs/pull/77)
   i see that with `RangeReadable.readTail` i could remove the file size and footer size from the table snapshot, which would be a good thing. To do this, i would need to assume some maximum for a "reasonable" footer size and do a `readTail` read with that size (i saw a suggestion to read 16kB, that would work).
   It wouldn't change number of requests sent (since currently  file size and footer size are known), but it would slightly reduce amount of data kept in a table.
   
   I assume we don't want to make such an improvement for some file systems while actually degrading performance on other file systems, right? What file systems is RangeReadable available on? 
   



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870885814


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadata.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+
+public class FileMetadata {
+  private final List<BlobMetadata> blobs;
+  private final Map<String, String> properties;
+
+  public FileMetadata(List<BlobMetadata> blobs, Map<String, String> properties) {
+    this.blobs = ImmutableList.copyOf(Preconditions.checkNotNull(blobs, "blobs is null"));

Review Comment:
   Style: We generally prefer to keep `Preconditions` first in the method, and not embed them in other statements.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871094304


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |

Review Comment:
   That's a different conversion, even if syntactically looks similar.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871095205


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {

Review Comment:
   no, that's part of StatsWriter API



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871379230


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);

Review Comment:
   Good point!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871382238


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();

Review Comment:
   What does "might consider" mean here? 



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870906679


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {

Review Comment:
   What do you think about using the same optimization as in Parquet and reading the last 16kb of the file, then reading more if the footer size is larger than that? I think we may as well start with that.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870908805


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);

Review Comment:
   This breaks forward compatibility if we ever decide to use the reserved bytes, which defeats the purpose. If we have to increment the format version to use them, we should just remove them.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870916478


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {

Review Comment:
   I think this should use the builder pattern that we use for other file formats. That way all of the metadata and config are set ahead of time and the writer just needs to append each payload. You can see `Parquet` for examples, but it would look like this:
   
   ```java
     try (FileAppender<Blob> writer = StatsFiles.write(outputFile)
         .set("key", "value")
         .compressFooter()
         .build()) {
       writer.addAll(blobs);
     }
   ```



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871522882


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();

Review Comment:
   Are there situations where we may create a reader but not use it? If so, then it would make sense for it to be lazy. If not, then this is fine.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871139213


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);

Review Comment:
   > This breaks forward compatibility
   
   Yes it does, intentionally. A reader cannot read a file version it doesn't support.
   This is same as with unrecognized flags. If there is a flag set a reader doesn't know meaning of, it cannot proceed.
   
   > If we have to increment the format version to use them, we should just remove them.
   
   It exists as reserved so that it's easier to add use of it.
   Footer has a fixed-layout portion which is hard to evolve, i.e. hard to maintain support for multiple different layouts in same code base. This is why this field is added from the start.
   
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857593965


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs

Review Comment:
   > we don't do vertical alignment usually
   
   That's what the formatter (imported from the definition linked in the project) produced for me, not my own art.



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856391410


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);

Review Comment:
   Maybe make 4 here a final constant FOOTER_MAGIC_OFFSET



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856378494


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");

Review Comment:
   Is this possible? Just wondering if ZstdDecompressor.getDecompressedSize() can return an invalid length. If so should we be throwing a "Corrupted file error or something like that?"



-- 
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


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889641104


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {

Review Comment:
   Feels like this would be better as a static member than a method since we're creating a new array each time it's called.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1144955855

   Thank you for the awesome review.
   
   Applied comments. looking forward to more review comments.


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r887891108


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+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.io.ByteStreams;
+import org.apache.iceberg.stats.StatsFormat.Flag;
+import org.apache.iceberg.util.Pair;
+
+public class StatsReader implements Closeable {

Review Comment:
   applied!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r887890068


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {

Review Comment:
   fixed



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893012794


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);
+    while (buffer.remaining() > 0) {
+      channel.write(buffer);
+    }
+  }
+
+  public long footerSize() {
+    return footerSize.orElseThrow(() -> new IllegalStateException("Footer not written yet"));
+  }
+
+  public long fileSize() {
+    return fileSize.orElseThrow(() -> new IllegalStateException("File not written yet"));

Review Comment:
   `written` -> `closed`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876698655


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;

Review Comment:
   This needs to be forward-compatible and extensible.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r884340183


##########
core/src/test/java/org/apache/iceberg/stats/StatsFormatTestUtil.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.stats;
+
+import java.io.InputStream;
+import java.net.URL;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public final class StatsFormatTestUtil {

Review Comment:
   I think most of what needs to be done is just renaming classes and variables that use "StatsFormat" or "STATS_FORMAT" since it isn't just stats.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897818638


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   fixed



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897823312


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer compress(Compressor compressor, ByteBuffer input) {
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  static ByteBuffer decompress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;

Review Comment:
   good idea, moving



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897824014


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    checkNotFinished();
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {

Review Comment:
   thanks for the suggestion!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897835869


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {
+  @Test
+  public void testWriteIntegerLittleEndian() throws Exception {
+    testWriteIntegerLittleEndian(0, bytes(0, 0, 0, 0));
+    testWriteIntegerLittleEndian(42, bytes(42, 0, 0, 0));
+    testWriteIntegerLittleEndian(Integer.MAX_VALUE - 5, bytes(0xFA, 0xFF, 0xFF, 0x7F));
+    testWriteIntegerLittleEndian(-7, bytes(0xF9, 0xFF, 0xFF, 0xFF));
+  }
+
+  private void testWriteIntegerLittleEndian(int value, byte[] expected) throws Exception {
+    // Sanity check: validate the expectation
+    ByteBuffer buffer = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN);
+    buffer.putInt(value);
+    buffer.flip();
+    byte[] written = new byte[4];
+    buffer.get(written);
+    Preconditions.checkState(Arrays.equals(written, expected), "Invalid expected value");

Review Comment:
   Throwing an AssertionError here could be interpreted as a bug in the implementation under test.
   i used assertions for the code under test.
   i use checkState to verify the test is configured properly. 
   
   > Because this will actually show expected/actual values if the check fails, whereas the precondition doesn't show that
   
   agreed, there is a cost to it



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897837490


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.puffin;
+
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.InMemoryOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.NONE;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.readTestResource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class TestPuffinWriter {
+  @Test
+  public void testEmptyFooterCompressed() {
+    InMemoryOutputFile outputFile = new InMemoryOutputFile();
+
+    PuffinWriter writer = Puffin.write(outputFile)
+        .compressFooter()
+        .build();
+    assertThatThrownBy(writer::footerSize)
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessage("Footer not written yet");
+    assertThatThrownBy(writer::finish)
+        .isInstanceOf(UnsupportedOperationException.class)
+        .hasMessage("Unsupported codec: LZ4");
+    assertThatThrownBy(writer::close)
+        .isInstanceOf(UnsupportedOperationException.class)
+        .hasMessage("Unsupported codec: LZ4");

Review Comment:
   > and it's not implemented
   
   coming -- https://github.com/airlift/aircompressor/pull/142
   
   > Should the default compressor maybe be configured to ZSTD?
   
   I don't believe it makes sense to compress the footer with ZSTD.
   
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r901341235


##########
core/src/main/java/org/apache/iceberg/puffin/BlobMetadata.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.puffin;
+
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+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.ImmutableMap;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+  private final Map<String, String> properties;
+
+  public BlobMetadata(
+      String type, List<Integer> inputFields, long offset, long length,
+      @Nullable String compressionCodec, Map<String, String> properties) {
+    Preconditions.checkNotNull(type, "type is null");
+    Preconditions.checkNotNull(inputFields, "inputFields is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    this.type = type;
+    this.inputFields = ImmutableList.copyOf(inputFields);
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+    this.properties = ImmutableMap.copyOf(properties);
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public List<Integer> inputFields() {

Review Comment:
   https://github.com/apache/iceberg/pull/5087 



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847671471


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;
+  private final long fileOffset;
+  private final long length;
+  private final Optional<String> compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,
+      @JsonProperty("columns") List<Integer> columns,
+      @JsonProperty("offset") long fileOffset,
+      @JsonProperty("length") long length,
+      @JsonProperty("compression_codec") Optional<String> compressionCodec) {
+    this.type = Objects.requireNonNull(type, "type is null");
+    this.columns = ImmutableList.copyOf(Objects.requireNonNull(columns, "columns is null"));
+    this.fileOffset = fileOffset;
+    this.length = length;
+    this.compressionCodec = Objects.requireNonNull(compressionCodec, "compressionCodec is null");
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public List<Integer> getColumns() {
+    return columns;
+  }
+
+  public long getFileOffset() {
+    return fileOffset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public Optional<String> getCompressionCodec() {

Review Comment:
   Why optional rather than returning `null`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872422076


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);

Review Comment:
   added constants for these



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872422996


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs

Review Comment:
   fixed by not throwing IOException anymore (per suggestion elsewhere)



##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {
+    if (!knownFooterSize.isPresent()) {
+      int footerTailSize = Math.toIntExact(Math.min(fileLength, 20));
+      byte[] footerTail = readInput(fileLength - footerTailSize, footerTailSize);
+
+      checkMagic(footerTail, footerTailSize - 4);

Review Comment:
   fixed



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856668213


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(

Review Comment:
   Is the `ZstdDecompressor` something that we can allocate once and share somehow?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876703463


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = 8;
+  static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  public static final String CREATED_BY_PROPERTY = "created-by";
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x42, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    if (codec == StatsCompressionCodec.NONE) {
+      return input.duplicate();
+    }
+    Compressor compressor = createCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor createCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case NONE:
+        break;

Review Comment:
   i don't get it



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877203398


##########
core/src/main/java/org/apache/iceberg/stats/Blob.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.stats;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public final class Blob {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final ByteBuffer blobData;
+  private final StatsCompressionCodec requestedCompression;
+
+  public Blob(String type, List<Integer> inputFields, ByteBuffer blobData) {
+    this(type, inputFields, blobData, null);
+  }
+
+  public Blob(
+      String type, List<Integer> inputFields, ByteBuffer blobData,
+      @Nullable StatsCompressionCodec requestedCompression) {

Review Comment:
   Is this still `@Nullable` now that there is a `NONE` symbol?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889756705


##########
core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.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.puffin;
+
+public final class StandardBlobTypes {
+  private StandardBlobTypes() {
+  }
+
+  /**
+   * 8-bytes integer stored little-endian and representing number of distinct values
+   */
+  public static final String NDV_LONG_LITTLE_ENDIAN = "ndv-long-little-endian";

Review Comment:
   This was removed from the spec, so we should also remove it from this commit.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r890871792


##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;

Review Comment:
   went ahead with `org.apache.iceberg.puffin`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893018158


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {

Review Comment:
   I think you should also validate these tests using the standard library. For example:
   
   ```java
     byte[] leBytes = bytes(42,0,0,0);
     ByteBuffer.wrap(leBytes).order(ByteOrder.LITTLE_ENDIAN).getInt() == readIntegerLittleEndian(leBytes, 0)
   ```



-- 
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


[GitHub] [iceberg] nastra commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897618171


##########
core/src/main/java/org/apache/iceberg/puffin/BlobMetadata.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.puffin;
+
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nullable;
+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.ImmutableMap;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> inputFields;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+  private final Map<String, String> properties;
+
+  public BlobMetadata(
+      String type, List<Integer> inputFields, long offset, long length,
+      @Nullable String compressionCodec, Map<String, String> properties) {
+    Preconditions.checkNotNull(type, "type is null");
+    Preconditions.checkNotNull(inputFields, "inputFields is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    this.type = type;
+    this.inputFields = ImmutableList.copyOf(inputFields);
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+    this.properties = ImmutableMap.copyOf(properties);
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public List<Integer> inputFields() {

Review Comment:
   the spec mentions that input fields are JSON longs, so I'm just wondering whether this should be a `List<Long>`?



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");

Review Comment:
   nit: should we mention what the actual byteNumber is and also what the requirement is? Same for the error message with bitNumber.
   
   `Preconditions.checkArgument(..., "Invalid byteNumber '%s': must be 0 <= byteNumber < %d", byteNumber, FOOTER_STRUCT_FLAGS_LENGTH)`



##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {
+  @Test
+  public void testWriteIntegerLittleEndian() throws Exception {
+    testWriteIntegerLittleEndian(0, bytes(0, 0, 0, 0));
+    testWriteIntegerLittleEndian(42, bytes(42, 0, 0, 0));
+    testWriteIntegerLittleEndian(Integer.MAX_VALUE - 5, bytes(0xFA, 0xFF, 0xFF, 0x7F));
+    testWriteIntegerLittleEndian(-7, bytes(0xF9, 0xFF, 0xFF, 0xFF));
+  }
+
+  private void testWriteIntegerLittleEndian(int value, byte[] expected) throws Exception {
+    // Sanity check: validate the expectation
+    ByteBuffer buffer = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN);
+    buffer.putInt(value);
+    buffer.flip();
+    byte[] written = new byte[4];
+    buffer.get(written);
+    Preconditions.checkState(Arrays.equals(written, expected), "Invalid expected value");

Review Comment:
   should that be just a `assertThat(written).isEqualTo(expected)`? Because this will actually show expected/actual values if the check fails, whereas the precondition doesn't show that



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    checkNotFinished();
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {

Review Comment:
   nit: Preconditions.checkState(..) maybe?



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);

Review Comment:
   nit: `Unsupported compression codec` maybe?



##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {
+  @Test
+  public void testWriteIntegerLittleEndian() throws Exception {
+    testWriteIntegerLittleEndian(0, bytes(0, 0, 0, 0));
+    testWriteIntegerLittleEndian(42, bytes(42, 0, 0, 0));
+    testWriteIntegerLittleEndian(Integer.MAX_VALUE - 5, bytes(0xFA, 0xFF, 0xFF, 0x7F));
+    testWriteIntegerLittleEndian(-7, bytes(0xF9, 0xFF, 0xFF, 0xFF));
+  }
+
+  private void testWriteIntegerLittleEndian(int value, byte[] expected) throws Exception {
+    // Sanity check: validate the expectation
+    ByteBuffer buffer = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN);
+    buffer.putInt(value);
+    buffer.flip();
+    byte[] written = new byte[4];
+    buffer.get(written);
+    Preconditions.checkState(Arrays.equals(written, expected), "Invalid expected value");
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    writeIntegerLittleEndian(outputStream, value);
+    assertThat(outputStream.toByteArray()).isEqualTo(expected);
+  }
+
+  @Test
+  public void testReadIntegerLittleEndian() {
+    testReadIntegerLittleEndian(bytes(0, 0, 0, 0), 0, 0);
+    testReadIntegerLittleEndian(bytes(42, 0, 0, 0), 0, 42);
+    testReadIntegerLittleEndian(bytes(13, 42, 0, 0, 0, 14), 1, 42);
+    testReadIntegerLittleEndian(bytes(13, 0xFa, 0xFF, 0xFF, 0x7F, 14), 1, Integer.MAX_VALUE - 5);
+    testReadIntegerLittleEndian(bytes(13, 0xF9, 0xFF, 0xFF, 0xFF, 14), 1, -7);
+  }
+
+  private void testReadIntegerLittleEndian(byte[] input, int offset, int expected) {
+    // Sanity check: validate the expectation
+    Preconditions.checkArgument(

Review Comment:
   same here, should this be just using `assertThat(...).isEqualTo(..)`?



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer compress(Compressor compressor, ByteBuffer input) {
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  static ByteBuffer decompress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");
+        return ByteBuffer.wrap(decompressed);
+      }
+    }
+
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);

Review Comment:
   nit: same as above



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Map<Pair<Integer, Integer>, Flag> BY_BYTE_AND_BIT = Stream.of(values())
+        .collect(ImmutableMap.toImmutableMap(
+            flag -> Pair.of(flag.byteNumber(), flag.bitNumber()), Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(
+          0 <= byteNumber && byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return BY_BYTE_AND_BIT.get(Pair.of(byteNumber, bitNumber));
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_START_MAGIC_OFFSET = 0;
+  static final int FOOTER_START_MAGIC_LENGTH = getMagic().length;
+
+  // "Footer struct" denotes the fixed-length portion of the Footer
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET + 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = FOOTER_STRUCT_FLAGS_OFFSET + FOOTER_STRUCT_FLAGS_LENGTH;
+  static final int FOOTER_STRUCT_LENGTH = FOOTER_STRUCT_MAGIC_OFFSET + getMagic().length;
+
+  static final PuffinCompressionCodec FOOTER_COMPRESSION_CODEC = PuffinCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x41, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & (value >> 8));
+    outputStream.write(0xFF & (value >> 16));
+    outputStream.write(0xFF & (value >> 24));
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+      case LZ4:
+        // TODO requires LZ4 frame compressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+      case ZSTD:
+        return compress(new ZstdCompressor(), input);
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer compress(Compressor compressor, ByteBuffer input) {
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  static ByteBuffer decompress(PuffinCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case NONE:
+        return input.duplicate();
+
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;

Review Comment:
   maybe move the entire block into it's own method as the case block is getting quite long?



##########
core/src/test/java/org/apache/iceberg/puffin/TestFileMetadataParser.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.puffin;
+
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestFileMetadataParser {

Review Comment:
   maybe add a few tests with invalid/missing json fields?



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    checkNotFinished();
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(
+        FileMetadataParser.toJson(fileMetadata, false).getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);
+    while (buffer.remaining() > 0) {
+      channel.write(buffer);
+    }
+  }
+
+  public long footerSize() {
+    return footerSize.orElseThrow(() -> new IllegalStateException("Footer not written yet"));
+  }
+
+  public long fileSize() {
+    return fileSize.orElseThrow(() -> new IllegalStateException("File not written yet"));
+  }
+
+  public List<BlobMetadata> writtenBlobsMetadata() {
+    return ImmutableList.copyOf(writtenBlobsMetadata);
+  }
+
+  private Set<Flag> fileFlags() {
+    EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+    if (footerCompression != PuffinCompressionCodec.NONE) {
+      flags.add(Flag.FOOTER_PAYLOAD_COMPRESSED);
+    }
+
+    return flags;
+  }
+
+  private void checkNotFinished() {
+    if (finished) {

Review Comment:
   nit: Preconditions.checkState(..) maybe?



##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;
+    this.input = inputFile.newStream();
+    if (footerSize != null) {
+      Preconditions.checkArgument(0 < footerSize && footerSize <= this.fileSize - MAGIC.length,
+          "Invalid footer size: %s", footerSize);

Review Comment:
   should this maybe mention what the valid size for the footer is?



##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.puffin;
+
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.InMemoryOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.NONE;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.readTestResource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class TestPuffinWriter {
+  @Test
+  public void testEmptyFooterCompressed() {
+    InMemoryOutputFile outputFile = new InMemoryOutputFile();
+
+    PuffinWriter writer = Puffin.write(outputFile)
+        .compressFooter()
+        .build();
+    assertThatThrownBy(writer::footerSize)
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessage("Footer not written yet");
+    assertThatThrownBy(writer::finish)
+        .isInstanceOf(UnsupportedOperationException.class)
+        .hasMessage("Unsupported codec: LZ4");
+    assertThatThrownBy(writer::close)
+        .isInstanceOf(UnsupportedOperationException.class)
+        .hasMessage("Unsupported codec: LZ4");

Review Comment:
   I'm not sure this is intuitive for users to understand why this exception is happening (given that currently LZ4 is the default when using `compressFooter()` and it's not implemented). Should the default compressor maybe be configured to ZSTD?



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1135819149

   the build has passed.


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857588013


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported

Review Comment:
   no, it's optional



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857590992


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);

Review Comment:
   Obviously there is a bunch of numbers that are hard to follow, and 4 is one of them.
   I figured that defining constants for each of them would only blur the code. 
   This is a low-level code and i don't think there are any abstractions here.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857690445


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }

Review Comment:
   ByteBuffers.toByteArray  is awesome, thanks!



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856281859


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Objects;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,
+      @JsonProperty("columns") Set<Integer> columns,
+      @JsonProperty("offset") long offset,
+      @JsonProperty("length") long length,
+      @JsonProperty("compression_codec") @Nullable String compressionCodec) {
+    this.type = Preconditions.checkNotNull(type, "type is null");
+    this.columns = ImmutableSet.copyOf(Preconditions.checkNotNull(columns, "columns is null"));
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public Set<Integer> columns() {

Review Comment:
   This is the set of fieldIds in the Iceberg table this blob applies to? (Probably should doc that) If these are field ID's is the association between this object and the table made at another level? (I'll probably find out if I keep reading)



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871144834


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {

Review Comment:
   > Why return a `Stream`? Is there a case where you wouldn't read all of the blobs into memory at once?
   
   i can imagine such future uses. Like merging stats files.
   
   > We also typically use Iterable rather than Stream.
   
   ok
   
   



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871380653


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {
+    this.type = Preconditions.checkNotNull(type, "type is null");
+    this.columns = ImmutableSet.copyOf(Preconditions.checkNotNull(columns, "columns is null"));

Review Comment:
   Good catch. In fact, blob input should also allow duplicate column inputs too



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870895948


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Preconditions.checkNotNull(codecName, "codecName is null");

Review Comment:
   I agree, we can use the `valueOf` with `toUpperCase`.
   
   Even if the spec uses lower case, this doesn't need to be that strict when interpreting input. Be liberal with what you accept and strict with what you produce.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870910726


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);

Review Comment:
   You can use Iceberg's `Pair` for this, rather than `Map.Entry`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870904508


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();

Review Comment:
   In Iceberg, we typically use `null` to signal that a lazy variable hasn't been initialized. That works cleanly with `transient`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870914062


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;

Review Comment:
   It doesn't look like you need a separate declaration anymore.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1124663159

   Reopened per discussion on slack: https://apache-iceberg.slack.com/archives/C02TZ3NMBRC/p1652299433000019


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871096061


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =

Review Comment:
   > Style: Iceberg adds empty lines between control flow blocks and the following statements.
   
   added
   
   
   
   > I guess it doesn't matter since we use the byte array path anyway.
   
   i don't understand



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871101028


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");

Review Comment:
   changed from `Optional<Integer> footerSize` to `Integer footerSize`.
   
   to hell with null safety.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871133582


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);

Review Comment:
   How to i get `RangeReadable` from an `InputFile`?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870913169


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {

Review Comment:
   Shouldn't compression be configured for the writer and then applied to every blob uniformly?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870893424


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import java.util.Set;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String COLUMNS = "columns";
+  private static final String OFFSET = "offset";
+  private static final String LENGTH = "length";
+  private static final String COMPRESSION_CODEC = "compression_codec";

Review Comment:
   As I noted on the spec, I think this should be `compression-codec` to align with the convention used in other JSON parsers.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877207995


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())

Review Comment:
   `static final` variables should be named using ALL_CAPS.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1132583000

   AC 


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870902264


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);

Review Comment:
   Does this modify `input`? If it does, then we should pass in `input.duplicate()` instead so that the original buffer is unchanged.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870899183


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported

Review Comment:
   If this isn't supported, then why list it in the spec? I think having it in the spec implies that it is required.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870899354


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {

Review Comment:
   Can this be package-private?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871518064


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");

Review Comment:
   > to hell with null safety.
   
   What do you mean?



-- 
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


[GitHub] [iceberg] alexjo2144 commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
alexjo2144 commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876220290


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;

Review Comment:
   this is called `columnsCovered` in the Blob class. Maybe unify the field names?



##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;

Review Comment:
   Should we have an enum for the Blob types?



##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Map;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class StatsFiles {
+  private StatsFiles() {
+  }
+
+  public static WriteBuilder write(OutputFile outputFile) {
+    return new WriteBuilder(outputFile);
+  }
+
+  public static class WriteBuilder {

Review Comment:
   Does it buy you anything to have this as a nested class? It's the only thing in the StatsFile class



##########
core/src/main/java/org/apache/iceberg/stats/FileMetadata.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+
+public class FileMetadata {
+  private final List<BlobMetadata> blobs;
+  private final Map<String, String> properties;
+
+  public FileMetadata(List<BlobMetadata> blobs, Map<String, String> properties) {
+    Preconditions.checkNotNull(blobs, "blobs is null");
+    Preconditions.checkNotNull(properties, "properties is null");

Review Comment:
   The spec file says `properties` is optional, so should it be nullable here?



##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = 8;
+  static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  public static final String CREATED_BY_PROPERTY = "created-by";
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x42, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    if (codec == StatsCompressionCodec.NONE) {
+      return input.duplicate();
+    }
+    Compressor compressor = createCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input.duplicate(), output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor createCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case NONE:
+        break;

Review Comment:
   Won't think fall through to the `throw UnsupportedOperationException`?



##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;

Review Comment:
   Is it worth verifying both are positive and bitNumber < 8?



##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;
+  static final int FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET = 0;
+  static final int FOOTER_STRUCT_FLAGS_OFFSET = 4;
+  static final int FOOTER_STRUCT_FLAGS_LENGTH = 4;
+  static final int FOOTER_STRUCT_MAGIC_OFFSET = 8;
+  static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  public static final String CREATED_BY_PROPERTY = "created-by";
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x42, 0x31};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);

Review Comment:
   Maybe worth putting some parenthesis in for clarity



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876694960


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;

Review Comment:
   good catch. in both places it should be "fields"-related. Will go with "inputFields".



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877831223


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+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.io.ByteStreams;
+import org.apache.iceberg.stats.StatsFormat.Flag;
+import org.apache.iceberg.util.Pair;
+
+public class StatsReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = StatsFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public StatsReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;
+    this.input = inputFile.newStream();
+    if (footerSize != null) {
+      Preconditions.checkArgument(0 < footerSize && footerSize <= this.fileSize - MAGIC.length,
+          "Invalid footer size: %s", footerSize);
+      this.knownFooterSize = Math.toIntExact(footerSize);
+    }
+  }
+
+  public FileMetadata fileMetadata() throws IOException {
+    if (knownFileMetadata == null) {
+      int footerSize = footerSize();
+      byte[] footer = readInput(fileSize - footerSize, footerSize);
+
+      checkMagic(footer, StatsFormat.FOOTER_MAGIC_OFFSET);
+      int footerStructOffset = footerSize - StatsFormat.FOOTER_STRUCT_LENGTH;
+      checkMagic(footer, footerStructOffset + StatsFormat.FOOTER_STRUCT_MAGIC_OFFSET);
+
+      StatsCompressionCodec footerCompression = StatsCompressionCodec.NONE;
+      for (Flag flag : decodeFlags(footer, footerStructOffset)) {
+        switch (flag) {
+          case FOOTER_PAYLOAD_COMPRESSED:
+            footerCompression = StatsFormat.FOOTER_COMPRESSION_CODEC;
+            break;
+          default:
+            throw new IllegalStateException("Unsupported flag: " + flag);
+        }
+      }
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(
+          footer,
+          footerStructOffset + StatsFormat.FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET);
+      Preconditions.checkState(footerSize == footerPayloadSize + StatsFormat.FOOTER_STRUCT_LENGTH + 4 /* magic */,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      ByteBuffer footerJson = StatsFormat.decompress(footerCompression, footerPayload);
+      this.knownFileMetadata = parseFileMetadata(footerJson);
+    }
+    return knownFileMetadata;
+  }
+
+  private Set<Flag> decodeFlags(byte[] footer, int footerStructOffset) {
+    EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+    for (int byteNumber = 0; byteNumber < StatsFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int flagByte =
+          Byte.toUnsignedInt(footer[footerStructOffset + StatsFormat.FOOTER_STRUCT_FLAGS_OFFSET + byteNumber]);
+      int bitNumber = 0;
+      while (flagByte != 0) {
+        if ((flagByte & 0x1) != 0) {
+          Flag flag = Flag.fromBit(byteNumber, bitNumber);
+          Preconditions.checkState(flag != null, "Unknown flag byte %s and bit %s set", byteNumber, bitNumber);
+          flags.add(flag);
+        }
+        flagByte = flagByte >> 1;
+        bitNumber++;
+      }
+    }
+    return flags;
+  }
+
+  public Iterable<Pair<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) {
+    if (blobs.isEmpty()) {
+      return ImmutableList.of();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return () -> blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer rawData = ByteBuffer.wrap(bytes);
+            StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+            ByteBuffer data = StatsFormat.decompress(codec, rawData);
+            return Pair.of(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        })
+        .iterator();
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    byte[] read = Arrays.copyOfRange(data, offset, offset + MAGIC.length);
+    if (!Arrays.equals(read, MAGIC)) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s: %s, but got %s",
+          offset, Arrays.toString(MAGIC), Arrays.toString(read)));
+    }
+  }
+
+  private int footerSize() throws IOException {
+    if (knownFooterSize == null) {
+      if (fileSize < StatsFormat.FOOTER_STRUCT_LENGTH) {
+        throw new IllegalStateException(String.format(
+            "Invalid file: file length %s is less tha minimal length of the footer tail %s",
+            fileSize,
+            StatsFormat.FOOTER_STRUCT_LENGTH));
+      }
+      byte[] footerStruct = readInput(fileSize - StatsFormat.FOOTER_STRUCT_LENGTH, StatsFormat.FOOTER_STRUCT_LENGTH);
+      checkMagic(footerStruct, StatsFormat.FOOTER_STRUCT_MAGIC_OFFSET);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(
+          footerStruct, StatsFormat.FOOTER_STRUCT_PAYLOAD_SIZE_OFFSET);
+      knownFooterSize = 4 /* magic */ + footerPayloadSize + StatsFormat.FOOTER_STRUCT_LENGTH;

Review Comment:
   done



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r896495547


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   if it is only about an IDE false positive warning about a Closeable not being closed, i think the right recourse is to add a suppression.
   
   in fact, i would prefer to avoid Channel at all, because Channels interact with interrupts in a manner that's not intuitive to me (eg https://github.com/delta-io/delta/commit/7a3f1e8ec626e80880d524c2b897a969c8b4d63a).
   
   Please let me know which approach would look awesome from Iceberg maintainers' point of view.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r901343296


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    checkNotFinished();
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    Preconditions.checkState(!footerSize.isPresent(), "footerSize already set");
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(
+        FileMetadataParser.toJson(fileMetadata, false).getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);

Review Comment:
   https://github.com/apache/iceberg/pull/5088



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897836035


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {
+  @Test
+  public void testWriteIntegerLittleEndian() throws Exception {
+    testWriteIntegerLittleEndian(0, bytes(0, 0, 0, 0));
+    testWriteIntegerLittleEndian(42, bytes(42, 0, 0, 0));
+    testWriteIntegerLittleEndian(Integer.MAX_VALUE - 5, bytes(0xFA, 0xFF, 0xFF, 0x7F));
+    testWriteIntegerLittleEndian(-7, bytes(0xF9, 0xFF, 0xFF, 0xFF));
+  }
+
+  private void testWriteIntegerLittleEndian(int value, byte[] expected) throws Exception {
+    // Sanity check: validate the expectation
+    ByteBuffer buffer = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN);
+    buffer.putInt(value);
+    buffer.flip();
+    byte[] written = new byte[4];
+    buffer.get(written);
+    Preconditions.checkState(Arrays.equals(written, expected), "Invalid expected value");
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    writeIntegerLittleEndian(outputStream, value);
+    assertThat(outputStream.toByteArray()).isEqualTo(expected);
+  }
+
+  @Test
+  public void testReadIntegerLittleEndian() {
+    testReadIntegerLittleEndian(bytes(0, 0, 0, 0), 0, 0);
+    testReadIntegerLittleEndian(bytes(42, 0, 0, 0), 0, 42);
+    testReadIntegerLittleEndian(bytes(13, 42, 0, 0, 0, 14), 1, 42);
+    testReadIntegerLittleEndian(bytes(13, 0xFa, 0xFF, 0xFF, 0x7F, 14), 1, Integer.MAX_VALUE - 5);
+    testReadIntegerLittleEndian(bytes(13, 0xF9, 0xFF, 0xFF, 0xFF, 14), 1, -7);
+  }
+
+  private void testReadIntegerLittleEndian(byte[] input, int offset, int expected) {
+    // Sanity check: validate the expectation
+    Preconditions.checkArgument(

Review Comment:
   replied here: https://github.com/apache/iceberg/pull/4537#discussion_r897835869



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1154880048

   Rebased after https://github.com/apache/iceberg/pull/5019 merged.


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r896118089


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(FileMetadataParser.toJson(fileMetadata, false)
+        .getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);

Review Comment:
   Got it. How about we wrap the stream and keep a reference to the channel instead?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897824363


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    checkNotFinished();
+    try {
+      writeHeaderIfNeeded();
+      long fileOffset = outputStream.getPos();
+      PuffinCompressionCodec codec = MoreObjects.firstNonNull(blob.requestedCompression(), defaultBlobCompression);
+      ByteBuffer rawData = PuffinFormat.compress(codec, blob.blobData());
+      int length = rawData.remaining();
+      writeFully(rawData);
+      writtenBlobsMetadata.add(new BlobMetadata(blob.type(), blob.inputFields(), fileOffset, length,
+          codec.codecName(), blob.properties()));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public Metrics metrics() {
+    return new Metrics();
+  }
+
+  @Override
+  public long length() {
+    return fileSize();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    this.outputStream.write(MAGIC);
+    this.headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+    if (footerSize.isPresent()) {
+      throw new IllegalStateException("footerSize already set");
+    }
+
+    long footerOffset = outputStream.getPos();
+    writeFooter();
+    this.footerSize = Optional.of(Math.toIntExact(outputStream.getPos() - footerOffset));
+    this.fileSize = Optional.of(outputStream.getPos());
+    this.finished = true;
+  }
+
+  private void writeFooter() throws IOException {
+    FileMetadata fileMetadata = new FileMetadata(writtenBlobsMetadata, properties);
+    ByteBuffer footerJson = ByteBuffer.wrap(
+        FileMetadataParser.toJson(fileMetadata, false).getBytes(StandardCharsets.UTF_8));
+    ByteBuffer footerPayload = PuffinFormat.compress(footerCompression, footerJson);
+    outputStream.write(MAGIC);
+    int footerPayloadLength = footerPayload.remaining();
+    writeFully(footerPayload);
+    PuffinFormat.writeIntegerLittleEndian(outputStream, footerPayloadLength);
+    writeFlags();
+    outputStream.write(MAGIC);
+  }
+
+  private void writeFlags() throws IOException {
+    Map<Integer, List<Flag>> flagsByByteNumber = fileFlags().stream()
+        .collect(Collectors.groupingBy(Flag::byteNumber));
+    for (int byteNumber = 0; byteNumber < PuffinFormat.FOOTER_STRUCT_FLAGS_LENGTH; byteNumber++) {
+      int byteFlag = 0;
+      for (Flag flag : flagsByByteNumber.getOrDefault(byteNumber, ImmutableList.of())) {
+        byteFlag |= 0x1 << flag.bitNumber();
+      }
+      outputStream.write(byteFlag);
+    }
+  }
+
+  private void writeFully(ByteBuffer buffer) throws IOException {
+    WritableByteChannel channel = Channels.newChannel(outputStream);
+    while (buffer.remaining() > 0) {
+      channel.write(buffer);
+    }
+  }
+
+  public long footerSize() {
+    return footerSize.orElseThrow(() -> new IllegalStateException("Footer not written yet"));
+  }
+
+  public long fileSize() {
+    return fileSize.orElseThrow(() -> new IllegalStateException("File not written yet"));
+  }
+
+  public List<BlobMetadata> writtenBlobsMetadata() {
+    return ImmutableList.copyOf(writtenBlobsMetadata);
+  }
+
+  private Set<Flag> fileFlags() {
+    EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
+    if (footerCompression != PuffinCompressionCodec.NONE) {
+      flags.add(Flag.FOOTER_PAYLOAD_COMPRESSED);
+    }
+
+    return flags;
+  }
+
+  private void checkNotFinished() {
+    if (finished) {

Review Comment:
   thanks for the suggestion!



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856350434


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);

Review Comment:
   Could we make a static final variable for this like FOOTER_COMPRESSION_CODEC, or something like that just to make it clear in the code?



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856391602


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);

Review Comment:
   8 => FOOTER_FILE_FORMAT_OFFSET



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856376860


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;

Review Comment:
   Should this be 
   `SUPPORTED_FLAGS = FLAG_COMPRESSED`
   Then when we add more flags it's just
   `SUPPORTED_FLAGS = FLAG_COMPRESSED | FLAG_OTHER`



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856374680


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(
+                inputBytes,
+                inputOffset,
+                inputLength,
+                decompressed,
+                0,
+                decompressed.length);
+        Preconditions.checkState(decompressedLength == decompressed.length, "Invalid decompressed length");
+        return ByteBuffer.wrap(decompressed);
+      }
+    }
+
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static byte[] swap(byte[] bytes) {

Review Comment:
   swap is a bit unclear here since we actually are just reversing the byte order here. Maybe "invert" or "reverse"?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857693488


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   `byte[]` is mutable, so i would need to `.clone()`/ copy when returning  from here.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857598232


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,

Review Comment:
   The footer contains a type of every blob. In particular, reading application will read only the blobs of types it can utilize for current operation. Same/similar information will be added to table metadata, so that reading application doesn't need to read stats file footer if no blobs are useful to it.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r853939944


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static byte[] decompressFooterPayload(byte[] footer, int offset, int length) {
+    return decompress(StatsCompressionCodec.LZ4, footer, offset, length);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static byte[] decompressBlob(StatsCompressionCodec codec, byte[] data, int dataOffset, int dataLength) {
+    return decompress(codec, data, dataOffset, dataLength);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input,  output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static byte[] decompress(StatsCompressionCodec codec, byte[] input, int inputOffset, int inputLength) {
+    switch (codec) {
+      case LZ4: {
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        throw new UnsupportedOperationException("LZ4 is not supported yet");

Review Comment:
   thanks for the tip



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847673771


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(MAGIC)).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  // TODO use LZ4 with frames, see https://trinodb.slack.com/archives/CP1MUNEUX/p1649676596198729
+  @Deprecated
+  static final int LZ4_UNCOMPRESSED_MAX_SIZE = 1024 * 1024;
+
+  // Not using JsonUtil.mapper() to avoid changing all JSON parsing
+  static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
+      .registerModule(new Jdk8Module());
+
+  static byte[] getMagic() {
+    return MAGIC.clone();
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static byte[] compressFooterPayload(byte[] payload) {

Review Comment:
   We don't generally work with `byte[]`, especially with no offset or length, because it requires new allocations all the time. Can these work with `ByteBuffer` instead?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847670242


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;
+  private final long fileOffset;
+  private final long length;
+  private final Optional<String> compressionCodec;
+
+  @JsonCreator

Review Comment:
   Because Iceberg is a library, we can't use annotations to control serialization. The JVM makes no guarantee that annotations are available at runtime and users are in control of the final classpath. So this brings in a new set of correctness problems. That's why we always serialize with a `SomethingParser` class.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847672579


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");

Review Comment:
   Why not just include the values from the spec here?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r849313220


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;
+  private final long fileOffset;
+  private final long length;
+  private final Optional<String> compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,
+      @JsonProperty("columns") List<Integer> columns,
+      @JsonProperty("offset") long fileOffset,
+      @JsonProperty("length") long length,
+      @JsonProperty("compression_codec") Optional<String> compressionCodec) {
+    this.type = Objects.requireNonNull(type, "type is null");
+    this.columns = ImmutableList.copyOf(Objects.requireNonNull(columns, "columns is null"));
+    this.fileOffset = fileOffset;
+    this.length = length;
+    this.compressionCodec = Objects.requireNonNull(compressionCodec, "compressionCodec is null");
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public List<Integer> getColumns() {
+    return columns;
+  }
+
+  public long getFileOffset() {
+    return fileOffset;
+  }
+
+  public long getLength() {
+    return length;
+  }
+
+  public Optional<String> getCompressionCodec() {

Review Comment:
   i would ask the question the other way around, but fine, changed to nullable.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847603057


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(MAGIC)).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  // TODO use LZ4 with frames, see https://trinodb.slack.com/archives/CP1MUNEUX/p1649676596198729

Review Comment:
   This needs to be addressed before the PR is merged.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847672888


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import io.airlift.compress.Compressor;
+import io.airlift.compress.lz4.Lz4Compressor;
+import io.airlift.compress.lz4.Lz4Decompressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import org.apache.commons.codec.binary.Base16;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  private static final byte[] MAGIC = new Base16().decode("50464953");
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(MAGIC)).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  // TODO use LZ4 with frames, see https://trinodb.slack.com/archives/CP1MUNEUX/p1649676596198729

Review Comment:
   And that should be documented in the spec, right?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r848084642


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;
+  private final long fileOffset;
+  private final long length;
+  private final Optional<String> compressionCodec;
+
+  @JsonCreator

Review Comment:
   Right, i took a shortcut here and forgot to mark it as a TODO item. Sorry for that.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1106115284

   AC & rebased after https://github.com/apache/iceberg/pull/4534 merged.


-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856339767


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;

Review Comment:
   empty JavaDoc



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856668849


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }

Review Comment:
   You might be interested in `org.apache.iceberg.util.ByteBuffers`, which has a `toByteArray` method that looks very similar to this.
   
   There are other methods in that class you might consider using too, or placing any additional new `ByteBuffer` utility functions.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857596435


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {
+    if (!knownFooterSize.isPresent()) {
+      int footerTailSize = Math.toIntExact(Math.min(fileLength, 20));

Review Comment:
   good point



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857601162


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {

Review Comment:
   None is not a compression. The file format spec defines there should be no mention of compression coded, when blob data is not compressed. Internally, in Java code we could still have an enum with additional value, but it would need to require special casing when reading, writing and compressing. Originally i had `Optional` here, but got a comment to use a nullable value instead.



##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = new byte[input.remaining()];
+          input.get(inputBytes);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =
+            new byte[Math.toIntExact(ZstdDecompressor.getDecompressedSize(inputBytes, inputOffset, inputLength))];
+        int decompressedLength =
+            new ZstdDecompressor().decompress(

Review Comment:
   it's mutable



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857599186


##########
build.gradle:
##########
@@ -217,6 +217,7 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'

Review Comment:
   I don't know when a new notice should be added, deferring to committers' judgement



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871523692


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {

Review Comment:
   Sounds reasonable. I like the idea of a default codec rather than having to pass it in with every blob.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893013294


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {

Review Comment:
   `checkNotFinished`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893353468


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.puffin;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.nio.charset.StandardCharsets;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class PuffinWriter implements FileAppender<Blob> {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final PositionOutputStream outputStream;
+  private final Map<String, String> properties;
+  private final PuffinCompressionCodec footerCompression;
+  private final PuffinCompressionCodec defaultBlobCompression;
+
+  private final List<BlobMetadata> writtenBlobsMetadata = Lists.newArrayList();
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+  private Optional<Long> fileSize = Optional.empty();
+
+  PuffinWriter(
+      OutputFile outputFile, Map<String, String> properties, boolean compressFooter,
+      PuffinCompressionCodec defaultBlobCompression) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    Preconditions.checkNotNull(properties, "properties is null");
+    Preconditions.checkNotNull(defaultBlobCompression, "defaultBlobCompression is null");
+    this.outputStream = outputFile.create();
+    this.properties = ImmutableMap.copyOf(properties);
+    this.footerCompression = compressFooter ? PuffinFormat.FOOTER_COMPRESSION_CODEC : PuffinCompressionCodec.NONE;
+    this.defaultBlobCompression = defaultBlobCompression;
+  }
+
+  @Override
+  public void add(Blob blob) {
+    Preconditions.checkNotNull(blob, "blob is null");
+    try {

Review Comment:
   added, thanks



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889867685


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {

Review Comment:
   yeah, i agree. i added helper there



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889751750


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinFormat.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.puffin;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class PuffinFormat {
+  private PuffinFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> BY_BYTE_AND_BIT = Stream.of(values())

Review Comment:
   Rather than adding a new Guava class, I think you could just use a `Map<Pair<Integer, Integer>, Flag>`. The lookup is hidden behind a static method anyway, so there isn't much of a cost to using `Pair`.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889870502


##########
core/src/main/java/org/apache/iceberg/puffin/StandardBlobTypes.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.puffin;
+
+public final class StandardBlobTypes {
+  private StandardBlobTypes() {
+  }
+
+  /**
+   * 8-bytes integer stored little-endian and representing number of distinct values
+   */
+  public static final String NDV_LONG_LITTLE_ENDIAN = "ndv-long-little-endian";

Review Comment:
   of course, removed!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r890871529


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;

Review Comment:
   added "Statistics are informational. A reader can choose to
   ignore statistics information. Statistics support is not required to read the table correctly." in https://github.com/apache/iceberg/pull/4945



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889755775


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {

Review Comment:
   Rather than hard-coding, or at least in addition, I think these tests should validate against the behavior of reading the little-endian value from a ByteBuffer. That will help readers see that it is correct.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870907944


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);

Review Comment:
   The new `RangeReadable` interface has a `readTail` method that you can use to avoid needing `fileLength`. S3 allows you to read the last N bytes of the file without knowing where to seek to.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871099987


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {

Review Comment:
   size of Footer (in bytes). Footer is defined in the file format spec https://github.com/apache/iceberg-docs/pull/69 
   ````
   ### Footer structure
   
   Footer has the following structure
   
   ```
   Magic FooterPayload FooterPayloadSize Reserved Flags FileFormatVersion Magic
   ```
   ````



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870893780


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import java.util.Set;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String COLUMNS = "columns";
+  private static final String OFFSET = "offset";
+  private static final String LENGTH = "length";
+  private static final String COMPRESSION_CODEC = "compression_codec";
+
+  public static String toJson(FileMetadata fileMetadata) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();
+      toJson(fileMetadata, generator);
+      generator.flush();
+      return writer.toString();
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to write json for: " + fileMetadata, e);
+    }
+  }
+
+  public static FileMetadata fromJson(String json) {
+    try {
+      return fromJson(JsonUtil.mapper().readValue(json, JsonNode.class));
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  public static FileMetadata fromJson(JsonNode json) {

Review Comment:
   Is this used elsewhere or can it be package-private?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870915101


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);

Review Comment:
   Looks like channels aren't guaranteed to write all of the bytes, so this should check the return value from `write`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870897570


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   While it is mutable, it should be fine to rely on people not modifying it. This isn't public.
   
   I'd just make this a constant, `MAGIC`.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871132118


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {

Review Comment:
   Footer size is known in practice: https://github.com/apache/iceberg-docs/pull/77
   
   it's made optional only for the manual inspection case when someone wants to read a standalone stats file, without having it's metadata/



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877826494


##########
bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java:
##########
@@ -73,6 +74,7 @@ public class GuavaClasses {
     ImmutableList.class.getName();
     ImmutableMap.class.getName();
     ImmutableSet.class.getName();
+    ImmutableTable.class.getName();

Review Comment:
   Table is a very good API that simplifies use wherever an engineer is tempted to use two-level nested Maps.
   
   Yes, i can go without Table, at the cost of readability. Up to you.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877211494


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(StatsCompressionCodec.LZ4, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(StatsCompressionCodec.LZ4, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported

Review Comment:
   If compression is optional, then why throw `UnsupportedOperationException` when the codec is `NONE`?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r901345504


##########
core/src/main/java/org/apache/iceberg/io/IOUtil.java:
##########
@@ -46,6 +50,21 @@ public static void readFully(InputStream stream, byte[] bytes, int offset, int l
     }
   }
 
+  /**
+   * Writes a buffer into a stream, making multiple write calls if necessary.
+   */
+  public static void writeFully(OutputStream outputStream, ByteBuffer buffer) throws IOException {
+    if (!buffer.hasRemaining()) {
+      return;
+    }
+    byte[] chunk = new byte[WRITE_CHUNK_SIZE];

Review Comment:
   > Alternatively, you could pass the temporary buffer in.
   
   This poses sizing challenge. I.e. the caller needs to provide a reasonably sized buffer.
   
   > Rather than allocating every time this is called, can you create a ThreadLocal to share this buffer?
   
   Sure, this is feasible. Do you happen to know what would be the expected reuse ratio for such a buffer?
   
   Alternatively we can have a static buffer pool that lends buffers to the current thread.
   (assuming we have a problem that we want to fix here)



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r888037661


##########
core/src/test/java/org/apache/iceberg/stats/StatsFormatTestUtil.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.stats;
+
+import java.io.InputStream;
+import java.net.URL;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public final class StatsFormatTestUtil {

Review Comment:
   updated



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r887886599


##########
core/src/main/java/org/apache/iceberg/stats/StatsFiles.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;

Review Comment:
   Which one you prefer?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870898219


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |

Review Comment:
   Why use `0xFF & value` above, but `Byte.toUnsignedInt` here?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870911710


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {

Review Comment:
   Why return a `Stream`? Is there a case where you wouldn't read all of the blobs into memory at once?
   
   We also typically use `Iterable` rather than `Stream`.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870912372


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(

Review Comment:
   Missing newline between methods.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871140650


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());

Review Comment:
   Forward compatibility for compression codecs. If we add e.g. LZ6, old reader can still make use of the file, it just cannot read portions of the file.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871514774


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =

Review Comment:
   I think I added the second comment as a follow up on the wrong thread. You can ignore it.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871512716


##########
build.gradle:
##########
@@ -217,6 +217,7 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'

Review Comment:
   No, the dependency is already listed in the LICENSE and NOTICE files so there is nothing more to do.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872420901


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);

Review Comment:
   thanks, fixed



##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {

Review Comment:
   added 



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872410250


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());

Review Comment:
   Yes, I can add `StatsCompressionCodec.UNKOWN`. To me it feels like a special value, very much like `-1`, which I would rather avoid. What would be the benefit from such a change? 
   
   If the user code was interested in compression information, a String is better than an UNKNOWN constant, since String conveys all information, while UNKNOWN already lost something.
   If the user code is not dealing with compression information directly, it doesn't matter, it's an impl detail.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872510624


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);

Review Comment:
   I think this is fine the way it is right now.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889754868


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  public PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {

Review Comment:
   I don't think this should be public. The reader should be created by helpers in `Puffin`.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889863959


##########
core/src/main/java/org/apache/iceberg/puffin/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.puffin;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String FIELDS = "fields";
+  private static final String OFFSET = "offset";
+  private static final String LENGTH = "length";
+  private static final String COMPRESSION_CODEC = "compression-codec";
+
+  public static String toJson(FileMetadata fileMetadata) {
+    try {
+      StringWriter writer = new StringWriter();
+      JsonGenerator generator = JsonUtil.factory().createGenerator(writer);
+      generator.useDefaultPrettyPrinter();

Review Comment:
   thanks!



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893380516


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinReader.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.puffin;
+
+import java.util.Map;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InMemoryInputFile;
+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.Streams;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.Pair;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.NONE;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.readTestResource;
+import static org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.toImmutableMap;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinReader {
+  @Test
+  public void testEmptyFooterUncompressed() throws Exception {
+    testEmpty("v1/empty-puffin-uncompressed.bin", EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE);
+  }
+
+  @Test
+  public void testEmptyWithUnknownFooterSize() throws Exception {
+    testEmpty("v1/empty-puffin-uncompressed.bin", null);
+  }
+
+  private void testEmpty(String resourceName, @Nullable Long footerSize) throws Exception {
+    InMemoryInputFile inputFile = new InMemoryInputFile(readTestResource(resourceName));
+    Puffin.ReadBuilder readBuilder = Puffin.read(inputFile)
+        .withFileSize(inputFile.getLength());
+    if (footerSize != null) {
+      readBuilder = readBuilder.withFooterSize(footerSize);

Review Comment:
   good idea, added



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893381050


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.puffin;
+
+import java.nio.ByteBuffer;
+import org.apache.iceberg.io.InMemoryOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.NONE;
+import static org.apache.iceberg.puffin.PuffinCompressionCodec.ZSTD;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.EMPTY_PUFFIN_UNCOMPRESSED_FOOTER_SIZE;
+import static org.apache.iceberg.puffin.PuffinFormatTestUtil.readTestResource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class TestPuffinWriter {
+  @Test
+  public void testEmptyFooterCompressed() {
+    InMemoryOutputFile outputFile = new InMemoryOutputFile();
+
+    PuffinWriter writer = Puffin.write(outputFile)
+        .compressFooter()
+        .build();
+    assertThatThrownBy(writer::footerSize)
+        .isInstanceOf(IllegalStateException.class)
+        .hasMessage("Footer not written yet");
+    assertThatThrownBy(writer::finish)

Review Comment:
   so that a user can read file size and footer size from the writer object.
   otherwise it wouldn't be possible to use writer with try-with-resources



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r876699653


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadata.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.stats;
+
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+
+public class FileMetadata {
+  private final List<BlobMetadata> blobs;
+  private final Map<String, String> properties;
+
+  public FileMetadata(List<BlobMetadata> blobs, Map<String, String> properties) {
+    Preconditions.checkNotNull(blobs, "blobs is null");
+    Preconditions.checkNotNull(properties, "properties is null");

Review Comment:
   `FileMetadataParser` sets empty map when properties are null and doesn't serialize anything if this is empty.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r897836694


##########
core/src/main/java/org/apache/iceberg/puffin/PuffinReader.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.puffin;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.RangeReadable;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.puffin.PuffinFormat.Flag;
+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.io.ByteStreams;
+import org.apache.iceberg.util.Pair;
+
+public class PuffinReader implements Closeable {
+  // Must not be modified
+  private static final byte[] MAGIC = PuffinFormat.getMagic();
+
+  private final long fileSize;
+  private final SeekableInputStream input;
+  private Integer knownFooterSize;
+  private FileMetadata knownFileMetadata;
+
+  PuffinReader(InputFile inputFile, @Nullable Long fileSize, @Nullable Long footerSize) {
+    Preconditions.checkNotNull(inputFile, "inputFile is null");
+    this.fileSize = fileSize == null ? inputFile.getLength() : fileSize;
+    this.input = inputFile.newStream();
+    if (footerSize != null) {
+      Preconditions.checkArgument(0 < footerSize && footerSize <= this.fileSize - MAGIC.length,
+          "Invalid footer size: %s", footerSize);

Review Comment:
   well, at this point we don't know what the valid footer size is.
   we only know the provided one is _obviously_ not valid



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1156375557

   Thanks @rdblue and @nastra for your reviews!
   
   Updated the code accordingly. Please take another look.


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r857592741


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);

Review Comment:
   As said above, defining the constants doesn't make the code clear, at least not in my eyes.
   The code is organized after and follows the raw footer layout. That's clearly indicated by the numbers subsequently used in the code (4, 8, 12, 16). Constants wouldn't convey that meaning, and wouldn't make evolving the code easier either.



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1108879409

   AC


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r893018158


##########
core/src/test/java/org/apache/iceberg/puffin/TestPuffinFormat.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.puffin;
+
+import java.io.ByteArrayOutputStream;
+import org.junit.Test;
+
+import static org.apache.iceberg.puffin.PuffinFormat.readIntegerLittleEndian;
+import static org.apache.iceberg.puffin.PuffinFormat.writeIntegerLittleEndian;
+import static org.apache.iceberg.relocated.com.google.common.base.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestPuffinFormat {

Review Comment:
   I think you should also validate these tests using the standard library. For example:
   
   ```java
     byte[] leBytes = bytes(42,0,0,0);
     ByteBuffer.wrap(leBytes).order(ByteOrder.LITTLE_ENDIAN).getInt() == readIntegerLittleEndian(leBytes, 0)
   ```
   
   Same with the write. You can write to a little endian byte buffer and then compare them to validate the writer, which is easier for readers to verify.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877216202


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())
+        .collect(ImmutableTable.toImmutableTable(Flag::byteNumber, Flag::bitNumber, Function.identity()));
+
+    private final int byteNumber;
+    private final int bitNumber;
+
+    Flag(int byteNumber, int bitNumber) {
+      Preconditions.checkArgument(0 <= byteNumber && byteNumber < StatsFormat.FOOTER_STRUCT_FLAGS_LENGTH,
+          "Invalid byteNumber");
+      Preconditions.checkArgument(0 <= bitNumber && bitNumber < Byte.SIZE, "Invalid bitNumber");
+      this.byteNumber = byteNumber;
+      this.bitNumber = bitNumber;
+    }
+
+    @Nullable
+    static Flag fromBit(int byteNumber, int bitNumber) {
+      return byByteAndBit.get(byteNumber, bitNumber);
+    }
+
+    public int byteNumber() {
+      return byteNumber;
+    }
+
+    public int bitNumber() {
+      return bitNumber;
+    }
+  }
+
+  static final int FOOTER_MAGIC_OFFSET = 0;
+  static final int FOOTER_STRUCT_LENGTH = 12;

Review Comment:
   What struct is this referring to?



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r898265357


##########
core/src/main/java/org/apache/iceberg/puffin/Puffin.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.puffin;
+
+import java.util.Map;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class Puffin {
+  private Puffin() {
+  }
+
+  public static WriteBuilder write(OutputFile outputFile) {
+    return new WriteBuilder(outputFile);
+  }
+
+  public static class WriteBuilder {
+    private final OutputFile outputFile;
+    private final Map<String, String> properties = Maps.newLinkedHashMap();
+    private boolean compressFooter = false;
+    private PuffinCompressionCodec defaultBlobCompression = PuffinCompressionCodec.NONE;
+
+    private WriteBuilder(OutputFile outputFile) {
+      this.outputFile = outputFile;
+    }
+
+    public WriteBuilder set(String property, String value) {
+      properties.put(property, value);
+      return this;
+    }
+
+    public WriteBuilder setAll(Map<String, String> props) {
+      this.properties.putAll(props);
+      return this;
+    }
+
+    public WriteBuilder compressFooter() {
+      this.compressFooter = true;
+      return this;
+    }
+
+    public WriteBuilder compressBlobs(PuffinCompressionCodec compression) {
+      this.defaultBlobCompression = compression;
+      return this;
+    }
+
+    public PuffinWriter build() {
+      return new PuffinWriter(outputFile, properties, compressFooter, defaultBlobCompression);

Review Comment:
   Why not set it automatically? It seems weird to require writers to remember to add it themselves.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r847670524


##########
build.gradle:
##########
@@ -217,9 +217,11 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'
     implementation 'org.apache.httpcomponents.client5:httpclient5'
     implementation "com.fasterxml.jackson.core:jackson-databind"
     implementation "com.fasterxml.jackson.core:jackson-core"
+    implementation "com.fasterxml.jackson.datatype:jackson-datatype-jdk8"

Review Comment:
   This shouldn't be needed.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r848632184


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+
+public class BlobMetadata {
+  private final String type;
+  private final List<Integer> columns;
+  private final long fileOffset;
+  private final long length;
+  private final Optional<String> compressionCodec;
+
+  @JsonCreator

Review Comment:
   No worries! This is a problem most people aren't aware of so it's pretty common.



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r855563164


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import java.util.Objects;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Objects.requireNonNull(codecName, "codecName is null");

Review Comment:
   Nit: Usually we use the relocated `Preconditions` form.
   
   Also, does it make more sense to just maintain a mapping between the two vs iterating over them?



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856303548


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Objects;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,
+      @JsonProperty("columns") Set<Integer> columns,
+      @JsonProperty("offset") long offset,
+      @JsonProperty("length") long length,
+      @JsonProperty("compression_codec") @Nullable String compressionCodec) {
+    this.type = Preconditions.checkNotNull(type, "type is null");
+    this.columns = ImmutableSet.copyOf(Preconditions.checkNotNull(columns, "columns is null"));
+    this.offset = offset;
+    this.length = length;
+    this.compressionCodec = compressionCodec;
+  }
+
+  public String type() {
+    return type;
+  }
+
+  public Set<Integer> columns() {

Review Comment:
   yes and yes



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856390934


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);

Review Comment:
   Checking for the magic at the beginning and end of the footer?



-- 
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


[GitHub] [iceberg] findepi commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1103691539

   ( squashed and rebased on top of current version of https://github.com/apache/iceberg/pull/4534, no other changes in this push )


-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856482432


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  @JsonCreator
+  public BlobMetadata(
+      @JsonProperty("type") String type,

Review Comment:
   What is the benefit of the multiple blobs in a file if we don't have footer information saying why we may or may not need to check a specific blob? I guess currently we can prune them based on "columns" but I feel like maybe we would benefit from other information? Like if a particular file had sketches that applied to different row groups in a file so the correct sketch to use would be based on min and max column values or something ? Just thinking about possible future directions.



-- 
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


[GitHub] [iceberg] rdblue commented on pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1156773135

   Thanks, @findepi! This looks good to me. We can still follow up, but I think the majority of the changes are ready so I've merged it to avoid keeping a big PR outstanding.


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r901346387


##########
core/src/main/java/org/apache/iceberg/puffin/Puffin.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.puffin;
+
+import java.util.Map;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public final class Puffin {
+  private Puffin() {
+  }
+
+  public static WriteBuilder write(OutputFile outputFile) {
+    return new WriteBuilder(outputFile);
+  }
+
+  public static class WriteBuilder {
+    private final OutputFile outputFile;
+    private final Map<String, String> properties = Maps.newLinkedHashMap();
+    private boolean compressFooter = false;
+    private PuffinCompressionCodec defaultBlobCompression = PuffinCompressionCodec.NONE;
+
+    private WriteBuilder(OutputFile outputFile) {
+      this.outputFile = outputFile;
+    }
+
+    public WriteBuilder set(String property, String value) {
+      properties.put(property, value);
+      return this;
+    }
+
+    public WriteBuilder setAll(Map<String, String> props) {
+      this.properties.putAll(props);
+      return this;
+    }
+
+    public WriteBuilder compressFooter() {
+      this.compressFooter = true;
+      return this;
+    }
+
+    public WriteBuilder compressBlobs(PuffinCompressionCodec compression) {
+      this.defaultBlobCompression = compression;
+      return this;
+    }
+
+    public PuffinWriter build() {
+      return new PuffinWriter(outputFile, properties, compressFooter, defaultBlobCompression);

Review Comment:
   How would you envision that on the Builder's API level?



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856344434


##########
core/src/main/java/org/apache/iceberg/stats/StatsCompressionCodec.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.stats;
+
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public enum StatsCompressionCodec {
+  /**
+   * LZ4 single compression frame with content size present
+   */
+  LZ4("lz4"),
+
+  /**
+   * Zstandard single compression frame with content size present
+   */
+  ZSTD("zstd"),
+  /**/;
+
+  private final String codecName;
+
+  StatsCompressionCodec(String codecName) {
+    this.codecName = codecName;
+  }
+
+  public String getCodecName() {
+    return codecName;
+  }
+
+  public static StatsCompressionCodec forName(String codecName) {
+    Preconditions.checkNotNull(codecName, "codecName is null");

Review Comment:
   we tend to do these like
   
   https://github.com/apache/iceberg/blob/c75ac359c1de6bf9fd4894b40009c5c42d2fee9d/api/src/main/java/org/apache/iceberg/DistributionMode.java#L51-L54
   
   Where we leverage `valueOf` to throw IllegalArgs for us and check for valid enum names
   
   



-- 
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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856667111


##########
core/src/main/java/org/apache/iceberg/stats/BlobMetadata.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.stats;
+
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+
+public class BlobMetadata {
+  private final String type;
+  private final Set<Integer> columns;
+  private final long offset;
+  private final long length;
+  private final String compressionCodec;
+
+  public BlobMetadata(String type, Set<Integer> columns, long offset, long length, @Nullable String compressionCodec) {

Review Comment:
   For the `compressionCodec`, instead of using null for uncompressed, would it make sense to use an enum and then define the possible compression types and have a value for none / uncompressed?
   
   This would be similar to `TableMetadataParser.Codec`, which has a `NONE` value, though there might be reasons not to do that.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877825395


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.util.function.Function;
+import java.util.stream.Stream;
+import javax.annotation.Nullable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableTable;
+import org.apache.iceberg.relocated.com.google.common.collect.Table;
+import org.apache.iceberg.util.ByteBuffers;
+
+public final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  enum Flag {
+    FOOTER_PAYLOAD_COMPRESSED(0, 0),
+    /**/;
+
+    private static final Table<Integer, Integer, Flag> byByteAndBit = Stream.of(values())

Review Comment:
   done



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r877824889


##########
core/src/main/java/org/apache/iceberg/stats/FileMetadataParser.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.Map;
+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.ImmutableMap;
+import org.apache.iceberg.util.JsonUtil;
+
+public final class FileMetadataParser {
+
+  private FileMetadataParser() {
+  }
+
+  private static final String BLOBS = "blobs";
+  private static final String PROPERTIES = "properties";
+
+  private static final String TYPE = "type";
+  private static final String COLUMNS = "columns";

Review Comment:
   Fixed. In the spec it's already `fields`



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add reader and writer for Puffin, indexes and stats file format

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r889752787


##########
bundled-guava/src/main/java/org/apache/iceberg/GuavaClasses.java:
##########
@@ -73,6 +74,7 @@ public class GuavaClasses {
     ImmutableList.class.getName();
     ImmutableMap.class.getName();
     ImmutableSet.class.getName();
+    ImmutableTable.class.getName();

Review Comment:
   I think we can replace this one fairly easily. But it looks like a good idea to add `Resources`.



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856409002


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {
+    if (!knownFooterSize.isPresent()) {
+      int footerTailSize = Math.toIntExact(Math.min(fileLength, 20));
+      byte[] footerTail = readInput(fileLength - footerTailSize, footerTailSize);
+
+      checkMagic(footerTail, footerTailSize - 4);

Review Comment:
   Like before I think it would be helpful if a lot of these magic numbers had Constants to label them.



##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {
+    if (!knownFooterSize.isPresent()) {
+      int footerTailSize = Math.toIntExact(Math.min(fileLength, 20));
+      byte[] footerTail = readInput(fileLength - footerTailSize, footerTailSize);
+
+      checkMagic(footerTail, footerTailSize - 4);

Review Comment:
   Like before I think it would be helpful if a lot of these magic numbers had constants to label them.



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856408088


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());
+              data = StatsFormat.decompressBlob(codec, data);
+            }
+            return new SimpleEntry<>(blobMetadata, data);
+          } catch (IOException e) {
+            throw new UncheckedIOException(e);
+          }
+        });
+  }
+
+  private static void checkMagic(byte[] data, int offset) {
+    int read = StatsFormat.readIntegerLittleEndian(data, offset);
+    if (read != StatsFormat.MAGIC_AS_NUMBER_LE) {
+      throw new IllegalStateException(String.format(
+          "Invalid file: expected magic at offset %s (%s i.e. %s as integer) but got %s",
+          offset, Arrays.toString(StatsFormat.getMagic()), StatsFormat.MAGIC_AS_NUMBER_LE, read));
+    }
+  }
+
+  private static boolean isFlagSet(int setFlags, int testedFlag) {
+    return (setFlags & testedFlag) == testedFlag;
+  }
+
+  private int getFooterSize() throws IOException {
+    if (!knownFooterSize.isPresent()) {
+      int footerTailSize = Math.toIntExact(Math.min(fileLength, 20));

Review Comment:
   What's the math here? If the file length is less than 20 I thought we can't have a valid footer? (Magic + Reserved + Flag + Format + Magic) * 4 = 20?



-- 
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


[GitHub] [iceberg] ggershinsky commented on pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#issuecomment-1107777980

   > CC: @ggershinsky Could you take a look at this as well I'm not sure how we fit this into our current encryption plans since we are compressing blob blocks within this file seperately we would end up with issues if we encrypted the entire file. Is there an easy way we can compress/encrypt the blobs inside the index files?
   
   Certainly, will do


-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r853943739


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Objects.requireNonNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Objects.requireNonNull(name, "name is null");
+    Objects.requireNonNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Objects.requireNonNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);
+    @Nullable String codecName = compression.map(StatsCompressionCodec::getCodecName).orElse(null);
+    blobs.add(new BlobMetadata(type, columnsCovered, fileOffset, length, codecName));
+  }
+
+  public void setCompressFooter(boolean compressFooter) {
+    this.compressFooter = compressFooter;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    outputStream.write(StatsFormat.getMagic());
+    headerWritten = true;
+  }
+
+  public void finish() throws IOException {
+    writeHeaderIfNeeded();
+    if (finished) {
+      throw new IllegalStateException("Already finished");
+    }

Review Comment:
   done



-- 
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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r856383582


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swap(getMagic())).intValueExact();
+
+  static final int SUPPORTED_FLAGS = 0b1;
+  static final int FLAG_COMPRESSED = 0b1;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};

Review Comment:
   This always allocates bytes for the Magic byte[] array, Is this because we want to re-use the array we are allocating here? Just wondering if this could be a static final byte array since it get's used in some contexts (like counting length) which don't require the allocation.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870839253


##########
build.gradle:
##########
@@ -217,6 +217,7 @@ project(':iceberg-core') {
       exclude group: 'org.tukaani' // xz compression is not supported
     }
 
+    implementation 'io.airlift:aircompressor'

Review Comment:
   This is already included because it is pulled in through ORC. Thanks for asking about it, @kbendick. We always need to be careful about this!



-- 
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


[GitHub] [iceberg] findepi closed pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi closed pull request #4537: Add Indexes and Stats format reader and writer
URL: https://github.com/apache/iceberg/pull/4537


-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870913772


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {
+    checkNotFinished();
+    writeHeaderIfNeeded();
+
+    Preconditions.checkNotNull(type, "type is null");
+    long fileOffset = outputStream.getPos();
+    ByteBuffer data;
+    data = compression.map(codec -> StatsFormat.compressBlob(codec, blobData))
+        .orElse(blobData);
+    int length = data.remaining();
+    Channels.newChannel(outputStream).write(data);
+    @Nullable String codecName = compression.map(StatsCompressionCodec::getCodecName).orElse(null);
+    blobs.add(new BlobMetadata(type, columnsCovered, fileOffset, length, codecName));
+  }
+
+  public void setCompressFooter(boolean compressFooter) {
+    this.compressFooter = compressFooter;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!finished) {
+      finish();
+    }
+
+    outputStream.close();
+  }
+
+  private void writeHeaderIfNeeded() throws IOException {
+    if (headerWritten) {
+      return;
+    }
+
+    outputStream.write(StatsFormat.getMagic());
+    headerWritten = true;

Review Comment:
   When modifying state by setting an instance field, we like to use `this.` so it is obvious that it isn't a local variable.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870902045


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);
+          inputOffset = 0;
+          inputLength = inputBytes.length;
+        }
+        byte[] decompressed =

Review Comment:
   I guess it doesn't matter since we use the byte array path anyway.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870898772


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {

Review Comment:
   Style: `get` in a method name.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871516414


##########
core/src/main/java/org/apache/iceberg/stats/StatsFormat.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.stats;
+
+import io.airlift.compress.Compressor;
+import io.airlift.compress.zstd.ZstdCompressor;
+import io.airlift.compress.zstd.ZstdDecompressor;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ByteBuffers;
+
+final class StatsFormat {
+  private StatsFormat() {
+  }
+
+  static final int CURRENT_FORMAT_VERSION = 1;
+
+  static final int MAGIC_AS_NUMBER_LE = new BigInteger(swapBytes(getMagic())).intValueExact();
+
+  static final int FLAG_COMPRESSED = 0b1;
+  static final int SUPPORTED_FLAGS = FLAG_COMPRESSED;
+
+  private static final StatsCompressionCodec FOOTER_COMPRESSION_CODEC = StatsCompressionCodec.LZ4;
+
+  static byte[] getMagic() {
+    return new byte[] {0x50, 0x46, 0x49, 0x53};
+  }
+
+  static void writeIntegerLittleEndian(OutputStream outputStream, int value) throws IOException {
+    outputStream.write(0xFF & value);
+    outputStream.write(0xFF & value >> 8);
+    outputStream.write(0xFF & value >> 16);
+    outputStream.write(0xFF & value >> 24);
+  }
+
+  static int readIntegerLittleEndian(byte[] data, int offset) {
+    return Byte.toUnsignedInt(data[offset]) |
+        (Byte.toUnsignedInt(data[offset + 1]) << 8) |
+        (Byte.toUnsignedInt(data[offset + 2]) << 16) |
+        (Byte.toUnsignedInt(data[offset + 3]) << 24);
+  }
+
+  static ByteBuffer compressFooterPayload(ByteBuffer payload) {
+    return compress(FOOTER_COMPRESSION_CODEC, payload);
+  }
+
+  static ByteBuffer decompressFooterPayload(ByteBuffer footer) {
+    return decompress(FOOTER_COMPRESSION_CODEC, footer);
+  }
+
+  static ByteBuffer compressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return compress(codec, data);
+  }
+
+  static ByteBuffer decompressBlob(StatsCompressionCodec codec, ByteBuffer data) {
+    return decompress(codec, data);
+  }
+
+  private static ByteBuffer compress(StatsCompressionCodec codec, ByteBuffer input) {
+    Compressor compressor = getCompressor(codec);
+    ByteBuffer output = ByteBuffer.allocate(compressor.maxCompressedLength(input.remaining()));
+    compressor.compress(input, output);
+    output.flip();
+    return output;
+  }
+
+  private static Compressor getCompressor(StatsCompressionCodec codec) {
+    switch (codec) {
+      case LZ4:
+        // TODO currently not supported
+        break;
+      case ZSTD:
+        return new ZstdCompressor();
+    }
+    throw new UnsupportedOperationException("Unsupported codec: " + codec);
+  }
+
+  private static ByteBuffer decompress(StatsCompressionCodec codec, ByteBuffer input) {
+    switch (codec) {
+      case LZ4:
+        // TODO requires LZ4 frame decompressor, e.g. https://github.com/airlift/aircompressor/pull/142
+        break;
+
+      case ZSTD: {
+        byte[] inputBytes;
+        int inputOffset;
+        int inputLength;
+        if (input.hasArray()) {
+          inputBytes = input.array();
+          inputOffset = input.arrayOffset();
+          inputLength = input.remaining();
+        } else {
+          // TODO implement ZstdDecompressor.getDecompressedSize for ByteBuffer to avoid copying
+          inputBytes = ByteBuffers.toByteArray(input);

Review Comment:
   This is where my other comment was supposed to go. This doesn't matter because the buffer isn't just used to get the size. It is also decompressed so it should be the whole blob.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871517437


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {

Review Comment:
   After going through the code, it's clear that this is the footer payload size + a constant. This should be documented in Javadoc.



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870910364


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);
+
+      int footerPayloadSize = StatsFormat.readIntegerLittleEndian(footer, footerSize - 20);
+      Preconditions.checkState(footerPayloadSize == footerSize - 24,
+          "Unexpected footer payload size value %s for footer size %s", footerPayloadSize, footerSize);
+
+      FileMetadata readFileMetadata;
+      ByteBuffer footerPayload = ByteBuffer.wrap(footer, 4, footerPayloadSize);
+      if (!compressed) {
+        readFileMetadata = parseFileMetadata(footerPayload);
+      } else {
+        ByteBuffer footerJson = StatsFormat.decompressFooterPayload(footerPayload);
+        readFileMetadata = parseFileMetadata(footerJson);
+      }
+      this.knownFileMetadata = Optional.of(readFileMetadata);
+    }
+    return knownFileMetadata.get();
+  }
+
+  /**
+   * @throws IOException          when I/O error occurs
+   * @throws UncheckedIOException when I/O error occurs
+   */
+  // Note: The method is marked as throwing IOException to allow future implementation evolution.
+  // Currently, the exception isn't being thrown.
+  public Stream<Map.Entry<BlobMetadata, ByteBuffer>> readAll(List<BlobMetadata> blobs) throws IOException {
+    if (blobs.isEmpty()) {
+      return Stream.empty();
+    }
+
+    // TODO inspect blob offsets and coalesce read regions close to each other
+
+    return blobs.stream()
+        .sorted(Comparator.comparingLong(BlobMetadata::offset))
+        .map((BlobMetadata blobMetadata) -> {
+          try {
+            input.seek(blobMetadata.offset());
+            byte[] bytes = new byte[Math.toIntExact(blobMetadata.length())];
+            ByteStreams.readFully(input, bytes);
+            ByteBuffer data = ByteBuffer.wrap(bytes);
+            if (blobMetadata.compressionCodec() != null) {
+              StatsCompressionCodec codec = StatsCompressionCodec.forName(blobMetadata.compressionCodec());

Review Comment:
   Why not translate to/from `StatsCompressionCodec` in the parser? That would make more sense to me.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r871146896


##########
core/src/main/java/org/apache/iceberg/stats/StatsWriter.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import javax.annotation.Nullable;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class StatsWriter implements Closeable {
+
+  private final PositionOutputStream outputStream;
+
+  private final Map<String, String> properties = Maps.newHashMap();
+  private final List<BlobMetadata> blobs = Lists.newArrayList();
+  private boolean compressFooter = false; // TODO compress footer by default when LZ4 support is added
+
+  private boolean headerWritten;
+  private boolean finished;
+  private Optional<Integer> footerSize = Optional.empty();
+
+  public StatsWriter(OutputFile outputFile) {
+    Preconditions.checkNotNull(outputFile, "outputFile is null");
+    this.outputStream = outputFile.create();
+  }
+
+  public void addFileProperty(String name, String value) {
+    Preconditions.checkNotNull(name, "name is null");
+    Preconditions.checkNotNull(value, "value is null");
+
+    if (properties.putIfAbsent(name, value) != null) {
+      throw new IllegalStateException(String.format("Property '%s' already set", name));
+    }
+  }
+  public void append(
+      String type,
+      Set<Integer> columnsCovered,
+      ByteBuffer blobData,
+      Optional<StatsCompressionCodec> compression) throws IOException {

Review Comment:
   i don't think same compression codec should be used irrespectively of the payload.
   
   i can see a use of default compression codec, but that's something that can be added later easily.



-- 
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


[GitHub] [iceberg] findepi commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
findepi commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r872424057


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {
+    this.fileLength = inputFile.getLength();
+    this.input = Preconditions.checkNotNull(inputFile, "inputFile is null").newStream();
+    this.knownFooterSize = Preconditions.checkNotNull(footerSize, "footerSize is null");
+    footerSize.ifPresent(size ->
+        Preconditions.checkArgument(0 < size && size <= fileLength - StatsFormat.getMagic().length,
+            "Invalid footer size: %s", size));
+  }
+
+  public FileMetadata getFileMetadata() throws IOException {
+    if (!knownFileMetadata.isPresent()) {
+      int footerSize = getFooterSize();
+      byte[] footer = readInput(fileLength - footerSize, footerSize);
+
+      checkMagic(footer, 0);
+      checkMagic(footer, footerSize - 4);
+
+      int fileFormatVersion = StatsFormat.readIntegerLittleEndian(footer, footerSize - 8);
+      Preconditions.checkState(
+          fileFormatVersion == StatsFormat.CURRENT_FORMAT_VERSION,
+          "Unsupported format version %s, %s is the latest supported",
+          fileFormatVersion,
+          StatsFormat.CURRENT_FORMAT_VERSION);
+
+      int flags = StatsFormat.readIntegerLittleEndian(footer, footerSize - 12);
+      Preconditions.checkState((flags & ~StatsFormat.SUPPORTED_FLAGS) == 0, "Unsupported flags: %s", flags);
+      boolean compressed = isFlagSet(flags, StatsFormat.FLAG_COMPRESSED);
+
+      int reserved = StatsFormat.readIntegerLittleEndian(footer, footerSize - 16);
+      Preconditions.checkState(reserved == 0, "Unexpected reserved bytes value: %s", reserved);

Review Comment:
   removed



-- 
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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4537: Add Indexes and Stats format reader and writer

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4537:
URL: https://github.com/apache/iceberg/pull/4537#discussion_r870903509


##########
core/src/main/java/org/apache/iceberg/stats/StatsReader.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.stats;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap.SimpleEntry;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Stream;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.io.ByteStreams;
+
+public class StatsReader implements Closeable {
+
+  private final long fileLength;
+  private final SeekableInputStream input;
+  private Optional<Integer> knownFooterSize;
+  private Optional<FileMetadata> knownFileMetadata = Optional.empty();
+
+  public StatsReader(InputFile inputFile, Optional<Integer> footerSize) {

Review Comment:
   What is expected for `footerSize`? Should that match `FooterPayloadSize` or be larger by some constant to account for magic and other non-payload bytes?



-- 
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