Skip to content

Commit c170a47

Browse files
committed
PARQUET-1647: Add logical type FLOAT16
1 parent 9b5a962 commit c170a47

File tree

19 files changed

+1228
-8
lines changed

19 files changed

+1228
-8
lines changed
Lines changed: 43 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,43 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.parquet.column.statistics;
21+
22+
import org.apache.parquet.schema.LogicalTypeAnnotation;
23+
import org.apache.parquet.schema.PrimitiveType;
24+
import org.apache.parquet.schema.Types;
25+
26+
public class Float16Statistics extends BinaryStatistics
27+
{
28+
// A fake type object to be used to generate the proper comparator
29+
private static final PrimitiveType DEFAULT_FAKE_TYPE = Types.optional(PrimitiveType.PrimitiveTypeName.BINARY)
30+
.named("fake_binary_float16_type").withLogicalTypeAnnotation(LogicalTypeAnnotation.float16Type());
31+
32+
/**
33+
* @deprecated will be removed in 2.0.0. Use {@link Statistics#createStats(org.apache.parquet.schema.Type)} instead
34+
*/
35+
@Deprecated
36+
public Float16Statistics() {
37+
this(DEFAULT_FAKE_TYPE);
38+
}
39+
40+
Float16Statistics(PrimitiveType type) {
41+
super(type);
42+
}
43+
}

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

Lines changed: 48 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,12 +21,13 @@
2121
import java.util.Arrays;
2222
import org.apache.parquet.column.UnknownColumnTypeException;
2323
import org.apache.parquet.io.api.Binary;
24+
import org.apache.parquet.schema.LogicalTypeAnnotation;
2425
import org.apache.parquet.schema.PrimitiveComparator;
2526
import org.apache.parquet.schema.PrimitiveStringifier;
2627
import org.apache.parquet.schema.PrimitiveType;
2728
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
2829
import org.apache.parquet.schema.Type;
29-
30+
import org.apache.parquet.type.Float16;
3031

