diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java index 6ce2f31a43..c0532f2961 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/BinaryPlainValuesReader.java @@ -19,36 +19,38 @@ package org.apache.parquet.column.values.plain; import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import org.apache.parquet.bytes.ByteBufferInputStream; -import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.ValuesReader; -import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.io.api.Binary; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Plain encoding reader for BINARY values. + * + *

Reads directly from a {@link ByteBuffer} with {@link ByteOrder#LITTLE_ENDIAN} byte order, + * using {@link ByteBuffer#getInt()} for the 4-byte length prefix instead of 4 individual + * {@code InputStream.read()} calls through {@link org.apache.parquet.bytes.BytesUtils#readIntLittleEndian}. + */ public class BinaryPlainValuesReader extends ValuesReader { private static final Logger LOG = LoggerFactory.getLogger(BinaryPlainValuesReader.class); - private ByteBufferInputStream in; + private ByteBuffer buffer; @Override public Binary readBytes() { - try { - int length = BytesUtils.readIntLittleEndian(in); - return Binary.fromConstantByteBuffer(in.slice(length)); - } catch (IOException | RuntimeException e) { - throw new ParquetDecodingException("could not read bytes at offset " + in.position(), e); - } + int length = buffer.getInt(); + ByteBuffer valueSlice = buffer.slice(); + valueSlice.limit(length); + buffer.position(buffer.position() + length); + return Binary.fromConstantByteBuffer(valueSlice); } @Override public void skip() { - try { - int length = BytesUtils.readIntLittleEndian(in); - in.skipFully(length); - } catch (IOException | RuntimeException e) { - throw new ParquetDecodingException("could not skip bytes at offset " + in.position(), e); - } + int length = buffer.getInt(); + buffer.position(buffer.position() + length); } @Override @@ -57,6 +59,11 @@ public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IO "init from page at offset {} for length {}", stream.position(), (stream.available() - stream.position())); - this.in = stream.remainingStream(); + int available = stream.available(); + if (available > 0) { + this.buffer = stream.slice(available).order(ByteOrder.LITTLE_ENDIAN); + } else { + this.buffer = ByteBuffer.allocate(0).order(ByteOrder.LITTLE_ENDIAN); + } } }