From 8a2713b51502f36c67c663dbbb771a949867df36 Mon Sep 17 00:00:00 2001 From: huaxingao Date: Tue, 8 Jul 2025 10:47:58 -0700 Subject: [PATCH 1/3] Add LogicalTypeAnnotation in ParquetColumnSpec --- .../comet/parquet/ParquetColumnSpec.java | 18 ++- .../java/org/apache/comet/parquet/Utils.java | 119 +++++++++++++++++- 2 files changed, 134 insertions(+), 3 deletions(-) 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..ed7932180f 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -19,6 +19,8 @@ package org.apache.comet.parquet; +import java.util.Map; + public class ParquetColumnSpec { private final String[] path; @@ -27,6 +29,8 @@ public class ParquetColumnSpec { private final boolean isRepeated; private final int maxDefinitionLevel; private final int maxRepetitionLevel; + private String logicalTypeName; + private Map logicalTypeParams; public ParquetColumnSpec( String[] path, @@ -34,13 +38,17 @@ public ParquetColumnSpec( int typeLength, boolean isRepeated, int maxDefinitionLevel, - int maxRepetitionLevel) { + int maxRepetitionLevel, + String logicalTypeName, + Map logicalTypeParams) { 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 String[] getPath() { @@ -66,4 +74,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..c64cad121d 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -24,6 +24,7 @@ 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 +291,129 @@ 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) + .named(name); } else { - primitiveType = new PrimitiveType(repetition, primType, name); + primitiveType = Types.primitive(primType, repetition).as(logicalType).named(name); } MessageType schema = new MessageType("root", primitiveType); return schema.getColumnDescription(columnSpec.getPath()); } + + 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": + 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": + boolean isUTC = Boolean.parseBoolean(params.getOrDefault("isAdjustedToUTC", "true")); + String timeUnitStr = params.getOrDefault("unit", "MICROS"); + + 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: + timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; + } + return LogicalTypeAnnotation.timeType(isUTC, timeUnit); + + // TIMESTAMP + case "TimestampLogicalTypeAnnotation": + boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); + String unitStr = params.getOrDefault("unit", "MICROS"); + + 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: + unit = LogicalTypeAnnotation.TimeUnit.MICROS; + } + return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); + + // INTEGER + case "IntLogicalTypeAnnotation": + 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); + } + } } From 64d534618d2b69b57b67d4559dde469f9d3c4de6 Mon Sep 17 00:00:00 2001 From: huaxingao Date: Tue, 8 Jul 2025 16:00:29 -0700 Subject: [PATCH 2/3] add id in ParquetColumnSpec --- .../apache/comet/parquet/ParquetColumnSpec.java | 9 +++++++++ .../main/java/org/apache/comet/parquet/Utils.java | 15 +++++++++++---- 2 files changed, 20 insertions(+), 4 deletions(-) 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 ed7932180f..49007f925d 100644 --- a/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java +++ b/common/src/main/java/org/apache/comet/parquet/ParquetColumnSpec.java @@ -23,16 +23,20 @@ public class ParquetColumnSpec { + private final int fieldId; private final String[] path; private final String physicalType; private final int typeLength; private final boolean isRepeated; 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, @@ -41,6 +45,7 @@ public ParquetColumnSpec( int maxRepetitionLevel, String logicalTypeName, Map logicalTypeParams) { + this.fieldId = fieldId; this.path = path; this.physicalType = physicalType; this.typeLength = typeLength; @@ -51,6 +56,10 @@ public ParquetColumnSpec( this.logicalTypeParams = logicalTypeParams; } + public int getFieldId() { + return fieldId; + } + public String[] getPath() { return path; } 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 c64cad121d..9d11cd0acc 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -21,7 +21,6 @@ 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; @@ -305,13 +304,21 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe Types.primitive(primType, repetition) .length(columnSpec.getTypeLength()) .as(logicalType) + .id(columnSpec.getFieldId()) .named(name); } else { - primitiveType = Types.primitive(primType, repetition).as(logicalType).named(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( From 5fdd82c1e9e5ec4d572a49b5302fe6586eddec68 Mon Sep 17 00:00:00 2001 From: huaxingao Date: Sun, 13 Jul 2025 14:34:45 -0700 Subject: [PATCH 3/3] address comments --- .../java/org/apache/comet/parquet/Utils.java | 27 +++- .../org/apache/comet/parquet/TestUtils.java | 149 ++++++++++++++++++ 2 files changed, 171 insertions(+), 5 deletions(-) create mode 100644 common/src/test/java/org/apache/comet/parquet/TestUtils.java 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 9d11cd0acc..fc7d1ab871 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -347,6 +347,10 @@ private static LogicalTypeAnnotation reconstructLogicalType( // 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); @@ -357,8 +361,13 @@ private static LogicalTypeAnnotation reconstructLogicalType( // TIME case "TimeLogicalTypeAnnotation": - boolean isUTC = Boolean.parseBoolean(params.getOrDefault("isAdjustedToUTC", "true")); - String timeUnitStr = params.getOrDefault("unit", "MICROS"); + 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) { @@ -372,14 +381,18 @@ private static LogicalTypeAnnotation reconstructLogicalType( timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; break; default: - timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; + 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.getOrDefault("unit", "MICROS"); + String unitStr = params.get("unit"); LogicalTypeAnnotation.TimeUnit unit; switch (unitStr) { @@ -393,12 +406,16 @@ private static LogicalTypeAnnotation reconstructLogicalType( unit = LogicalTypeAnnotation.TimeUnit.NANOS; break; default: - unit = LogicalTypeAnnotation.TimeUnit.MICROS; + 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); 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); + } +}