Skip to content

Commit 44a6a56

Browse files
committed
GH-3493: Deprecate LittleEndianDataInputStream and migrate last test usage
After GH-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.)
1 parent 36b508d commit 44a6a56

2 files changed

Lines changed: 17 additions & 10 deletions

File tree

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

Lines changed: 15 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,10 +21,24 @@
2121
import java.io.EOFException;
2222
import java.io.IOException;
2323
import java.io.InputStream;
24+
import java.nio.ByteBuffer;
25+
import java.nio.ByteOrder;
2426

2527
/**
26-
* Based on DataInputStream but little endian and without the String/char methods
28+
* Based on DataInputStream but little endian and without the String/char methods.
29+
*
30+
* @deprecated This class has no remaining production usages and is
31+
* significantly slower than reading directly from a {@link ByteBuffer} configured with
32+
* {@link ByteOrder#LITTLE_ENDIAN}. Each {@link #readInt()} performs four virtual
33+
* {@code in.read()} calls and reassembles the value with bit shifts, while
34+
* {@link ByteBuffer#getInt()} on a little-endian buffer is a HotSpot intrinsic that
35+
* compiles to a single unaligned load on x86/ARM. For new code, prefer
36+
* {@link ByteBufferInputStream#slice(int)} followed by
37+
* {@code buffer.order(ByteOrder.LITTLE_ENDIAN)} and {@link ByteBuffer#getInt()} /
38+
* {@link ByteBuffer#getLong()} / {@link ByteBuffer#getFloat()} /
39+
* {@link ByteBuffer#getDouble()}. This class will be removed in a future release.
2740
*/
41+
@Deprecated
2842
public final class LittleEndianDataInputStream extends InputStream {
2943

3044
private final InputStream in;

parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestColumnChunkPageWriteStore.java

Lines changed: 2 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -38,9 +38,8 @@
3838
import static org.mockito.ArgumentMatchers.same;
3939
import static org.mockito.Mockito.inOrder;
4040

41-
import java.io.ByteArrayInputStream;
42-
import java.io.ByteArrayOutputStream;
4341
import java.io.IOException;
42+
import java.nio.ByteOrder;
4443
import java.util.HashMap;
4544
import org.apache.hadoop.conf.Configuration;
4645
import org.apache.hadoop.fs.FileSystem;
@@ -49,7 +48,6 @@
4948
import org.apache.parquet.bytes.BytesInput;
5049
import org.apache.parquet.bytes.DirectByteBufferAllocator;
5150
import org.apache.parquet.bytes.HeapByteBufferAllocator;
52-
import org.apache.parquet.bytes.LittleEndianDataInputStream;
5351
import org.apache.parquet.bytes.TrackingByteBufferAllocator;
5452
import org.apache.parquet.column.ColumnDescriptor;
5553
import org.apache.parquet.column.Encoding;
@@ -249,12 +247,7 @@ public void test(Configuration config, ByteBufferAllocator allocator) throws Exc
249247
}
250248

251249
private int intValue(BytesInput in) throws IOException {
252-
ByteArrayOutputStream baos = new ByteArrayOutputStream();
253-
in.writeAllTo(baos);
254-
LittleEndianDataInputStream os = new LittleEndianDataInputStream(new ByteArrayInputStream(baos.toByteArray()));
255-
int i = os.readInt();
256-
os.close();
257-
return i;
250+
return in.toByteBuffer().order(ByteOrder.LITTLE_ENDIAN).getInt();
258251
}
259252

260253
@Test

0 commit comments

Comments
 (0)