GH-3493: Optimize PlainValuesReader with direct ByteBuffer reads#3494
Open
iemejia wants to merge 2 commits intoapache:masterfrom
Open
GH-3493: Optimize PlainValuesReader with direct ByteBuffer reads#3494iemejia wants to merge 2 commits intoapache:masterfrom
iemejia wants to merge 2 commits intoapache:masterfrom
Conversation
arouel
approved these changes
Apr 19, 2026
Replace the LittleEndianDataInputStream wrapper with direct ByteBuffer
access using LITTLE_ENDIAN byte order in PlainValuesReader. Each
read{Integer,Long,Float,Double}() previously dispatched through 4
in.read() calls per value and assembled the result with manual bit
shifts; it now compiles to a single ByteBuffer get*() JVM intrinsic.
In initFromPage, the page data is obtained as a single contiguous
ByteBuffer via ByteBufferInputStream.slice(available). The
ByteBufferInputStream.slice() method handles both single-buffer
(zero-copy view) and multi-buffer (copy into contiguous buffer) cases
transparently. In practice page data is almost always a single
contiguous buffer.
Benchmark (IntEncodingBenchmark.decodePlain, 100,000 INT32 values per
invocation):
Pattern Before (ops/s) After (ops/s) Speedup
SEQUENTIAL 92,918,297 1,143,149,235 12.3x
RANDOM 92,126,888 1,147,547,093 12.5x
LOW_CARDINALITY 93,005,451 1,142,666,760 12.3x
HIGH_CARDINALITY 93,312,596 1,144,681,876 12.3x
The improvement is consistent regardless of data distribution because
the bottleneck was entirely in the dispatch overhead. All four numeric
plain reader types (int, long, float, double) benefit equally.
All 573 parquet-column tests pass.
… test usage After apacheGH-3493 replaced the only production usage of LittleEndianDataInputStream in PlainValuesReader with direct ByteBuffer reads, the class has no remaining production callers. Mark it @deprecated and document the faster alternative. Migrate the only remaining usage in TestColumnChunkPageWriteStore.intValue() to ByteBuffer.getInt() with LITTLE_ENDIAN order, reading directly from BytesInput.toByteBuffer() instead of round-tripping through a ByteArrayOutputStream + ByteArrayInputStream + LittleEndianDataInputStream. Per-call readInt() on the deprecated class performs 4 virtual in.read() dispatches and manually reassembles the value with bit shifts. The ByteBuffer.getInt() replacement is a HotSpot intrinsic that compiles to a single unaligned load on x86/ARM. The class is left in place (only @deprecated) for source/binary compatibility of any downstream code that may still reference it. It can be removed in a future major release. All 308 parquet-common, 573 parquet-column, and TestColumnChunkPageWriteStore column-order tests pass. (The two pre-existing JDK Hadoop getSubject failures in TestColumnChunkPageWriteStore are unrelated to this change.)
44a6a56 to
9c06a94
Compare
This was referenced Apr 19, 2026
Optimize ByteStreamSplitValuesWriter: remove per-value allocation and batch single-byte writes
#3503
Open
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Rationale for this change
Closes #3493.
PlainValuesReader(used for PLAIN-encoded INT32, INT64, FLOAT, and DOUBLE columns, and for decoding the dictionary page of every dictionary-encoded numeric column) currently reads each value through aLittleEndianDataInputStreamwrapper around aByteBufferInputStream. Per value,readInt()performs 4 separate virtualin.read()calls and reassembles the result with bit shifts. TheLittleEndianDataInputStream.readInt()method itself carries a TODO comment from years ago suggesting exactly this kind of replacement.What changes are included in this PR?
Commit 1 —
Optimize PlainValuesReader with direct ByteBuffer readsIn
PlainValuesReader.initFromPage(), obtain the page data as a single contiguousByteBufferviastream.slice(stream.available())withByteOrder.LITTLE_ENDIAN, and call the correspondingByteBufferaccessor directly per value:readInteger()→buffer.getInt()readLong()→buffer.getLong()readFloat()→buffer.getFloat()readDouble()→buffer.getDouble()skip(n)→buffer.position(buffer.position() + n * typeSize)ByteBuffer.getInt()with the appropriate byte order is a HotSpot intrinsic that compiles to a single unaligned load instruction on x86/ARM — no virtual dispatch, no per-byte assembly, no checkedIOExceptionon the per-value path.ByteBufferInputStream.slice()already handles both single-buffer (zero-copy view) and multi-buffer (single contiguous copy) cases transparently.Commit 2 —
Deprecate LittleEndianDataInputStream and migrate last test usageAfter commit 1,
LittleEndianDataInputStreamhas no remaining production usages. This commit:@Deprecatedand detailed javadoc pointing to the fasterByteBuffer+LITTLE_ENDIANalternativeTestColumnChunkPageWriteStore.intValue()toBytesInput.toByteBuffer().order(LITTLE_ENDIAN).getInt(), eliminating aBytesInput → ByteArrayOutputStream → ByteArrayInputStream → LittleEndianDataInputStreamround-tripThe class is left in place (only
@Deprecated) for source/binary compatibility of any downstream code that may still reference it. It can be removed in a future major release.Benchmark results
IntEncodingBenchmark.decodePlain(100,000 INT32 values per invocation, JMH-wi 3 -i 5 -f 1):The speedup is consistent across data patterns because the bottleneck is entirely in the per-value dispatch overhead, not the data itself. All four numeric plain reader types (int, long, float, double) benefit equally.
Are these changes tested?
Yes. All 573
parquet-columnand 308parquet-commontests pass. The migratedTestColumnChunkPageWriteStore.testColumnOrderV1test passes; the two pre-existinggetSubjectfailures inTestColumnChunkPageWriteStoreon JDK 18+ are unrelated and reproduce onmasterwithout these changes.Are there any user-facing changes?
LittleEndianDataInputStreamis now@Deprecated. No behavioral or binary-compatibility changes for existing callers.