From 785ad34664f2c45cdb26d25603240a3e9f3725d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 21 Apr 2026 16:48:00 +0000 Subject: [PATCH] GH-3518: Bulk write in LittleEndianDataOutputStream.writeInt/writeShort Replace per-byte out.write(int) calls with a single out.write(byte[], 0, N) using the existing writeBuffer[] field, matching the pattern already used by writeLong. For writeInt this collapses 4 bookkeeping trips through the underlying stream (hasRemaining check, Math.addExact, slab-grow check, single-byte store) into 1. Resolves the long-standing TODO comment in writeInt that flagged this as a potential improvement. Benchmark (IntEncodingBenchmark.encodePlain when routed through LittleEndianDataOutputStream): ~20.9M -> ~28.2M ops/s (+35%) Note: PR #3496 deprecates this class because Parquet's own writers no longer use it. This change benefits external Parquet-format producers that still use the class until they migrate. Validation: parquet-common 308 tests pass. Built with -Dspotless.check.skip=true -Drat.skip=true -Djapicmp.skip=true. --- .../bytes/LittleEndianDataOutputStream.java | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/parquet-common/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java b/parquet-common/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java index ef6c71bc86..baf7109836 100644 --- a/parquet-common/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java +++ b/parquet-common/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java @@ -30,6 +30,7 @@ public class LittleEndianDataOutputStream extends OutputStream { private static final Logger LOG = LoggerFactory.getLogger(LittleEndianDataOutputStream.class); private final OutputStream out; + private byte writeBuffer[] = new byte[8]; /** * Creates a new data output stream to write data to the specified @@ -130,8 +131,9 @@ public final void writeByte(int v) throws IOException { * @see java.io.FilterOutputStream#out */ public final void writeShort(int v) throws IOException { - out.write((v >>> 0) & 0xFF); - out.write((v >>> 8) & 0xFF); + writeBuffer[0] = (byte) (v >>> 0); + writeBuffer[1] = (byte) (v >>> 8); + out.write(writeBuffer, 0, 2); } /** @@ -144,17 +146,13 @@ public final void writeShort(int v) throws IOException { * @see java.io.FilterOutputStream#out */ public final void writeInt(int v) throws IOException { - // TODO: see note in LittleEndianDataInputStream: maybe faster - // to use Integer.reverseBytes() and then writeInt, or a ByteBuffer - // approach - out.write((v >>> 0) & 0xFF); - out.write((v >>> 8) & 0xFF); - out.write((v >>> 16) & 0xFF); - out.write((v >>> 24) & 0xFF); + writeBuffer[0] = (byte) (v >>> 0); + writeBuffer[1] = (byte) (v >>> 8); + writeBuffer[2] = (byte) (v >>> 16); + writeBuffer[3] = (byte) (v >>> 24); + out.write(writeBuffer, 0, 4); } - private byte writeBuffer[] = new byte[8]; - /** * Writes a long to the underlying output stream as eight * bytes, low byte first. In no exception is thrown, the counter