diff --git a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java index 7faa6e62b3..49007f925d 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -19,8 +19,11 @@ package org.apache.comet.parquet; +import java.util.Map; + public class ParquetColumnSpec { + private final int fieldId; private final String[] path; private final String physicalType; private final int typeLength; @@ -28,19 +31,33 @@ public class ParquetColumnSpec { private final int maxDefinitionLevel; private final int maxRepetitionLevel; + // Logical type info + private String logicalTypeName; + private Map logicalTypeParams; + public ParquetColumnSpec( + int fieldId, String[] path, String physicalType, int typeLength, boolean isRepeated, int maxDefinitionLevel, - int maxRepetitionLevel) { + int maxRepetitionLevel, + String logicalTypeName, + Map logicalTypeParams) { + this.fieldId = fieldId; this.path = path; this.physicalType = physicalType; this.typeLength = typeLength; this.isRepeated = isRepeated; this.maxDefinitionLevel = maxDefinitionLevel; this.maxRepetitionLevel = maxRepetitionLevel; + this.logicalTypeName = logicalTypeName; + this.logicalTypeParams = logicalTypeParams; + } + + public int getFieldId() { + return fieldId; } public String[] getPath() { @@ -66,4 +83,12 @@ public int getMaxRepetitionLevel() { public int getMaxDefinitionLevel() { return maxDefinitionLevel; } + + public String getLogicalTypeName() { + return logicalTypeName; + } + + public Map getLogicalTypeParams() { + return logicalTypeParams; + } } diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java index d64ab371a3..fc7d1ab871 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -21,9 +21,9 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; import org.apache.spark.sql.types.*; import org.apache.comet.CometSchemaImporter; @@ -290,15 +290,154 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe } String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; + // Reconstruct the logical type from parameters + LogicalTypeAnnotation logicalType = null; + if (columnSpec.getLogicalTypeName() != null) { + logicalType = + reconstructLogicalType( + columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); + } PrimitiveType primitiveType; if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { - primitiveType = new PrimitiveType(repetition, primType, columnSpec.getTypeLength(), name); + primitiveType = + Types.primitive(primType, repetition) + .length(columnSpec.getTypeLength()) + .as(logicalType) + .id(columnSpec.getFieldId()) + .named(name); } else { - primitiveType = new PrimitiveType(repetition, primType, name); + primitiveType = + Types.primitive(primType, repetition) + .as(logicalType) + .id(columnSpec.getFieldId()) + .named(name); } - MessageType schema = new MessageType("root", primitiveType); - return schema.getColumnDescription(columnSpec.getPath()); + return new ColumnDescriptor( + columnSpec.getPath(), + primitiveType, + columnSpec.getMaxRepetitionLevel(), + columnSpec.getMaxDefinitionLevel()); + } + + private static LogicalTypeAnnotation reconstructLogicalType( + String logicalTypeName, java.util.Map params) { + + switch (logicalTypeName) { + // MAP + case "MapLogicalTypeAnnotation": + return LogicalTypeAnnotation.mapType(); + + // LIST + case "ListLogicalTypeAnnotation": + return LogicalTypeAnnotation.listType(); + + // STRING + case "StringLogicalTypeAnnotation": + return LogicalTypeAnnotation.stringType(); + + // MAP_KEY_VALUE + case "MapKeyValueLogicalTypeAnnotation": + return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); + + // ENUM + case "EnumLogicalTypeAnnotation": + return LogicalTypeAnnotation.enumType(); + + // DECIMAL + case "DecimalLogicalTypeAnnotation": + if (!params.containsKey("scale") || !params.containsKey("precision")) { + throw new IllegalArgumentException( + "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); + } + int scale = Integer.parseInt(params.get("scale")); + int precision = Integer.parseInt(params.get("precision")); + return LogicalTypeAnnotation.decimalType(scale, precision); + + // DATE + case "DateLogicalTypeAnnotation": + return LogicalTypeAnnotation.dateType(); + + // TIME + case "TimeLogicalTypeAnnotation": + if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { + throw new IllegalArgumentException( + "Missing required parameters for TimeLogicalTypeAnnotation: " + params); + } + + boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); + String timeUnitStr = params.get("unit"); + + LogicalTypeAnnotation.TimeUnit timeUnit; + switch (timeUnitStr) { + case "MILLIS": + timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; + break; + case "MICROS": + timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; + break; + case "NANOS": + timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; + break; + default: + throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); + } + return LogicalTypeAnnotation.timeType(isUTC, timeUnit); + + // TIMESTAMP + case "TimestampLogicalTypeAnnotation": + if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { + throw new IllegalArgumentException( + "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); + } + boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); + String unitStr = params.get("unit"); + + LogicalTypeAnnotation.TimeUnit unit; + switch (unitStr) { + case "MILLIS": + unit = LogicalTypeAnnotation.TimeUnit.MILLIS; + break; + case "MICROS": + unit = LogicalTypeAnnotation.TimeUnit.MICROS; + break; + case "NANOS": + unit = LogicalTypeAnnotation.TimeUnit.NANOS; + break; + default: + throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); + } + return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); + + // INTEGER + case "IntLogicalTypeAnnotation": + if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { + throw new IllegalArgumentException( + "Missing required parameters for IntLogicalTypeAnnotation: " + params); + } + boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); + int bitWidth = Integer.parseInt(params.get("bitWidth")); + return LogicalTypeAnnotation.intType(bitWidth, isSigned); + + // JSON + case "JsonLogicalTypeAnnotation": + return LogicalTypeAnnotation.jsonType(); + + // BSON + case "BsonLogicalTypeAnnotation": + return LogicalTypeAnnotation.bsonType(); + + // UUID + case "UUIDLogicalTypeAnnotation": + return LogicalTypeAnnotation.uuidType(); + + // INTERVAL + case "IntervalLogicalTypeAnnotation": + return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); + } } } diff --git a/common/src/test/java/org/apache/comet/parquet/TestUtils.java b/common/src/test/java/org/apache/comet/parquet/TestUtils.java new file mode 100644 index 0000000000..6daa9a254b --- /dev/null +++ b/common/src/test/java/org/apache/comet/parquet/TestUtils.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.parquet; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; + +import static org.junit.Assert.*; + +public class TestUtils { + + @Test + public void testBuildColumnDescriptorWithTimestamp() { + Map params = new HashMap<>(); + params.put("isAdjustedToUTC", "true"); + params.put("unit", "MICROS"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 10, + new String[] {"event_time"}, + "INT64", + 0, + false, + 0, + 0, + "TimestampLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + assertNotNull(descriptor); + + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT64, primitiveType.getPrimitiveTypeName()); + assertTrue( + primitiveType.getLogicalTypeAnnotation() + instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation); + + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation ts = + (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) + primitiveType.getLogicalTypeAnnotation(); + assertTrue(ts.isAdjustedToUTC()); + assertEquals(LogicalTypeAnnotation.TimeUnit.MICROS, ts.getUnit()); + } + + @Test + public void testBuildColumnDescriptorWithDecimal() { + Map params = new HashMap<>(); + params.put("precision", "10"); + params.put("scale", "2"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 11, + new String[] {"price"}, + "FIXED_LEN_BYTE_ARRAY", + 5, + false, + 0, + 0, + "DecimalLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + assertEquals( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, primitiveType.getPrimitiveTypeName()); + + LogicalTypeAnnotation.DecimalLogicalTypeAnnotation dec = + (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) + primitiveType.getLogicalTypeAnnotation(); + assertEquals(10, dec.getPrecision()); + assertEquals(2, dec.getScale()); + } + + @Test + public void testBuildColumnDescriptorWithIntLogicalType() { + Map params = new HashMap<>(); + params.put("bitWidth", "32"); + params.put("isSigned", "true"); + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 12, + new String[] {"count"}, + "INT32", + 0, + false, + 0, + 0, + "IntLogicalTypeAnnotation", + params); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.INT32, primitiveType.getPrimitiveTypeName()); + + LogicalTypeAnnotation.IntLogicalTypeAnnotation ann = + (LogicalTypeAnnotation.IntLogicalTypeAnnotation) primitiveType.getLogicalTypeAnnotation(); + assertEquals(32, ann.getBitWidth()); + assertTrue(ann.isSigned()); + } + + @Test + public void testBuildColumnDescriptorWithStringLogicalType() { + ParquetColumnSpec spec = + new ParquetColumnSpec( + 13, + new String[] {"name"}, + "BINARY", + 0, + false, + 0, + 0, + "StringLogicalTypeAnnotation", + Collections.emptyMap()); + + ColumnDescriptor descriptor = Utils.buildColumnDescriptor(spec); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + assertEquals(PrimitiveType.PrimitiveTypeName.BINARY, primitiveType.getPrimitiveTypeName()); + assertTrue( + primitiveType.getLogicalTypeAnnotation() + instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation); + } +}