the-other-tim-brown commented on code in PR #669: URL: https://github.com/apache/incubator-xtable/pull/669#discussion_r2057284571
########## xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetStatsExtractor.java: ########## @@ -0,0 +1,327 @@ +/* + * 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.xtable.parquet; +import static org.apache.parquet.column.Encoding.BIT_PACKED; +import static org.apache.parquet.column.Encoding.PLAIN; +import static org.junit.jupiter.api.Assertions.*; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +import lombok.Builder; +import org.apache.parquet.column.statistics.BooleanStatistics; +import org.apache.hadoop.conf.Configuration; + +import java.nio.file.Paths; + +import java.nio.file.Files; +//import java.nio.file.Path; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.statistics.IntStatistics; +import org.apache.parquet.column.statistics.BinaryStatistics; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.*; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.stat.ColumnStat; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.model.storage.FileFormat; +import org.apache.xtable.model.storage.InternalDataFile; + +import org.junit.jupiter.api.io.TempDir; + +public class TestParquetStatsExtractor { + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + @TempDir + static java.nio.file.Path tempDir = Paths.get("./"); + + + + public static List<ColumnStat> initBooleanFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + MessageTypeParser.parseMessageType("message m { required group a {required boolean b;}}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + BooleanStatistics stats = new BooleanStatistics(); + stats.updateStats(true); + stats.updateStats(false); + + + +/* + byte byteTrue = (byte)(true?1:0); + byte byteFalse = (byte)(false?1:0); +*/ + + + // write the string columned file + + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(c1, 5, codec); + //w.startColumn(c1, 2, codec); + w.writeDataPage(2, 4, BytesInput.fromInt(1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.fromInt(0), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(c1, 8, codec); + //w.startColumn(c1, 1, codec); + w.writeDataPage(7, 4, BytesInput.fromInt(0), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + //byte[] minStat = stats.getMinBytes(); + boolean minStat = stats.genericGetMin(); + //byte[] maxStat = stats.getMaxBytes(); + boolean maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BOOLEAN, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(5, 8));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(54, 27)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; + } + + public static List<ColumnStat> initStringFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + MessageTypeParser.parseMessageType("message m { required group a {required fixed_len_byte_array(10) b;}}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + BinaryStatistics stats = new BinaryStatistics(); + stats.updateStats(Binary.fromString("1")); + stats.updateStats(Binary.fromString("2")); + stats.updateStats(Binary.fromString("5")); + + byte[] bytes1 = "First string".getBytes();//{0, 1, 2, 3}; + byte[] bytes2 = "Second string".getBytes();//{2, 3, 4, 5}; + + + // write the string columned file + + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(c1, 5, codec); + //w.startColumn(c1, 2, codec); + w.writeDataPage(2, 4, BytesInput.from(bytes1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(c1, 8, codec); + //w.startColumn(c1, 1, codec); + w.writeDataPage(7, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + //byte[] minStat = stats.getMinBytes(); + Binary minStat = stats.genericGetMin(); + //byte[] maxStat = stats.getMaxBytes(); + Binary maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(5, 8));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(71, 36)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; + } + + public static List<ColumnStat> initFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + //MessageTypeParser.parseMessageType("message m { required group a {required binary b;}}"); + MessageTypeParser.parseMessageType("message m { required group a {required int32 b;}}"); + //MessageType schema = MessageTypeParser.parseMessageType( + // "message m { required group a {required binary b;} required group " + "c { required int64 d; }}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + + byte[] bytes1 = {0, 1, 2, 3}; + byte[] bytes2 = {2, 3, 4, 5}; + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + + // if the schema col is of primitiveType then the stat should be of that same type (except binary schema which enable stats to be int or binary..) + // include statics using update() + IntStatistics stats = new IntStatistics(); // or BinaryStatistics + stats.updateStats(1); + stats.updateStats(2); + stats.updateStats(5); + +// BinaryStatistics stats = new BinaryStatistics(); +// stats.updateStats(Binary.fromString("1")); +// stats.updateStats(Binary.fromString("2")); +// stats.updateStats(Binary.fromString("5")); + + // to simplify the test we keep the same stats for both columns + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + //w.startColumn(c1, 5, codec); + w.startColumn(c1, 2, codec); + //w.writeDataPage(2, 4, BytesInput.from(bytes1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(3), stats, BIT_PACKED, BIT_PACKED, PLAIN);//bytes of int 3 are encoded as int32 (the primitive type of the schema) + //w.writeDataPage(3, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + //w.startColumn(c1, 8, codec); + w.startColumn(c1, 1, codec); + //w.writeDataPage(7, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + byte[] minStat = stats.getMinBytes(); + //java.lang.Integer minStat = stats.genericGetMin(); + byte[] maxStat = stats.getMaxBytes(); + //java.lang.Integer maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(2, 1));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(54, 27)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; // new ParquetFileReader(configuration, path, w.getFooter()); + } + + @Test + public void testToInternalDataFile() throws IOException { + + + ParquetFileReader fileReader = null; + InternalDataFile internalDataFile = null; + Configuration configuration = new Configuration(); + List<ColumnStat> testColumnStats = new ArrayList<>(); + java.nio.file.Path path = tempDir.resolve("parquet-test-files"); + File file = path.toFile(); + file.deleteOnExit(); + //testColumnStats = initFileTest(file); + //testColumnStats = initStringFileTest(file); + testColumnStats = initBooleanFileTest(file); + Path hadoopPath = new Path(file.toURI()); + // statsExtractor toInternalDataFile testing + internalDataFile = ParquetStatsExtractor.toInternalDataFile(configuration, hadoopPath); + //System.out.println(file.getAbsolutePath()); + InternalDataFile testInternalFile = + InternalDataFile.builder() + .physicalPath( + "file:/".concat(file.toPath().normalize().toAbsolutePath().toString().replace("\\", "/"))) // TODO hard coded path to file method + .columnStats(testColumnStats) + .fileFormat(FileFormat.APACHE_PARQUET) + .lastModified(file.lastModified()) + .fileSizeBytes(file.length()) + .recordCount(8) //2 + .build(); + + Assertions.assertEquals(true, testInternalFile.equals(internalDataFile)); + } + + public void main() throws IOException { + testToInternalDataFile(); + } Review Comment: Remove this since it is not necessary ########## xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java: ########## @@ -0,0 +1,159 @@ +/* + * 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.xtable.parquet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import lombok.Builder; +import lombok.Value; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; + +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.stat.ColumnStat; +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.model.storage.FileFormat; +import org.apache.xtable.model.storage.InternalDataFile; + +@Value +@Builder +public class ParquetStatsExtractor { + + private static final ParquetStatsExtractor INSTANCE = null; // new ParquetStatsExtractor(); + + private static final ParquetPartitionValueExtractor partitionExtractor = + ParquetPartitionValueExtractor.getInstance(); + + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + + private static final InputPartitionFields partitions = null; + + public static ParquetStatsExtractor getInstance() { + return INSTANCE; + } + + public static List<ColumnStat> getColumnStatsForaFile(ParquetMetadata footer) { + return getStatsForFile(footer).values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } + + private static Optional<Long> getMaxFromColumnStats(List<ColumnStat> columnStats) { + return columnStats.stream() + .filter(entry -> entry.getField().getParentPath() == null) + .map(ColumnStat::getNumValues) + .filter(numValues -> numValues > 0) + .max(Long::compareTo); + } + + public static Map<ColumnDescriptor, List<ColumnStat>> getStatsForFile(ParquetMetadata footer) { + Map<ColumnDescriptor, List<ColumnStat>> columnDescStats = new HashMap<>(); + MessageType schema = parquetMetadataExtractor.getSchema(footer); + List<ColumnChunkMetaData> columns = new ArrayList<>(); + columns = + footer.getBlocks().stream() + .flatMap(blockMetaData -> blockMetaData.getColumns().stream()) + .collect(Collectors.toList()); + columnDescStats = + columns.stream() + .collect( + Collectors.groupingBy( + columnMetaData -> + schema.getColumnDescription(columnMetaData.getPath().toArray()), + Collectors.mapping( + columnMetaData -> + ColumnStat.builder() + .field( + InternalField.builder() + .name(columnMetaData.getPrimitiveType().getName()) + .fieldId( + columnMetaData.getPrimitiveType().getId() == null + ? null + : columnMetaData + .getPrimitiveType() + .getId() + .intValue()) + .parentPath(null) + .schema( + schemaExtractor.toInternalSchema( + columnMetaData.getPrimitiveType(), + columnMetaData.getPath().toDotString())) + .build()) + .numValues(columnMetaData.getValueCount()) + .totalSize(columnMetaData.getTotalSize()) + .range( + Range.vector( + columnMetaData.getStatistics().genericGetMin(), + columnMetaData + .getStatistics() + .genericGetMax())) + .build(), + Collectors.toList()))); + return columnDescStats; + } + + private static InputPartitionFields initPartitionInfo() { + return partitions; + } + + public static InternalDataFile toInternalDataFile(Configuration hadoopConf, Path parentPath) + throws java.io.IOException { Review Comment: Import the exception instead of using the fully qualified path ########## xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java: ########## @@ -31,9 +33,20 @@ @Value @Builder(toBuilder = true) public class ColumnStat { - InternalField field; - Range range; - long numNulls; - long numValues; - long totalSize; + InternalField field; + Range range; + long numNulls; + long numValues; + long totalSize; + + public boolean equals(ColumnStat colStat) { Review Comment: It will automatically generate the code with the plugin when compiling ########## xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetStatsExtractor.java: ########## @@ -0,0 +1,327 @@ +/* + * 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.xtable.parquet; +import static org.apache.parquet.column.Encoding.BIT_PACKED; +import static org.apache.parquet.column.Encoding.PLAIN; +import static org.junit.jupiter.api.Assertions.*; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +import lombok.Builder; +import org.apache.parquet.column.statistics.BooleanStatistics; +import org.apache.hadoop.conf.Configuration; + +import java.nio.file.Paths; + +import java.nio.file.Files; +//import java.nio.file.Path; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.statistics.IntStatistics; +import org.apache.parquet.column.statistics.BinaryStatistics; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.*; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type.Repetition; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.stat.ColumnStat; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.model.storage.FileFormat; +import org.apache.xtable.model.storage.InternalDataFile; + +import org.junit.jupiter.api.io.TempDir; + +public class TestParquetStatsExtractor { + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + @TempDir + static java.nio.file.Path tempDir = Paths.get("./"); + + + + public static List<ColumnStat> initBooleanFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + MessageTypeParser.parseMessageType("message m { required group a {required boolean b;}}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + BooleanStatistics stats = new BooleanStatistics(); + stats.updateStats(true); + stats.updateStats(false); + + + +/* + byte byteTrue = (byte)(true?1:0); + byte byteFalse = (byte)(false?1:0); +*/ + + + // write the string columned file + + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(c1, 5, codec); + //w.startColumn(c1, 2, codec); + w.writeDataPage(2, 4, BytesInput.fromInt(1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.fromInt(0), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(c1, 8, codec); + //w.startColumn(c1, 1, codec); + w.writeDataPage(7, 4, BytesInput.fromInt(0), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + //byte[] minStat = stats.getMinBytes(); + boolean minStat = stats.genericGetMin(); + //byte[] maxStat = stats.getMaxBytes(); + boolean maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BOOLEAN, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(5, 8));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(54, 27)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; + } + + public static List<ColumnStat> initStringFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + MessageTypeParser.parseMessageType("message m { required group a {required fixed_len_byte_array(10) b;}}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + BinaryStatistics stats = new BinaryStatistics(); + stats.updateStats(Binary.fromString("1")); + stats.updateStats(Binary.fromString("2")); + stats.updateStats(Binary.fromString("5")); + + byte[] bytes1 = "First string".getBytes();//{0, 1, 2, 3}; + byte[] bytes2 = "Second string".getBytes();//{2, 3, 4, 5}; + + + // write the string columned file + + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + w.startColumn(c1, 5, codec); + //w.startColumn(c1, 2, codec); + w.writeDataPage(2, 4, BytesInput.from(bytes1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + w.startColumn(c1, 8, codec); + //w.startColumn(c1, 1, codec); + w.writeDataPage(7, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + //byte[] minStat = stats.getMinBytes(); + Binary minStat = stats.genericGetMin(); + //byte[] maxStat = stats.getMaxBytes(); + Binary maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(5, 8));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(71, 36)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; + } + + public static List<ColumnStat> initFileTest(File file) throws IOException { + // create the parquet file by parsing a schema + Path path = new Path(file.toURI()); + Configuration configuration = new Configuration(); + + MessageType schema = + //MessageTypeParser.parseMessageType("message m { required group a {required binary b;}}"); + MessageTypeParser.parseMessageType("message m { required group a {required int32 b;}}"); + //MessageType schema = MessageTypeParser.parseMessageType( + // "message m { required group a {required binary b;} required group " + "c { required int64 d; }}"); + String[] columnPath = {"a", "b"}; + ColumnDescriptor c1 = schema.getColumnDescription(columnPath); + + byte[] bytes1 = {0, 1, 2, 3}; + byte[] bytes2 = {2, 3, 4, 5}; + CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED; + + // if the schema col is of primitiveType then the stat should be of that same type (except binary schema which enable stats to be int or binary..) + // include statics using update() + IntStatistics stats = new IntStatistics(); // or BinaryStatistics + stats.updateStats(1); + stats.updateStats(2); + stats.updateStats(5); + +// BinaryStatistics stats = new BinaryStatistics(); +// stats.updateStats(Binary.fromString("1")); +// stats.updateStats(Binary.fromString("2")); +// stats.updateStats(Binary.fromString("5")); + + // to simplify the test we keep the same stats for both columns + ParquetFileWriter w = new ParquetFileWriter(configuration, schema, path); + w.start(); + w.startBlock(3); + //w.startColumn(c1, 5, codec); + w.startColumn(c1, 2, codec); + //w.writeDataPage(2, 4, BytesInput.from(bytes1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(3), stats, BIT_PACKED, BIT_PACKED, PLAIN);//bytes of int 3 are encoded as int32 (the primitive type of the schema) + //w.writeDataPage(3, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.startBlock(4); + //w.startColumn(c1, 8, codec); + w.startColumn(c1, 1, codec); + //w.writeDataPage(7, 4, BytesInput.from(bytes2), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.writeDataPage(3, 3, BytesInput.fromInt(1), stats, BIT_PACKED, BIT_PACKED, PLAIN); + w.endColumn(); + w.endBlock(); + w.end(new HashMap<String, String>()); + + // reconstruct the stats for the InternalDataFile testing object + byte[] minStat = stats.getMinBytes(); + //java.lang.Integer minStat = stats.genericGetMin(); + byte[] maxStat = stats.getMaxBytes(); + //java.lang.Integer maxStat = stats.genericGetMax(); + PrimitiveType primitiveType = + // new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.BINARY, "b"); + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "b"); + List<Integer> col1NumValTotSize = new ArrayList<>(Arrays.asList(2, 1));//(5, 8)// start column indexes + List<Integer> col2NumValTotSize = new ArrayList<>(Arrays.asList(54, 27)); + List<ColumnStat> testColumnStats = new ArrayList<>(); + String[] columnDotPath = {"a.b", "a.b"}; + for (int i = 0; i < columnDotPath.length; i++) { + testColumnStats.add( + ColumnStat.builder() + .field( + InternalField.builder() + .name(primitiveType.getName()) + .parentPath(null) + .schema(schemaExtractor.toInternalSchema(primitiveType, columnDotPath[i])) + .build()) + .numValues(col1NumValTotSize.get(i)) + .totalSize(col2NumValTotSize.get(i)) + .range(Range.vector(minStat, maxStat)) + .build()); + } + + return testColumnStats; // new ParquetFileReader(configuration, path, w.getFooter()); + } + + @Test + public void testToInternalDataFile() throws IOException { + + + ParquetFileReader fileReader = null; + InternalDataFile internalDataFile = null; + Configuration configuration = new Configuration(); + List<ColumnStat> testColumnStats = new ArrayList<>(); + java.nio.file.Path path = tempDir.resolve("parquet-test-files"); + File file = path.toFile(); + file.deleteOnExit(); + //testColumnStats = initFileTest(file); Review Comment: You can make a separate test case for the int and string stats so it is clear what each case handles. For example have: ``` @Test void testIntStatistics() ... @Test void testStringStatistics() .... ``` ########## xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java: ########## @@ -0,0 +1,159 @@ +/* + * 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.xtable.parquet; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import lombok.Builder; +import lombok.Value; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; + +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.stat.ColumnStat; +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.model.storage.FileFormat; +import org.apache.xtable.model.storage.InternalDataFile; + +@Value +@Builder +public class ParquetStatsExtractor { + + private static final ParquetStatsExtractor INSTANCE = null; // new ParquetStatsExtractor(); Review Comment: I think you'll want to uncomment this line right? ########## xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java: ########## @@ -0,0 +1,496 @@ +/* + * 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.xtable.parquet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumMap; +import java.util.List; +import java.util.Map; + +import lombok.AccessLevel; +import lombok.NoArgsConstructor; + +import org.apache.avro.Schema; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Type.Repetition; +import org.apache.parquet.schema.Types; + +import org.apache.xtable.collectors.CustomCollectors; +import org.apache.xtable.exception.SchemaExtractorException; +import org.apache.xtable.exception.UnsupportedSchemaTypeException; +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.schema.InternalType; +import org.apache.xtable.schema.SchemaUtils; + +/** + * Class that converts parquet Schema {@link Schema} to Canonical Schema {@link InternalSchema} and + * vice-versa. This conversion is fully reversible and there is a strict 1 to 1 mapping between + * parquet data types and canonical data types. + */ +@NoArgsConstructor(access = AccessLevel.PRIVATE) +public class ParquetSchemaExtractor { + // parquet only supports string keys in maps + private static final InternalField MAP_KEY_FIELD = + InternalField.builder() + .name(InternalField.Constants.MAP_KEY_FIELD_NAME) + .schema( + InternalSchema.builder() + .name("map_key") + .dataType(InternalType.STRING) + .isNullable(false) + .build()) + .defaultValue("") + .build(); + private static final ParquetSchemaExtractor INSTANCE = new ParquetSchemaExtractor(); + private static final String ELEMENT = "element"; + private static final String KEY = "key"; + private static final String VALUE = "value"; + + public static ParquetSchemaExtractor getInstance() { + return INSTANCE; + } + + private static boolean isNullable(Type schema) { + return schema.getRepetition() != Repetition.REQUIRED; + } + + /** + * Converts the parquet {@link Schema} to {@link InternalSchema}. + * + * @param schema The schema being converted + * @param parentPath If this schema is nested within another, this will be a dot separated string + * representing the path from the top most field to the current schema. + * @return a converted schema + */ + public InternalSchema toInternalSchema(Type schema, String parentPath) { + InternalType newDataType = null; + Type.Repetition currentRepetition = null; + List<InternalField> subFields = null; + PrimitiveType primitiveType; + LogicalTypeAnnotation logicalType; + Map<InternalSchema.MetadataKey, Object> metadata = + new EnumMap<>(InternalSchema.MetadataKey.class); + String elementName = schema.getName(); + if (schema.isPrimitive()) { + primitiveType = schema.asPrimitiveType(); + switch (primitiveType.getPrimitiveTypeName()) { + // PrimitiveTypes + case INT64: + logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = + ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType).getUnit(); + boolean isAdjustedToUTC = + ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType) + .isAdjustedToUTC(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, + InternalSchema.MetadataValue.MICROS); + } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + if (isAdjustedToUTC) { + newDataType = InternalType.TIMESTAMP; + } else { + newDataType = InternalType.TIMESTAMP_NTZ; + } Review Comment: Let's move this up to line 107 so we don't need to repeat it for micros and nanos as well. ########## xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java: ########## @@ -52,4 +52,21 @@ public class InternalDataFile extends InternalFile { @Builder.Default @NonNull List<ColumnStat> columnStats = Collections.emptyList(); // last modified time in millis since epoch long lastModified; + public static InternalDataFileBuilder builderFrom(InternalDataFile dataFile) { + return dataFile.toBuilder(); + } + + public boolean equals(InternalDataFile obj2) { Review Comment: Since this equality check is critical for testing, I don't want to change it without good reason. If the issue is with on of the equality checks for this class's fields we can update that but let's avoid rewriting all these checks. Lombok will automatically add new fields to the equals method which helps with maintaining the repo. ########## xtable-core/pom.xml: ########## @@ -61,6 +61,12 @@ <groupId>org.apache.avro</groupId> <artifactId>avro</artifactId> </dependency> + <!-- https://mvnrepository.com/artifact/org.apache.parquet/parquet-avro --> + <dependency> + <groupId>org.apache.parquet</groupId> + <artifactId>parquet-avro</artifactId> + <version>1.15.1</version> Review Comment: The version is defined in the parent pom so you should not add the version here. The dependency management section in the parent pom helps us keep the versions aligned across the modules and the project's transitive dependencies ########## xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java: ########## @@ -0,0 +1,354 @@ +/* + * 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.xtable.parquet; + +import static org.junit.jupiter.api.Assertions.*; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.parquet.schema.*; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Type.Repetition; +import org.apache.parquet.schema.Types; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.schema.InternalType; + +public class TestParquetSchemaExtractor { + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + + @Test + public void testPrimitiveTypes() { + + InternalSchema primitive1 = + InternalSchema.builder().name("integer").dataType(InternalType.INT).build(); + InternalSchema primitive2 = + InternalSchema.builder().name("string").dataType(InternalType.STRING).build(); + + Map<InternalSchema.MetadataKey, Object> fixedDecimalMetadata = new HashMap<>(); + fixedDecimalMetadata.put(InternalSchema.MetadataKey.DECIMAL_PRECISION, 6); + fixedDecimalMetadata.put(InternalSchema.MetadataKey.DECIMAL_SCALE, 5); + InternalSchema decimalType = + InternalSchema.builder() + .name("decimal") + .dataType(InternalType.DECIMAL) + .isNullable(false) + .metadata(fixedDecimalMetadata) + .build(); + + Type stringPrimitiveType = + Types.required(PrimitiveTypeName.BINARY) + .as(LogicalTypeAnnotation.stringType()) // .named("string") Review Comment: remove the `// .named("string")` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@xtable.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org