diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/ColumnOrder.java b/parquet-column/src/main/java/org/apache/parquet/schema/ColumnOrder.java index 94a1275569..35ef0ec9d2 100644 --- a/parquet-column/src/main/java/org/apache/parquet/schema/ColumnOrder.java +++ b/parquet-column/src/main/java/org/apache/parquet/schema/ColumnOrder.java @@ -36,11 +36,16 @@ public enum ColumnOrderName { /** * Type defined order meaning that the comparison order of the elements are based on its type. */ - TYPE_DEFINED_ORDER + TYPE_DEFINED_ORDER, + /** + * The column order is defined by the IEEE 754 standard. + */ + IEEE_754_TOTAL_ORDER, } private static final ColumnOrder UNDEFINED_COLUMN_ORDER = new ColumnOrder(ColumnOrderName.UNDEFINED); private static final ColumnOrder TYPE_DEFINED_COLUMN_ORDER = new ColumnOrder(ColumnOrderName.TYPE_DEFINED_ORDER); + private static final ColumnOrder IEEE_754_TOTAL_ORDER = new ColumnOrder(ColumnOrderName.IEEE_754_TOTAL_ORDER); /** * @return a {@link ColumnOrder} instance representing an undefined order @@ -58,6 +63,14 @@ public static ColumnOrder typeDefined() { return TYPE_DEFINED_COLUMN_ORDER; } + /** + * @return a {@link ColumnOrder} instance representing an IEEE 754 total order + * @see ColumnOrderName#IEEE_754_TOTAL_ORDER + */ + public static ColumnOrder ieee754TotalOrder() { + return IEEE_754_TOTAL_ORDER; + } + private final ColumnOrderName columnOrderName; private ColumnOrder(ColumnOrderName columnOrderName) { diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java b/parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java index be98e071f6..6c6433e407 100644 --- a/parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java +++ b/parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java @@ -20,6 +20,7 @@ import static java.util.Arrays.asList; import static java.util.Optional.empty; +import static org.apache.parquet.schema.ColumnOrder.ColumnOrderName.IEEE_754_TOTAL_ORDER; import static org.apache.parquet.schema.ColumnOrder.ColumnOrderName.TYPE_DEFINED_ORDER; import static org.apache.parquet.schema.ColumnOrder.ColumnOrderName.UNDEFINED; import static org.apache.parquet.schema.PrimitiveStringifier.TIMESTAMP_MICROS_STRINGIFIER; @@ -1051,6 +1052,13 @@ LogicalTypeToken getType() { PrimitiveStringifier valueStringifier(PrimitiveType primitiveType) { return PrimitiveStringifier.FLOAT16_STRINGIFIER; } + + @Override + boolean isValidColumnOrder(ColumnOrder columnOrder) { + return columnOrder.getColumnOrderName() == UNDEFINED + || columnOrder.getColumnOrderName() == TYPE_DEFINED_ORDER + || columnOrder.getColumnOrderName() == IEEE_754_TOTAL_ORDER; + } } public static class UnknownLogicalTypeAnnotation extends LogicalTypeAnnotation { diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java index 50c4acd4c9..9d22d25312 100644 --- a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java +++ b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java @@ -293,4 +293,65 @@ public String toString() { return "BINARY_AS_FLOAT16_COMPARATOR"; } }; + + static final PrimitiveComparator FLOAT_IEEE_754_TOTAL_ORDER_COMPARATOR = new PrimitiveComparator() { + @Override + int compareNotNulls(Float o1, Float o2) { + return compare(o1.floatValue(), o2.floatValue()); + } + + @Override + public int compare(float f1, float f2) { + int f1Int = Float.floatToRawIntBits(f1); + int f2Int = Float.floatToRawIntBits(f2); + f1Int ^= ((f1Int >> 31) >>> 1); + f2Int ^= ((f2Int >> 31) >>> 1); + return Integer.compare(f1Int, f2Int); + } + + @Override + public String toString() { + return "FLOAT_IEEE_754_TOTAL_ORDER_COMPARATOR"; + } + }; + + static final PrimitiveComparator DOUBLE_IEEE_754_TOTAL_ORDER_COMPARATOR = + new PrimitiveComparator() { + @Override + int compareNotNulls(Double o1, Double o2) { + return compare(o1.doubleValue(), o2.doubleValue()); + } + + @Override + public int compare(double d1, double d2) { + long d1Long = Double.doubleToRawLongBits(d1); + long d2Long = Double.doubleToRawLongBits(d2); + d1Long ^= ((d1Long >> 63) >>> 1); + d2Long ^= ((d2Long >> 63) >>> 1); + return Long.compare(d1Long, d2Long); + } + + @Override + public String toString() { + return "DOUBLE_IEEE_754_TOTAL_ORDER_COMPARATOR"; + } + }; + + static final PrimitiveComparator BINARY_AS_FLOAT16_IEEE_754_TOTAL_ORDER_COMPARATOR = + new BinaryComparator() { + + @Override + int compareBinary(Binary b1, Binary b2) { + int b1Short = b1.get2BytesLittleEndian(); + int b2Short = b2.get2BytesLittleEndian(); + b1Short ^= ((b1Short >> 15) >>> 1); + b2Short ^= ((b2Short >> 15) >>> 1); + return Integer.compare(b1Short, b2Short); + } + + @Override + public String toString() { + return "BINARY_AS_FLOAT16_IEEE_754_TOTAL_ORDER_COMPARATOR"; + } + }; } diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java index 6beff4da93..4a4fdf2599 100644 --- a/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java +++ b/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveType.java @@ -88,7 +88,7 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { if (logicalType == null) { return PrimitiveComparator.SIGNED_INT64_COMPARATOR; } @@ -146,7 +146,7 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { if (logicalType == null) { return PrimitiveComparator.SIGNED_INT32_COMPARATOR; } @@ -210,7 +210,7 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { return PrimitiveComparator.BOOLEAN_COMPARATOR; } }, @@ -236,7 +236,7 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { if (logicalType == null) { return PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR; } @@ -310,8 +310,10 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { - return PrimitiveComparator.FLOAT_COMPARATOR; + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { + return columnOrder.getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER + ? PrimitiveComparator.FLOAT_IEEE_754_TOTAL_ORDER_COMPARATOR + : PrimitiveComparator.FLOAT_COMPARATOR; } }, DOUBLE("getDouble", Double.TYPE) { @@ -336,8 +338,10 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { - return PrimitiveComparator.DOUBLE_COMPARATOR; + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { + return columnOrder.getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER + ? PrimitiveComparator.DOUBLE_IEEE_754_TOTAL_ORDER_COMPARATOR + : PrimitiveComparator.DOUBLE_COMPARATOR; } }, INT96("getBinary", Binary.class) { @@ -362,7 +366,7 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { return PrimitiveComparator.BINARY_AS_SIGNED_INTEGER_COMPARATOR; } }, @@ -388,11 +392,16 @@ public T convert(PrimitiveTypeNameConverter conve } @Override - PrimitiveComparator comparator(LogicalTypeAnnotation logicalType) { + PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder) { if (logicalType == null) { return PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR; } + if (logicalType.getType() == LogicalTypeAnnotation.LogicalTypeToken.FLOAT16 + && columnOrder.getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER) { + return PrimitiveComparator.UNSIGNED_LEXICOGRAPHICAL_BINARY_COMPARATOR; + } + return logicalType .accept(new LogicalTypeAnnotation.LogicalTypeAnnotationVisitor() { @Override @@ -453,7 +462,7 @@ public abstract void addValueToPrimitiveConverter( public abstract T convert(PrimitiveTypeNameConverter converter) throws E; - abstract PrimitiveComparator comparator(LogicalTypeAnnotation logicalType); + abstract PrimitiveComparator comparator(LogicalTypeAnnotation logicalType, ColumnOrder columnOrder); } private final PrimitiveTypeName primitive; @@ -545,6 +554,12 @@ public PrimitiveType( columnOrder = primitive == PrimitiveTypeName.INT96 || originalType == OriginalType.INTERVAL ? ColumnOrder.undefined() : ColumnOrder.typeDefined(); + } else if (columnOrder.getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER) { + Preconditions.checkArgument( + primitive == PrimitiveTypeName.FLOAT || primitive == PrimitiveTypeName.DOUBLE, + "The column order %s is not supported by type %s", + columnOrder, + primitive); } this.columnOrder = requireValidColumnOrder(columnOrder); } @@ -591,6 +606,17 @@ public PrimitiveType( || logicalTypeAnnotation instanceof LogicalTypeAnnotation.IntervalLogicalTypeAnnotation ? ColumnOrder.undefined() : ColumnOrder.typeDefined(); + } else if (columnOrder.getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER) { + Preconditions.checkArgument( + primitive == PrimitiveTypeName.FLOAT + || primitive == PrimitiveTypeName.DOUBLE + || (logicalTypeAnnotation != null + && logicalTypeAnnotation.getType() + == LogicalTypeAnnotation.LogicalTypeToken.FLOAT16), + "The column order %s is not supported by type %s logical type %s", + columnOrder, + primitive, + logicalTypeAnnotation); } this.columnOrder = requireValidColumnOrder(columnOrder); } @@ -631,6 +657,15 @@ public PrimitiveType withLogicalTypeAnnotation(LogicalTypeAnnotation logicalType return new PrimitiveType(getRepetition(), primitive, length, getName(), logicalType, getId()); } + /** + * @param columnOrder the column order + * @return the same type with the column order set + */ + public Type withColumnOrder(ColumnOrder columnOrder) { + return new PrimitiveType( + getRepetition(), primitive, length, getName(), getLogicalTypeAnnotation(), getId(), columnOrder); + } + /** * @return the primitive type */ @@ -845,7 +880,7 @@ protected Type union(Type toMerge, boolean strict) { */ @SuppressWarnings("unchecked") public PrimitiveComparator comparator() { - return (PrimitiveComparator) getPrimitiveTypeName().comparator(getLogicalTypeAnnotation()); + return (PrimitiveComparator) getPrimitiveTypeName().comparator(getLogicalTypeAnnotation(), columnOrder()); } /** diff --git a/parquet-column/src/test/java/org/apache/parquet/schema/TestPrimitiveComparator.java b/parquet-column/src/test/java/org/apache/parquet/schema/TestPrimitiveComparator.java index d3d1b15bc6..a86a6d6335 100644 --- a/parquet-column/src/test/java/org/apache/parquet/schema/TestPrimitiveComparator.java +++ b/parquet-column/src/test/java/org/apache/parquet/schema/TestPrimitiveComparator.java @@ -19,10 +19,13 @@ package org.apache.parquet.schema; import static org.apache.parquet.schema.PrimitiveComparator.BINARY_AS_FLOAT16_COMPARATOR; +import static org.apache.parquet.schema.PrimitiveComparator.BINARY_AS_FLOAT16_IEEE_754_TOTAL_ORDER_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.BINARY_AS_SIGNED_INTEGER_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.BOOLEAN_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.DOUBLE_COMPARATOR; +import static org.apache.parquet.schema.PrimitiveComparator.DOUBLE_IEEE_754_TOTAL_ORDER_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.FLOAT_COMPARATOR; +import static org.apache.parquet.schema.PrimitiveComparator.FLOAT_IEEE_754_TOTAL_ORDER_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.SIGNED_INT32_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.SIGNED_INT64_COMPARATOR; import static org.apache.parquet.schema.PrimitiveComparator.UNSIGNED_INT32_COMPARATOR; @@ -144,6 +147,22 @@ private void testInt64Comparator(PrimitiveComparator comparator, Long... v checkThrowingUnsupportedException(comparator, Long.TYPE); } + private void testFloatComparator(PrimitiveComparator comparator, Float... valuesInAscendingOrder) { + for (int i = 0; i < valuesInAscendingOrder.length; ++i) { + for (int j = 0; j < valuesInAscendingOrder.length; ++j) { + Float vi = valuesInAscendingOrder[i]; + Float vj = valuesInAscendingOrder[j]; + int exp = i - j; + assertSignumEquals(vi, vj, exp, comparator.compare(vi, vj)); + if (vi != null && vj != null) { + assertSignumEquals(vi, vj, exp, comparator.compare(vi.floatValue(), vj.floatValue())); + } + } + } + + checkThrowingUnsupportedException(comparator, Float.TYPE); + } + @Test public void testFloatComparator() { Float[] valuesInAscendingOrder = { @@ -159,19 +178,46 @@ public void testFloatComparator() { Float.POSITIVE_INFINITY }; + testFloatComparator(FLOAT_COMPARATOR, valuesInAscendingOrder); + } + + @Test + public void testFloatIEEE754TotalOrderComparator() { + Float[] valuesInAscendingOrder = { + null, + Float.intBitsToFloat(0xFFFFFFFF), // -NaN (smallest) + Float.intBitsToFloat(0xFFF00001), // -NaN (largest) + Float.NEGATIVE_INFINITY, + -Float.MAX_VALUE, + -1234.5678F, + -Float.MIN_VALUE, + -0.0F, + 0.0F, + Float.MIN_VALUE, + 1234.5678F, + Float.MAX_VALUE, + Float.POSITIVE_INFINITY, + Float.intBitsToFloat(0x7FF00001), // +NaN (smallest) + Float.intBitsToFloat(0x7FFFFFFF), // +NaN (largest) + }; + + testFloatComparator(FLOAT_IEEE_754_TOTAL_ORDER_COMPARATOR, valuesInAscendingOrder); + } + + private void testDoubleComparator(PrimitiveComparator comparator, Double... valuesInAscendingOrder) { for (int i = 0; i < valuesInAscendingOrder.length; ++i) { for (int j = 0; j < valuesInAscendingOrder.length; ++j) { - Float vi = valuesInAscendingOrder[i]; - Float vj = valuesInAscendingOrder[j]; + Double vi = valuesInAscendingOrder[i]; + Double vj = valuesInAscendingOrder[j]; int exp = i - j; - assertSignumEquals(vi, vj, exp, FLOAT_COMPARATOR.compare(vi, vj)); + assertSignumEquals(vi, vj, exp, comparator.compare(vi, vj)); if (vi != null && vj != null) { - assertSignumEquals(vi, vj, exp, FLOAT_COMPARATOR.compare(vi.floatValue(), vj.floatValue())); + assertSignumEquals(vi, vj, exp, comparator.compare(vi.doubleValue(), vj.doubleValue())); } } } - checkThrowingUnsupportedException(FLOAT_COMPARATOR, Float.TYPE); + checkThrowingUnsupportedException(comparator, Double.TYPE); } @Test @@ -189,19 +235,30 @@ public void testDoubleComparator() { Double.POSITIVE_INFINITY }; - for (int i = 0; i < valuesInAscendingOrder.length; ++i) { - for (int j = 0; j < valuesInAscendingOrder.length; ++j) { - Double vi = valuesInAscendingOrder[i]; - Double vj = valuesInAscendingOrder[j]; - int exp = i - j; - assertSignumEquals(vi, vj, exp, DOUBLE_COMPARATOR.compare(vi, vj)); - if (vi != null && vj != null) { - assertSignumEquals(vi, vj, exp, DOUBLE_COMPARATOR.compare(vi.doubleValue(), vj.doubleValue())); - } - } - } + testDoubleComparator(DOUBLE_COMPARATOR, valuesInAscendingOrder); + } - checkThrowingUnsupportedException(DOUBLE_COMPARATOR, Double.TYPE); + @Test + public void testDoubleIEEE754TotalOrderComparator() { + Double[] valuesInAscendingOrder = { + null, + Double.longBitsToDouble(0xFFFFFFFFFFFFFFFFL), // -NaN (smallest) + Double.longBitsToDouble(0xFFF0000000000001L), // -NaN (largest) + Double.NEGATIVE_INFINITY, + -Double.MAX_VALUE, + -123456.7890123456789, + -Double.MIN_VALUE, + -0.0, + +0.0, + Double.MIN_VALUE, + 123456.7890123456789, + Double.MAX_VALUE, + Double.POSITIVE_INFINITY, + Double.longBitsToDouble(0x7FF0000000000001L), // +NaN (smallest) + Double.longBitsToDouble(0x7FFFFFFFFFFFFFFFL), // +NaN (largest) + }; + + testDoubleComparator(DOUBLE_IEEE_754_TOTAL_ORDER_COMPARATOR, valuesInAscendingOrder); } @Test @@ -301,6 +358,34 @@ public void testFloat16Comparator() { } } + @Test + public void testBinaryAsFloat16IEEE754TotalOrderComparator() { + Binary[] valuesInAscendingOrder = { + null, + Binary.fromConstantByteArray(new byte[] {(byte) 0xff, (byte) 0xff}), // -NaN (smallest) + Binary.fromConstantByteArray(new byte[] {(byte) 0x01, (byte) 0xfc}), // -NaN (largest) + Binary.fromConstantByteArray(new byte[] {0x00, (byte) 0xfc}), // -Infinity + Binary.fromConstantByteArray(new byte[] {0x00, (byte) 0xc0}), // -2.0 + Binary.fromConstantByteArray(new byte[] {(byte) 0x01, (byte) 0x84}), // -6.109476E-5 + Binary.fromConstantByteArray(new byte[] {0x00, (byte) 0x80}), // -0 + Binary.fromConstantByteArray(new byte[] {0x00, 0x00}), // +0 + Binary.fromConstantByteArray(new byte[] {(byte) 0x01, (byte) 0x00}), // 5.9604645E-8 + Binary.fromConstantByteArray(new byte[] {(byte) 0xff, (byte) 0x7b}), // 65504.0 + Binary.fromConstantByteArray(new byte[] {(byte) 0x00, (byte) 0x7c}), // Infinity + Binary.fromConstantByteArray(new byte[] {0x01, 0x7c}), // +NaN (smallest) + Binary.fromConstantByteArray(new byte[] {(byte) 0xff, 0x7f}) // +NaN (largest) + }; + + for (int i = 0; i < valuesInAscendingOrder.length; ++i) { + for (int j = 0; j < valuesInAscendingOrder.length; ++j) { + Binary vi = valuesInAscendingOrder[i]; + Binary vj = valuesInAscendingOrder[j]; + int exp = i - j; + assertSignumEquals(vi, vj, exp, BINARY_AS_FLOAT16_IEEE_754_TOTAL_ORDER_COMPARATOR.compare(vi, vj)); + } + } + } + private void testObjectComparator(PrimitiveComparator comparator, T... valuesInAscendingOrder) { for (int i = 0; i < valuesInAscendingOrder.length; ++i) { for (int j = 0; j < valuesInAscendingOrder.length; ++j) { diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java index d20ac7faeb..27dce8e93c 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java @@ -47,6 +47,7 @@ import org.apache.parquet.CorruptStatistics; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.Preconditions; +import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.EncodingStats; import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.column.statistics.BinaryStatistics; @@ -86,6 +87,7 @@ import org.apache.parquet.format.GeographyType; import org.apache.parquet.format.GeometryType; import org.apache.parquet.format.GeospatialStatistics; +import org.apache.parquet.format.IEEE754TotalOrder; import org.apache.parquet.format.IntType; import org.apache.parquet.format.KeyValue; import org.apache.parquet.format.LogicalType; @@ -143,6 +145,7 @@ public class ParquetMetadataConverter { private static final TypeDefinedOrder TYPE_DEFINED_ORDER = new TypeDefinedOrder(); + private static final IEEE754TotalOrder IEEE_754_TOTAL_ORDER = new IEEE754TotalOrder(); public static final MetadataFilter NO_FILTER = new NoFilter(); public static final MetadataFilter SKIP_ROW_GROUPS = new SkipMetadataFilter(); public static final long MAX_STATS_SIZE = 4096; // limit stats to 4k @@ -249,11 +252,23 @@ public FileMetaData toParquetMetadata( private List getColumnOrders(MessageType schema) { List columnOrders = new ArrayList<>(); - // Currently, only TypeDefinedOrder is supported, so we create a column order for each columns with - // TypeDefinedOrder even if some types (e.g. INT96) have undefined column orders. - for (int i = 0, n = schema.getPaths().size(); i < n; ++i) { + for (ColumnDescriptor column : schema.getColumns()) { ColumnOrder columnOrder = new ColumnOrder(); - columnOrder.setTYPE_ORDER(TYPE_DEFINED_ORDER); + switch (column.getPrimitiveType().columnOrder().getColumnOrderName()) { + case TYPE_DEFINED_ORDER: + columnOrder.setTYPE_ORDER(TYPE_DEFINED_ORDER); + break; + case IEEE_754_TOTAL_ORDER: + columnOrder.setIEEE_754_TOTAL_ORDER(IEEE_754_TOTAL_ORDER); + break; + case UNDEFINED: + // Use TypeDefinedOrder if some types (e.g. INT96) have undefined column orders. + columnOrder.setTYPE_ORDER(TYPE_DEFINED_ORDER); + break; + default: + throw new IllegalArgumentException( + "Unknown column order: " + column.getPrimitiveType().columnOrder()); + } columnOrders.add(columnOrder); } return columnOrders; @@ -860,7 +875,8 @@ private static byte[] tuncateMax(BinaryTruncator truncator, int truncateLength, } private static boolean isMinMaxStatsSupported(PrimitiveType type) { - return type.columnOrder().getColumnOrderName() == ColumnOrderName.TYPE_DEFINED_ORDER; + return type.columnOrder().getColumnOrderName() == ColumnOrderName.TYPE_DEFINED_ORDER + || type.columnOrder().getColumnOrderName() == ColumnOrderName.IEEE_754_TOTAL_ORDER; } /** @@ -2051,6 +2067,9 @@ private static org.apache.parquet.schema.ColumnOrder fromParquetColumnOrder(Colu if (columnOrder.isSetTYPE_ORDER()) { return org.apache.parquet.schema.ColumnOrder.typeDefined(); } + if (columnOrder.isSetIEEE_754_TOTAL_ORDER()) { + return org.apache.parquet.schema.ColumnOrder.ieee754TotalOrder(); + } // The column order is not yet supported by this API return org.apache.parquet.schema.ColumnOrder.undefined(); } diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java index 2529f06ada..d1dcd2a54f 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java @@ -1961,4 +1961,37 @@ public void testEdgeInterpolationAlgorithmConversion() { assertNull(ParquetMetadataConverter.fromParquetEdgeInterpolationAlgorithm(null)); assertNull(ParquetMetadataConverter.toParquetEdgeInterpolationAlgorithm(null)); } + + @Test + public void testIEEE754TotalOrderColumnOrder() throws IOException { + MessageType schema = Types.buildMessage() + .required(PrimitiveTypeName.FLOAT) + .columnOrder(ColumnOrder.ieee754TotalOrder()) + .named("float_ieee754") + .required(PrimitiveTypeName.DOUBLE) + .columnOrder(ColumnOrder.ieee754TotalOrder()) + .named("double_ieee754") + .named("Message"); + + org.apache.parquet.hadoop.metadata.FileMetaData fileMetaData = + new org.apache.parquet.hadoop.metadata.FileMetaData(schema, new HashMap(), null); + ParquetMetadata metadata = new ParquetMetadata(fileMetaData, new ArrayList()); + ParquetMetadataConverter converter = new ParquetMetadataConverter(); + FileMetaData formatMetadata = converter.toParquetMetadata(1, metadata); + + List columnOrders = formatMetadata.getColumn_orders(); + assertEquals(2, columnOrders.size()); + for (org.apache.parquet.format.ColumnOrder columnOrder : columnOrders) { + assertTrue(columnOrder.isSetIEEE_754_TOTAL_ORDER()); + } + + MessageType resultSchema = + converter.fromParquetMetadata(formatMetadata).getFileMetaData().getSchema(); + assertEquals( + ColumnOrder.ieee754TotalOrder(), + resultSchema.getType("float_ieee754").asPrimitiveType().columnOrder()); + assertEquals( + ColumnOrder.ieee754TotalOrder(), + resultSchema.getType("double_ieee754").asPrimitiveType().columnOrder()); + } } diff --git a/pom.xml b/pom.xml index 75050c4453..d77ca49976 100644 --- a/pom.xml +++ b/pom.xml @@ -85,7 +85,7 @@ shaded.parquet 3.3.0 - 2.11.0 + 2.12.0-SNAPSHOT 1.15.1 thrift ${thrift.executable}