Skip to content

Commit 0e8b821

Browse files
committed
GH-3495: Deprecate LittleEndianDataOutputStream and remove remaining wrapper usages
This is an API cleanup commit with no measurable performance impact; it removes the last two production usages of LittleEndianDataOutputStream so the class can be deprecated. After the previous commit removed LittleEndianDataOutputStream from PlainValuesWriter, two production usages remained: - FixedLenByteArrayPlainValuesWriter wrapped its CapacityByteArrayOutputStream in a LittleEndianDataOutputStream solely to call Binary.writeTo(out) for the fixed-length payload. The fixed-length encoding has no length prefix and the wrapper exposed no LE-specific behavior used here -- Binary.writeTo() only invokes OutputStream.write(byte[], int, int), which the wrapper passes through unchanged. The wrapper has been removed and the writer now writes the binary payload directly to the underlying CapacityByteArrayOutputStream. The wrapper-specific flush() in getBytes() is also gone (CBOS does not buffer). - DeltaLengthByteArrayValuesWriter had the same pattern: a wrapper used only for v.writeTo(out) on the concatenated byte-array payload, with lengths written through a separate DeltaBinaryPackingValuesWriterForInteger. The wrapper has been removed for the same reasons. With no remaining production usages, LittleEndianDataOutputStream is marked @deprecated. The class is retained for binary compatibility (it is part of the public parquet-common API) and will be removed in a future major release. The javadoc directs producers of PLAIN-encoded data to write little-endian values directly into a ByteBuffer with ByteOrder.LITTLE_ENDIAN, which compiles to a single intrinsic store on little-endian architectures and avoids the per-call byte decomposition and virtual dispatch performed by this class.
1 parent 39fc18c commit 0e8b821

3 files changed

Lines changed: 13 additions & 19 deletions

File tree

parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesWriter.java

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import org.apache.parquet.bytes.ByteBufferAllocator;
2323
import org.apache.parquet.bytes.BytesInput;
2424
import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
25-
import org.apache.parquet.bytes.LittleEndianDataOutputStream;
2625
import org.apache.parquet.column.Encoding;
2726
import org.apache.parquet.column.values.ValuesWriter;
2827
import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriter;
@@ -46,11 +45,9 @@ public class DeltaLengthByteArrayValuesWriter extends ValuesWriter {
4645

4746
private ValuesWriter lengthWriter;
4847
private CapacityByteArrayOutputStream arrayOut;
49-
private LittleEndianDataOutputStream out;
5048

5149
public DeltaLengthByteArrayValuesWriter(int initialSize, int pageSize, ByteBufferAllocator allocator) {
5250
arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize, allocator);
53-
out = new LittleEndianDataOutputStream(arrayOut);
5451
lengthWriter = new DeltaBinaryPackingValuesWriterForInteger(
5552
DeltaBinaryPackingValuesWriter.DEFAULT_NUM_BLOCK_VALUES,
5653
DeltaBinaryPackingValuesWriter.DEFAULT_NUM_MINIBLOCKS,
@@ -63,7 +60,7 @@ public DeltaLengthByteArrayValuesWriter(int initialSize, int pageSize, ByteBuffe
6360
public void writeBytes(Binary v) {
6461
try {
6562
lengthWriter.writeInteger(v.length());
66-
v.writeTo(out);
63+
v.writeTo(arrayOut);
6764
} catch (IOException e) {
6865
throw new ParquetEncodingException("could not write bytes", e);
6966
}
@@ -76,11 +73,6 @@ public long getBufferedSize() {
7673

7774
@Override
7875
public BytesInput getBytes() {
79-
try {
80-
out.flush();
81-
} catch (IOException e) {
82-
throw new ParquetEncodingException("could not write page", e);
83-
}
8476
LOG.debug("writing a buffer of size {}", arrayOut.size());
8577
return BytesInput.concat(lengthWriter.getBytes(), BytesInput.from(arrayOut));
8678
}

parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesWriter.java

Lines changed: 1 addition & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import org.apache.parquet.bytes.ByteBufferAllocator;
2323
import org.apache.parquet.bytes.BytesInput;
2424
import org.apache.parquet.bytes.CapacityByteArrayOutputStream;
25-
import org.apache.parquet.bytes.LittleEndianDataOutputStream;
2625
import org.apache.parquet.column.Encoding;
2726
import org.apache.parquet.column.values.ValuesWriter;
2827
import org.apache.parquet.io.ParquetEncodingException;
@@ -37,7 +36,6 @@ public class FixedLenByteArrayPlainValuesWriter extends ValuesWriter {
3736
private static final Logger LOG = LoggerFactory.getLogger(PlainValuesWriter.class);
3837

3938
private CapacityByteArrayOutputStream arrayOut;
40-
private LittleEndianDataOutputStream out;
4139
private int length;
4240
private ByteBufferAllocator allocator;
4341

@@ -46,7 +44,6 @@ public FixedLenByteArrayPlainValuesWriter(
4644
this.length = length;
4745
this.allocator = allocator;
4846
this.arrayOut = new CapacityByteArrayOutputStream(initialSize, pageSize, this.allocator);
49-
this.out = new LittleEndianDataOutputStream(arrayOut);
5047
}
5148

5249
@Override
@@ -56,7 +53,7 @@ public final void writeBytes(Binary v) {
5653
"Fixed Binary size " + v.length() + " does not match field type length " + length);
5754
}
5855
try {
59-
v.writeTo(out);
56+
v.writeTo(arrayOut);
6057
} catch (IOException e) {
6158
throw new ParquetEncodingException("could not write fixed bytes", e);
6259
}
@@ -69,11 +66,6 @@ public long getBufferedSize() {
6966

7067
@Override
7168
public BytesInput getBytes() {
72-
try {
73-
out.flush();
74-
} catch (IOException e) {
75-
throw new ParquetEncodingException("could not write page", e);
76-
}
7769
LOG.debug("writing a buffer of size {}", arrayOut.size());
7870
return BytesInput.from(arrayOut);
7971
}

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

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,8 +24,18 @@
2424
import org.slf4j.LoggerFactory;
2525

2626
/**
27-
* Based on DataOutputStream but in little endian and without the String/char methods
27+
* Based on DataOutputStream but in little endian and without the String/char methods.
28+
*
29+
* @deprecated As of release following the {@link CapacityByteArrayOutputStream#writeInt(int)}
30+
* and {@link CapacityByteArrayOutputStream#writeLong(long)} additions, this class is no
31+
* longer used by Parquet's own writers. Producers of PLAIN-encoded data should write
32+
* little-endian values directly into a {@link java.nio.ByteBuffer} configured with
33+
* {@link java.nio.ByteOrder#LITTLE_ENDIAN}, which compiles to a single intrinsic store on
34+
* little-endian architectures and avoids the per-call byte decomposition and virtual
35+
* dispatch performed here. This class is retained for binary compatibility and will be
36+
* removed in a future release.
2837
*/
38+
@Deprecated
2939
public class LittleEndianDataOutputStream extends OutputStream {
3040

3141
private static final Logger LOG = LoggerFactory.getLogger(LittleEndianDataOutputStream.class);

0 commit comments

Comments
 (0)