Skip to content

Commit 0d127a7

Browse files
author
asingh
committed
Add unmodfied and Reused versions for creating a Binary. Add copy() to Binary.
1 parent b4e2950 commit 0d127a7

24 files changed

Lines changed: 110 additions & 61 deletions

File tree

parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -252,9 +252,9 @@ private void writeValue(Type type, Schema avroSchema, Object value) {
252252
recordConsumer.addDouble(((Number) value).doubleValue());
253253
} else if (avroType.equals(Schema.Type.BYTES)) {
254254
if (value instanceof byte[]) {
255-
recordConsumer.addBinary(Binary.fromByteArray((byte[]) value));
255+
recordConsumer.addBinary(Binary.fromUnmodifiedByteArray((byte[]) value));
256256
} else {
257-
recordConsumer.addBinary(Binary.fromByteBuffer((ByteBuffer) value));
257+
recordConsumer.addBinary(Binary.fromUnmodifiedByteBuffer((ByteBuffer) value));
258258
}
259259
} else if (avroType.equals(Schema.Type.STRING)) {
260260
recordConsumer.addBinary(fromAvroString(value));
@@ -269,14 +269,14 @@ private void writeValue(Type type, Schema avroSchema, Object value) {
269269
} else if (avroType.equals(Schema.Type.UNION)) {
270270
writeUnion(type.asGroupType(), nonNullAvroSchema, value);
271271
} else if (avroType.equals(Schema.Type.FIXED)) {
272-
recordConsumer.addBinary(Binary.fromByteArray(((GenericFixed) value).bytes()));
272+
recordConsumer.addBinary(Binary.fromUnmodifiedByteArray(((GenericFixed) value).bytes()));
273273
}
274274
}
275275

276276
private Binary fromAvroString(Object value) {
277277
if (value instanceof Utf8) {
278278
Utf8 utf8 = (Utf8) value;
279-
return Binary.fromByteArray(utf8.getBytes(), 0, utf8.getByteLength());
279+
return Binary.fromUnmodifiedByteArray(utf8.getBytes(), 0, utf8.getByteLength());
280280
}
281281
return Binary.fromString(value.toString());
282282
}

parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -362,7 +362,8 @@ public void write(Map<String, Object> record) {
362362
recordConsumer.endField("mydouble", index++);
363363

364364
recordConsumer.startField("mybytes", index);
365-
recordConsumer.addBinary(Binary.fromByteBuffer((ByteBuffer) record.get("mybytes")));
365+
recordConsumer.addBinary(
366+
Binary.fromUnmodifiedByteBuffer((ByteBuffer) record.get("mybytes")));
366367
recordConsumer.endField("mybytes", index++);
367368

368369
recordConsumer.startField("mystring", index);
@@ -457,7 +458,7 @@ public void write(Map<String, Object> record) {
457458
recordConsumer.endField("mymap", index++);
458459

459460
recordConsumer.startField("myfixed", index);
460-
recordConsumer.addBinary(Binary.fromByteArray((byte[]) record.get("myfixed")));
461+
recordConsumer.addBinary(Binary.fromUnmodifiedByteArray((byte[]) record.get("myfixed")));
461462
recordConsumer.endField("myfixed", index++);
462463

463464
recordConsumer.endMessage();

parquet-benchmarks/src/main/java/org/apache/parquet/benchmarks/DataGenerator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ public void generateData(Path outFile, Configuration configuration, ParquetPrope
107107
.append("float_field", 1.0f)
108108
.append("double_field", 2.0d)
109109
.append("flba_field", new String(chars))
110-
.append("int96_field", Binary.fromByteArray(new byte[12]))
110+
.append("int96_field", Binary.fromUnmodifiedByteArray(new byte[12]))
111111
);
112112
}
113113
writer.close();

parquet-column/src/main/java/org/apache/parquet/column/statistics/BinaryStatistics.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -46,8 +46,8 @@ public void mergeStatisticsMinMax(Statistics stats) {
4646

4747
@Override
4848
public void setMinMaxFromBytes(byte[] minBytes, byte[] maxBytes) {
49-
max = Binary.fromByteArray(maxBytes);
50-
min = Binary.fromByteArray(minBytes);
49+
max = Binary.fromUnmodifiedByteArray(maxBytes);
50+
min = Binary.fromUnmodifiedByteArray(minBytes);
5151
this.markAsNotEmpty();
5252
}
5353

@@ -72,13 +72,13 @@ else if (!this.isEmpty())
7272
}
7373

7474
public void updateStats(Binary min_value, Binary max_value) {
75-
if (min.compareTo(min_value) > 0) { min = Binary.fromByteArray(min_value.getBytes()); }
76-
if (max.compareTo(max_value) < 0) { max = Binary.fromByteArray(max_value.getBytes()); }
75+
if (min.compareTo(min_value) > 0) { min = min_value.copy(); }
76+
if (max.compareTo(max_value) < 0) { max = max_value.copy(); }
7777
}
7878

7979
public void initializeStats(Binary min_value, Binary max_value) {
80-
min = Binary.fromByteArray(min_value.getBytes());
81-
max = Binary.fromByteArray(max_value.getBytes());
80+
min = min_value.copy();
81+
max = max_value.copy();
8282
this.markAsNotEmpty();
8383
}
8484

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ public Binary readBytes() {
5959
int length = lengthReader.readInteger();
6060
int start = offset;
6161
offset = start + length;
62-
return Binary.fromByteArray(in, start, length);
62+
return Binary.fromUnmodifiedByteArray(in, start, length);
6363
}
6464

6565
@Override

parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayReader.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ public class DeltaByteArrayReader extends ValuesReader {
4040
public DeltaByteArrayReader() {
4141
this.prefixLengthReader = new DeltaBinaryPackingValuesReader();
4242
this.suffixReader = new DeltaLengthByteArrayValuesReader();
43-
this.previous = Binary.fromByteArray(new byte[0]);
43+
this.previous = Binary.fromUnmodifiedByteArray(new byte[0]);
4444
}
4545

4646
@Override
@@ -69,7 +69,7 @@ public Binary readBytes() {
6969
byte[] out = new byte[length];
7070
System.arraycopy(previous.getBytesUnsafe(), 0, out, 0, prefixLength);
7171
System.arraycopy(suffix.getBytesUnsafe(), 0, out, prefixLength, suffix.length());
72-
previous = Binary.fromByteArray(out);
72+
previous = Binary.fromUnmodifiedByteArray(out);
7373
} else {
7474
previous = suffix;
7575
}

parquet-column/src/main/java/org/apache/parquet/column/values/deltastrings/DeltaByteArrayWriter.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -87,7 +87,7 @@ public void writeBytes(Binary v) {
8787
int length = previous.length < vb.length ? previous.length : vb.length;
8888
for(i = 0; (i < length) && (previous[i] == vb[i]); i++);
8989
prefixLengthWriter.writeInteger(i);
90-
suffixWriter.writeBytes(Binary.fromByteArray(vb, i, vb.length - i));
90+
suffixWriter.writeBytes(Binary.fromUnmodifiedByteArray(vb, i, vb.length - i));
9191
previous = vb;
9292
}
9393
}

parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/DictionaryValuesWriter.java

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -235,7 +235,7 @@ public void writeBytes(Binary v) {
235235
int id = binaryDictionaryContent.getInt(v);
236236
if (id == -1) {
237237
id = binaryDictionaryContent.size();
238-
binaryDictionaryContent.put(copy(v), id);
238+
binaryDictionaryContent.put(v.copy(), id);
239239
// length as int (4 bytes) + actual bytes
240240
dictionaryByteSize += 4 + v.length();
241241
}
@@ -283,10 +283,6 @@ public void fallBackDictionaryEncodedData(ValuesWriter writer) {
283283
writer.writeBytes(reverseDictionary[id]);
284284
}
285285
}
286-
287-
protected static Binary copy(Binary binary) {
288-
return Binary.fromByteArray(binary.getBytes());
289-
}
290286
}
291287

292288
/**
@@ -310,7 +306,7 @@ public void writeBytes(Binary value) {
310306
int id = binaryDictionaryContent.getInt(value);
311307
if (id == -1) {
312308
id = binaryDictionaryContent.size();
313-
binaryDictionaryContent.put(copy(value), id);
309+
binaryDictionaryContent.put(value.copy(), id);
314310
dictionaryByteSize += length;
315311
}
316312
encodedValues.add(id);

parquet-column/src/main/java/org/apache/parquet/column/values/dictionary/PlainValuesDictionary.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,7 @@ public PlainBinaryDictionary(DictionaryPage dictionaryPage, Integer length) thro
9696
// read the length
9797
offset += 4;
9898
// wrap the content in a binary
99-
binaryDictionaryContent[i] = Binary.fromByteArray(dictionaryBytes, offset, len);
99+
binaryDictionaryContent[i] = Binary.fromUnmodifiedByteArray(dictionaryBytes, offset, len);
100100
// increment to the next value
101101
offset += len;
102102
}
@@ -106,7 +106,7 @@ public PlainBinaryDictionary(DictionaryPage dictionaryPage, Integer length) thro
106106
"Invalid byte array length: " + length);
107107
for (int i = 0; i < binaryDictionaryContent.length; i++) {
108108
// wrap the content in a Binary
109-
binaryDictionaryContent[i] = Binary.fromByteArray(
109+
binaryDictionaryContent[i] = Binary.fromUnmodifiedByteArray(
110110
dictionaryBytes, offset, length);
111111
// increment to the next value
112112
offset += length;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ public Binary readBytes() {
3939
int length = BytesUtils.readIntLittleEndian(in, offset);
4040
int start = offset + 4;
4141
offset = start + length;
42-
return Binary.fromByteArray(in, start, length);
42+
return Binary.fromUnmodifiedByteArray(in, start, length);
4343
} catch (IOException e) {
4444
throw new ParquetDecodingException("could not read bytes at offset " + offset, e);
4545
} catch (RuntimeException e) {

0 commit comments

Comments
 (0)