Skip to content

Commit 860485d

Browse files
committed
GH-3495: 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 trips through the underlying stream's bookkeeping (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. Although this PR also deprecates the class, the optimization is worth landing for any external Parquet-format producer still using it until they migrate to direct ByteBuffer writes. The change is minimal (~10 lines) and obviously correct (matches the existing writeLong pattern). Benchmark (IntEncodingBenchmark.encodePlain when routed through LittleEndianDataOutputStream): ~20.9M -> ~28.2M ops/s (+35%) Validation: parquet-common 308 tests pass. Built with -Dspotless.check.skip=true -Drat.skip=true -Djapicmp.skip=true.
1 parent 0e8b821 commit 860485d

1 file changed

Lines changed: 9 additions & 11 deletions

File tree

parquet-common/src/main/java/org/apache/parquet/bytes/LittleEndianDataOutputStream.java

Lines changed: 9 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ public class LittleEndianDataOutputStream extends OutputStream {
4040

4141
private static final Logger LOG = LoggerFactory.getLogger(LittleEndianDataOutputStream.class);
4242
private final OutputStream out;
43+
private byte writeBuffer[] = new byte[8];
4344

4445
/**
4546
* Creates a new data output stream to write data to the specified
@@ -140,8 +141,9 @@ public final void writeByte(int v) throws IOException {
140141
* @see java.io.FilterOutputStream#out
141142
*/
142143
public final void writeShort(int v) throws IOException {
143-
out.write((v >>> 0) & 0xFF);
144-
out.write((v >>> 8) & 0xFF);
144+
writeBuffer[0] = (byte) (v >>> 0);
145+
writeBuffer[1] = (byte) (v >>> 8);
146+
out.write(writeBuffer, 0, 2);
145147
}
146148

147149
/**
@@ -154,17 +156,13 @@ public final void writeShort(int v) throws IOException {
154156
* @see java.io.FilterOutputStream#out
155157
*/
156158
public final void writeInt(int v) throws IOException {
157-
// TODO: see note in LittleEndianDataInputStream: maybe faster
158-
// to use Integer.reverseBytes() and then writeInt, or a ByteBuffer
159-
// approach
160-
out.write((v >>> 0) & 0xFF);
161-
out.write((v >>> 8) & 0xFF);
162-
out.write((v >>> 16) & 0xFF);
163-
out.write((v >>> 24) & 0xFF);
159+
writeBuffer[0] = (byte) (v >>> 0);
160+
writeBuffer[1] = (byte) (v >>> 8);
161+
writeBuffer[2] = (byte) (v >>> 16);
162+
writeBuffer[3] = (byte) (v >>> 24);
163+
out.write(writeBuffer, 0, 4);
164164
}
165165

166-
private byte writeBuffer[] = new byte[8];
167-
168166
/**
169167
* Writes a <code>long</code> to the underlying output stream as eight
170168
* bytes, low byte first. In no exception is thrown, the counter

0 commit comments

Comments
 (0)