3132
/**
3233
* Statistics class to keep track of statistics in parquet pages and column chunks
@@ -139,6 +140,43 @@ public Statistics<?> build() {
139140
}
140141
}
141142

143+
// Builder for FLOAT16 type to handle special cases of min/max values like NaN, -0.0, and 0.0
144+
private static class Float16Builder extends Builder {
145+
public Float16Builder(PrimitiveType type) {
146+
super(type);
147+
assert type.getPrimitiveTypeName() == PrimitiveTypeName.BINARY;
148+
}
149+
150+
@Override
151+
public Statistics<?> build() {
152+
Float16Statistics stats = (Float16Statistics) super.build();
153+
if (stats.hasNonNullValue()) {
154+
Binary bMin = stats.genericGetMin();
155+
Binary bMax = stats.genericGetMax();
156+
short min = Float16.fromBytesLittleEndian(bMin.getBytes());
157+
short max = Float16.fromBytesLittleEndian(bMax.getBytes());
158+
// Drop min/max values in case of NaN as the sorting order of values is undefined for this case
159+
if (Float16.isNaN(min) || Float16.isNaN(max)) {
160+
bMin = Binary.fromConstantByteArray(Float16.toBytesLittleEndian(Float16.POSITIVE_ZERO));
161+
bMax = Binary.fromConstantByteArray(Float16.toBytesLittleEndian(Float16.POSITIVE_ZERO));
162+
stats.setMinMax(bMin, bMax);
163+
((Statistics<?>) stats).hasNonNullValue = false;
164+
} else {
165+
// Updating min to -0.0 and max to +0.0 to ensure that no 0.0 values would be skipped
166+
if (Float16.equals(min, Float16.POSITIVE_ZERO)) {
167+
bMin = Binary.fromConstantByteArray(Float16.toBytesLittleEndian(Float16.NEGATIVE_ZERO));
168+
stats.setMinMax(bMin, bMax);
169+
}
170+
if (Float16.equals(max, Float16.NEGATIVE_ZERO)) {
171+
bMax = Binary.fromConstantByteArray(Float16.toBytesLittleEndian(Float16.POSITIVE_ZERO));
172+
stats.setMinMax(bMin, bMax);
173+
}
174+
}
175+
}
176+
return stats;
177+
}
178+
}
179+
142180
private final PrimitiveType type;
143181
private final PrimitiveComparator<T> comparator;
144182
private boolean hasNonNullValue;
@@ -207,6 +245,10 @@ public static Statistics<?> createStats(Type type) {
207245
case BINARY:
208246
case INT96:
209247
case FIXED_LEN_BYTE_ARRAY:
248+
LogicalTypeAnnotation logicalTypeAnnotation = type.getLogicalTypeAnnotation();
249+
if (logicalTypeAnnotation instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) {
250+
return new Float16Statistics(primitive);
251+
}
210252
return new BinaryStatistics(primitive);
211253
default:
212254
throw new UnknownColumnTypeException(primitive.getPrimitiveTypeName());
@@ -226,6 +268,11 @@ public static Builder getBuilderForReading(PrimitiveType type) {
226268
return new FloatBuilder(type);
227269
case DOUBLE:
228270
return new DoubleBuilder(type);
271+
case BINARY:
272+
LogicalTypeAnnotation logicalTypeAnnotation = type.getLogicalTypeAnnotation();
273+
if (logicalTypeAnnotation instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) {
274+
return new Float16Builder(type);
275+
}
229276
default:
230277
return new Builder(type);
231278
}

parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java

Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,12 @@ protected LogicalTypeAnnotation fromString(List<String> params) {
141141
protected LogicalTypeAnnotation fromString(List<String> params) {
142142
return IntervalLogicalTypeAnnotation.getInstance();
143143
}
144+
},
145+
FLOAT16 {
146+
@Override
147+
protected LogicalTypeAnnotation fromString(List<String> params) {
148+
return float16Type();
149+
}
144150
};
145151

146152
protected abstract LogicalTypeAnnotation fromString(List<String> params);
@@ -296,6 +302,10 @@ public static UUIDLogicalTypeAnnotation uuidType() {
296302
return UUIDLogicalTypeAnnotation.INSTANCE;
297303
}
298304

305+
public static Float16LogicalTypeAnnotation float16Type() {
306+
return Float16LogicalTypeAnnotation.INSTANCE;
307+
}
308+
299309
public static class StringLogicalTypeAnnotation extends LogicalTypeAnnotation {
300310
private static final StringLogicalTypeAnnotation INSTANCE = new StringLogicalTypeAnnotation();
301311

@@ -901,6 +911,36 @@ PrimitiveStringifier valueStringifier(PrimitiveType primitiveType) {
901911
}
902912
}
903913

914+
public static class Float16LogicalTypeAnnotation extends LogicalTypeAnnotation {
915+
private static final Float16LogicalTypeAnnotation INSTANCE = new Float16LogicalTypeAnnotation();
916+
public static final int BYTES = 2;
917+
918+
private Float16LogicalTypeAnnotation() {
919+
}
920+
921+
@Override
922+
@InterfaceAudience.Private
923+
public OriginalType toOriginalType() {
924+
// No OriginalType for Float16
925+
return null;
926+
}
927+
928+
@Override
929+
public <T> Optional<T> accept(LogicalTypeAnnotationVisitor<T> logicalTypeAnnotationVisitor) {
930+
return logicalTypeAnnotationVisitor.visit(this);
931+
}
932+
933+
@Override
934+
LogicalTypeToken getType() {
935+
return LogicalTypeToken.FLOAT16;
936+
}
937+
938+
@Override
939+
PrimitiveStringifier valueStringifier(PrimitiveType primitiveType) {
940+
return PrimitiveStringifier.FLOAT16_STRINGIFIER;
941+
}
942+
}
943+
904944
// This logical type annotation is implemented to support backward compatibility with ConvertedType.
905945
// The new logical type representation in parquet-format doesn't have any interval type,
906946
// thus this annotation is mapped to UNKNOWN.
@@ -1060,5 +1100,9 @@ default Optional<T> visit(IntervalLogicalTypeAnnotation intervalLogicalType) {
10601100
default Optional<T> visit(MapKeyValueTypeAnnotation mapKeyValueLogicalType) {
10611101
return empty();
10621102
}
1103+
1104+
default Optional<T> visit(Float16LogicalTypeAnnotation float16LogicalType) {
1105+
return empty();
1106+
}
10631107
}
10641108
}

parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,8 @@
2424
import java.nio.ByteBuffer;
2525
import java.util.Comparator;
2626

27+
import org.apache.parquet.type.Float16;
28+
2729
/**
2830
* {@link Comparator} implementation that also supports the comparison of the related primitive type to avoid the
2931
* performance penalty of boxing/unboxing. The {@code compare} methods for the not supported primitive types throw
@@ -276,4 +278,22 @@ public String toString() {
276278
return "BINARY_AS_SIGNED_INTEGER_COMPARATOR";
277279
}
278280
};
281+
282+
/**
283+
* This comparator is for comparing two float16 values represented in 2 bytes binary.
284+
*/
285+
static final PrimitiveComparator<Binary> BINARY_AS_FLOAT16_COMPARATOR = new BinaryComparator() {
286+
287+
@Override
288+
int compareBinary(Binary b1, Binary b2)
289+
{
290+
return Float16.compare(Float16.fromBytesLittleEndian(b1.getBytes()),
291+
Float16.fromBytesLittleEndian(b2.getBytes()));
292+
}
293+
294+
@Override
295+
public String toString() {
296+
return "BINARY_AS_FLOAT16_COMPARATOR";
297+
}
298+
};
279299
}

parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveStringifier.java

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,12 +32,12 @@
3232
import java.time.Instant;
3333
import java.time.ZoneOffset;
3434
import java.time.format.DateTimeFormatter;
35-
import java.util.UUID;
3635
import java.util.concurrent.TimeUnit;
3736

3837
import javax.naming.OperationNotSupportedException;
3938

4039
import org.apache.parquet.io.api.Binary;
40+
import org.apache.parquet.type.Float16;
4141

4242
/**
4343
* Class that provides string representations for the primitive values. These string values are to be used for
@@ -448,4 +448,12 @@ private void appendHex(byte[] array, int offset, int length, StringBuilder build
448448
}
449449
}
450450
};
451+
452+
static final PrimitiveStringifier FLOAT16_STRINGIFIER = new BinaryStringifierBase("FLOAT16_STRINGIFIER") {
453+
454+
@Override
455+
String stringifyNotNull(Binary value) {
456+
return Float16.toFloatString(Float16.fromBytesLittleEndian(value.getBytes()));
457+
}
458+
};
451459
}

parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java

Lines changed: 18 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@
3232
import org.apache.parquet.io.api.PrimitiveConverter;
3333
import org.apache.parquet.io.api.RecordConsumer;
3434
import org.apache.parquet.schema.ColumnOrder.ColumnOrderName;
35-
import org.apache.parquet.schema.LogicalTypeAnnotation.LogicalTypeAnnotationVisitor;
3635
import org.apache.parquet.schema.LogicalTypeAnnotation.UUIDLogicalTypeAnnotation;
3736

3837
import static java.util.Optional.empty;
@@ -261,6 +260,11 @@ public Optional<PrimitiveComparator> visit(LogicalTypeAnnotation.JsonLogicalType
261260
public Optional<PrimitiveComparator> visit(LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) {
262261
return of(PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR);
263262
}
263+
264+
@Override
265+
public Optional<PrimitiveComparator> visit(LogicalTypeAnnotation.Float16LogicalTypeAnnotation float16LogicalType) {
266+
return of(PrimitiveComparator.BINARY_AS_FLOAT16_COMPARATOR);
267+
}
264268
}).orElseThrow(() -> new ShouldNeverHappenException("No comparator logic implemented for BINARY logical type: " + logicalType));
265269
}
266270
},
@@ -390,6 +394,11 @@ public Optional<PrimitiveComparator> visit(LogicalTypeAnnotation.IntervalLogical
390394
public Optional<PrimitiveComparator> visit(UUIDLogicalTypeAnnotation uuidLogicalType) {
391395
return of(PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR);
392396
}
397+
398+
@Override
399+
public Optional<PrimitiveComparator> visit(LogicalTypeAnnotation.Float16LogicalTypeAnnotation float16LogicalType) {
400+
return of(PrimitiveComparator.BINARY_AS_FLOAT16_COMPARATOR);
401+
}
393402
}).orElseThrow(() -> new ShouldNeverHappenException(
394403
"No comparator logic implemented for FIXED_LEN_BYTE_ARRAY logical type: " + logicalType));
395404
}
@@ -564,6 +573,14 @@ public PrimitiveType withId(int id) {
564573
columnOrder);
565574
}
566575

576+
/**
577+
* @param logicalType LogicalTypeAnnotation
578+
* @return a new PrimitiveType with the same fields and a new id null
579+
*/
580+
public PrimitiveType withLogicalTypeAnnotation(LogicalTypeAnnotation logicalType) {
581+
return new PrimitiveType(getRepetition(), primitive, length, getName(), logicalType, getId());
582+
}
583+
567584
/**
568585
* @return the primitive type
569586
*/

