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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