parquet-column/src/main/java/org/apache/parquet/schema/Types.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -465,6 +465,11 @@ public Optional<Boolean> visit(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation u
465465
return checkFixedPrimitiveType(LogicalTypeAnnotation.UUIDLogicalTypeAnnotation.BYTES, uuidLogicalType);
466466
}
467467

468+
@Override
469+
public Optional<Boolean> visit(LogicalTypeAnnotation.Float16LogicalTypeAnnotation float16LogicalType) {
470+
return checkFixedPrimitiveType(LogicalTypeAnnotation.Float16LogicalTypeAnnotation.BYTES, float16LogicalType);
471+
}
472+
468473
@Override
469474
public Optional<Boolean> visit(LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimalLogicalType) {
470475
Preconditions.checkState(

parquet-column/src/test/java/org/apache/parquet/schema/TestPrimitiveComparator.java

Lines changed: 33 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,13 +19,16 @@
1919
package org.apache.parquet.schema;
2020

2121
import org.apache.parquet.io.api.Binary;
22+
import org.apache.parquet.type.Float16;
2223
import org.junit.Test;
2324

2425
import java.math.BigInteger;
2526
import java.nio.ByteBuffer;
27+
import java.nio.ByteOrder;
2628
import java.util.ArrayList;
2729
import java.util.List;
2830

31+
import static org.apache.parquet.schema.PrimitiveComparator.BINARY_AS_FLOAT16_COMPARATOR;
2932
import static org.apache.parquet.schema.PrimitiveComparator.BOOLEAN_COMPARATOR;
3033
import static org.apache.parquet.schema.PrimitiveComparator.DOUBLE_COMPARATOR;
3134
import static org.apache.parquet.schema.PrimitiveComparator.FLOAT_COMPARATOR;
@@ -268,6 +271,36 @@ public void testBinaryAsSignedIntegerComparatorWithEquals() {
268271
}
269272
}
270273

274+
@Test
275+
public void testFloat16Comparator() {
276+
short[] valuesInAscendingOrder = {
277+
(short) 0xfc00,
278+
Float16.MIN_VALUE,
279+
-Float16.MAX_VALUE,
280+
(short) 0xc000,
281+
-Float16.MIN_VALUE,
282+
0,
283+
Float16.MIN_VALUE,
284+
(short) 0x7bff,
285+
Float16.MAX_VALUE,
286+
(short) 0x7c00};
287+
288+
for (int i = 0; i < valuesInAscendingOrder.length; ++i) {
289+
for (int j = 0; j < valuesInAscendingOrder.length; ++j) {
290+
short vi = valuesInAscendingOrder[i];
291+
short vj = valuesInAscendingOrder[j];
292+
ByteBuffer bbi = ByteBuffer.allocate(2).order(ByteOrder.LITTLE_ENDIAN);
293+
bbi.putShort(vi).flip();
294+
ByteBuffer bbj = ByteBuffer.allocate(2).order(ByteOrder.LITTLE_ENDIAN);
295+
bbj.putShort(vj).flip();
296+
float fi = Float16.toFloat(vi);
297+
float fj = Float16.toFloat(vj);
298+
assertEquals(Float.compare(fi, fj), BINARY_AS_FLOAT16_COMPARATOR.compare(
299+
Binary.fromConstantByteArray(bbi.array()), Binary.fromConstantByteArray(bbj.array())));
300+
}
301+
}
302+
}
303+
271304
private <T> void testObjectComparator(PrimitiveComparator<T> comparator, T... valuesInAscendingOrder) {
272305
for (int i = 0; i < valuesInAscendingOrder.length; ++i) {
273306
for (int j = 0; j < valuesInAscendingOrder.length; ++j) {

0 commit comments

Comments
 (0)