From 74cbc83e3ffee7db6ac5a34d48a72de0a65f0a72 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 15 Feb 2025 17:09:57 +0100 Subject: [PATCH 01/49] First commit on supporting parquet --- .../parquet/ParquetConversionSource.java | 249 ++++++++++ .../parquet/ParquetSchemaExtractror.java | 469 ++++++++++++++++++ .../xtable/parquet/ParquetTableExtractor.java | 69 +++ 3 files changed, 787 insertions(+) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java new file mode 100644 index 000000000..d3d80e96b --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -0,0 +1,249 @@ +package org.apache.xtable.parquet; + +import java.io.IOException; +import java.time.Instant; +import java.util.*; +import java.util.stream.Collectors; +import lombok.Builder; +import lombok.NonNull; +import org.apache.parquet.Schema; +import org.apache.parquet.SchemaBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.xtable.parquet.ParquetSchemaConverter; +import org.apache.xtable.model.*; +import org.apache.xtable.model.schema.InternalPartitionField; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.storage.*; +import org.apache.xtable.spi.extractor.ConversionSource; + +@Builder +public class ParquetConversionSource implements ConversionSource { + + private final String tableName; + private final String basePath; + @NonNull private final Configuration hadoopConf; + + @Builder.Default + private static final ParquetSchemaConverter schemaExtractor = ParquetSchemaConverter.getInstance(); + + + + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + + @Builder.Default + private static final ParquetPartitionHelper parquetPartitionHelper = + ParquetPartitionHelper.getInstance(); + + private Map> initPartitionInfo() { + return getPartitionFromDirectoryStructure( + hadoopConf, basePath, Collections.emptyMap()); + } + + /** + * To infer schema getting the latest file assumption is that latest file will have new fields + * + * @param modificationTime the commit to consider for reading the table state + * @return + */ + @Override + public InternalTable getTable(Long modificationTime) { + + Optional latestFile = + + getParquetFiles(hadoopConf, basePath) + .max(Comparator.comparing(FileStatus::getModificationTime)); + + ParquetMetadata parquetMetadata = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); + Schema tableSchema = + new org.apache.parquet.parquet.ParquetSchemaConverter() + .convert(parquetMetadata.getFileMetaData().getSchema()); + + Set partitionKeys = initPartitionInfo().keySet(); + + // merge schema of partition into original as partition is not part of parquet fie + if (!partitionKeys.isEmpty()) { + tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + } + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); + + List partitionFields = + partitionKeys.isEmpty() + ? Collections.emptyList() + : parquetPartitionHelper.getInternalPartitionField(partitionKeys, schema); + DataLayoutStrategy dataLayoutStrategy = + partitionFields.isEmpty() + ? DataLayoutStrategy.FLAT + : DataLayoutStrategy.HIVE_STYLE_PARTITION; + return InternalTable.builder() + .tableFormat(TableFormat.PARQUET) + .basePath(basePath) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) + .build(); + } + + /** + * Here to get current snapshot listing all files hence the -1 is being passed + * + * @return + */ + @Override + public InternalSnapshot getCurrentSnapshot() { + + List latestFile = + getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); + Map> partitionInfo = initPartitionInfo(); + InternalTable table = getTable(-1L); + List internalDataFiles = + latestFile.stream() + .map( + file -> + InternalDataFile.builder() + .physicalPath(file.getPath().toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .fileSizeBytes(file.getLen()) + .partitionValues( + parquetPartitionHelper.getPartitionValue( + basePath, + file.getPath().toString(), + table.getReadSchema(), + partitionInfo)) + .lastModified(file.getModificationTime()) + .columnStats( + parquetMetadataExtractor.getColumnStatsForaFile( + hadoopConf, file, table)) + .build()) + .collect(Collectors.toList()); + + return InternalSnapshot.builder() + .table(table) + .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) + .build(); + } + + /** + * Whenever new file is added , condition to get new file is listing files whose modification time + * is greater than previous ysnc + * + * @param modificationTime commit to capture table changes for. + * @return + */ + @Override + public TableChange getTableChangeForCommit(Long modificationTime) { + List tableChanges = + + getParquetFiles(hadoopConf, basePath) + .filter(fileStatus -> fileStatus.getModificationTime() > modificationTime) + .collect(Collectors.toList()); + // TODO avoid doing full list of directory to get schema , just argument of modification time + // needs to be tweaked + InternalTable internalTable = getTable(-1L); + Set internalDataFiles = new HashSet<>(); + Map> partitionInfo = initPartitionInfo(); + for (FileStatus tableStatus : tableChanges) { + internalDataFiles.add( + InternalDataFile.builder() + .physicalPath(tableStatus.getPath().toString()) + .partitionValues( + parquetPartitionHelper.getPartitionValue( + basePath, + tableStatus.getPath().toString(), + internalTable.getReadSchema(), + partitionInfo)) + .lastModified(tableStatus.getModificationTime()) + .fileSizeBytes(tableStatus.getLen()) + .columnStats( + parquetMetadataExtractor.getColumnStatsForaFile( + hadoopConf, tableStatus, internalTable)) + .build()); + } + + return TableChange.builder() + .tableAsOfChange(internalTable) + .filesDiff(DataFilesDiff.builder().filesAdded(internalDataFiles).build()) + .build(); + } + + @Override + public CommitsBacklog getCommitsBacklog( + InstantsForIncrementalSync instantsForIncrementalSync) { + + List commitsToProcess = + Collections.singletonList(instantsForIncrementalSync.getLastSyncInstant().toEpochMilli()); + + return CommitsBacklog.builder().commitsToProcess(commitsToProcess).build(); + } + + // TODO Need to understnad how this needs to be implemented should _SUCCESS or .staging dir needs + // to be checked + @Override + public boolean isIncrementalSyncSafeFrom(Instant instant) { + return true; + } + + @Override + public void close() throws IOException {} + + private Schema mergeParquetSchema(Schema internalSchema, Set parititonFields) { + + SchemaBuilder.FieldAssembler fieldAssembler = + SchemaBuilder.record(internalSchema.getName()).fields(); + for (Schema.Field field : internalSchema.getFields()) { + fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); + } + + for (String paritionKey : parititonFields) { + fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); + } + + return fieldAssembler.endRecord(); + } + + + public Stream getParquetFiles(Configuration hadoopConf, String basePath) { + try { + FileSystem fs = FileSystem.get(hadoopConf); + RemoteIterator iterator = fs.listFiles(new Path(basePath), true); + return remoteIteratorToStream(iterator) + .filter(file -> file.getPath().getName().endsWith("parquet")); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public Map> getPartitionFromDirectoryStructure( + Configuration hadoopConf, String basePath, Map> partitionMap) { + + try { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); + Map> currentPartitionMap = new HashMap<>(partitionMap); + + for (FileStatus dirStatus : baseFileStatus) { + if (dirStatus.isDirectory()) { + String partitionPath = dirStatus.getPath().getName(); + if (partitionPath.contains("=")) { + String[] partitionKeyValue = partitionPath.split("="); + currentPartitionMap + .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) + .add(partitionKeyValue[1]); + getPartitionFromDirectoryStructure( + hadoopConf, dirStatus.getPath().toString(), partitionMap); + } + } + } + return currentPartitionMap; + + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java new file mode 100644 index 000000000..3839395f8 --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -0,0 +1,469 @@ +/* + * 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.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +import lombok.AccessLevel; +import lombok.NoArgsConstructor; + +import org.apache.parquet.LogicalType; +import org.apache.parquet.LogicalTypes; +import org.apache.parquet.Schema; + +import org.apache.xtable.collectors.CustomCollectors; +import org.apache.xtable.exception.SchemaExtractorException; +import org.apache.xtable.exception.UnsupportedSchemaTypeException; +import org.apache.xtable.hudi.idtracking.IdTracker; +import org.apache.xtable.hudi.idtracking.models.IdMapping; +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 ParquetSchemaConverter { + // 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 ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); + private static final String ELEMENT = "element"; + private static final String KEY = "key"; + private static final String VALUE = "value"; + + public static parquetSchemaConverter getInstance() { + return INSTANCE; + } + + public InternalSchema toInternalSchema(Schema schema) { + Map fieldNameToIdMapping = + IdTracker.getInstance() + .getIdTracking(schema) + .map( + idTracking -> + idTracking.getIdMappings().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) + .orElse(Collections.emptyMap()); + return toInternalSchema(schema, null, fieldNameToIdMapping); + } + + /** + * 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. + * @param fieldNameToIdMapping map of fieldName to IdMapping to track field IDs provided by the + * source schema. If source schema does not contain IdMappings, map will be empty. + * @return a converted schema + */ + private InternalSchema toInternalSchema( + Schema schema, String parentPath, Map fieldNameToIdMapping) { + // TODO - Does not handle recursion in parquet schema + InternalType newDataType; + Map metadata = new HashMap<>(); + switch (schema.getType()) { + case INT: + LogicalType logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.Date) { + newDataType = InternalType.DATE; + } else { + newDataType = InternalType.INT; + } + break; + case STRING: + newDataType = InternalType.STRING; + break; + case BOOLEAN: + newDataType = InternalType.BOOLEAN; + break; + case BYTES: + case JSON: + case BSON: + case FIXED: + logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.Decimal) { + metadata.put( + InternalSchema.MetadataKey.DECIMAL_PRECISION, + ((LogicalTypes.Decimal) logicalType).getPrecision()); + metadata.put( + InternalSchema.MetadataKey.DECIMAL_SCALE, + ((LogicalTypes.Decimal) logicalType).getScale()); + if (schema.getType() == Schema.Type.FIXED) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + } + newDataType = InternalType.DECIMAL; + break; + } + if (schema.getType() == Schema.Type.FIXED) { + String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); + if ("uuid".equals(xtableLogicalType)) { + newDataType = InternalType.UUID; + } else { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + newDataType = InternalType.FIXED; + } + } else { + newDataType = InternalType.BYTES; + } + break; + case DOUBLE: + newDataType = InternalType.DOUBLE; + break; + case FLOAT: + newDataType = InternalType.FLOAT; + break; + case LONG: + logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.TimestampMillis) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else { + newDataType = InternalType.LONG; + } + break; + case ENUM: + metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.getEnumSymbols()); + newDataType = InternalType.ENUM; + break; + case NULL: + newDataType = InternalType.NULL; + break; + case RECORD: + List subFields = new ArrayList<>(schema.getFields().size()); + for (Schema.Field parquetField : schema.getFields()) { + IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); + InternalSchema subFieldSchema = + toInternalSchema( + parquetField.schema(), + SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), + getChildIdMap(idMapping)); + Object defaultValue = getDefaultValue(parquetField); + subFields.add( + InternalField.builder() + .parentPath(parentPath) + .name(parquetField.name()) + .schema(subFieldSchema) + .defaultValue(defaultValue) + .fieldId(idMapping == null ? null : idMapping.getId()) + .build()); + } + return InternalSchema.builder() + .name(schema.getName()) + .comment(schema.getDoc()) + .dataType(InternalType.RECORD) + .fields(subFields) + .isNullable(schema.isNullable()) + .build(); + case ARRAY: + IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + InternalSchema elementSchema = + toInternalSchema( + schema.getElementType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), + getChildIdMap(elementMapping)); + InternalField elementField = + InternalField.builder() + .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) + .parentPath(parentPath) + .schema(elementSchema) + .fieldId(elementMapping == null ? null : elementMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.LIST) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields(Collections.singletonList(elementField)) + .build(); + case MAP: + IdMapping keyMapping = fieldNameToIdMapping.get(KEY); + IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); + InternalSchema valueSchema = + toInternalSchema( + schema.getValueType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), + getChildIdMap(valueMapping)); + InternalField valueField = + InternalField.builder() + .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) + .parentPath(parentPath) + .schema(valueSchema) + .fieldId(valueMapping == null ? null : valueMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.MAP) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields( + Arrays.asList( + MAP_KEY_FIELD.toBuilder() + .parentPath(parentPath) + .fieldId(keyMapping == null ? null : keyMapping.getId()) + .build(), + valueField)) + .build(); + case UNION: + boolean containsUnionWithNull = + schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); + if (containsUnionWithNull) { + List remainingSchemas = + schema.getTypes().stream() + .filter(t -> t.getType() != Schema.Type.NULL) + .collect(Collectors.toList()); + if (remainingSchemas.size() == 1) { + InternalSchema restSchema = + toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); + return InternalSchema.builderFrom(restSchema).isNullable(true).build(); + } else { + return InternalSchema.builderFrom( + toInternalSchema(Schema.createUnion(remainingSchemas))) + .isNullable(true) + .build(); + } + } else { + throw new UnsupportedSchemaTypeException( + String.format("Unsupported complex union type %s", schema)); + } + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); + } + return InternalSchema.builder() + .name(schema.getName()) + .dataType(newDataType) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .metadata(metadata.isEmpty() ? null : metadata) + .build(); + } + + private Map getChildIdMap(IdMapping idMapping) { + if (idMapping == null) { + return Collections.emptyMap(); + } + return idMapping.getFields().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity())); + } + + private static Object getDefaultValue(Schema.Field parquetField) { + return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) + ? InternalField.Constants.NULL_DEFAULT_VALUE + : parquetField.defaultVal(); + } + + /** + * Converts the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @return an parquet schema + */ + public Schema fromInternalSchema(InternalSchema internalSchema) { + return fromInternalSchema(internalSchema, null); + } + + /** + * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @param currentPath If this schema is nested within another, this will be a dot separated + * string. This is used for the parquet namespace to guarantee unique names for nested records. + * @return an parquet schema + */ + private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { + switch (internalSchema.getDataType()) { + case RECORD: + List fields = + internalSchema.getFields().stream() + .map( + field -> + new Schema.Field( + field.getName(), + fromInternalSchema( + field.getSchema(), + SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), + field.getSchema().getComment(), + InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() + ? Schema.Field.NULL_VALUE + : field.getDefaultValue())) + .collect(CustomCollectors.toList(internalSchema.getFields().size())); + return finalizeSchema( + Schema.createRecord( + internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), + internalSchema); + case BYTES: + return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); + case BOOLEAN: + return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); + case INT: + return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); + case LONG: + return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); + case STRING: + return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); + case FLOAT: + return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); + case DOUBLE: + return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); + case ENUM: + return finalizeSchema( + Schema.createEnum( + internalSchema.getName(), + internalSchema.getComment(), + null, + (List) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), + null), + internalSchema); + case DATE: + return finalizeSchema( + LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); + case TIMESTAMP: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case TIMESTAMP_NTZ: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case LIST: + InternalField elementField = + internalSchema.getFields().stream() + .filter( + field -> + InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); + return finalizeSchema( + Schema.createArray( + fromInternalSchema(elementField.getSchema(), elementField.getPath())), + internalSchema); + case MAP: + InternalField valueField = + internalSchema.getFields().stream() + .filter( + field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); + return finalizeSchema( + Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), + internalSchema); + case DECIMAL: + int precision = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); + int scale = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); + Integer size = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + if (size == null) { + return finalizeSchema( + LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.decimal(precision, scale) + .addToSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, size)), + internalSchema); + } + case FIXED: + Integer fixedSize = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + return finalizeSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, fixedSize), + internalSchema); + case UUID: + Schema uuidSchema = + Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); + uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); + return finalizeSchema(uuidSchema, internalSchema); + default: + throw new UnsupportedSchemaTypeException( + "Encountered unhandled type during InternalSchema to parquet conversion: " + + internalSchema.getDataType()); + } + } + + private String buildCurrentPath(InternalField field, String parentPath) { + return Optional.ofNullable(parentPath) + .map(path -> path + "." + field.getName()) + .orElse(field.getName()); + } + + private static Schema finalizeSchema(Schema targetSchema, InternalSchema inputSchema) { + if (inputSchema.isNullable()) { + return Schema.createUnion(Schema.create(Schema.Type.NULL), targetSchema); + } + return targetSchema; + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java new file mode 100644 index 000000000..0be978274 --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -0,0 +1,69 @@ +/* + * 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.delta; + +import java.time.Instant; +import java.util.List; + +import lombok.Builder; + +import org.apache.spark.sql.delta.DeltaLog; +import org.apache.spark.sql.delta.Snapshot; + +import scala.Option; + +import org.apache.xtable.model.InternalTable; +import org.apache.xtable.model.schema.InternalPartitionField; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.storage.DataLayoutStrategy; +import org.apache.xtable.model.storage.TableFormat; +import org.apache.parquet.format.FileMetaData; +import org.apache.parquet.hadoop.ParquetFileReader + +/** + * Extracts {@link InternalTable} canonical representation of a table at a point in time for Delta. + */ +@Builder +public class ParquetTableExtractor { + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); + + public InternalTable table(String tableName, Long version) { + ParquetMetadata readFooter = ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + MessageType schema = readFooter.getFileMetaData().getSchema(); + InternalSchema schema = schemaExtractor.toInternalSchema(schema); + // TODO check partitionSchema of Parquet File + List partitionFields = + ParquetPartitionExtractor.getInstance() + .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); + DataLayoutStrategy dataLayoutStrategy = + !partitionFields.isEmpty() + ? DataLayoutStrategy.HIVE_STYLE_PARTITION + : DataLayoutStrategy.FLAT; + return InternalTable.builder() + .tableFormat(TableFormat.APACHE_PARQUET) + .basePath(snapshot.deltaLog().dataPath().toString()) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(snapshot.timestamp())) + .build(); + } +} From 79bd222c8dd827d654eb51dbee557e881cd77790 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Mon, 17 Feb 2025 21:49:00 +0100 Subject: [PATCH 02/49] catch file not found exception --- .../java/org/apache/xtable/parquet/ParquetConversionSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index d3d80e96b..0c9fda6c6 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -214,7 +214,7 @@ public Stream getParquetFiles(Configuration hadoopConf, Strin RemoteIterator iterator = fs.listFiles(new Path(basePath), true); return remoteIteratorToStream(iterator) .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException e) { + } catch (IOException|FileNotFoundException e) { throw new RuntimeException(e); } } From 2143c998d3fb6e68ac664fcbc7777b2c52d7ac01 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 18 Feb 2025 00:21:15 +0100 Subject: [PATCH 03/49] executed mvn spotless:apply --- .../xtable/conversion/ExternalTable.java | 4 + .../catalog/HierarchicalTableIdentifier.java | 12 +- .../ThreePartHierarchicalTableIdentifier.java | 1 + .../xtable/model/schema/InternalField.java | 2 + .../parquet/ParquetConversionSource.java | 439 +++++----- .../parquet/ParquetSchemaExtractror.java | 801 +++++++++--------- .../xtable/parquet/ParquetTableExtractor.java | 58 +- .../apache/xtable/TestAbstractHudiTable.java | 1 + .../org/apache/xtable/TestJavaHudiTable.java | 1 + .../apache/xtable/iceberg/StubCatalog.java | 1 + .../xtable/utilities/RunCatalogSync.java | 6 + 11 files changed, 680 insertions(+), 646 deletions(-) diff --git a/xtable-api/src/main/java/org/apache/xtable/conversion/ExternalTable.java b/xtable-api/src/main/java/org/apache/xtable/conversion/ExternalTable.java index 939c59c09..a6c97d8fa 100644 --- a/xtable-api/src/main/java/org/apache/xtable/conversion/ExternalTable.java +++ b/xtable-api/src/main/java/org/apache/xtable/conversion/ExternalTable.java @@ -34,12 +34,16 @@ class ExternalTable { /** The name of the table. */ protected final @NonNull String name; + /** The format of the table (e.g. DELTA, ICEBERG, HUDI) */ protected final @NonNull String formatName; + /** The path to the root of the table or the metadata directory depending on the format */ protected final @NonNull String basePath; + /** Optional namespace for the table */ protected final String[] namespace; + /** The configuration for interacting with the catalog that manages this table */ protected final CatalogConfig catalogConfig; diff --git a/xtable-api/src/main/java/org/apache/xtable/model/catalog/HierarchicalTableIdentifier.java b/xtable-api/src/main/java/org/apache/xtable/model/catalog/HierarchicalTableIdentifier.java index e6d7cf0eb..2dfc2cd89 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/catalog/HierarchicalTableIdentifier.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/catalog/HierarchicalTableIdentifier.java @@ -23,12 +23,18 @@ * table names. Some catalogs may omit the catalog name. */ public interface HierarchicalTableIdentifier extends CatalogTableIdentifier { - /** @return the catalog name if present, otherwise null */ + /** + * @return the catalog name if present, otherwise null + */ String getCatalogName(); - /** @return the database (or schema) name; required */ + /** + * @return the database (or schema) name; required + */ String getDatabaseName(); - /** @return the table name; required */ + /** + * @return the table name; required + */ String getTableName(); } diff --git a/xtable-api/src/main/java/org/apache/xtable/model/catalog/ThreePartHierarchicalTableIdentifier.java b/xtable-api/src/main/java/org/apache/xtable/model/catalog/ThreePartHierarchicalTableIdentifier.java index 2608d36a3..f387c7d36 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/catalog/ThreePartHierarchicalTableIdentifier.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/catalog/ThreePartHierarchicalTableIdentifier.java @@ -42,6 +42,7 @@ public class ThreePartHierarchicalTableIdentifier implements HierarchicalTableId * name varies depending on the catalogType. */ String catalogName; + /** * Catalogs have the ability to group tables logically, databaseName is the identifier for such * logical classification. The alternate names for this field include namespace, schemaName etc. diff --git a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalField.java b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalField.java index 16b6da8aa..31eb0ed41 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalField.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalField.java @@ -43,9 +43,11 @@ public class InternalField { // The id field for the field. This is used to identify the field in the schema even after // renames. Integer fieldId; + // represents the fully qualified path to the field (dot separated) @Getter(lazy = true) String path = createPath(); + // splits the dot separated path into parts @Getter(lazy = true) String[] pathParts = splitPath(); diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 0c9fda6c6..973ccff30 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -1,17 +1,37 @@ +/* + * 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.io.IOException; import java.time.Instant; import java.util.*; import java.util.stream.Collectors; + import lombok.Builder; import lombok.NonNull; -import org.apache.parquet.Schema; -import org.apache.parquet.SchemaBuilder; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; +import org.apache.parquet.Schema; +import org.apache.parquet.SchemaBuilder; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.xtable.parquet.ParquetSchemaConverter; + import org.apache.xtable.model.*; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; @@ -21,229 +41,224 @@ @Builder public class ParquetConversionSource implements ConversionSource { - private final String tableName; - private final String basePath; - @NonNull private final Configuration hadoopConf; - - @Builder.Default - private static final ParquetSchemaConverter schemaExtractor = ParquetSchemaConverter.getInstance(); - - - - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - - @Builder.Default - private static final ParquetPartitionHelper parquetPartitionHelper = - ParquetPartitionHelper.getInstance(); - - private Map> initPartitionInfo() { - return getPartitionFromDirectoryStructure( - hadoopConf, basePath, Collections.emptyMap()); + private final String tableName; + private final String basePath; + @NonNull private final Configuration hadoopConf; + + @Builder.Default + private static final ParquetSchemaConverter schemaExtractor = + ParquetSchemaConverter.getInstance(); + + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + + @Builder.Default + private static final ParquetPartitionHelper parquetPartitionHelper = + ParquetPartitionHelper.getInstance(); + + private Map> initPartitionInfo() { + return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); + } + + /** + * To infer schema getting the latest file assumption is that latest file will have new fields + * + * @param modificationTime the commit to consider for reading the table state + * @return + */ + @Override + public InternalTable getTable(Long modificationTime) { + + Optional latestFile = + getParquetFiles(hadoopConf, basePath) + .max(Comparator.comparing(FileStatus::getModificationTime)); + + ParquetMetadata parquetMetadata = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); + Schema tableSchema = + new org.apache.parquet.parquet.ParquetSchemaConverter() + .convert(parquetMetadata.getFileMetaData().getSchema()); + + Set partitionKeys = initPartitionInfo().keySet(); + + // merge schema of partition into original as partition is not part of parquet fie + if (!partitionKeys.isEmpty()) { + tableSchema = mergeParquetSchema(tableSchema, partitionKeys); } - - /** - * To infer schema getting the latest file assumption is that latest file will have new fields - * - * @param modificationTime the commit to consider for reading the table state - * @return - */ - @Override - public InternalTable getTable(Long modificationTime) { - - Optional latestFile = - - getParquetFiles(hadoopConf, basePath) - .max(Comparator.comparing(FileStatus::getModificationTime)); - - ParquetMetadata parquetMetadata = - parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - Schema tableSchema = - new org.apache.parquet.parquet.ParquetSchemaConverter() - .convert(parquetMetadata.getFileMetaData().getSchema()); - - Set partitionKeys = initPartitionInfo().keySet(); - - // merge schema of partition into original as partition is not part of parquet fie - if (!partitionKeys.isEmpty()) { - tableSchema = mergeParquetSchema(tableSchema, partitionKeys); - } - InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); - - List partitionFields = - partitionKeys.isEmpty() - ? Collections.emptyList() - : parquetPartitionHelper.getInternalPartitionField(partitionKeys, schema); - DataLayoutStrategy dataLayoutStrategy = - partitionFields.isEmpty() - ? DataLayoutStrategy.FLAT - : DataLayoutStrategy.HIVE_STYLE_PARTITION; - return InternalTable.builder() - .tableFormat(TableFormat.PARQUET) - .basePath(basePath) - .name(tableName) - .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) - .build(); - } - - /** - * Here to get current snapshot listing all files hence the -1 is being passed - * - * @return - */ - @Override - public InternalSnapshot getCurrentSnapshot() { - - List latestFile = - getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - Map> partitionInfo = initPartitionInfo(); - InternalTable table = getTable(-1L); - List internalDataFiles = - latestFile.stream() - .map( - file -> - InternalDataFile.builder() - .physicalPath(file.getPath().toString()) - .fileFormat(FileFormat.APACHE_PARQUET) - .fileSizeBytes(file.getLen()) - .partitionValues( - parquetPartitionHelper.getPartitionValue( - basePath, - file.getPath().toString(), - table.getReadSchema(), - partitionInfo)) - .lastModified(file.getModificationTime()) - .columnStats( - parquetMetadataExtractor.getColumnStatsForaFile( - hadoopConf, file, table)) - .build()) - .collect(Collectors.toList()); - - return InternalSnapshot.builder() - .table(table) - .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) - .build(); - } - - /** - * Whenever new file is added , condition to get new file is listing files whose modification time - * is greater than previous ysnc - * - * @param modificationTime commit to capture table changes for. - * @return - */ - @Override - public TableChange getTableChangeForCommit(Long modificationTime) { - List tableChanges = - - getParquetFiles(hadoopConf, basePath) - .filter(fileStatus -> fileStatus.getModificationTime() > modificationTime) - .collect(Collectors.toList()); - // TODO avoid doing full list of directory to get schema , just argument of modification time - // needs to be tweaked - InternalTable internalTable = getTable(-1L); - Set internalDataFiles = new HashSet<>(); - Map> partitionInfo = initPartitionInfo(); - for (FileStatus tableStatus : tableChanges) { - internalDataFiles.add( + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); + + List partitionFields = + partitionKeys.isEmpty() + ? Collections.emptyList() + : parquetPartitionHelper.getInternalPartitionField(partitionKeys, schema); + DataLayoutStrategy dataLayoutStrategy = + partitionFields.isEmpty() + ? DataLayoutStrategy.FLAT + : DataLayoutStrategy.HIVE_STYLE_PARTITION; + return InternalTable.builder() + .tableFormat(TableFormat.PARQUET) + .basePath(basePath) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) + .build(); + } + + /** + * Here to get current snapshot listing all files hence the -1 is being passed + * + * @return + */ + @Override + public InternalSnapshot getCurrentSnapshot() { + + List latestFile = + getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); + Map> partitionInfo = initPartitionInfo(); + InternalTable table = getTable(-1L); + List internalDataFiles = + latestFile.stream() + .map( + file -> InternalDataFile.builder() - .physicalPath(tableStatus.getPath().toString()) - .partitionValues( - parquetPartitionHelper.getPartitionValue( - basePath, - tableStatus.getPath().toString(), - internalTable.getReadSchema(), - partitionInfo)) - .lastModified(tableStatus.getModificationTime()) - .fileSizeBytes(tableStatus.getLen()) - .columnStats( - parquetMetadataExtractor.getColumnStatsForaFile( - hadoopConf, tableStatus, internalTable)) - .build()); - } - - return TableChange.builder() - .tableAsOfChange(internalTable) - .filesDiff(DataFilesDiff.builder().filesAdded(internalDataFiles).build()) - .build(); + .physicalPath(file.getPath().toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .fileSizeBytes(file.getLen()) + .partitionValues( + parquetPartitionHelper.getPartitionValue( + basePath, + file.getPath().toString(), + table.getReadSchema(), + partitionInfo)) + .lastModified(file.getModificationTime()) + .columnStats( + parquetMetadataExtractor.getColumnStatsForaFile( + hadoopConf, file, table)) + .build()) + .collect(Collectors.toList()); + + return InternalSnapshot.builder() + .table(table) + .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) + .build(); + } + + /** + * Whenever new file is added , condition to get new file is listing files whose modification time + * is greater than previous ysnc + * + * @param modificationTime commit to capture table changes for. + * @return + */ + @Override + public TableChange getTableChangeForCommit(Long modificationTime) { + List tableChanges = + getParquetFiles(hadoopConf, basePath) + .filter(fileStatus -> fileStatus.getModificationTime() > modificationTime) + .collect(Collectors.toList()); + // TODO avoid doing full list of directory to get schema , just argument of modification time + // needs to be tweaked + InternalTable internalTable = getTable(-1L); + Set internalDataFiles = new HashSet<>(); + Map> partitionInfo = initPartitionInfo(); + for (FileStatus tableStatus : tableChanges) { + internalDataFiles.add( + InternalDataFile.builder() + .physicalPath(tableStatus.getPath().toString()) + .partitionValues( + parquetPartitionHelper.getPartitionValue( + basePath, + tableStatus.getPath().toString(), + internalTable.getReadSchema(), + partitionInfo)) + .lastModified(tableStatus.getModificationTime()) + .fileSizeBytes(tableStatus.getLen()) + .columnStats( + parquetMetadataExtractor.getColumnStatsForaFile( + hadoopConf, tableStatus, internalTable)) + .build()); } - @Override - public CommitsBacklog getCommitsBacklog( - InstantsForIncrementalSync instantsForIncrementalSync) { + return TableChange.builder() + .tableAsOfChange(internalTable) + .filesDiff(DataFilesDiff.builder().filesAdded(internalDataFiles).build()) + .build(); + } - List commitsToProcess = - Collections.singletonList(instantsForIncrementalSync.getLastSyncInstant().toEpochMilli()); + @Override + public CommitsBacklog getCommitsBacklog( + InstantsForIncrementalSync instantsForIncrementalSync) { - return CommitsBacklog.builder().commitsToProcess(commitsToProcess).build(); - } + List commitsToProcess = + Collections.singletonList(instantsForIncrementalSync.getLastSyncInstant().toEpochMilli()); - // TODO Need to understnad how this needs to be implemented should _SUCCESS or .staging dir needs - // to be checked - @Override - public boolean isIncrementalSyncSafeFrom(Instant instant) { - return true; - } - - @Override - public void close() throws IOException {} + return CommitsBacklog.builder().commitsToProcess(commitsToProcess).build(); + } - private Schema mergeParquetSchema(Schema internalSchema, Set parititonFields) { + // TODO Need to understnad how this needs to be implemented should _SUCCESS or .staging dir needs + // to be checked + @Override + public boolean isIncrementalSyncSafeFrom(Instant instant) { + return true; + } - SchemaBuilder.FieldAssembler fieldAssembler = - SchemaBuilder.record(internalSchema.getName()).fields(); - for (Schema.Field field : internalSchema.getFields()) { - fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); - } + @Override + public void close() throws IOException {} - for (String paritionKey : parititonFields) { - fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); - } + private Schema mergeParquetSchema(Schema internalSchema, Set parititonFields) { - return fieldAssembler.endRecord(); + SchemaBuilder.FieldAssembler fieldAssembler = + SchemaBuilder.record(internalSchema.getName()).fields(); + for (Schema.Field field : internalSchema.getFields()) { + fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); } - - public Stream getParquetFiles(Configuration hadoopConf, String basePath) { - try { - FileSystem fs = FileSystem.get(hadoopConf); - RemoteIterator iterator = fs.listFiles(new Path(basePath), true); - return remoteIteratorToStream(iterator) - .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException|FileNotFoundException e) { - throw new RuntimeException(e); - } + for (String paritionKey : parititonFields) { + fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); } - public Map> getPartitionFromDirectoryStructure( - Configuration hadoopConf, String basePath, Map> partitionMap) { - - try { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); - Map> currentPartitionMap = new HashMap<>(partitionMap); - - for (FileStatus dirStatus : baseFileStatus) { - if (dirStatus.isDirectory()) { - String partitionPath = dirStatus.getPath().getName(); - if (partitionPath.contains("=")) { - String[] partitionKeyValue = partitionPath.split("="); - currentPartitionMap - .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) - .add(partitionKeyValue[1]); - getPartitionFromDirectoryStructure( - hadoopConf, dirStatus.getPath().toString(), partitionMap); - } - } - } - return currentPartitionMap; - - } catch (IOException e) { - throw new RuntimeException(e); + return fieldAssembler.endRecord(); + } + + public Stream getParquetFiles(Configuration hadoopConf, String basePath) { + try { + FileSystem fs = FileSystem.get(hadoopConf); + RemoteIterator iterator = fs.listFiles(new Path(basePath), true); + return remoteIteratorToStream(iterator) + .filter(file -> file.getPath().getName().endsWith("parquet")); + } catch (IOException | FileNotFoundException e) { + throw new RuntimeException(e); + } + } + + public Map> getPartitionFromDirectoryStructure( + Configuration hadoopConf, String basePath, Map> partitionMap) { + + try { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); + Map> currentPartitionMap = new HashMap<>(partitionMap); + + for (FileStatus dirStatus : baseFileStatus) { + if (dirStatus.isDirectory()) { + String partitionPath = dirStatus.getPath().getName(); + if (partitionPath.contains("=")) { + String[] partitionKeyValue = partitionPath.split("="); + currentPartitionMap + .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) + .add(partitionKeyValue[1]); + getPartitionFromDirectoryStructure( + hadoopConf, dirStatus.getPath().toString(), partitionMap); + } } + } + return currentPartitionMap; + + } catch (IOException e) { + throw new RuntimeException(e); } -} \ No newline at end of file + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 3839395f8..6c70c8241 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.util.ArrayList; @@ -47,423 +47,424 @@ /** * 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. + * 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 ParquetSchemaConverter { - // 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 ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); - private static final String ELEMENT = "element"; - private static final String KEY = "key"; - private static final String VALUE = "value"; + // 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 ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); + private static final String ELEMENT = "element"; + private static final String KEY = "key"; + private static final String VALUE = "value"; - public static parquetSchemaConverter getInstance() { - return INSTANCE; - } + public static parquetSchemaConverter getInstance() { + return INSTANCE; + } - public InternalSchema toInternalSchema(Schema schema) { - Map fieldNameToIdMapping = - IdTracker.getInstance() - .getIdTracking(schema) - .map( - idTracking -> - idTracking.getIdMappings().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) - .orElse(Collections.emptyMap()); - return toInternalSchema(schema, null, fieldNameToIdMapping); - } + public InternalSchema toInternalSchema(Schema schema) { + Map fieldNameToIdMapping = + IdTracker.getInstance() + .getIdTracking(schema) + .map( + idTracking -> + idTracking.getIdMappings().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) + .orElse(Collections.emptyMap()); + return toInternalSchema(schema, null, fieldNameToIdMapping); + } - /** - * 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. - * @param fieldNameToIdMapping map of fieldName to IdMapping to track field IDs provided by the - * source schema. If source schema does not contain IdMappings, map will be empty. - * @return a converted schema - */ - private InternalSchema toInternalSchema( - Schema schema, String parentPath, Map fieldNameToIdMapping) { - // TODO - Does not handle recursion in parquet schema - InternalType newDataType; - Map metadata = new HashMap<>(); - switch (schema.getType()) { - case INT: - LogicalType logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.Date) { - newDataType = InternalType.DATE; - } else { - newDataType = InternalType.INT; - } - break; - case STRING: - newDataType = InternalType.STRING; - break; - case BOOLEAN: - newDataType = InternalType.BOOLEAN; - break; - case BYTES: - case JSON: - case BSON: - case FIXED: - logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.Decimal) { - metadata.put( - InternalSchema.MetadataKey.DECIMAL_PRECISION, - ((LogicalTypes.Decimal) logicalType).getPrecision()); - metadata.put( - InternalSchema.MetadataKey.DECIMAL_SCALE, - ((LogicalTypes.Decimal) logicalType).getScale()); - if (schema.getType() == Schema.Type.FIXED) { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - } - newDataType = InternalType.DECIMAL; - break; - } - if (schema.getType() == Schema.Type.FIXED) { - String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); - if ("uuid".equals(xtableLogicalType)) { - newDataType = InternalType.UUID; - } else { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - newDataType = InternalType.FIXED; - } - } else { - newDataType = InternalType.BYTES; - } - break; - case DOUBLE: - newDataType = InternalType.DOUBLE; - break; - case FLOAT: - newDataType = InternalType.FLOAT; - break; - case LONG: - logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.TimestampMillis) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else { - newDataType = InternalType.LONG; - } - break; - case ENUM: - metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.getEnumSymbols()); - newDataType = InternalType.ENUM; - break; - case NULL: - newDataType = InternalType.NULL; - break; - case RECORD: - List subFields = new ArrayList<>(schema.getFields().size()); - for (Schema.Field parquetField : schema.getFields()) { - IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); - InternalSchema subFieldSchema = - toInternalSchema( - parquetField.schema(), - SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), - getChildIdMap(idMapping)); - Object defaultValue = getDefaultValue(parquetField); - subFields.add( - InternalField.builder() - .parentPath(parentPath) - .name(parquetField.name()) - .schema(subFieldSchema) - .defaultValue(defaultValue) - .fieldId(idMapping == null ? null : idMapping.getId()) - .build()); - } - return InternalSchema.builder() - .name(schema.getName()) - .comment(schema.getDoc()) - .dataType(InternalType.RECORD) - .fields(subFields) - .isNullable(schema.isNullable()) - .build(); - case ARRAY: - IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); - InternalSchema elementSchema = - toInternalSchema( - schema.getElementType(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), - getChildIdMap(elementMapping)); - InternalField elementField = - InternalField.builder() - .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) - .parentPath(parentPath) - .schema(elementSchema) - .fieldId(elementMapping == null ? null : elementMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.LIST) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .fields(Collections.singletonList(elementField)) - .build(); - case MAP: - IdMapping keyMapping = fieldNameToIdMapping.get(KEY); - IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); - InternalSchema valueSchema = - toInternalSchema( - schema.getValueType(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), - getChildIdMap(valueMapping)); - InternalField valueField = - InternalField.builder() - .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) - .parentPath(parentPath) - .schema(valueSchema) - .fieldId(valueMapping == null ? null : valueMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.MAP) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .fields( - Arrays.asList( - MAP_KEY_FIELD.toBuilder() - .parentPath(parentPath) - .fieldId(keyMapping == null ? null : keyMapping.getId()) - .build(), - valueField)) - .build(); - case UNION: - boolean containsUnionWithNull = - schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); - if (containsUnionWithNull) { - List remainingSchemas = - schema.getTypes().stream() - .filter(t -> t.getType() != Schema.Type.NULL) - .collect(Collectors.toList()); - if (remainingSchemas.size() == 1) { - InternalSchema restSchema = - toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); - return InternalSchema.builderFrom(restSchema).isNullable(true).build(); - } else { - return InternalSchema.builderFrom( - toInternalSchema(Schema.createUnion(remainingSchemas))) - .isNullable(true) - .build(); - } - } else { - throw new UnsupportedSchemaTypeException( - String.format("Unsupported complex union type %s", schema)); - } - default: - throw new UnsupportedSchemaTypeException( - String.format("Unsupported schema type %s", schema)); + /** + * 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. + * @param fieldNameToIdMapping map of fieldName to IdMapping to track field IDs provided by the + * source schema. If source schema does not contain IdMappings, map will be empty. + * @return a converted schema + */ + private InternalSchema toInternalSchema( + Schema schema, String parentPath, Map fieldNameToIdMapping) { + // TODO - Does not handle recursion in parquet schema + InternalType newDataType; + Map metadata = new HashMap<>(); + switch (schema.getType()) { + case INT: + LogicalType logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.Date) { + newDataType = InternalType.DATE; + } else { + newDataType = InternalType.INT; + } + break; + case STRING: + newDataType = InternalType.STRING; + break; + case BOOLEAN: + newDataType = InternalType.BOOLEAN; + break; + case BYTES: + case JSON: + case BSON: + case FIXED: + logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.Decimal) { + metadata.put( + InternalSchema.MetadataKey.DECIMAL_PRECISION, + ((LogicalTypes.Decimal) logicalType).getPrecision()); + metadata.put( + InternalSchema.MetadataKey.DECIMAL_SCALE, + ((LogicalTypes.Decimal) logicalType).getScale()); + if (schema.getType() == Schema.Type.FIXED) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + } + newDataType = InternalType.DECIMAL; + break; + } + if (schema.getType() == Schema.Type.FIXED) { + String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); + if ("uuid".equals(xtableLogicalType)) { + newDataType = InternalType.UUID; + } else { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + newDataType = InternalType.FIXED; + } + } else { + newDataType = InternalType.BYTES; + } + break; + case DOUBLE: + newDataType = InternalType.DOUBLE; + break; + case FLOAT: + newDataType = InternalType.FLOAT; + break; + case LONG: + logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.TimestampMillis) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else { + newDataType = InternalType.LONG; + } + break; + case ENUM: + metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.getEnumSymbols()); + newDataType = InternalType.ENUM; + break; + case NULL: + newDataType = InternalType.NULL; + break; + case RECORD: + List subFields = new ArrayList<>(schema.getFields().size()); + for (Schema.Field parquetField : schema.getFields()) { + IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); + InternalSchema subFieldSchema = + toInternalSchema( + parquetField.schema(), + SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), + getChildIdMap(idMapping)); + Object defaultValue = getDefaultValue(parquetField); + subFields.add( + InternalField.builder() + .parentPath(parentPath) + .name(parquetField.name()) + .schema(subFieldSchema) + .defaultValue(defaultValue) + .fieldId(idMapping == null ? null : idMapping.getId()) + .build()); } return InternalSchema.builder() - .name(schema.getName()) - .dataType(newDataType) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .metadata(metadata.isEmpty() ? null : metadata) + .name(schema.getName()) + .comment(schema.getDoc()) + .dataType(InternalType.RECORD) + .fields(subFields) + .isNullable(schema.isNullable()) + .build(); + case ARRAY: + IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + InternalSchema elementSchema = + toInternalSchema( + schema.getElementType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), + getChildIdMap(elementMapping)); + InternalField elementField = + InternalField.builder() + .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) + .parentPath(parentPath) + .schema(elementSchema) + .fieldId(elementMapping == null ? null : elementMapping.getId()) .build(); - } - - private Map getChildIdMap(IdMapping idMapping) { - if (idMapping == null) { - return Collections.emptyMap(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.LIST) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields(Collections.singletonList(elementField)) + .build(); + case MAP: + IdMapping keyMapping = fieldNameToIdMapping.get(KEY); + IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); + InternalSchema valueSchema = + toInternalSchema( + schema.getValueType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), + getChildIdMap(valueMapping)); + InternalField valueField = + InternalField.builder() + .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) + .parentPath(parentPath) + .schema(valueSchema) + .fieldId(valueMapping == null ? null : valueMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.MAP) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields( + Arrays.asList( + MAP_KEY_FIELD.toBuilder() + .parentPath(parentPath) + .fieldId(keyMapping == null ? null : keyMapping.getId()) + .build(), + valueField)) + .build(); + case UNION: + boolean containsUnionWithNull = + schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); + if (containsUnionWithNull) { + List remainingSchemas = + schema.getTypes().stream() + .filter(t -> t.getType() != Schema.Type.NULL) + .collect(Collectors.toList()); + if (remainingSchemas.size() == 1) { + InternalSchema restSchema = + toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); + return InternalSchema.builderFrom(restSchema).isNullable(true).build(); + } else { + return InternalSchema.builderFrom( + toInternalSchema(Schema.createUnion(remainingSchemas))) + .isNullable(true) + .build(); + } + } else { + throw new UnsupportedSchemaTypeException( + String.format("Unsupported complex union type %s", schema)); } - return idMapping.getFields().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity())); + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); } + return InternalSchema.builder() + .name(schema.getName()) + .dataType(newDataType) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .metadata(metadata.isEmpty() ? null : metadata) + .build(); + } - private static Object getDefaultValue(Schema.Field parquetField) { - return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) - ? InternalField.Constants.NULL_DEFAULT_VALUE - : parquetField.defaultVal(); + private Map getChildIdMap(IdMapping idMapping) { + if (idMapping == null) { + return Collections.emptyMap(); } + return idMapping.getFields().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity())); + } - /** - * Converts the {@link InternalSchema} to parquet {@link Schema}. - * - * @param internalSchema internal schema representation - * @return an parquet schema - */ - public Schema fromInternalSchema(InternalSchema internalSchema) { - return fromInternalSchema(internalSchema, null); - } + private static Object getDefaultValue(Schema.Field parquetField) { + return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) + ? InternalField.Constants.NULL_DEFAULT_VALUE + : parquetField.defaultVal(); + } - /** - * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. - * - * @param internalSchema internal schema representation - * @param currentPath If this schema is nested within another, this will be a dot separated - * string. This is used for the parquet namespace to guarantee unique names for nested records. - * @return an parquet schema - */ - private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { - switch (internalSchema.getDataType()) { - case RECORD: - List fields = - internalSchema.getFields().stream() - .map( - field -> - new Schema.Field( - field.getName(), - fromInternalSchema( - field.getSchema(), - SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), - field.getSchema().getComment(), - InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() - ? Schema.Field.NULL_VALUE - : field.getDefaultValue())) - .collect(CustomCollectors.toList(internalSchema.getFields().size())); - return finalizeSchema( - Schema.createRecord( - internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), - internalSchema); - case BYTES: - return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); - case BOOLEAN: - return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); - case INT: - return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); - case LONG: - return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); - case STRING: - return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); - case FLOAT: - return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); - case DOUBLE: - return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); - case ENUM: - return finalizeSchema( - Schema.createEnum( - internalSchema.getName(), - internalSchema.getComment(), - null, - (List) - internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null), - internalSchema); - case DATE: - return finalizeSchema( - LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); - case TIMESTAMP: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } - case TIMESTAMP_NTZ: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } - case LIST: - InternalField elementField = - internalSchema.getFields().stream() - .filter( - field -> - InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); - return finalizeSchema( - Schema.createArray( - fromInternalSchema(elementField.getSchema(), elementField.getPath())), - internalSchema); - case MAP: - InternalField valueField = - internalSchema.getFields().stream() - .filter( - field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); - return finalizeSchema( - Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), - internalSchema); - case DECIMAL: - int precision = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); - int scale = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); - Integer size = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - if (size == null) { - return finalizeSchema( - LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.decimal(precision, scale) - .addToSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, size)), - internalSchema); - } - case FIXED: - Integer fixedSize = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - return finalizeSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, fixedSize), - internalSchema); - case UUID: - Schema uuidSchema = - Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); - uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); - return finalizeSchema(uuidSchema, internalSchema); - default: - throw new UnsupportedSchemaTypeException( - "Encountered unhandled type during InternalSchema to parquet conversion: " - + internalSchema.getDataType()); + /** + * Converts the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @return an parquet schema + */ + public Schema fromInternalSchema(InternalSchema internalSchema) { + return fromInternalSchema(internalSchema, null); + } + + /** + * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @param currentPath If this schema is nested within another, this will be a dot separated + * string. This is used for the parquet namespace to guarantee unique names for nested + * records. + * @return an parquet schema + */ + private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { + switch (internalSchema.getDataType()) { + case RECORD: + List fields = + internalSchema.getFields().stream() + .map( + field -> + new Schema.Field( + field.getName(), + fromInternalSchema( + field.getSchema(), + SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), + field.getSchema().getComment(), + InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() + ? Schema.Field.NULL_VALUE + : field.getDefaultValue())) + .collect(CustomCollectors.toList(internalSchema.getFields().size())); + return finalizeSchema( + Schema.createRecord( + internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), + internalSchema); + case BYTES: + return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); + case BOOLEAN: + return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); + case INT: + return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); + case LONG: + return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); + case STRING: + return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); + case FLOAT: + return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); + case DOUBLE: + return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); + case ENUM: + return finalizeSchema( + Schema.createEnum( + internalSchema.getName(), + internalSchema.getComment(), + null, + (List) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), + null), + internalSchema); + case DATE: + return finalizeSchema( + LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); + case TIMESTAMP: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case TIMESTAMP_NTZ: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case LIST: + InternalField elementField = + internalSchema.getFields().stream() + .filter( + field -> + InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); + return finalizeSchema( + Schema.createArray( + fromInternalSchema(elementField.getSchema(), elementField.getPath())), + internalSchema); + case MAP: + InternalField valueField = + internalSchema.getFields().stream() + .filter( + field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); + return finalizeSchema( + Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), + internalSchema); + case DECIMAL: + int precision = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); + int scale = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); + Integer size = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + if (size == null) { + return finalizeSchema( + LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.decimal(precision, scale) + .addToSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, size)), + internalSchema); } + case FIXED: + Integer fixedSize = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + return finalizeSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, fixedSize), + internalSchema); + case UUID: + Schema uuidSchema = + Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); + uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); + return finalizeSchema(uuidSchema, internalSchema); + default: + throw new UnsupportedSchemaTypeException( + "Encountered unhandled type during InternalSchema to parquet conversion: " + + internalSchema.getDataType()); } + } - private String buildCurrentPath(InternalField field, String parentPath) { - return Optional.ofNullable(parentPath) - .map(path -> path + "." + field.getName()) - .orElse(field.getName()); - } + private String buildCurrentPath(InternalField field, String parentPath) { + return Optional.ofNullable(parentPath) + .map(path -> path + "." + field.getName()) + .orElse(field.getName()); + } - private static Schema finalizeSchema(Schema targetSchema, InternalSchema inputSchema) { - if (inputSchema.isNullable()) { - return Schema.createUnion(Schema.create(Schema.Type.NULL), targetSchema); - } - return targetSchema; + private static Schema finalizeSchema(Schema targetSchema, InternalSchema inputSchema) { + if (inputSchema.isNullable()) { + return Schema.createUnion(Schema.create(Schema.Type.NULL), targetSchema); } + return targetSchema; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 0be978274..90f8b52df 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.delta; import java.time.Instant; @@ -23,47 +23,43 @@ import lombok.Builder; -import org.apache.spark.sql.delta.DeltaLog; -import org.apache.spark.sql.delta.Snapshot; - -import scala.Option; +import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.xtable.model.InternalTable; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.storage.DataLayoutStrategy; import org.apache.xtable.model.storage.TableFormat; -import org.apache.parquet.format.FileMetaData; -import org.apache.parquet.hadoop.ParquetFileReader /** * Extracts {@link InternalTable} canonical representation of a table at a point in time for Delta. */ @Builder public class ParquetTableExtractor { - @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); - public InternalTable table(String tableName, Long version) { - ParquetMetadata readFooter = ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); - MessageType schema = readFooter.getFileMetaData().getSchema(); - InternalSchema schema = schemaExtractor.toInternalSchema(schema); - // TODO check partitionSchema of Parquet File - List partitionFields = - ParquetPartitionExtractor.getInstance() - .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); - DataLayoutStrategy dataLayoutStrategy = - !partitionFields.isEmpty() - ? DataLayoutStrategy.HIVE_STYLE_PARTITION - : DataLayoutStrategy.FLAT; - return InternalTable.builder() - .tableFormat(TableFormat.APACHE_PARQUET) - .basePath(snapshot.deltaLog().dataPath().toString()) - .name(tableName) - .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(snapshot.timestamp())) - .build(); - } + public InternalTable table(String tableName, Long version) { + ParquetMetadata readFooter = + ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + MessageType schema = readFooter.getFileMetaData().getSchema(); + InternalSchema schema = schemaExtractor.toInternalSchema(schema); + // TODO check partitionSchema of Parquet File + List partitionFields = + ParquetPartitionExtractor.getInstance() + .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); + DataLayoutStrategy dataLayoutStrategy = + !partitionFields.isEmpty() + ? DataLayoutStrategy.HIVE_STYLE_PARTITION + : DataLayoutStrategy.FLAT; + return InternalTable.builder() + .tableFormat(TableFormat.APACHE_PARQUET) + .basePath(snapshot.deltaLog().dataPath().toString()) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(snapshot.timestamp())) + .build(); + } } diff --git a/xtable-core/src/test/java/org/apache/xtable/TestAbstractHudiTable.java b/xtable-core/src/test/java/org/apache/xtable/TestAbstractHudiTable.java index 3e9a133a2..1e50066a6 100644 --- a/xtable-core/src/test/java/org/apache/xtable/TestAbstractHudiTable.java +++ b/xtable-core/src/test/java/org/apache/xtable/TestAbstractHudiTable.java @@ -127,6 +127,7 @@ public abstract class TestAbstractHudiTable throw new UncheckedIOException(ex); } } + // Name of the table protected String tableName; // Base path for the table diff --git a/xtable-core/src/test/java/org/apache/xtable/TestJavaHudiTable.java b/xtable-core/src/test/java/org/apache/xtable/TestJavaHudiTable.java index ce3b25bda..abbe7fe67 100644 --- a/xtable-core/src/test/java/org/apache/xtable/TestJavaHudiTable.java +++ b/xtable-core/src/test/java/org/apache/xtable/TestJavaHudiTable.java @@ -66,6 +66,7 @@ public class TestJavaHudiTable extends TestAbstractHudiTable { private HoodieJavaWriteClient writeClient; private final Configuration conf; + /** * Create a test table instance for general testing. The table is created with the schema defined * in basic_schema.avsc which contains many data types to ensure they are handled correctly. diff --git a/xtable-core/src/test/java/org/apache/xtable/iceberg/StubCatalog.java b/xtable-core/src/test/java/org/apache/xtable/iceberg/StubCatalog.java index a25063c42..9475d5038 100644 --- a/xtable-core/src/test/java/org/apache/xtable/iceberg/StubCatalog.java +++ b/xtable-core/src/test/java/org/apache/xtable/iceberg/StubCatalog.java @@ -36,6 +36,7 @@ public class StubCatalog implements Catalog { public static void registerMock(String catalogName, Catalog catalog) { REGISTERED_MOCKS.put(catalogName, catalog); } + // use a mocked catalog instance to more easily test private Catalog mockedCatalog; diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunCatalogSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunCatalogSync.java index 349b5ca93..33bf1c80b 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunCatalogSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunCatalogSync.java @@ -283,11 +283,13 @@ public static class DatasetConfig { * necessary connection and access details for describing and listing tables */ ExternalCatalogConfig sourceCatalog; + /** * Defines configuration one or more target catalogs, to which XTable will write or update * tables. Unlike the source, these catalogs must be writable */ List targetCatalogs; + /** A list of datasets that specify how a source table maps to one or more target tables. */ List datasets; @@ -300,6 +302,7 @@ public static class DatasetConfig { public static class Dataset { /** Identifies the source table in sourceCatalog. */ SourceTableIdentifier sourceCatalogTableIdentifier; + /** A list of one or more targets that this source table should be written to. */ List targetCatalogTableIdentifiers; } @@ -310,6 +313,7 @@ public static class Dataset { public static class SourceTableIdentifier { /** Specifies the table identifier in the source catalog. */ TableIdentifier tableIdentifier; + /** * (Optional) Provides direct storage details such as a table’s base path (like an S3 * location) and the partition specification. This allows reading from a source even if it is @@ -327,11 +331,13 @@ public static class TargetTableIdentifier { * updated */ String catalogId; + /** * The target table format (e.g., DELTA, HUDI, ICEBERG), specifying how the data will be * stored at the target. */ String tableFormat; + /** Specifies the table identifier in the target catalog. */ TableIdentifier tableIdentifier; } From 4f1ea77a31bc2834fdb6920b65fce6f1d52d31f2 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 18 Feb 2025 21:52:41 +0100 Subject: [PATCH 04/49] added byte_array data type --- .../xtable/model/schema/InternalType.java | 7 +++++- .../parquet/ParquetSchemaExtractror.java | 22 ++++++++++++++++--- 2 files changed, 25 insertions(+), 4 deletions(-) diff --git a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java index e1b1049d0..2a2ad7eef 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java @@ -51,7 +51,12 @@ public enum InternalType { DATE, DECIMAL, TIMESTAMP, - TIMESTAMP_NTZ; + TIMESTAMP_NTZ, + JSON, + BSON, + VARIANT, + GEOMETRY, + GEOGRAPHY; private final String name; InternalType() { diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 6c70c8241..48162413a 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -115,9 +115,25 @@ private InternalSchema toInternalSchema( case BOOLEAN: newDataType = InternalType.BOOLEAN; break; - case BYTES: - case JSON: - case BSON: + case BYTE_ARRAY: + logicalType = schema.getLogicalType(); + // TODO: any metadata to add ? + if (logicalType== LogicalTypes.JSON) { + newDataType = InternalType.JSON; + } + else if (logicalType instanceof LogicalTypes.BSON) { + newDataType = InternalType.BSON; + } + else if (logicalType instanceof LogicalTypes.VARIANT) { + newDataType = InternalType.VARIANT; + } + else if (logicalType instanceof LogicalTypes.GEOMETRY) { + newDataType = InternalType.GEOMETRY; + } + else if (logicalType instanceof LogicalTypes.GEOGRAPHY) { + newDataType = InternalType.GEOGRAPHY; + } + break; case FIXED: logicalType = schema.getLogicalType(); if (logicalType instanceof LogicalTypes.Decimal) { From f71610b2d315bfc2a5d20984081b02c34b35b9ee Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Thu, 20 Feb 2025 12:51:56 +0100 Subject: [PATCH 05/49] added ParquetStatsExtractor --- .../xtable/parquet/ParquetStatsExtractor.java | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java new file mode 100644 index 000000000..169180ba1 --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -0,0 +1,33 @@ + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HiddenFileFilter; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.schema.MessageType; + +// TODO add other methods of stats (row group vs columns) +public class ParquetTableExtractor { + private static void add(ParquetMetadata footer) { + for (BlockMetaData blockMetaData : footer.getBlocks()) { + + MessageType schema = footer.getFileMetaData().getSchema(); + recordCount += blockMetaData.getRowCount(); + List columns = blockMetaData.getColumns(); + for (ColumnChunkMetaData columnMetaData : columns) { + ColumnDescriptor desc = + schema.getColumnDescription(columnMetaData.getPath().toArray()); + add( + desc, + columnMetaData.getValueCount(), + columnMetaData.getTotalSize(), + columnMetaData.getTotalUncompressedSize(), + columnMetaData.getEncodings(), + columnMetaData.getStatistics()); + } + } + } +} \ No newline at end of file From c57a42f97063ea3157a612abbef6cdd6cd888e8e Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 22 Feb 2025 21:07:40 +0100 Subject: [PATCH 06/49] added InternalDataFile population from parquet metadata --- .../model/storage/InternalDataFile.java | 4 ++ .../parquet/ParquetConversionSource.java | 8 ++- .../parquet/ParquetMetadataExtractor.java | 21 ++++++ .../xtable/parquet/ParquetStatsExtractor.java | 66 ++++++++++++++++++- .../xtable/parquet/ParquetTableExtractor.java | 9 ++- 5 files changed, 99 insertions(+), 9 deletions(-) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java diff --git a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java index 3aee766ef..de6e98fb9 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java @@ -49,4 +49,8 @@ public class InternalDataFile { @Builder.Default @NonNull List columnStats = Collections.emptyList(); // last modified time in millis since epoch long lastModified; + + public static InternalDataFileBuilder builderFrom(InternalDataFile dataFile) { + return dataFile.toBuilder(); + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 973ccff30..d243d9668 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -46,8 +46,8 @@ public class ParquetConversionSource implements ConversionSource { @NonNull private final Configuration hadoopConf; @Builder.Default - private static final ParquetSchemaConverter schemaExtractor = - ParquetSchemaConverter.getInstance(); + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = @@ -78,7 +78,7 @@ public InternalTable getTable(Long modificationTime) { parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); Schema tableSchema = new org.apache.parquet.parquet.ParquetSchemaConverter() - .convert(parquetMetadata.getFileMetaData().getSchema()); + .convert(parquetMetadataExtractor.getSchema(parquetMetadata)); Set partitionKeys = initPartitionInfo().keySet(); @@ -127,6 +127,7 @@ public InternalSnapshot getCurrentSnapshot() { .physicalPath(file.getPath().toString()) .fileFormat(FileFormat.APACHE_PARQUET) .fileSizeBytes(file.getLen()) + //TODO create parquetPartitionHelper Class .partitionValues( parquetPartitionHelper.getPartitionValue( basePath, @@ -134,6 +135,7 @@ public InternalSnapshot getCurrentSnapshot() { table.getReadSchema(), partitionInfo)) .lastModified(file.getModificationTime()) + //TODO create getColumnStatsForaFile method in parquetMetadataExtractor class .columnStats( parquetMetadataExtractor.getColumnStatsForaFile( hadoopConf, file, table)) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java new file mode 100644 index 000000000..cbe031f1e --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java @@ -0,0 +1,21 @@ +import org.apache.parquet.schema.MessageType; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.parquet.Schema; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; + +public class ParquetMetadataExtractor { + + private static MessageType getSchema(ParquetMetadata footer) { + MessageType schema = footer.getFileMetaData().getSchema(); + return schema + } + + private static ParquetMetadata readParquetMetadata(HadoopConf conf, BasePath path) { + ParquetMetadata footer = + ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + return footer + + } +} \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index 169180ba1..963b39759 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -8,13 +8,43 @@ import org.apache.parquet.hadoop.util.HiddenFileFilter; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.MessageType; - +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.statistics.Statistics; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.*; // TODO add other methods of stats (row group vs columns) -public class ParquetTableExtractor { +public class ParquetStatsExtractor { + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + + private static Map stats = new LinkedHashMap(); + private InternalDataFile toInternalDataFile(Configuration hadoopConf, + String parentPath, Map stats) { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus file = fs.getFileStatus(new Path(parentPath)); + + return InternalDataFile.builder() + .physicalPath(parentPath) + .fileFormat(FileFormat.APACHE_PARQUET) + //TODO create parquetPartitionHelper Class getPartitionValue( + // basePath, + // file.getPath().toString(), + // table.getReadSchema(), + // partitionInfo)) + .partitionValues(schema.getDoc()) + .fileSizeBytes(file.getLen()) + // TODO check record count of Parquet + .recordCount() + .columnStats(stats.values().stream().collect(Collectors.toList())) + .lastModified(file.getModificationTime()) + .build(); + } private static void add(ParquetMetadata footer) { for (BlockMetaData blockMetaData : footer.getBlocks()) { - MessageType schema = footer.getFileMetaData().getSchema(); + MessageType schema = parquetMetadataExtractor.getSchema(footer) recordCount += blockMetaData.getRowCount(); List columns = blockMetaData.getColumns(); for (ColumnChunkMetaData columnMetaData : columns) { @@ -30,4 +60,34 @@ private static void add(ParquetMetadata footer) { } } } + private static class Stats { + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + long total = 0; + + public void add(long length) { + min = Math.min(length, min); + max = Math.max(length, max); + total += length; + } + } + private static class ColStats { + + Stats valueCountStats = new Stats(); + Stats allStats = new Stats(); + Stats uncStats = new Stats(); + Set encodings = new TreeSet(); + Statistics colValuesStats = null; + int blocks = 0; + + public void add( + long valueCount, long size, long uncSize, Collection encodings, Statistics colValuesStats) { + ++blocks; + valueCountStats.add(valueCount); + allStats.add(size); + uncStats.add(uncSize); + this.encodings.addAll(encodings); + this.colValuesStats = colValuesStats; + } + } } \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 90f8b52df..f35312c51 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -38,11 +38,14 @@ public class ParquetTableExtractor { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); public InternalTable table(String tableName, Long version) { - ParquetMetadata readFooter = - ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); - MessageType schema = readFooter.getFileMetaData().getSchema(); + ParquetMetadata footer = + parquetMetadataExtractor.readParquetMetadata(conf, path, ParquetMetadataConverter.NO_FILTER); + MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); // TODO check partitionSchema of Parquet File List partitionFields = From 1557ea3f1cad45bf2728739e1072aaa06628f44a Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 22 Feb 2025 23:56:03 +0100 Subject: [PATCH 07/49] added col stats for parquet --- .../parquet/ParquetConversionSource.java | 13 ++++++---- .../xtable/parquet/ParquetStatsExtractor.java | 24 +++++++++++++++---- .../xtable/parquet/ParquetTableExtractor.java | 2 +- 3 files changed, 28 insertions(+), 11 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index d243d9668..7a776b0f1 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -53,9 +53,13 @@ public class ParquetConversionSource implements ConversionSource { private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - @Builder.Default + /* @Builder.Default private static final ParquetPartitionHelper parquetPartitionHelper = - ParquetPartitionHelper.getInstance(); + ParquetPartitionHelper.getInstance();*/ + + @Builder.Default + private static final ParquetStatsExtractor parquetStatsExtractor = + ParquetStatsExtractor.getInstance(); private Map> initPartitionInfo() { return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); @@ -135,10 +139,9 @@ public InternalSnapshot getCurrentSnapshot() { table.getReadSchema(), partitionInfo)) .lastModified(file.getModificationTime()) - //TODO create getColumnStatsForaFile method in parquetMetadataExtractor class .columnStats( - parquetMetadataExtractor.getColumnStatsForaFile( - hadoopConf, file, table)) + parquetStatsExtractor.getColumnStatsForaFile(parquetMetadataExtractor.readParquetMetadata( + hadoopConf, file.getPath().toString())) .build()) .collect(Collectors.toList()); diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index 963b39759..e2967db6c 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -20,6 +20,7 @@ public class ParquetStatsExtractor { ParquetMetadataExtractor.getInstance(); private static Map stats = new LinkedHashMap(); + private static long recordCount = 0; private InternalDataFile toInternalDataFile(Configuration hadoopConf, String parentPath, Map stats) { FileSystem fs = FileSystem.get(hadoopConf); @@ -33,15 +34,14 @@ private InternalDataFile toInternalDataFile(Configuration hadoopConf, // file.getPath().toString(), // table.getReadSchema(), // partitionInfo)) - .partitionValues(schema.getDoc()) + .partitionValues(/*schema.getDoc()*/) .fileSizeBytes(file.getLen()) - // TODO check record count of Parquet - .recordCount() + .recordCount(recordCount) .columnStats(stats.values().stream().collect(Collectors.toList())) .lastModified(file.getModificationTime()) .build(); } - private static void add(ParquetMetadata footer) { + private static void getColumnStatsForaFile(ParquetMetadata footer) { for (BlockMetaData blockMetaData : footer.getBlocks()) { MessageType schema = parquetMetadataExtractor.getSchema(footer) @@ -50,7 +50,7 @@ private static void add(ParquetMetadata footer) { for (ColumnChunkMetaData columnMetaData : columns) { ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); - add( + ColStats.add( desc, columnMetaData.getValueCount(), columnMetaData.getTotalSize(), @@ -89,5 +89,19 @@ public void add( this.encodings.addAll(encodings); this.colValuesStats = colValuesStats; } + private static void add( + ColumnDescriptor desc, + long valueCount, + long size, + long uncSize, + Collection encodings, + Statistics colValuesStats) { + ColStats colStats = stats.get(desc); + if (colStats == null) { + colStats = new ColStats(); + stats.put(desc, colStats); + } + colStats.add(valueCount, size, uncSize, encodings, colValuesStats); + } } } \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index f35312c51..6fcc1089d 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -47,7 +47,7 @@ public InternalTable table(String tableName, Long version) { parquetMetadataExtractor.readParquetMetadata(conf, path, ParquetMetadataConverter.NO_FILTER); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); - // TODO check partitionSchema of Parquet File + // TODO check ParquetPartitionExtractor,partitionSchema of Parquet File List partitionFields = ParquetPartitionExtractor.getInstance() .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); From 24c474ada6d2aa798f1fe1b030d7478226ca5b74 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 23 Feb 2025 00:18:50 +0100 Subject: [PATCH 08/49] set todos --- .../org/apache/xtable/parquet/ParquetConversionSource.java | 4 ++-- .../java/org/apache/xtable/parquet/ParquetTableExtractor.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 7a776b0f1..5525d5354 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -53,9 +53,9 @@ public class ParquetConversionSource implements ConversionSource { private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - /* @Builder.Default + @Builder.Default private static final ParquetPartitionHelper parquetPartitionHelper = - ParquetPartitionHelper.getInstance();*/ + ParquetPartitionHelper.getInstance(); @Builder.Default private static final ParquetStatsExtractor parquetStatsExtractor = diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 6fcc1089d..aad2508f7 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -47,7 +47,7 @@ public InternalTable table(String tableName, Long version) { parquetMetadataExtractor.readParquetMetadata(conf, path, ParquetMetadataConverter.NO_FILTER); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); - // TODO check ParquetPartitionExtractor,partitionSchema of Parquet File + // TODO check ParquetPartitionExtractor, how to get partitionSchema of Parquet File List partitionFields = ParquetPartitionExtractor.getInstance() .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); From e1a3f35743e2a8822a677a4a15390c15a1b93e92 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 23 Feb 2025 15:17:14 +0100 Subject: [PATCH 09/49] integrated ParquetPartitionExtractor.java --- .../parquet/ParquetConversionSource.java | 11 ++- .../parquet/ParquetPartitionExtractor.java | 69 +++++++++++++++++++ .../xtable/parquet/ParquetTableExtractor.java | 7 +- 3 files changed, 79 insertions(+), 8 deletions(-) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 5525d5354..f0c30ee3f 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -54,8 +54,8 @@ public class ParquetConversionSource implements ConversionSource { ParquetMetadataExtractor.getInstance(); @Builder.Default - private static final ParquetPartitionHelper parquetPartitionHelper = - ParquetPartitionHelper.getInstance(); + private static final ParquetPartitionExtractor parquetPartitionExtractor = + ParquetPartitionExtractor.getInstance(); @Builder.Default private static final ParquetStatsExtractor parquetStatsExtractor = @@ -95,7 +95,7 @@ public InternalTable getTable(Long modificationTime) { List partitionFields = partitionKeys.isEmpty() ? Collections.emptyList() - : parquetPartitionHelper.getInternalPartitionField(partitionKeys, schema); + : parquetPartitionExtractor.getInternalPartitionField(partitionKeys, schema); DataLayoutStrategy dataLayoutStrategy = partitionFields.isEmpty() ? DataLayoutStrategy.FLAT @@ -131,9 +131,8 @@ public InternalSnapshot getCurrentSnapshot() { .physicalPath(file.getPath().toString()) .fileFormat(FileFormat.APACHE_PARQUET) .fileSizeBytes(file.getLen()) - //TODO create parquetPartitionHelper Class .partitionValues( - parquetPartitionHelper.getPartitionValue( + parquetPartitionExtractor.getPartitionValue( basePath, file.getPath().toString(), table.getReadSchema(), @@ -174,7 +173,7 @@ public TableChange getTableChangeForCommit(Long modificationTime) { InternalDataFile.builder() .physicalPath(tableStatus.getPath().toString()) .partitionValues( - parquetPartitionHelper.getPartitionValue( + parquetPartitionExtractor.getPartitionValue( basePath, tableStatus.getPath().toString(), internalTable.getReadSchema(), diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java new file mode 100644 index 000000000..99f5d8d86 --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -0,0 +1,69 @@ +package org.apache.xtable.parquet; + +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.xtable.model.schema.InternalPartitionField; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.schema.PartitionTransformType; +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.schema.SchemaFieldFinder; + +public class ParquetPartitionExtractor { + private static final ParquetPartitionExtractor INSTANCE = new ParquetPartitionExtractor(); + + public static ParquetPartitionExtractor getInstance() { + return INSTANCE; + } + + public List getInternalPartitionField( + Set partitionList, InternalSchema schema) { + List partitionFields = new ArrayList<>(); + + for (String partitionKey : partitionList) { + + partitionFields.add( + InternalPartitionField.builder() + .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) + .transformType(PartitionTransformType.VALUE) + .build()); + } + + return partitionFields; + } + + // TODO logic is too complicated can be simplified + public List getPartitionValue( + String basePath, + String filePath, + InternalSchema schema, + Map> partitionInfo) { + List partitionValues = new ArrayList<>(); + java.nio.file.Path base = Paths.get(basePath).normalize(); + java.nio.file.Path file = Paths.get(filePath).normalize(); + java.nio.file.Path relative = base.relativize(file); + for (Map.Entry> entry : partitionInfo.entrySet()) { + String key = entry.getKey(); + List values = entry.getValue(); + for (String value : values) { + String pathCheck = key + "=" + value; + if (relative.startsWith(pathCheck)) { + System.out.println("Relative " + relative + " " + pathCheck); + partitionValues.add( + PartitionValue.builder() + .partitionField( + InternalPartitionField.builder() + .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, key)) + .transformType(PartitionTransformType.VALUE) + .build()) + .range(Range.scalar(value)) + .build()); + } + } + } + return partitionValues; + } +} \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index aad2508f7..5c7012490 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -41,16 +41,19 @@ public class ParquetTableExtractor { @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); + private Map> initPartitionInfo() { + return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); + } public InternalTable table(String tableName, Long version) { ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(conf, path, ParquetMetadataConverter.NO_FILTER); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); - // TODO check ParquetPartitionExtractor, how to get partitionSchema of Parquet File + Set partitionKeys = initPartitionInfo().keySet(); List partitionFields = ParquetPartitionExtractor.getInstance() - .convertFromParquetPartitionFormat(schema, fileMetaData.metadata().partitionSchema()); + .convertFromParquetPartitionFormat(partitionKeys,schema); DataLayoutStrategy dataLayoutStrategy = !partitionFields.isEmpty() ? DataLayoutStrategy.HIVE_STYLE_PARTITION From fbbd1eb2f8c9afbb57a120d40a2f8e332705e9c8 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 23 Feb 2025 15:43:10 +0100 Subject: [PATCH 10/49] added partitionValues to StatsExtractor builder --- .../xtable/parquet/ParquetStatsExtractor.java | 22 +++++++++++++------ .../xtable/parquet/ParquetTableExtractor.java | 8 ++++--- 2 files changed, 20 insertions(+), 10 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index e2967db6c..957b3a922 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -13,28 +13,36 @@ import org.apache.parquet.column.statistics.Statistics; import java.util.stream.Collectors; import org.apache.hadoop.fs.*; -// TODO add other methods of stats (row group vs columns) + public class ParquetStatsExtractor { @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); + @Builder.Default + private static final ParquetPartitionExtractor partitionExtractor = ParquetPartitionExtractor.getInstance(); private static Map stats = new LinkedHashMap(); private static long recordCount = 0; + private Map> initPartitionInfo() { + return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); + } private InternalDataFile toInternalDataFile(Configuration hadoopConf, String parentPath, Map stats) { FileSystem fs = FileSystem.get(hadoopConf); FileStatus file = fs.getFileStatus(new Path(parentPath)); + Map> partitionInfo = initPartitionInfo(); + ParquetMetadata footer = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); + MessageType schema = parquetMetadataExtractor.getSchema(footer); + InternalSchema schema = schemaExtractor.toInternalSchema(schema); + List partitionValues = + partitionExtractor + .getPartitionValue(parentPath,file.getPath().toString(),schema,partitionInfo); return InternalDataFile.builder() .physicalPath(parentPath) .fileFormat(FileFormat.APACHE_PARQUET) - //TODO create parquetPartitionHelper Class getPartitionValue( - // basePath, - // file.getPath().toString(), - // table.getReadSchema(), - // partitionInfo)) - .partitionValues(/*schema.getDoc()*/) + .partitionValues(partitionValues) .fileSizeBytes(file.getLen()) .recordCount(recordCount) .columnStats(stats.values().stream().collect(Collectors.toList())) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 5c7012490..49321125f 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -39,6 +39,8 @@ public class ParquetTableExtractor { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); @Builder.Default + private static final ParquetPartitionExtractor partitionExtractor = ParquetPartitionExtractor.getInstance(); + @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); private Map> initPartitionInfo() { @@ -47,13 +49,13 @@ private Map> initPartitionInfo() { public InternalTable table(String tableName, Long version) { ParquetMetadata footer = - parquetMetadataExtractor.readParquetMetadata(conf, path, ParquetMetadataConverter.NO_FILTER); + parquetMetadataExtractor.readParquetMetadata(conf, path); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); Set partitionKeys = initPartitionInfo().keySet(); List partitionFields = - ParquetPartitionExtractor.getInstance() - .convertFromParquetPartitionFormat(partitionKeys,schema); + partitionExtractor + .getInternalPartitionField(partitionKeys,schema); DataLayoutStrategy dataLayoutStrategy = !partitionFields.isEmpty() ? DataLayoutStrategy.HIVE_STYLE_PARTITION From 40c5e670576640001ecbd392c0d881123b85bf09 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 23 Feb 2025 18:30:04 +0100 Subject: [PATCH 11/49] added the parquet conversion source provider --- .../ParquetConversionSourceProvider.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java new file mode 100644 index 000000000..9eeca51fd --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java @@ -0,0 +1,36 @@ +/* + * 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 org.apache.hadoop.conf.Configuration; +import org.apache.xtable.conversion.ConversionSourceProvider; +import org.apache.xtable.conversion.SourceTable; + +/** A concrete implementation of {@link ConversionSourceProvider} for Delta Lake table format. */ +public class ParquetConversionSourceProvider extends ConversionSourceProvider { + @Override + public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { + + return ParquetConversionSource.builder() + .tableName(sourceTable.getName()) + .basePath(sourceTable.getBasePath()) + .hadoopConf(new Configuration()) + .build(); + } +} \ No newline at end of file From ec222de29003f200ade2ce228a1680a547eaf2da Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Mon, 24 Feb 2025 10:32:18 +0100 Subject: [PATCH 12/49] run mvn spotless:apply --- .../parquet/ParquetConversionSource.java | 16 +- .../ParquetConversionSourceProvider.java | 21 +- .../parquet/ParquetMetadataExtractor.java | 44 ++-- .../parquet/ParquetPartitionExtractor.java | 111 +++++---- .../parquet/ParquetSchemaExtractror.java | 14 +- .../xtable/parquet/ParquetStatsExtractor.java | 216 ++++++++++-------- .../xtable/parquet/ParquetTableExtractor.java | 16 +- 7 files changed, 247 insertions(+), 191 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index f0c30ee3f..08e6a6444 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -59,7 +59,7 @@ public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetStatsExtractor parquetStatsExtractor = - ParquetStatsExtractor.getInstance(); + ParquetStatsExtractor.getInstance(); private Map> initPartitionInfo() { return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); @@ -132,17 +132,19 @@ public InternalSnapshot getCurrentSnapshot() { .fileFormat(FileFormat.APACHE_PARQUET) .fileSizeBytes(file.getLen()) .partitionValues( - parquetPartitionExtractor.getPartitionValue( + parquetPartitionExtractor.getPartitionValue( basePath, file.getPath().toString(), table.getReadSchema(), partitionInfo)) .lastModified(file.getModificationTime()) .columnStats( - parquetStatsExtractor.getColumnStatsForaFile(parquetMetadataExtractor.readParquetMetadata( - hadoopConf, file.getPath().toString())) - .build()) - .collect(Collectors.toList()); + parquetStatsExtractor + .getColumnStatsForaFile( + parquetMetadataExtractor.readParquetMetadata( + hadoopConf, file.getPath().toString())) + .build()) + .collect(Collectors.toList())); return InternalSnapshot.builder() .table(table) @@ -173,7 +175,7 @@ public TableChange getTableChangeForCommit(Long modificationTime) { InternalDataFile.builder() .physicalPath(tableStatus.getPath().toString()) .partitionValues( - parquetPartitionExtractor.getPartitionValue( + parquetPartitionExtractor.getPartitionValue( basePath, tableStatus.getPath().toString(), internalTable.getReadSchema(), diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java index 9eeca51fd..e3bfd20a2 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java @@ -15,22 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import org.apache.hadoop.conf.Configuration; + import org.apache.xtable.conversion.ConversionSourceProvider; import org.apache.xtable.conversion.SourceTable; /** A concrete implementation of {@link ConversionSourceProvider} for Delta Lake table format. */ public class ParquetConversionSourceProvider extends ConversionSourceProvider { - @Override - public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { + @Override + public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { - return ParquetConversionSource.builder() - .tableName(sourceTable.getName()) - .basePath(sourceTable.getBasePath()) - .hadoopConf(new Configuration()) - .build(); - } -} \ No newline at end of file + return ParquetConversionSource.builder() + .tableName(sourceTable.getName()) + .basePath(sourceTable.getBasePath()) + .hadoopConf(new Configuration()) + .build(); + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java index cbe031f1e..c5790d642 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java @@ -1,21 +1,35 @@ -import org.apache.parquet.schema.MessageType; - -import org.apache.hadoop.conf.Configuration; +/* + * 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. + */ + import org.apache.hadoop.fs.*; -import org.apache.parquet.Schema; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; public class ParquetMetadataExtractor { - private static MessageType getSchema(ParquetMetadata footer) { - MessageType schema = footer.getFileMetaData().getSchema(); - return schema - } - - private static ParquetMetadata readParquetMetadata(HadoopConf conf, BasePath path) { - ParquetMetadata footer = - ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); - return footer + private static MessageType getSchema(ParquetMetadata footer) { + MessageType schema = footer.getFileMetaData().getSchema(); + return schema; + } - } -} \ No newline at end of file + private static ParquetMetadata readParquetMetadata(HadoopConf conf, BasePath path) { + ParquetMetadata footer = + ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + return footer; + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 99f5d8d86..2c7e8f3c9 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -1,3 +1,21 @@ +/* + * 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.nio.file.Paths; @@ -5,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Set; + import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.schema.PartitionTransformType; @@ -13,57 +32,57 @@ import org.apache.xtable.schema.SchemaFieldFinder; public class ParquetPartitionExtractor { - private static final ParquetPartitionExtractor INSTANCE = new ParquetPartitionExtractor(); + private static final ParquetPartitionExtractor INSTANCE = new ParquetPartitionExtractor(); - public static ParquetPartitionExtractor getInstance() { - return INSTANCE; - } + public static ParquetPartitionExtractor getInstance() { + return INSTANCE; + } - public List getInternalPartitionField( - Set partitionList, InternalSchema schema) { - List partitionFields = new ArrayList<>(); + public List getInternalPartitionField( + Set partitionList, InternalSchema schema) { + List partitionFields = new ArrayList<>(); - for (String partitionKey : partitionList) { + for (String partitionKey : partitionList) { - partitionFields.add( - InternalPartitionField.builder() - .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) - .transformType(PartitionTransformType.VALUE) - .build()); - } - - return partitionFields; + partitionFields.add( + InternalPartitionField.builder() + .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) + .transformType(PartitionTransformType.VALUE) + .build()); } - // TODO logic is too complicated can be simplified - public List getPartitionValue( - String basePath, - String filePath, - InternalSchema schema, - Map> partitionInfo) { - List partitionValues = new ArrayList<>(); - java.nio.file.Path base = Paths.get(basePath).normalize(); - java.nio.file.Path file = Paths.get(filePath).normalize(); - java.nio.file.Path relative = base.relativize(file); - for (Map.Entry> entry : partitionInfo.entrySet()) { - String key = entry.getKey(); - List values = entry.getValue(); - for (String value : values) { - String pathCheck = key + "=" + value; - if (relative.startsWith(pathCheck)) { - System.out.println("Relative " + relative + " " + pathCheck); - partitionValues.add( - PartitionValue.builder() - .partitionField( - InternalPartitionField.builder() - .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, key)) - .transformType(PartitionTransformType.VALUE) - .build()) - .range(Range.scalar(value)) - .build()); - } - } + return partitionFields; + } + + // TODO logic is too complicated can be simplified + public List getPartitionValue( + String basePath, + String filePath, + InternalSchema schema, + Map> partitionInfo) { + List partitionValues = new ArrayList<>(); + java.nio.file.Path base = Paths.get(basePath).normalize(); + java.nio.file.Path file = Paths.get(filePath).normalize(); + java.nio.file.Path relative = base.relativize(file); + for (Map.Entry> entry : partitionInfo.entrySet()) { + String key = entry.getKey(); + List values = entry.getValue(); + for (String value : values) { + String pathCheck = key + "=" + value; + if (relative.startsWith(pathCheck)) { + System.out.println("Relative " + relative + " " + pathCheck); + partitionValues.add( + PartitionValue.builder() + .partitionField( + InternalPartitionField.builder() + .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, key)) + .transformType(PartitionTransformType.VALUE) + .build()) + .range(Range.scalar(value)) + .build()); } - return partitionValues; + } } -} \ No newline at end of file + return partitionValues; + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 48162413a..b983973f5 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -118,19 +118,15 @@ private InternalSchema toInternalSchema( case BYTE_ARRAY: logicalType = schema.getLogicalType(); // TODO: any metadata to add ? - if (logicalType== LogicalTypes.JSON) { + if (logicalType == LogicalTypes.JSON) { newDataType = InternalType.JSON; - } - else if (logicalType instanceof LogicalTypes.BSON) { + } else if (logicalType instanceof LogicalTypes.BSON) { newDataType = InternalType.BSON; - } - else if (logicalType instanceof LogicalTypes.VARIANT) { + } else if (logicalType instanceof LogicalTypes.VARIANT) { newDataType = InternalType.VARIANT; - } - else if (logicalType instanceof LogicalTypes.GEOMETRY) { + } else if (logicalType instanceof LogicalTypes.GEOMETRY) { newDataType = InternalType.GEOMETRY; - } - else if (logicalType instanceof LogicalTypes.GEOGRAPHY) { + } else if (logicalType instanceof LogicalTypes.GEOGRAPHY) { newDataType = InternalType.GEOGRAPHY; } break; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index 957b3a922..d9ba1e842 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -1,115 +1,139 @@ +/* + * 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. + */ + +import java.util.stream.Collectors; +import org.apache.hadoop.fs.*; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Encoding; import org.apache.parquet.column.statistics.Statistics; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.HiddenFileFilter; -import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.Encoding; -import org.apache.parquet.column.statistics.Statistics; -import java.util.stream.Collectors; -import org.apache.hadoop.fs.*; public class ParquetStatsExtractor { - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - @Builder.Default - private static final ParquetPartitionExtractor partitionExtractor = ParquetPartitionExtractor.getInstance(); + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); - private static Map stats = new LinkedHashMap(); - private static long recordCount = 0; - private Map> initPartitionInfo() { - return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); - } - private InternalDataFile toInternalDataFile(Configuration hadoopConf, - String parentPath, Map stats) { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus file = fs.getFileStatus(new Path(parentPath)); - Map> partitionInfo = initPartitionInfo(); + @Builder.Default + private static final ParquetPartitionExtractor partitionExtractor = + ParquetPartitionExtractor.getInstance(); - ParquetMetadata footer = - parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); - MessageType schema = parquetMetadataExtractor.getSchema(footer); - InternalSchema schema = schemaExtractor.toInternalSchema(schema); - List partitionValues = - partitionExtractor - .getPartitionValue(parentPath,file.getPath().toString(),schema,partitionInfo); - return InternalDataFile.builder() - .physicalPath(parentPath) - .fileFormat(FileFormat.APACHE_PARQUET) - .partitionValues(partitionValues) - .fileSizeBytes(file.getLen()) - .recordCount(recordCount) - .columnStats(stats.values().stream().collect(Collectors.toList())) - .lastModified(file.getModificationTime()) - .build(); - } - private static void getColumnStatsForaFile(ParquetMetadata footer) { - for (BlockMetaData blockMetaData : footer.getBlocks()) { + private static Map stats = + new LinkedHashMap(); + private static long recordCount = 0; - MessageType schema = parquetMetadataExtractor.getSchema(footer) - recordCount += blockMetaData.getRowCount(); - List columns = blockMetaData.getColumns(); - for (ColumnChunkMetaData columnMetaData : columns) { - ColumnDescriptor desc = - schema.getColumnDescription(columnMetaData.getPath().toArray()); - ColStats.add( - desc, - columnMetaData.getValueCount(), - columnMetaData.getTotalSize(), - columnMetaData.getTotalUncompressedSize(), - columnMetaData.getEncodings(), - columnMetaData.getStatistics()); - } - } + private Map> initPartitionInfo() { + return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); + } + + private InternalDataFile toInternalDataFile( + Configuration hadoopConf, String parentPath, Map stats) { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus file = fs.getFileStatus(new Path(parentPath)); + Map> partitionInfo = initPartitionInfo(); + + ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); + MessageType schema = parquetMetadataExtractor.getSchema(footer); + InternalSchema schema = schemaExtractor.toInternalSchema(schema); + List partitionValues = + partitionExtractor.getPartitionValue( + parentPath, file.getPath().toString(), schema, partitionInfo); + return InternalDataFile.builder() + .physicalPath(parentPath) + .fileFormat(FileFormat.APACHE_PARQUET) + .partitionValues(partitionValues) + .fileSizeBytes(file.getLen()) + .recordCount(recordCount) + .columnStats(stats.values().stream().collect(Collectors.toList())) + .lastModified(file.getModificationTime()) + .build(); + } + + private static void getColumnStatsForaFile(ParquetMetadata footer) { + for (BlockMetaData blockMetaData : footer.getBlocks()) { + + MessageType schema = parquetMetadataExtractor.getSchema(footer); + recordCount += blockMetaData.getRowCount(); + List columns = blockMetaData.getColumns(); + for (ColumnChunkMetaData columnMetaData : columns) { + ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); + ColStats.add( + desc, + columnMetaData.getValueCount(), + columnMetaData.getTotalSize(), + columnMetaData.getTotalUncompressedSize(), + columnMetaData.getEncodings(), + columnMetaData.getStatistics()); + } } - private static class Stats { - long min = Long.MAX_VALUE; - long max = Long.MIN_VALUE; - long total = 0; + } - public void add(long length) { - min = Math.min(length, min); - max = Math.max(length, max); - total += length; - } + private static class Stats { + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + long total = 0; + + public void add(long length) { + min = Math.min(length, min); + max = Math.max(length, max); + total += length; } - private static class ColStats { + } + + private static class ColStats { - Stats valueCountStats = new Stats(); - Stats allStats = new Stats(); - Stats uncStats = new Stats(); - Set encodings = new TreeSet(); - Statistics colValuesStats = null; - int blocks = 0; + Stats valueCountStats = new Stats(); + Stats allStats = new Stats(); + Stats uncStats = new Stats(); + Set encodings = new TreeSet(); + Statistics colValuesStats = null; + int blocks = 0; + + public void add( + long valueCount, + long size, + long uncSize, + Collection encodings, + Statistics colValuesStats) { + ++blocks; + valueCountStats.add(valueCount); + allStats.add(size); + uncStats.add(uncSize); + this.encodings.addAll(encodings); + this.colValuesStats = colValuesStats; + } - public void add( - long valueCount, long size, long uncSize, Collection encodings, Statistics colValuesStats) { - ++blocks; - valueCountStats.add(valueCount); - allStats.add(size); - uncStats.add(uncSize); - this.encodings.addAll(encodings); - this.colValuesStats = colValuesStats; - } - private static void add( - ColumnDescriptor desc, - long valueCount, - long size, - long uncSize, - Collection encodings, - Statistics colValuesStats) { - ColStats colStats = stats.get(desc); - if (colStats == null) { - colStats = new ColStats(); - stats.put(desc, colStats); - } - colStats.add(valueCount, size, uncSize, encodings, colValuesStats); - } + private static void add( + ColumnDescriptor desc, + long valueCount, + long size, + long uncSize, + Collection encodings, + Statistics colValuesStats) { + ColStats colStats = stats.get(desc); + if (colStats == null) { + colStats = new ColStats(); + stats.put(desc, colStats); + } + colStats.add(valueCount, size, uncSize, encodings, colValuesStats); } -} \ No newline at end of file + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 49321125f..ad5f36bb1 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -23,8 +23,6 @@ import lombok.Builder; -import org.apache.parquet.hadoop.ParquetFileReader; - import org.apache.xtable.model.InternalTable; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; @@ -38,24 +36,26 @@ public class ParquetTableExtractor { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); + @Builder.Default - private static final ParquetPartitionExtractor partitionExtractor = ParquetPartitionExtractor.getInstance(); + private static final ParquetPartitionExtractor partitionExtractor = + ParquetPartitionExtractor.getInstance(); + @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); + ParquetMetadataExtractor.getInstance(); + private Map> initPartitionInfo() { return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); } public InternalTable table(String tableName, Long version) { - ParquetMetadata footer = - parquetMetadataExtractor.readParquetMetadata(conf, path); + ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(conf, path); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); Set partitionKeys = initPartitionInfo().keySet(); List partitionFields = - partitionExtractor - .getInternalPartitionField(partitionKeys,schema); + partitionExtractor.getInternalPartitionField(partitionKeys, schema); DataLayoutStrategy dataLayoutStrategy = !partitionFields.isEmpty() ? DataLayoutStrategy.HIVE_STYLE_PARTITION From e0fbca892c3d262a54b3c3d2379508925440beff Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 5 Mar 2025 02:11:44 +0100 Subject: [PATCH 13/49] edited ParquetSchemaExtractor to include some other LogicalTypes and added YAML reader for partition fields user configuration --- .../xtable/model/schema/InternalType.java | 6 +- .../parquet/ParquetPartitionExtractor.java | 23 ++++++ .../parquet/ParquetSchemaExtractror.java | 74 ++++++++++--------- .../xtable/parquet/ParquetTableExtractor.java | 1 + .../parquet/PartitionConfiguration.java | 21 ++++++ 5 files changed, 87 insertions(+), 38 deletions(-) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java diff --git a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java index 2a2ad7eef..312c9d13d 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java @@ -52,11 +52,7 @@ public enum InternalType { DECIMAL, TIMESTAMP, TIMESTAMP_NTZ, - JSON, - BSON, - VARIANT, - GEOMETRY, - GEOGRAPHY; + JSON; private final String name; InternalType() { diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 2c7e8f3c9..38bce7735 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -31,12 +31,35 @@ import org.apache.xtable.model.stat.Range; import org.apache.xtable.schema.SchemaFieldFinder; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectReader; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; + public class ParquetPartitionExtractor { private static final ParquetPartitionExtractor INSTANCE = new ParquetPartitionExtractor(); public static ParquetPartitionExtractor getInstance() { return INSTANCE; } + public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); + + + public PartitionConfiguration getPartitionsFromUserConfiguration(String configPath) throws IOException { + PartitionConfiguration partitionConfiguration = new PartitionConfiguration(); + try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { + ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); + objectReader.readValue(inputStream); + return partitionConfiguration; + } + } + public List getInternalPartitionField( Set partitionList, InternalSchema schema) { diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index b983973f5..74d138174 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -34,6 +34,12 @@ import org.apache.parquet.LogicalType; import org.apache.parquet.LogicalTypes; import org.apache.parquet.Schema; +import org.apache.parquet.Schema.Type; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.format.NullType; +import org.apache.parquet.schema.LogicalTypeAnnotation; + + import org.apache.xtable.collectors.CustomCollectors; import org.apache.xtable.exception.SchemaExtractorException; @@ -96,38 +102,49 @@ public InternalSchema toInternalSchema(Schema schema) { * @return a converted schema */ private InternalSchema toInternalSchema( - Schema schema, String parentPath, Map fieldNameToIdMapping) { + Type schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; Map metadata = new HashMap<>(); - switch (schema.getType()) { - case INT: - LogicalType logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.Date) { + switch (schema.getName()) { + case INT64: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + newDataType = InternalType.TIMESTAMP; + } + break; + case INT32: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; - } else { - newDataType = InternalType.INT; } + // is also a TIME break; - case STRING: - newDataType = InternalType.STRING; + case FLOAT: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { + newDataType = InternalType.FLOAT; + } break; - case BOOLEAN: - newDataType = InternalType.BOOLEAN; + case BYTES: + logicalType = schema.getLogicalType(); break; + case FIXED_LEN_BYTE_ARRAY: + logicalType=schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + newDataType = InternalType.UUID; + } + //TODO how to add INTERVAL ? case BYTE_ARRAY: - logicalType = schema.getLogicalType(); - // TODO: any metadata to add ? + // includes string json,BSON, Variant,GEOMETRY, geography, + logicalType=schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + newDataType = InternalType.ENUM; + } if (logicalType == LogicalTypes.JSON) { - newDataType = InternalType.JSON; - } else if (logicalType instanceof LogicalTypes.BSON) { - newDataType = InternalType.BSON; - } else if (logicalType instanceof LogicalTypes.VARIANT) { - newDataType = InternalType.VARIANT; - } else if (logicalType instanceof LogicalTypes.GEOMETRY) { - newDataType = InternalType.GEOMETRY; - } else if (logicalType instanceof LogicalTypes.GEOGRAPHY) { - newDataType = InternalType.GEOGRAPHY; + newDataType = InternalType.BYTES; + } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { + newDataType = InternalType.BYTES; } break; case FIXED: @@ -157,12 +174,6 @@ private InternalSchema toInternalSchema( newDataType = InternalType.BYTES; } break; - case DOUBLE: - newDataType = InternalType.DOUBLE; - break; - case FLOAT: - newDataType = InternalType.FLOAT; - break; case LONG: logicalType = schema.getLogicalType(); if (logicalType instanceof LogicalTypes.TimestampMillis) { @@ -185,10 +196,6 @@ private InternalSchema toInternalSchema( newDataType = InternalType.LONG; } break; - case ENUM: - metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.getEnumSymbols()); - newDataType = InternalType.ENUM; - break; case NULL: newDataType = InternalType.NULL; break; @@ -473,8 +480,9 @@ private String buildCurrentPath(InternalField field, String parentPath) { .orElse(field.getName()); } - private static Schema finalizeSchema(Schema targetSchema, InternalSchema inputSchema) { + private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { + return targetSchema.union(Type("NULL","optional")) return Schema.createUnion(Schema.create(Schema.Type.NULL), targetSchema); } return targetSchema; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index ad5f36bb1..608f0dba4 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -54,6 +54,7 @@ public InternalTable table(String tableName, Long version) { MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema schema = schemaExtractor.toInternalSchema(schema); Set partitionKeys = initPartitionInfo().keySet(); + // TODO getInternalPartitionField() to be replaced with the new YAML based partition fields List partitionFields = partitionExtractor.getInternalPartitionField(partitionKeys, schema); DataLayoutStrategy dataLayoutStrategy = diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java b/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java new file mode 100644 index 000000000..abb79dc1d --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java @@ -0,0 +1,21 @@ +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +public class PartitionConfiguration { + public String partition; + + + public PartitionConfiguration(String partition) { + this.partition = partition; + } + + public PartitionConfiguration() { + + } + + public String getPartition() { + return this.partition; + } + +} From 6e2fc66509310e1b528686e14bf238577082ae7e Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 5 Mar 2025 04:47:47 +0100 Subject: [PATCH 14/49] ParquetSchemaExtractor few fixes --- .../org/apache/xtable/parquet/ParquetSchemaExtractror.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 74d138174..d6306116a 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -134,6 +134,7 @@ private InternalSchema toInternalSchema( if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { newDataType = InternalType.UUID; } + break; //TODO how to add INTERVAL ? case BYTE_ARRAY: // includes string json,BSON, Variant,GEOMETRY, geography, @@ -482,8 +483,7 @@ private String buildCurrentPath(InternalField field, String parentPath) { private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { - return targetSchema.union(Type("NULL","optional")) - return Schema.createUnion(Schema.create(Schema.Type.NULL), targetSchema); + return targetSchema.union(Type(null,"optional")) } return targetSchema; } From b4c49b78c33b9d4e7388db9562ed3735d0dbcf29 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 5 Mar 2025 14:35:55 +0100 Subject: [PATCH 15/49] ParquetSchemaExtractor NULL type added --- .../xtable/model/schema/InternalType.java | 3 +- .../parquet/ParquetSchemaExtractror.java | 843 +++++++++--------- 2 files changed, 429 insertions(+), 417 deletions(-) diff --git a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java index 312c9d13d..e1b1049d0 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalType.java @@ -51,8 +51,7 @@ public enum InternalType { DATE, DECIMAL, TIMESTAMP, - TIMESTAMP_NTZ, - JSON; + TIMESTAMP_NTZ; private final String name; InternalType() { diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index d6306116a..d411eeaa3 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.util.ArrayList; @@ -40,7 +40,6 @@ import org.apache.parquet.schema.LogicalTypeAnnotation; - import org.apache.xtable.collectors.CustomCollectors; import org.apache.xtable.exception.SchemaExtractorException; import org.apache.xtable.exception.UnsupportedSchemaTypeException; @@ -58,433 +57,447 @@ */ @NoArgsConstructor(access = AccessLevel.PRIVATE) public class ParquetSchemaConverter { - // 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 ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); - private static final String ELEMENT = "element"; - private static final String KEY = "key"; - private static final String VALUE = "value"; + // 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 ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); + private static final String ELEMENT = "element"; + private static final String KEY = "key"; + private static final String VALUE = "value"; - public static parquetSchemaConverter getInstance() { - return INSTANCE; - } + public static parquetSchemaConverter getInstance() { + return INSTANCE; + } - public InternalSchema toInternalSchema(Schema schema) { - Map fieldNameToIdMapping = - IdTracker.getInstance() - .getIdTracking(schema) - .map( - idTracking -> - idTracking.getIdMappings().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) - .orElse(Collections.emptyMap()); - return toInternalSchema(schema, null, fieldNameToIdMapping); - } + private static Object getDefaultValue(Schema.Field parquetField) { + return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) + ? InternalField.Constants.NULL_DEFAULT_VALUE + : parquetField.defaultVal(); + } - /** - * 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. - * @param fieldNameToIdMapping map of fieldName to IdMapping to track field IDs provided by the - * source schema. If source schema does not contain IdMappings, map will be empty. - * @return a converted schema - */ - private InternalSchema toInternalSchema( - Type schema, String parentPath, Map fieldNameToIdMapping) { - // TODO - Does not handle recursion in parquet schema - InternalType newDataType; - Map metadata = new HashMap<>(); - switch (schema.getName()) { - case INT64: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - newDataType = InternalType.TIMESTAMP; - } - break; - case INT32: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { - newDataType = InternalType.DATE; - } - // is also a TIME - break; - case FLOAT: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { - newDataType = InternalType.FLOAT; - } - break; - case BYTES: - logicalType = schema.getLogicalType(); - break; - case FIXED_LEN_BYTE_ARRAY: - logicalType=schema.getLogicalTypeAnnotation() - if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { - newDataType = InternalType.UUID; + private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { + if (inputSchema.isNullable()) { + return targetSchema.union(LogicalTypeAnnotation.unknownType()) } - break; - //TODO how to add INTERVAL ? - case BYTE_ARRAY: - // includes string json,BSON, Variant,GEOMETRY, geography, - logicalType=schema.getLogicalTypeAnnotation() - if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - newDataType = InternalType.ENUM; - } - if (logicalType == LogicalTypes.JSON) { - newDataType = InternalType.BYTES; - } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { - newDataType = InternalType.BYTES; - } - break; - case FIXED: - logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.Decimal) { - metadata.put( - InternalSchema.MetadataKey.DECIMAL_PRECISION, - ((LogicalTypes.Decimal) logicalType).getPrecision()); - metadata.put( - InternalSchema.MetadataKey.DECIMAL_SCALE, - ((LogicalTypes.Decimal) logicalType).getScale()); - if (schema.getType() == Schema.Type.FIXED) { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - } - newDataType = InternalType.DECIMAL; - break; - } - if (schema.getType() == Schema.Type.FIXED) { - String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); - if ("uuid".equals(xtableLogicalType)) { - newDataType = InternalType.UUID; - } else { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - newDataType = InternalType.FIXED; - } - } else { - newDataType = InternalType.BYTES; - } - break; - case LONG: - logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.TimestampMillis) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else { - newDataType = InternalType.LONG; - } - break; - case NULL: - newDataType = InternalType.NULL; - break; - case RECORD: - List subFields = new ArrayList<>(schema.getFields().size()); - for (Schema.Field parquetField : schema.getFields()) { - IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); - InternalSchema subFieldSchema = - toInternalSchema( - parquetField.schema(), - SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), - getChildIdMap(idMapping)); - Object defaultValue = getDefaultValue(parquetField); - subFields.add( - InternalField.builder() - .parentPath(parentPath) - .name(parquetField.name()) - .schema(subFieldSchema) - .defaultValue(defaultValue) - .fieldId(idMapping == null ? null : idMapping.getId()) - .build()); + return targetSchema; + } + + public InternalSchema toInternalSchema(Schema schema) { + Map fieldNameToIdMapping = + IdTracker.getInstance() + .getIdTracking(schema) + .map( + idTracking -> + idTracking.getIdMappings().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) + .orElse(Collections.emptyMap()); + return toInternalSchema(schema, null, fieldNameToIdMapping); + } + + /** + * 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. + * @param fieldNameToIdMapping map of fieldName to IdMapping to track field IDs provided by the + * source schema. If source schema does not contain IdMappings, map will be empty. + * @return a converted schema + */ + private InternalSchema toInternalSchema( + Type schema, String parentPath, Map fieldNameToIdMapping) { + // TODO - Does not handle recursion in parquet schema + InternalType newDataType; + Map metadata = new HashMap<>(); + switch (schema.getName()) { + case INT64: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + newDataType = InternalType.TIMESTAMP; + } + break; + case INT32: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { + newDataType = InternalType.DATE; + } + // is also a TIME + break; + case FLOAT: + LogicalType logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { + newDataType = InternalType.FLOAT; + } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + metadata.put( + InternalSchema.MetadataKey.DECIMAL_PRECISION, + ((LogicalTypes.Decimal) logicalType).getPrecision()); + metadata.put( + InternalSchema.MetadataKey.DECIMAL_SCALE, + ((LogicalTypes.Decimal) logicalType).getScale()); + if (schema.getType() == Schema.Type.FIXED) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + } + newDataType = InternalType.DECIMAL; + } + break; + case BYTES: + logicalType = schema.getLogicalType(); + break; + case FIXED_LEN_BYTE_ARRAY: + logicalType = schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + newDataType = InternalType.UUID; + } + break; + //TODO add other logicalTypes ? + case BYTE_ARRAY: + // includes metadata (?) for json,BSON, Variant,GEOMETRY, geography, + logicalType = schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + newDataType = InternalType.ENUM; + } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { + newDataType = InternalType.BYTES; + } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { + newDataType = InternalType.BYTES; + } + break; + case BOOLEAN: + newDataType = InternalType.BOOLEAN; + break; + case UNKNOWN: + newDataType = InternalType.NULL; + break; + // TODO check the rest of the types starting from here + case FIXED: + logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + metadata.put( + InternalSchema.MetadataKey.DECIMAL_PRECISION, + ((LogicalTypes.Decimal) logicalType).getPrecision()); + metadata.put( + InternalSchema.MetadataKey.DECIMAL_SCALE, + ((LogicalTypes.Decimal) logicalType).getScale()); + if (schema.getType() == Schema.Type.FIXED) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + } + newDataType = InternalType.DECIMAL; + break; + } + if (schema.getType() == Schema.Type.FIXED) { + String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); + if ("uuid".equals(xtableLogicalType)) { + newDataType = InternalType.UUID; + } else { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); + newDataType = InternalType.FIXED; + } + } else { + newDataType = InternalType.BYTES; + } + break; + case LONG: + logicalType = schema.getLogicalType(); + if (logicalType instanceof LogicalTypes.TimestampMillis) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else { + newDataType = InternalType.LONG; + } + break; + case RECORD: + List subFields = new ArrayList<>(schema.getFields().size()); + for (Schema.Field parquetField : schema.getFields()) { + IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); + InternalSchema subFieldSchema = + toInternalSchema( + parquetField.schema(), + SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), + getChildIdMap(idMapping)); + Object defaultValue = getDefaultValue(parquetField); + subFields.add( + InternalField.builder() + .parentPath(parentPath) + .name(parquetField.name()) + .schema(subFieldSchema) + .defaultValue(defaultValue) + .fieldId(idMapping == null ? null : idMapping.getId()) + .build()); + } + return InternalSchema.builder() + .name(schema.getName()) + .comment(schema.getDoc()) + .dataType(InternalType.RECORD) + .fields(subFields) + .isNullable(schema.isNullable()) + .build(); + case ARRAY: + IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + InternalSchema elementSchema = + toInternalSchema( + schema.getElementType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), + getChildIdMap(elementMapping)); + InternalField elementField = + InternalField.builder() + .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) + .parentPath(parentPath) + .schema(elementSchema) + .fieldId(elementMapping == null ? null : elementMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.LIST) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields(Collections.singletonList(elementField)) + .build(); + case MAP: + IdMapping keyMapping = fieldNameToIdMapping.get(KEY); + IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); + InternalSchema valueSchema = + toInternalSchema( + schema.getValueType(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), + getChildIdMap(valueMapping)); + InternalField valueField = + InternalField.builder() + .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) + .parentPath(parentPath) + .schema(valueSchema) + .fieldId(valueMapping == null ? null : valueMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.MAP) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .fields( + Arrays.asList( + MAP_KEY_FIELD.toBuilder() + .parentPath(parentPath) + .fieldId(keyMapping == null ? null : keyMapping.getId()) + .build(), + valueField)) + .build(); + case UNION: + boolean containsUnionWithNull = + schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); + if (containsUnionWithNull) { + List remainingSchemas = + schema.getTypes().stream() + .filter(t -> t.getType() != Schema.Type.NULL) + .collect(Collectors.toList()); + if (remainingSchemas.size() == 1) { + InternalSchema restSchema = + toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); + return InternalSchema.builderFrom(restSchema).isNullable(true).build(); + } else { + return InternalSchema.builderFrom( + toInternalSchema(Schema.createUnion(remainingSchemas))) + .isNullable(true) + .build(); + } + } else { + throw new UnsupportedSchemaTypeException( + String.format("Unsupported complex union type %s", schema)); + } + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); } return InternalSchema.builder() - .name(schema.getName()) - .comment(schema.getDoc()) - .dataType(InternalType.RECORD) - .fields(subFields) - .isNullable(schema.isNullable()) - .build(); - case ARRAY: - IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); - InternalSchema elementSchema = - toInternalSchema( - schema.getElementType(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), - getChildIdMap(elementMapping)); - InternalField elementField = - InternalField.builder() - .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) - .parentPath(parentPath) - .schema(elementSchema) - .fieldId(elementMapping == null ? null : elementMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.LIST) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .fields(Collections.singletonList(elementField)) - .build(); - case MAP: - IdMapping keyMapping = fieldNameToIdMapping.get(KEY); - IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); - InternalSchema valueSchema = - toInternalSchema( - schema.getValueType(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), - getChildIdMap(valueMapping)); - InternalField valueField = - InternalField.builder() - .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) - .parentPath(parentPath) - .schema(valueSchema) - .fieldId(valueMapping == null ? null : valueMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.MAP) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .fields( - Arrays.asList( - MAP_KEY_FIELD.toBuilder() - .parentPath(parentPath) - .fieldId(keyMapping == null ? null : keyMapping.getId()) - .build(), - valueField)) - .build(); - case UNION: - boolean containsUnionWithNull = - schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); - if (containsUnionWithNull) { - List remainingSchemas = - schema.getTypes().stream() - .filter(t -> t.getType() != Schema.Type.NULL) - .collect(Collectors.toList()); - if (remainingSchemas.size() == 1) { - InternalSchema restSchema = - toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); - return InternalSchema.builderFrom(restSchema).isNullable(true).build(); - } else { - return InternalSchema.builderFrom( - toInternalSchema(Schema.createUnion(remainingSchemas))) - .isNullable(true) + .name(schema.getName()) + .dataType(newDataType) + .comment(schema.getDoc()) + .isNullable(schema.isNullable()) + .metadata(metadata.isEmpty() ? null : metadata) .build(); - } - } else { - throw new UnsupportedSchemaTypeException( - String.format("Unsupported complex union type %s", schema)); - } - default: - throw new UnsupportedSchemaTypeException( - String.format("Unsupported schema type %s", schema)); } - return InternalSchema.builder() - .name(schema.getName()) - .dataType(newDataType) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) - .metadata(metadata.isEmpty() ? null : metadata) - .build(); - } - private Map getChildIdMap(IdMapping idMapping) { - if (idMapping == null) { - return Collections.emptyMap(); + private Map getChildIdMap(IdMapping idMapping) { + if (idMapping == null) { + return Collections.emptyMap(); + } + return idMapping.getFields().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity())); } - return idMapping.getFields().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity())); - } - private static Object getDefaultValue(Schema.Field parquetField) { - return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) - ? InternalField.Constants.NULL_DEFAULT_VALUE - : parquetField.defaultVal(); - } - - /** - * Converts the {@link InternalSchema} to parquet {@link Schema}. - * - * @param internalSchema internal schema representation - * @return an parquet schema - */ - public Schema fromInternalSchema(InternalSchema internalSchema) { - return fromInternalSchema(internalSchema, null); - } + /** + * Converts the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @return an parquet schema + */ + public Schema fromInternalSchema(InternalSchema internalSchema) { + return fromInternalSchema(internalSchema, null); + } - /** - * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. - * - * @param internalSchema internal schema representation - * @param currentPath If this schema is nested within another, this will be a dot separated - * string. This is used for the parquet namespace to guarantee unique names for nested - * records. - * @return an parquet schema - */ - private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { - switch (internalSchema.getDataType()) { - case RECORD: - List fields = - internalSchema.getFields().stream() - .map( - field -> - new Schema.Field( - field.getName(), - fromInternalSchema( - field.getSchema(), - SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), - field.getSchema().getComment(), - InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() - ? Schema.Field.NULL_VALUE - : field.getDefaultValue())) - .collect(CustomCollectors.toList(internalSchema.getFields().size())); - return finalizeSchema( - Schema.createRecord( - internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), - internalSchema); - case BYTES: - return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); - case BOOLEAN: - return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); - case INT: - return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); - case LONG: - return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); - case STRING: - return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); - case FLOAT: - return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); - case DOUBLE: - return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); - case ENUM: - return finalizeSchema( - Schema.createEnum( - internalSchema.getName(), - internalSchema.getComment(), - null, - (List) - internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null), - internalSchema); - case DATE: - return finalizeSchema( - LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); - case TIMESTAMP: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); + /** + * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. + * + * @param internalSchema internal schema representation + * @param currentPath If this schema is nested within another, this will be a dot separated + * string. This is used for the parquet namespace to guarantee unique names for nested + * records. + * @return an parquet schema + */ + private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { + switch (internalSchema.getDataType()) { + case RECORD: + List fields = + internalSchema.getFields().stream() + .map( + field -> + new Schema.Field( + field.getName(), + fromInternalSchema( + field.getSchema(), + SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), + field.getSchema().getComment(), + InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() + ? Schema.Field.NULL_VALUE + : field.getDefaultValue())) + .collect(CustomCollectors.toList(internalSchema.getFields().size())); + return finalizeSchema( + Schema.createRecord( + internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), + internalSchema); + case BYTES: + return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); + case BOOLEAN: + return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); + case INT: + return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); + case LONG: + return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); + case STRING: + return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); + case FLOAT: + return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); + case DOUBLE: + return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); + case ENUM: + return finalizeSchema( + Schema.createEnum( + internalSchema.getName(), + internalSchema.getComment(), + null, + (List) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), + null), + internalSchema); + case DATE: + return finalizeSchema( + LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); + case TIMESTAMP: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case TIMESTAMP_NTZ: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return finalizeSchema( + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + internalSchema); + } + case LIST: + InternalField elementField = + internalSchema.getFields().stream() + .filter( + field -> + InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); + return finalizeSchema( + Schema.createArray( + fromInternalSchema(elementField.getSchema(), elementField.getPath())), + internalSchema); + case MAP: + InternalField valueField = + internalSchema.getFields().stream() + .filter( + field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) + .findFirst() + .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); + return finalizeSchema( + Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), + internalSchema); + case DECIMAL: + int precision = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); + int scale = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); + Integer size = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + if (size == null) { + return finalizeSchema( + LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), + internalSchema); + } else { + return finalizeSchema( + LogicalTypes.decimal(precision, scale) + .addToSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, size)), + internalSchema); + } + case FIXED: + Integer fixedSize = + (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); + return finalizeSchema( + Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, fixedSize), + internalSchema); + case UUID: + Schema uuidSchema = + Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); + uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); + return finalizeSchema(uuidSchema, internalSchema); + default: + throw new UnsupportedSchemaTypeException( + "Encountered unhandled type during InternalSchema to parquet conversion: " + + internalSchema.getDataType()); } - case TIMESTAMP_NTZ: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } - case LIST: - InternalField elementField = - internalSchema.getFields().stream() - .filter( - field -> - InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); - return finalizeSchema( - Schema.createArray( - fromInternalSchema(elementField.getSchema(), elementField.getPath())), - internalSchema); - case MAP: - InternalField valueField = - internalSchema.getFields().stream() - .filter( - field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); - return finalizeSchema( - Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), - internalSchema); - case DECIMAL: - int precision = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); - int scale = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); - Integer size = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - if (size == null) { - return finalizeSchema( - LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.decimal(precision, scale) - .addToSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, size)), - internalSchema); - } - case FIXED: - Integer fixedSize = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - return finalizeSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, fixedSize), - internalSchema); - case UUID: - Schema uuidSchema = - Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); - uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); - return finalizeSchema(uuidSchema, internalSchema); - default: - throw new UnsupportedSchemaTypeException( - "Encountered unhandled type during InternalSchema to parquet conversion: " - + internalSchema.getDataType()); } - } - - private String buildCurrentPath(InternalField field, String parentPath) { - return Optional.ofNullable(parentPath) - .map(path -> path + "." + field.getName()) - .orElse(field.getName()); - } - private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { - if (inputSchema.isNullable()) { - return targetSchema.union(Type(null,"optional")) + private String buildCurrentPath(InternalField field, String parentPath) { + return Optional.ofNullable(parentPath) + .map(path -> path + "." + field.getName()) + .orElse(field.getName()); } - return targetSchema; - } } From cac552a53b764aa7a632ca6983cd683c44892664 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 5 Mar 2025 20:34:05 +0100 Subject: [PATCH 16/49] ParquetSchemaExtractor Numeric and time types OK, TODO : Arrays and Maps --- .../parquet/ParquetSchemaExtractror.java | 68 ++++++++++--------- 1 file changed, 36 insertions(+), 32 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index d411eeaa3..b5022ffd8 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -78,11 +78,11 @@ public static parquetSchemaConverter getInstance() { return INSTANCE; } - private static Object getDefaultValue(Schema.Field parquetField) { + /* private static Object getDefaultValue(Schema.Field parquetField) { return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) ? InternalField.Constants.NULL_DEFAULT_VALUE : parquetField.defaultVal(); - } + }*/ private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { @@ -91,8 +91,8 @@ private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema return targetSchema; } - public InternalSchema toInternalSchema(Schema schema) { - Map fieldNameToIdMapping = + /* public InternalSchema toInternalSchema(Schema schema) { + Map fieldNameToIdMapping = IdTracker.getInstance() .getIdTracking(schema) .map( @@ -100,7 +100,7 @@ public InternalSchema toInternalSchema(Schema schema) { idTracking.getIdMappings().stream() .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) .orElse(Collections.emptyMap()); - return toInternalSchema(schema, null, fieldNameToIdMapping); + return toInternalSchema(schema, null, fieldNameToIdMapping);*/ } /** @@ -117,40 +117,43 @@ private InternalSchema toInternalSchema( Type schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; + LogicalTypeAnnotation logicalType; Map metadata = new HashMap<>(); switch (schema.getName()) { - case INT64: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); + case "INT64": + logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { newDataType = InternalType.TIMESTAMP; + } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation){ + newDataType = InternalType.INT; } break; - case INT32: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); + case "INT32": + logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; } // is also a TIME break; - case FLOAT: - LogicalType logicalType = schema.getLogicalTypeAnnotation(); + case "INT96": + newDataType = InternalType.INT; + break; + case "FLOAT": + logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { newDataType = InternalType.FLOAT; } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { metadata.put( InternalSchema.MetadataKey.DECIMAL_PRECISION, - ((LogicalTypes.Decimal) logicalType).getPrecision()); + logicalType.getPrecision()); metadata.put( InternalSchema.MetadataKey.DECIMAL_SCALE, - ((LogicalTypes.Decimal) logicalType).getScale()); - if (schema.getType() == Schema.Type.FIXED) { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - } + logicalType.getScale()); newDataType = InternalType.DECIMAL; } break; - case BYTES: - logicalType = schema.getLogicalType(); + case "BYTES": + newDataType = InternalType.BYTES; break; case FIXED_LEN_BYTE_ARRAY: logicalType = schema.getLogicalTypeAnnotation() @@ -159,7 +162,7 @@ private InternalSchema toInternalSchema( } break; //TODO add other logicalTypes ? - case BYTE_ARRAY: + case "BYTE_ARRAY": // includes metadata (?) for json,BSON, Variant,GEOMETRY, geography, logicalType = schema.getLogicalTypeAnnotation() if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { @@ -170,14 +173,14 @@ private InternalSchema toInternalSchema( newDataType = InternalType.BYTES; } break; - case BOOLEAN: + case "BOOLEAN": newDataType = InternalType.BOOLEAN; break; - case UNKNOWN: + case "UNKNOWN": newDataType = InternalType.NULL; break; - // TODO check the rest of the types starting from here - case FIXED: + // TODO: check the rest of the types starting from here + /*case FIXED: logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { metadata.put( @@ -203,8 +206,8 @@ private InternalSchema toInternalSchema( } else { newDataType = InternalType.BYTES; } - break; - case LONG: + break;*/ + /* case LONG: logicalType = schema.getLogicalType(); if (logicalType instanceof LogicalTypes.TimestampMillis) { newDataType = InternalType.TIMESTAMP; @@ -225,8 +228,8 @@ private InternalSchema toInternalSchema( } else { newDataType = InternalType.LONG; } - break; - case RECORD: + break;*/ + /*case RECORD: List subFields = new ArrayList<>(schema.getFields().size()); for (Schema.Field parquetField : schema.getFields()) { IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); @@ -251,7 +254,7 @@ private InternalSchema toInternalSchema( .dataType(InternalType.RECORD) .fields(subFields) .isNullable(schema.isNullable()) - .build(); + .build();*/ case ARRAY: IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); InternalSchema elementSchema = @@ -303,7 +306,7 @@ private InternalSchema toInternalSchema( .build(), valueField)) .build(); - case UNION: + /* case UNION: boolean containsUnionWithNull = schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); if (containsUnionWithNull) { @@ -324,7 +327,7 @@ private InternalSchema toInternalSchema( } else { throw new UnsupportedSchemaTypeException( String.format("Unsupported complex union type %s", schema)); - } + }*/ default: throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); @@ -367,7 +370,8 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { */ private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { - case RECORD: + // TODO consider RECORD Type? + /* case RECORD: List fields = internalSchema.getFields().stream() .map( @@ -385,7 +389,7 @@ private Schema fromInternalSchema(InternalSchema internalSchema, String currentP return finalizeSchema( Schema.createRecord( internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), - internalSchema); + internalSchema);*/ case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); case BOOLEAN: From 004d7631c2c3d79c448f63f45078bf52eff0d930 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 5 Mar 2025 21:53:38 +0100 Subject: [PATCH 17/49] ParquetSchemaExtractor added groupTypes Map and List: TODO: tests --- .../parquet/ParquetSchemaExtractror.java | 36 ++++++++++++------- 1 file changed, 24 insertions(+), 12 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index b5022ffd8..9ee997a3d 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -91,6 +91,15 @@ private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema return targetSchema; } + private static boolean groupTypeContainsNull(Type schema) { + for (Type field in schema.getFields()){ + if (field == null) { + return True; + } + } + return False; + } + /* public InternalSchema toInternalSchema(Schema schema) { Map fieldNameToIdMapping = IdTracker.getInstance() @@ -101,7 +110,7 @@ private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) .orElse(Collections.emptyMap()); return toInternalSchema(schema, null, fieldNameToIdMapping);*/ - } +} /** * Converts the parquet {@link Schema} to {@link InternalSchema}. @@ -120,11 +129,12 @@ private InternalSchema toInternalSchema( LogicalTypeAnnotation logicalType; Map metadata = new HashMap<>(); switch (schema.getName()) { + //PrimitiveTypes case "INT64": logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { newDataType = InternalType.TIMESTAMP; - } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation){ + } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; } break; @@ -145,10 +155,10 @@ private InternalSchema toInternalSchema( } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { metadata.put( InternalSchema.MetadataKey.DECIMAL_PRECISION, - logicalType.getPrecision()); + logicalType.getPrecision()); metadata.put( InternalSchema.MetadataKey.DECIMAL_SCALE, - logicalType.getScale()); + logicalType.getScale()); newDataType = InternalType.DECIMAL; } break; @@ -255,11 +265,12 @@ private InternalSchema toInternalSchema( .fields(subFields) .isNullable(schema.isNullable()) .build();*/ - case ARRAY: + //GroupTypes + case "LIST": IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); InternalSchema elementSchema = toInternalSchema( - schema.getElementType(), + schema.getName(), SchemaUtils.getFullyQualifiedPath( parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), getChildIdMap(elementMapping)); @@ -273,16 +284,17 @@ private InternalSchema toInternalSchema( return InternalSchema.builder() .name(schema.getName()) .dataType(InternalType.LIST) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) .fields(Collections.singletonList(elementField)) .build(); - case MAP: + + case "MAP": IdMapping keyMapping = fieldNameToIdMapping.get(KEY); IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); InternalSchema valueSchema = toInternalSchema( - schema.getValueType(), + schema.getName(), SchemaUtils.getFullyQualifiedPath( parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), getChildIdMap(valueMapping)); @@ -296,8 +308,8 @@ private InternalSchema toInternalSchema( return InternalSchema.builder() .name(schema.getName()) .dataType(InternalType.MAP) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) .fields( Arrays.asList( MAP_KEY_FIELD.toBuilder() From 4b4593b2d062ce3ac11f96ebf99ccde39367c2fa Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Thu, 6 Mar 2025 14:32:02 +0100 Subject: [PATCH 18/49] added -write parquet- to test Parquet types --- .../parquet/TestParquetSchemaExtractor.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java new file mode 100644 index 000000000..91b70e138 --- /dev/null +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -0,0 +1,71 @@ +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.SimpleGroup; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.schema.*; + + +import java.util.ArrayList; +import java.util.List; + + +public class TestParquetSchemaExtractor { + static final GroupType mapGroupType = new GroupType(Type.Repetition.REPEATED, "key_value", + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakevalue")); + static final GroupType groupType = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType); + static final GroupType mapGroupType2 = new GroupType(Type.Repetition.REPEATED, "key_value", + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), + groupType); + static final GroupType groupType2 = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType2); + static final MessageType messageType = new MessageType("schema", groupType2); + public static void main(String[] args) { + generateParquetFileFor(); + } + + private static void generateParquetFileFor() { + try { + MessageType schema = parseSchema(); + Path path = new Path("test.parquet"); + + List recordList = generateRecords(); + + ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(path); + try (ParquetWriter writer = builder.withType(schema).build()) { + for (Group record : recordList) { + writer.write(record); + } + } + } catch (Exception ex) { + ex.printStackTrace(System.out); + } + } + + private static MessageType parseSchema() { + return messageType; + } + + private static List generateRecords() { + + List recordList = new ArrayList<>(); + + for(int i = 1; i <= 4; i++) { + Group mapGroup = new SimpleGroup(mapGroupType); + mapGroup.add("fakekey", i*i); + mapGroup.add("fakevalue", i*i*i); + Group group = new SimpleGroup(groupType); + group.add("key_value", mapGroup); + Group mapGroup2 = new SimpleGroup(mapGroupType2); + mapGroup2.add("fakekey", i); + mapGroup2.add("my_map", group); + Group group2 = new SimpleGroup(groupType2); + group2.add("key_value", mapGroup2); + Group mType = new SimpleGroup(messageType); + mType.add("my_map", group2); + recordList.add(mType); + } + + return recordList; + } +} \ No newline at end of file From 9d56c21a3c22ae4ccd94cbac5a5cbe3b9da86fb6 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 8 Mar 2025 08:25:15 +0100 Subject: [PATCH 19/49] added first test for primitive types --- .../parquet/ParquetSchemaExtractror.java | 6 +-- .../parquet/TestParquetSchemaExtractor.java | 48 ++++++++++++++++++- 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 9ee997a3d..dcbd9532c 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -56,7 +56,7 @@ * parquet data types and canonical data types. */ @NoArgsConstructor(access = AccessLevel.PRIVATE) -public class ParquetSchemaConverter { +public class ParquetSchemaExtractor { // parquet only supports string keys in maps private static final InternalField MAP_KEY_FIELD = InternalField.builder() @@ -69,12 +69,12 @@ public class ParquetSchemaConverter { .build()) .defaultValue("") .build(); - private static final ParquetSchemaConverter INSTANCE = new ParquetSchemaConverter(); + 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 parquetSchemaConverter getInstance() { + public static ParquetSchemaExtractor getInstance() { return INSTANCE; } diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java index 91b70e138..46b8ee809 100644 --- a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -1,9 +1,12 @@ +package org.apache.xtable.parquet; + import org.apache.hadoop.fs.Path; import org.apache.parquet.example.data.Group; import org.apache.parquet.example.data.simple.SimpleGroup; import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.ExampleParquetWriter; import org.apache.parquet.schema.*; +import org.apache.parquet.schema.Type.Repetition; import java.util.ArrayList; @@ -13,17 +16,60 @@ public class TestParquetSchemaExtractor { static final GroupType mapGroupType = new GroupType(Type.Repetition.REPEATED, "key_value", new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), - new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakevalue")); + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.FLOAT, "fakevalue")); static final GroupType groupType = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType); static final GroupType mapGroupType2 = new GroupType(Type.Repetition.REPEATED, "key_value", new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), groupType); static final GroupType groupType2 = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType2); static final MessageType messageType = new MessageType("schema", groupType2); + private static final ParquetSchemaExtractor SCHEMA_EXTRACTOR = + ParquetSchemaExtractor.getInstance(); public static void main(String[] args) { generateParquetFileFor(); } + @Test + public void testPrimitiveTypes() { + /* Map requiredEnumMetadata = + Collections.singletonMap( + InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("ONE", "TWO")); + Map optionalEnumMetadata = + Collections.singletonMap( + InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("THREE", "FOUR"));*/ + + InternalSchema schemaWithPrimitiveTypes = + InternalSchema.builder() + .dataType(InternalType.RECORD) + .fields( + Arrays.asList( + InternalField.builder() + .name("int") + .schema( + InternalSchema.builder() + .name("REQUIRED_int") + .dataType(InternalType.INT) + .isNullable(false) + .metadata(null) + .build()) + .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) + .build(), + InternalField.builder() + .name("float") + .schema( + InternalSchema.builder() + .name("REQUIRED_double") + .dataType(InternalType.FLOAT) + .isNullable(true) + .metadata(null) + .build()) + .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) + .build())) + .build(); + //Type expectedSchema = mapGroupType; + assertTrue(TestParquetSchemaExtractor.mapGroupType.equals(SCHEMA_EXTRACTOR.toInternalSchema(schemaWithPrimitiveTypes))); + } + private static void generateParquetFileFor() { try { MessageType schema = parseSchema(); From 18ef0374720ebd4706acfc925baa93ee2714d467 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Mon, 10 Mar 2025 00:51:28 +0100 Subject: [PATCH 20/49] cleanups --- .../parquet/ParquetSchemaExtractror.java | 135 ------------------ 1 file changed, 135 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index dcbd9532c..67b2e1151 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -78,12 +78,6 @@ public static ParquetSchemaExtractor getInstance() { return INSTANCE; } - /* private static Object getDefaultValue(Schema.Field parquetField) { - return Schema.Field.NULL_VALUE.equals(parquetField.defaultVal()) - ? InternalField.Constants.NULL_DEFAULT_VALUE - : parquetField.defaultVal(); - }*/ - private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { return targetSchema.union(LogicalTypeAnnotation.unknownType()) @@ -99,17 +93,6 @@ private static boolean groupTypeContainsNull(Type schema) { } return False; } - - /* public InternalSchema toInternalSchema(Schema schema) { - Map fieldNameToIdMapping = - IdTracker.getInstance() - .getIdTracking(schema) - .map( - idTracking -> - idTracking.getIdMappings().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) - .orElse(Collections.emptyMap()); - return toInternalSchema(schema, null, fieldNameToIdMapping);*/ } /** @@ -189,82 +172,6 @@ private InternalSchema toInternalSchema( case "UNKNOWN": newDataType = InternalType.NULL; break; - // TODO: check the rest of the types starting from here - /*case FIXED: - logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - metadata.put( - InternalSchema.MetadataKey.DECIMAL_PRECISION, - ((LogicalTypes.Decimal) logicalType).getPrecision()); - metadata.put( - InternalSchema.MetadataKey.DECIMAL_SCALE, - ((LogicalTypes.Decimal) logicalType).getScale()); - if (schema.getType() == Schema.Type.FIXED) { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - } - newDataType = InternalType.DECIMAL; - break; - } - if (schema.getType() == Schema.Type.FIXED) { - String xtableLogicalType = schema.getProp(InternalSchema.XTABLE_LOGICAL_TYPE); - if ("uuid".equals(xtableLogicalType)) { - newDataType = InternalType.UUID; - } else { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, schema.getFixedSize()); - newDataType = InternalType.FIXED; - } - } else { - newDataType = InternalType.BYTES; - } - break;*/ - /* case LONG: - logicalType = schema.getLogicalType(); - if (logicalType instanceof LogicalTypes.TimestampMillis) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.TimestampMicros) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else { - newDataType = InternalType.LONG; - } - break;*/ - /*case RECORD: - List subFields = new ArrayList<>(schema.getFields().size()); - for (Schema.Field parquetField : schema.getFields()) { - IdMapping idMapping = fieldNameToIdMapping.get(parquetField.name()); - InternalSchema subFieldSchema = - toInternalSchema( - parquetField.schema(), - SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.name()), - getChildIdMap(idMapping)); - Object defaultValue = getDefaultValue(parquetField); - subFields.add( - InternalField.builder() - .parentPath(parentPath) - .name(parquetField.name()) - .schema(subFieldSchema) - .defaultValue(defaultValue) - .fieldId(idMapping == null ? null : idMapping.getId()) - .build()); - } - return InternalSchema.builder() - .name(schema.getName()) - .comment(schema.getDoc()) - .dataType(InternalType.RECORD) - .fields(subFields) - .isNullable(schema.isNullable()) - .build();*/ //GroupTypes case "LIST": IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); @@ -318,28 +225,6 @@ private InternalSchema toInternalSchema( .build(), valueField)) .build(); - /* case UNION: - boolean containsUnionWithNull = - schema.getTypes().stream().anyMatch(t -> t.getType() == Schema.Type.NULL); - if (containsUnionWithNull) { - List remainingSchemas = - schema.getTypes().stream() - .filter(t -> t.getType() != Schema.Type.NULL) - .collect(Collectors.toList()); - if (remainingSchemas.size() == 1) { - InternalSchema restSchema = - toInternalSchema(remainingSchemas.get(0), parentPath, fieldNameToIdMapping); - return InternalSchema.builderFrom(restSchema).isNullable(true).build(); - } else { - return InternalSchema.builderFrom( - toInternalSchema(Schema.createUnion(remainingSchemas))) - .isNullable(true) - .build(); - } - } else { - throw new UnsupportedSchemaTypeException( - String.format("Unsupported complex union type %s", schema)); - }*/ default: throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); @@ -382,26 +267,6 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { */ private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { - // TODO consider RECORD Type? - /* case RECORD: - List fields = - internalSchema.getFields().stream() - .map( - field -> - new Schema.Field( - field.getName(), - fromInternalSchema( - field.getSchema(), - SchemaUtils.getFullyQualifiedPath(currentPath, field.getName())), - field.getSchema().getComment(), - InternalField.Constants.NULL_DEFAULT_VALUE == field.getDefaultValue() - ? Schema.Field.NULL_VALUE - : field.getDefaultValue())) - .collect(CustomCollectors.toList(internalSchema.getFields().size())); - return finalizeSchema( - Schema.createRecord( - internalSchema.getName(), internalSchema.getComment(), currentPath, false, fields), - internalSchema);*/ case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); case BOOLEAN: From bd11c67fe1384e4e96e3edbf7ba807a950e134ca Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 04:23:17 +0100 Subject: [PATCH 21/49] added timestamp metadata (millis, micros, nanos) --- .../xtable/model/schema/InternalSchema.java | 3 ++- .../xtable/parquet/ParquetSchemaExtractror.java | 17 ++++++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalSchema.java b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalSchema.java index 20af37e0c..8b7e0fc59 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalSchema.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/schema/InternalSchema.java @@ -75,7 +75,8 @@ public enum MetadataKey { public enum MetadataValue { MICROS, - MILLIS + MILLIS, + NANOS } public static final String XTABLE_LOGICAL_TYPE = "xtableLogicalType"; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 67b2e1151..b6143fa60 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -93,6 +93,7 @@ private static boolean groupTypeContainsNull(Type schema) { } return False; } + } /** @@ -116,7 +117,21 @@ private InternalSchema toInternalSchema( case "INT64": logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - newDataType = InternalType.TIMESTAMP; + LogicalTypeAnnotation.TimeUnit time_unit = logicalType.getUnit(); + if (time_unit == LogicalTypeAnnotation.TimeUnit.MICROS) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else if (time_unit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (time_unit == LogicalTypeAnnotation.TimeUnit.NANOS) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); + } + //newDataType = InternalType.TIMESTAMP; } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; } From 0dbedb0ec0b88e426c671d846f57d5f19a75cb24 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 04:32:10 +0100 Subject: [PATCH 22/49] added else type for each switch case --- .../parquet/ParquetSchemaExtractror.java | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index b6143fa60..3c353f722 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -117,16 +117,16 @@ private InternalSchema toInternalSchema( case "INT64": logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit time_unit = logicalType.getUnit(); - if (time_unit == LogicalTypeAnnotation.TimeUnit.MICROS) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { newDataType = InternalType.TIMESTAMP; metadata.put( InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else if (time_unit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { newDataType = InternalType.TIMESTAMP_NTZ; metadata.put( InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (time_unit == LogicalTypeAnnotation.TimeUnit.NANOS) { + } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { newDataType = InternalType.TIMESTAMP_NTZ; metadata.put( InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); @@ -134,12 +134,16 @@ private InternalSchema toInternalSchema( //newDataType = InternalType.TIMESTAMP; } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; + } else { + newDataType = InternalType.INT; } break; case "INT32": logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; + } else { + newDataType = InternalType.INT; } // is also a TIME break; @@ -158,6 +162,8 @@ private InternalSchema toInternalSchema( InternalSchema.MetadataKey.DECIMAL_SCALE, logicalType.getScale()); newDataType = InternalType.DECIMAL; + } else { + newDataType = InternalType.FLOAT; } break; case "BYTES": @@ -167,6 +173,8 @@ private InternalSchema toInternalSchema( logicalType = schema.getLogicalTypeAnnotation() if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { newDataType = InternalType.UUID; + } else { + newDataType = InternalType.BYTES; } break; //TODO add other logicalTypes ? @@ -179,6 +187,8 @@ private InternalSchema toInternalSchema( newDataType = InternalType.BYTES; } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; + } else { + newDataType = InternalType.BYTES; } break; case "BOOLEAN": From 0233d549ec92fddaa6678f1899db06caa95197dd Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 07:46:41 +0100 Subject: [PATCH 23/49] added string type --- .../org/apache/xtable/parquet/ParquetSchemaExtractror.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 3c353f722..6662f5f1e 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -187,7 +187,10 @@ private InternalSchema toInternalSchema( newDataType = InternalType.BYTES; } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; - } else { + } else if (logicalType instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation){ + newDataType = InternalType.STRING; + } + else { newDataType = InternalType.BYTES; } break; From 8fc6a9573089e2f93dece2d71cc10ff122daf7d3 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 08:14:38 +0100 Subject: [PATCH 24/49] added Time type --- .../parquet/ParquetSchemaExtractror.java | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 6662f5f1e..644e311b0 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -131,9 +131,14 @@ private InternalSchema toInternalSchema( metadata.put( InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); } - //newDataType = InternalType.TIMESTAMP; } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; + } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS || timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { + // check if INT is the InternalType needed here + newDataType = InternalType.INT; + } } else { newDataType = InternalType.INT; } @@ -142,6 +147,12 @@ private InternalSchema toInternalSchema( logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; + } else if (logicalType instanceof TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + // check if INT is the InternalType needed here + newDataType = InternalType.INT; + } } else { newDataType = InternalType.INT; } @@ -187,10 +198,9 @@ private InternalSchema toInternalSchema( newDataType = InternalType.BYTES; } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; - } else if (logicalType instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation){ + } else if (logicalType instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { newDataType = InternalType.STRING; - } - else { + } else { newDataType = InternalType.BYTES; } break; From c88fb25a4b8cd5e77047253881768c66efa278af Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 08:36:39 +0100 Subject: [PATCH 25/49] added metadata for ENUM and FIXED --- .../apache/xtable/parquet/ParquetSchemaExtractror.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 644e311b0..c7b6f8140 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -184,15 +184,17 @@ private InternalSchema toInternalSchema( logicalType = schema.getLogicalTypeAnnotation() if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { newDataType = InternalType.UUID; - } else { - newDataType = InternalType.BYTES; + } else if (logicalType instanceof LogicalTypeAnnotation.IntervalLogicalTypeAnnotation) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, 12); + newDataType = InternalType.FIXED; } break; //TODO add other logicalTypes ? case "BYTE_ARRAY": - // includes metadata (?) for json,BSON, Variant,GEOMETRY, geography, + //? Variant,GEOMETRY, GEOGRAPHY, logicalType = schema.getLogicalTypeAnnotation() if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.toOriginalType().values()); newDataType = InternalType.ENUM; } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; From 6c04cc7a6d07da7644557def55cae127cc8ba811 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 11 Mar 2025 21:47:12 +0100 Subject: [PATCH 26/49] adjusted primitive type detection --- .../parquet/ParquetSchemaExtractror.java | 305 +++++++++--------- 1 file changed, 156 insertions(+), 149 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index c7b6f8140..89a5d790e 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -110,164 +110,171 @@ private InternalSchema toInternalSchema( Type schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; + PrimitiveType typeName; LogicalTypeAnnotation logicalType; Map metadata = new HashMap<>(); - switch (schema.getName()) { - //PrimitiveTypes - case "INT64": - logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); - if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { - newDataType = InternalType.TIMESTAMP; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); - } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); - } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { - newDataType = InternalType.TIMESTAMP_NTZ; - metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); - } - } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { - newDataType = InternalType.INT; - } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); - if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS || timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { - // check if INT is the InternalType needed here + if (schema.isPrimitive()) { + typeName = schema.asPrimitiveType(); + switch (typeName.getPrimitiveTypeName()) { + //PrimitiveTypes + case INT64: + logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { + newDataType = InternalType.TIMESTAMP; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { + newDataType = InternalType.TIMESTAMP_NTZ; + metadata.put( + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); + } + } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { + newDataType = InternalType.INT; + } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS || timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { + // check if INT is the InternalType needed here + newDataType = InternalType.INT; + } + } else { newDataType = InternalType.INT; } - } else { - newDataType = InternalType.INT; - } - break; - case "INT32": - logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { - newDataType = InternalType.DATE; - } else if (logicalType instanceof TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); - if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { - // check if INT is the InternalType needed here + break; + case INT32: + logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { + newDataType = InternalType.DATE; + } else if (logicalType instanceof TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { + // check if INT is the InternalType needed here + newDataType = InternalType.INT; + } + } else { newDataType = InternalType.INT; } - } else { + break; + case INT96: newDataType = InternalType.INT; - } - // is also a TIME - break; - case "INT96": - newDataType = InternalType.INT; - break; - case "FLOAT": - logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { - newDataType = InternalType.FLOAT; - } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { - metadata.put( - InternalSchema.MetadataKey.DECIMAL_PRECISION, - logicalType.getPrecision()); - metadata.put( - InternalSchema.MetadataKey.DECIMAL_SCALE, - logicalType.getScale()); - newDataType = InternalType.DECIMAL; - } else { - newDataType = InternalType.FLOAT; - } - break; - case "BYTES": - newDataType = InternalType.BYTES; - break; - case FIXED_LEN_BYTE_ARRAY: - logicalType = schema.getLogicalTypeAnnotation() - if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { - newDataType = InternalType.UUID; - } else if (logicalType instanceof LogicalTypeAnnotation.IntervalLogicalTypeAnnotation) { - metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, 12); - newDataType = InternalType.FIXED; - } - break; - //TODO add other logicalTypes ? - case "BYTE_ARRAY": - //? Variant,GEOMETRY, GEOGRAPHY, - logicalType = schema.getLogicalTypeAnnotation() - if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.toOriginalType().values()); - newDataType = InternalType.ENUM; - } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { - newDataType = InternalType.BYTES; - } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { - newDataType = InternalType.BYTES; - } else if (logicalType instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { - newDataType = InternalType.STRING; - } else { - newDataType = InternalType.BYTES; - } - break; - case "BOOLEAN": - newDataType = InternalType.BOOLEAN; - break; - case "UNKNOWN": - newDataType = InternalType.NULL; - break; + break; + case FLOAT: + logicalType = schema.getLogicalTypeAnnotation(); + if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { + newDataType = InternalType.FLOAT; + } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + metadata.put( + InternalSchema.MetadataKey.DECIMAL_PRECISION, + logicalType.getPrecision()); + metadata.put( + InternalSchema.MetadataKey.DECIMAL_SCALE, + logicalType.getScale()); + newDataType = InternalType.DECIMAL; + } else { + newDataType = InternalType.FLOAT; + } + break; + case FIXED_LEN_BYTE_ARRAY: + logicalType = schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { + newDataType = InternalType.UUID; + } else if (logicalType instanceof LogicalTypeAnnotation.IntervalLogicalTypeAnnotation) { + metadata.put(InternalSchema.MetadataKey.FIXED_BYTES_SIZE, 12); + newDataType = InternalType.FIXED; + } + break; + //TODO add other logicalTypes? + case BINARY: + //? Variant,GEOMETRY, GEOGRAPHY, + logicalType = schema.getLogicalTypeAnnotation() + if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { + metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.toOriginalType().values()); + newDataType = InternalType.ENUM; + } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { + newDataType = InternalType.BYTES; + } else if (logicalType instanceof LogicalTypeAnnotation.BsonLogicalTypeAnnotation) { + newDataType = InternalType.BYTES; + } else if (logicalType instanceof LogicalTypeAnnotation.StringLogicalTypeAnnotation) { + newDataType = InternalType.STRING; + } else { + newDataType = InternalType.BYTES; + } + break; + case BOOLEAN: + newDataType = InternalType.BOOLEAN; + break; + case UNKNOWN: + newDataType = InternalType.NULL; + break; + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); + } + }else { //GroupTypes - case "LIST": - IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); - InternalSchema elementSchema = - toInternalSchema( - schema.getName(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), - getChildIdMap(elementMapping)); - InternalField elementField = - InternalField.builder() - .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) - .parentPath(parentPath) - .schema(elementSchema) - .fieldId(elementMapping == null ? null : elementMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.LIST) - .comment(schema.toString()) - .isNullable(groupTypeContainsNull(schema)) - .fields(Collections.singletonList(elementField)) - .build(); + typeName = schema.asGroupType(); + switch (typeName.getOriginalType()) { + case LIST: + IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + InternalSchema elementSchema = + toInternalSchema( + schema.getName(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), + getChildIdMap(elementMapping)); + InternalField elementField = + InternalField.builder() + .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) + .parentPath(parentPath) + .schema(elementSchema) + .fieldId(elementMapping == null ? null : elementMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.LIST) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) + .fields(Collections.singletonList(elementField)) + .build(); - case "MAP": - IdMapping keyMapping = fieldNameToIdMapping.get(KEY); - IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); - InternalSchema valueSchema = - toInternalSchema( - schema.getName(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), - getChildIdMap(valueMapping)); - InternalField valueField = - InternalField.builder() - .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) - .parentPath(parentPath) - .schema(valueSchema) - .fieldId(valueMapping == null ? null : valueMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.MAP) - .comment(schema.toString()) - .isNullable(groupTypeContainsNull(schema)) - .fields( - Arrays.asList( - MAP_KEY_FIELD.toBuilder() - .parentPath(parentPath) - .fieldId(keyMapping == null ? null : keyMapping.getId()) - .build(), - valueField)) - .build(); - default: - throw new UnsupportedSchemaTypeException( - String.format("Unsupported schema type %s", schema)); + case MAP: + IdMapping keyMapping = fieldNameToIdMapping.get(KEY); + IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); + InternalSchema valueSchema = + toInternalSchema( + schema.getName(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), + getChildIdMap(valueMapping)); + InternalField valueField = + InternalField.builder() + .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) + .parentPath(parentPath) + .schema(valueSchema) + .fieldId(valueMapping == null ? null : valueMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.MAP) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) + .fields( + Arrays.asList( + MAP_KEY_FIELD.toBuilder() + .parentPath(parentPath) + .fieldId(keyMapping == null ? null : keyMapping.getId()) + .build(), + valueField)) + .build(); + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); + } } return InternalSchema.builder() .name(schema.getName()) From 9bdd972b0dc85a996ea5d2a39ff1c5cc8f09026a Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 12 Mar 2025 00:33:59 +0100 Subject: [PATCH 27/49] adjusted primitive types for fromInternalSchema sync, TODO: ENUM, LIST, MAP --- .../parquet/ParquetSchemaExtractror.java | 80 +++++++++++-------- 1 file changed, 46 insertions(+), 34 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 89a5d790e..55bbf858a 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -215,7 +215,7 @@ private InternalSchema toInternalSchema( throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); } - }else { + } else { //GroupTypes typeName = schema.asGroupType(); switch (typeName.getOriginalType()) { @@ -314,20 +314,25 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { */ private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { - case BYTES: + /*case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); case BOOLEAN: - return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema); + return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ case INT: - return finalizeSchema(Schema.create(Schema.Type.INT), internalSchema); + return finalizeSchema(LogicalTypeAnnotation.intType(32), internalSchema); case LONG: - return finalizeSchema(Schema.create(Schema.Type.LONG), internalSchema); + return finalizeSchema(LogicalTypeAnnotation.intType(64), internalSchema); case STRING: - return finalizeSchema(Schema.create(Schema.Type.STRING), internalSchema); + return finalizeSchema(LogicalTypeAnnotation.stringType(), internalSchema); case FLOAT: - return finalizeSchema(Schema.create(Schema.Type.FLOAT), internalSchema); + return finalizeSchema(LogicalTypeAnnotation.float16Type(), internalSchema); case DOUBLE: - return finalizeSchema(Schema.create(Schema.Type.DOUBLE), internalSchema); + int precision = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); + int scale = + (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); + return finalizeSchema(LogicalTypeAnnotation.decimalType(scale, precision), internalSchema); + // TODO check how to create ENUM case ENUM: return finalizeSchema( Schema.createEnum( @@ -340,29 +345,36 @@ private Schema fromInternalSchema(InternalSchema internalSchema, String currentP internalSchema); case DATE: return finalizeSchema( - LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)), internalSchema); + LogicalTypeAnnotation.dateType(), internalSchema); case TIMESTAMP: if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { return finalizeSchema( - LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + , LogicalTypeAnnotation.timestampType(True, MICROS) internalSchema); - } else { - return finalizeSchema( - LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), - internalSchema); - } + } if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MILLIS) { + return finalizeSchema( + , LogicalTypeAnnotation.timestampType(True, MILLIS) + internalSchema); + } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.NANOS) { + return finalizeSchema( + LogicalTypeAnnotation.timestampType(True, NANOS), + internalSchema); + } case TIMESTAMP_NTZ: if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { return finalizeSchema( - LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)), + LogicalTypeAnnotation.timestampType(True, MICROS), internalSchema); } else { return finalizeSchema( - LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)), + LogicalTypeAnnotation.timestampType(True, MILLIS), internalSchema); } + // TODO check from here FIXED, LIST and MAP types (still to todo) case LIST: InternalField elementField = internalSchema.getFields().stream() @@ -390,20 +402,20 @@ private Schema fromInternalSchema(InternalSchema internalSchema, String currentP (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); int scale = (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); - Integer size = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - if (size == null) { - return finalizeSchema( - LogicalTypes.decimal(precision, scale).addToSchema(Schema.create(Schema.Type.BYTES)), - internalSchema); - } else { - return finalizeSchema( - LogicalTypes.decimal(precision, scale) - .addToSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, size)), - internalSchema); - } +// Integer size = +// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); +// if (size == null) { + return finalizeSchema( + LogicalTypeAnnotation.decimalType(scale, precision), + internalSchema); +// } else { +// return finalizeSchema( +// LogicalTypes.decimal(precision, scale) +// .addToSchema( +// Schema.createFixed( +// internalSchema.getName(), internalSchema.getComment(), null, size)), +// internalSchema); +// } case FIXED: Integer fixedSize = (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); @@ -412,10 +424,10 @@ private Schema fromInternalSchema(InternalSchema internalSchema, String currentP internalSchema.getName(), internalSchema.getComment(), null, fixedSize), internalSchema); case UUID: - Schema uuidSchema = + /*Schema uuidSchema = Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); - uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid"); - return finalizeSchema(uuidSchema, internalSchema); + uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid");*/ + return finalizeSchema(LogicalTypeAnnotation.uuidType(), internalSchema); default: throw new UnsupportedSchemaTypeException( "Encountered unhandled type during InternalSchema to parquet conversion: " From 924db34ec72bf064e1d35550e37f155e67fbc2c0 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 03:13:34 +0100 Subject: [PATCH 28/49] logic for partitionFields (from user configuration) and updated Conversion Source accordingly, TODO: tests --- .../parquet/ParquetConversionSource.java | 373 ++++++++---------- .../parquet/ParquetPartitionExtractor.java | 17 - .../ParquetPartitionValueExtractor.java | 94 +++++ .../xtable/parquet/ParquetTableExtractor.java | 29 +- .../org/apache/xtable/utilities/RunSync.java | 6 + 5 files changed, 286 insertions(+), 233 deletions(-) create mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 08e6a6444..ca70ae5e2 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.io.IOException; @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.parquet.Schema; +import org.apache.parquet.Type; import org.apache.parquet.SchemaBuilder; import org.apache.parquet.hadoop.metadata.ParquetMetadata; @@ -37,234 +37,191 @@ import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.storage.*; import org.apache.xtable.spi.extractor.ConversionSource; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.Type.Repetition; @Builder public class ParquetConversionSource implements ConversionSource { - private final String tableName; - private final String basePath; - @NonNull private final Configuration hadoopConf; - - @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = - ParquetSchemaExtractor.getInstance(); - - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - - @Builder.Default - private static final ParquetPartitionExtractor parquetPartitionExtractor = - ParquetPartitionExtractor.getInstance(); - - @Builder.Default - private static final ParquetStatsExtractor parquetStatsExtractor = - ParquetStatsExtractor.getInstance(); - - private Map> initPartitionInfo() { - return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); - } - - /** - * To infer schema getting the latest file assumption is that latest file will have new fields - * - * @param modificationTime the commit to consider for reading the table state - * @return - */ - @Override - public InternalTable getTable(Long modificationTime) { - - Optional latestFile = - getParquetFiles(hadoopConf, basePath) - .max(Comparator.comparing(FileStatus::getModificationTime)); - - ParquetMetadata parquetMetadata = - parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - Schema tableSchema = - new org.apache.parquet.parquet.ParquetSchemaConverter() - .convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - - Set partitionKeys = initPartitionInfo().keySet(); - - // merge schema of partition into original as partition is not part of parquet fie - if (!partitionKeys.isEmpty()) { - tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + private static final ParquetSchemaConverter parquetSchemaConverter = + ParquetSchemaConverter.getInstance(); + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + @Builder.Default + private static final ParquetPartitionExtractor parquetPartitionExtractor = + ParquetPartitionExtractor.getInstance(); + @Builder.Default + private static final ParquetStatsExtractor parquetStatsExtractor = + ParquetStatsExtractor.getInstance(); + private final String tableName; + private final String basePath; + // user config path of the parquet file (partitions) + private final String configPath; + @NonNull + private final Configuration hadoopConf; + + private Map> initPartitionInfo() { + return getPartitionFromConfiguration(); } - InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); - - List partitionFields = - partitionKeys.isEmpty() - ? Collections.emptyList() - : parquetPartitionExtractor.getInternalPartitionField(partitionKeys, schema); - DataLayoutStrategy dataLayoutStrategy = - partitionFields.isEmpty() - ? DataLayoutStrategy.FLAT - : DataLayoutStrategy.HIVE_STYLE_PARTITION; - return InternalTable.builder() - .tableFormat(TableFormat.PARQUET) - .basePath(basePath) - .name(tableName) - .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) - .build(); - } - - /** - * Here to get current snapshot listing all files hence the -1 is being passed - * - * @return - */ - @Override - public InternalSnapshot getCurrentSnapshot() { - List latestFile = - getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - Map> partitionInfo = initPartitionInfo(); - InternalTable table = getTable(-1L); - List internalDataFiles = - latestFile.stream() - .map( - file -> - InternalDataFile.builder() - .physicalPath(file.getPath().toString()) - .fileFormat(FileFormat.APACHE_PARQUET) - .fileSizeBytes(file.getLen()) - .partitionValues( - parquetPartitionExtractor.getPartitionValue( - basePath, - file.getPath().toString(), - table.getReadSchema(), - partitionInfo)) - .lastModified(file.getModificationTime()) - .columnStats( - parquetStatsExtractor - .getColumnStatsForaFile( - parquetMetadataExtractor.readParquetMetadata( - hadoopConf, file.getPath().toString())) - .build()) - .collect(Collectors.toList())); - - return InternalSnapshot.builder() - .table(table) - .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) - .build(); - } - - /** - * Whenever new file is added , condition to get new file is listing files whose modification time - * is greater than previous ysnc - * - * @param modificationTime commit to capture table changes for. - * @return - */ - @Override - public TableChange getTableChangeForCommit(Long modificationTime) { - List tableChanges = - getParquetFiles(hadoopConf, basePath) - .filter(fileStatus -> fileStatus.getModificationTime() > modificationTime) - .collect(Collectors.toList()); - // TODO avoid doing full list of directory to get schema , just argument of modification time - // needs to be tweaked - InternalTable internalTable = getTable(-1L); - Set internalDataFiles = new HashSet<>(); - Map> partitionInfo = initPartitionInfo(); - for (FileStatus tableStatus : tableChanges) { - internalDataFiles.add( - InternalDataFile.builder() - .physicalPath(tableStatus.getPath().toString()) - .partitionValues( - parquetPartitionExtractor.getPartitionValue( - basePath, - tableStatus.getPath().toString(), - internalTable.getReadSchema(), - partitionInfo)) - .lastModified(tableStatus.getModificationTime()) - .fileSizeBytes(tableStatus.getLen()) - .columnStats( - parquetMetadataExtractor.getColumnStatsForaFile( - hadoopConf, tableStatus, internalTable)) - .build()); + public Map> getPartitionFromConfiguration() { + List partitionFields = parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + Map> partitionsMap = new HashMap<>(); + for (InputPartitionField partition : partitionFields) { + partitionsMap + .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) + .addAll(partition.partitionFieldValues()); + } + return partitionsMap; } - return TableChange.builder() - .tableAsOfChange(internalTable) - .filesDiff(DataFilesDiff.builder().filesAdded(internalDataFiles).build()) - .build(); - } - - @Override - public CommitsBacklog getCommitsBacklog( - InstantsForIncrementalSync instantsForIncrementalSync) { + /** + * To infer schema getting the latest file assumption is that latest file will have new fields + * + * @param modificationTime the commit to consider for reading the table state + * @return + */ + @Override + public InternalTable getTable(Long modificationTime) { - List commitsToProcess = - Collections.singletonList(instantsForIncrementalSync.getLastSyncInstant().toEpochMilli()); + Optional latestFile = + getParquetFiles(hadoopConf, basePath) + .max(Comparator.comparing(FileStatus::getModificationTime)); - return CommitsBacklog.builder().commitsToProcess(commitsToProcess).build(); - } + ParquetMetadata parquetMetadata = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); + Type tableSchema = + parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - // TODO Need to understnad how this needs to be implemented should _SUCCESS or .staging dir needs - // to be checked - @Override - public boolean isIncrementalSyncSafeFrom(Instant instant) { - return true; - } - @Override - public void close() throws IOException {} + Set partitionKeys = initPartitionInfo().keySet(); - private Schema mergeParquetSchema(Schema internalSchema, Set parititonFields) { - - SchemaBuilder.FieldAssembler fieldAssembler = - SchemaBuilder.record(internalSchema.getName()).fields(); - for (Schema.Field field : internalSchema.getFields()) { - fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); + // merge schema of partition into original as partition is not part of parquet fie + if (!partitionKeys.isEmpty()) { + tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + } + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); + + List partitionFields = + partitionKeys.isEmpty() + ? Collections.emptyList() + : parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + DataLayoutStrategy dataLayoutStrategy = + partitionFields.isEmpty() + ? DataLayoutStrategy.FLAT + : DataLayoutStrategy.HIVE_STYLE_PARTITION; + return InternalTable.builder() + .tableFormat(TableFormat.PARQUET) + .basePath(basePath) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) + .build(); } - for (String paritionKey : parititonFields) { - fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); + public List getInternalDataFiles(){ + List parquetFiles = + getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); + Map> partitionInfo = initPartitionInfo(); + InternalTable table = getTable(-1L); + List internalDataFiles = + parquetFiles.stream() + .map( + file -> + InternalDataFile.builder() + .physicalPath(file.getPath().toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .fileSizeBytes(file.getLen()) + .partitionValues( + parquetPartitionExtractor.getPartitionValue( + basePath, + file.getPath().toString(), + table.getReadSchema(), + partitionInfo)) + .lastModified(file.getModificationTime()) + .columnStats( + parquetStatsExtractor + .getColumnStatsForaFile( + parquetMetadataExtractor.readParquetMetadata( + hadoopConf, file.getPath().toString())) + .build()) + .collect(Collectors.toList())); + return internalDataFiles; } - return fieldAssembler.endRecord(); - } - - public Stream getParquetFiles(Configuration hadoopConf, String basePath) { - try { - FileSystem fs = FileSystem.get(hadoopConf); - RemoteIterator iterator = fs.listFiles(new Path(basePath), true); - return remoteIteratorToStream(iterator) - .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException | FileNotFoundException e) { - throw new RuntimeException(e); + /** + * Here to get current snapshot listing all files hence the -1 is being passed + * + * @return + */ + @Override + public InternalSnapshot getCurrentSnapshot() { + List internalDataFiles = getInternalDataFiles(); + return InternalSnapshot.builder() + .table(table) + .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) + .build(); } - } - public Map> getPartitionFromDirectoryStructure( - Configuration hadoopConf, String basePath, Map> partitionMap) { + private Type mergeParquetSchema(Type internalSchema, List parititonFields) { - try { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); - Map> currentPartitionMap = new HashMap<>(partitionMap); + List listOfAllFields = internalSchema.getFields(); + Type fieldsToMerge = listOfAllFields.get(0); + listOfAllFields.remove(0); + // start the merge + for (Type field : internalSchema.getFields()) { + fieldsToMerge = fieldsToMerge.union(field); + } + for (String parition : parititonFields) { + //create Type from partiton, TODO: check further... + fieldsToMerge = fieldsToMerge.union(Type(partition, Repetition.REQUIRED)) + } + + return fieldsToMerge; + } - for (FileStatus dirStatus : baseFileStatus) { - if (dirStatus.isDirectory()) { - String partitionPath = dirStatus.getPath().getName(); - if (partitionPath.contains("=")) { - String[] partitionKeyValue = partitionPath.split("="); - currentPartitionMap - .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) - .add(partitionKeyValue[1]); - getPartitionFromDirectoryStructure( - hadoopConf, dirStatus.getPath().toString(), partitionMap); - } + public Stream getParquetFiles(Configuration hadoopConf, String basePath) { + try { + FileSystem fs = FileSystem.get(hadoopConf); + RemoteIterator iterator = fs.listFiles(new Path(basePath), true); + return remoteIteratorToStream(iterator) + .filter(file -> file.getPath().getName().endsWith("parquet")); + } catch (IOException | FileNotFoundException e) { + throw new RuntimeException(e); } - } - return currentPartitionMap; + } - } catch (IOException e) { - throw new RuntimeException(e); + public Map> getPartitionFromDirectoryStructure( + Configuration hadoopConf, String basePath, Map> partitionMap) { + + try { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); + Map> currentPartitionMap = new HashMap<>(partitionMap); + + for (FileStatus dirStatus : baseFileStatus) { + if (dirStatus.isDirectory()) { + String partitionPath = dirStatus.getPath().getName(); + if (partitionPath.contains("=")) { + String[] partitionKeyValue = partitionPath.split("="); + currentPartitionMap + .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) + .add(partitionKeyValue[1]); + getPartitionFromDirectoryStructure( + hadoopConf, dirStatus.getPath().toString(), partitionMap); + } + } + } + return currentPartitionMap; + + } catch (IOException e) { + throw new RuntimeException(e); + } } - } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 38bce7735..7d152ff38 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -60,23 +60,6 @@ public PartitionConfiguration getPartitionsFromUserConfiguration(String configPa } } - - public List getInternalPartitionField( - Set partitionList, InternalSchema schema) { - List partitionFields = new ArrayList<>(); - - for (String partitionKey : partitionList) { - - partitionFields.add( - InternalPartitionField.builder() - .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) - .transformType(PartitionTransformType.VALUE) - .build()); - } - - return partitionFields; - } - // TODO logic is too complicated can be simplified public List getPartitionValue( String basePath, diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java new file mode 100644 index 000000000..d4251ba09 --- /dev/null +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -0,0 +1,94 @@ +/* + * 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.iceberg; + +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import lombok.AccessLevel; +import lombok.NoArgsConstructor; + +//import org.apache.iceberg.StructLike; + +import org.apache.xtable.model.schema.InternalPartitionField; +import org.apache.xtable.model.stat.Range; +import org.apache.xtable.schema.SchemaFieldFinder; + +/** + * Partition value extractor for Parquet. + */ +@NoArgsConstructor(access = AccessLevel.PRIVATE) +public class ParquetPartitionValueExtractor { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final ParquetPartitionValueExtractor INSTANCE = + new ParquetPartitionValueExtractor(); + + public static ParquetPartitionValueExtractor getInstance() { + return INSTANCE; + } + + public List getInternalPartitionField( + Set partitionList, InternalSchema schema) { + List partitionFields = new ArrayList<>(); + + for (String partitionKey : partitionList) { + partitionFields.add( + InternalPartitionField.builder() + //TODO check if this still is true for parquet (get sourceField from shcema and partitionKey?) + .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) + .transformType(PartitionTransformType.VALUE) + .build()); + } + + return partitionFields; + } + + /*public Map extractPartitionValues( + List partitionFields, Type schema) { + Map partitionValues = new HashMap<>(); + for (int i = 0; i < partitionFields.size(); i++) { + InternalPartitionField partitionField = partitionFields.get(i); + Object value; + // Convert date based partitions into millis since epoch + switch (partitionField.getTransformType()) { + case YEAR: + value = EPOCH.plusYears(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + break; + case MONTH: + value = EPOCH.plusMonths(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + break; + case DAY: + value = EPOCH.plusDays(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + break; + case HOUR: + value = EPOCH.plusHours(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + break; + default: + value = structLike.get(i, Object.class); + } + + partitionValues.put(partitionFields.get(i), Range.scalar(value)); + } + return partitionValues; + }*/ +} \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 608f0dba4..e6014d5b4 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -41,6 +41,14 @@ public class ParquetTableExtractor { private static final ParquetPartitionExtractor partitionExtractor = ParquetPartitionExtractor.getInstance(); + @Builder.Default + private static final ParquetPartitionValueExtractor partitionValueExtractor = + ParquetPartitionValueExtractor.getInstance(); + + @Builder.Default + private static final ParquetConversionSource parquetConversionSource = + ParquetConversionSource.getInstance(); + @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); @@ -48,27 +56,32 @@ public class ParquetTableExtractor { private Map> initPartitionInfo() { return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); } + public String getBasePathFromLastModifiedTable(){ + InternalTable table = parquetConversionSource.getTable(-1L); + return table.getBasePath(); + } - public InternalTable table(String tableName, Long version) { + public InternalTable table(String tableName, Set partitionKeys) { ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(conf, path); MessageType schema = parquetMetadataExtractor.getSchema(footer); - InternalSchema schema = schemaExtractor.toInternalSchema(schema); - Set partitionKeys = initPartitionInfo().keySet(); - // TODO getInternalPartitionField() to be replaced with the new YAML based partition fields + InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema); List partitionFields = - partitionExtractor.getInternalPartitionField(partitionKeys, schema); + partitionValueExtractor.getInternalPartitionField(partitionKeys, internalSchema); + InternalTable snapshot = parquetConversionSource.getTable(-1L); + // Assuming InternalTable.java has its getters + Instant lastCommit = snapshot.latestCommitTime(); DataLayoutStrategy dataLayoutStrategy = !partitionFields.isEmpty() ? DataLayoutStrategy.HIVE_STYLE_PARTITION : DataLayoutStrategy.FLAT; return InternalTable.builder() .tableFormat(TableFormat.APACHE_PARQUET) - .basePath(snapshot.deltaLog().dataPath().toString()) + .basePath(getBasePathFromLastModifiedTable()) .name(tableName) .layoutStrategy(dataLayoutStrategy) .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(snapshot.timestamp())) + .readSchema(internalSchema) + .latestCommitTime(Instant.ofEpochMilli(lastCommit)) .build(); } } diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java index 1a7bda874..390a68ab3 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java @@ -271,6 +271,12 @@ public static class Table { String tableName; String partitionSpec; String namespace; + + public class InputPartitionField { + String partitionFieldName; + List partitionFieldValues; + PartitionTransformType transformType; + } } } From 271756ea8ea767cb7a2dd699d03dfd962c4e661e Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 04:16:18 +0100 Subject: [PATCH 29/49] adjusted data class for reading user config --- .../apache/xtable/parquet/ParquetPartitionExtractor.java | 4 ++-- .../src/main/java/org/apache/xtable/utilities/RunSync.java | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 7d152ff38..20fd8e4b8 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -51,8 +51,8 @@ public static ParquetPartitionExtractor getInstance() { public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - public PartitionConfiguration getPartitionsFromUserConfiguration(String configPath) throws IOException { - PartitionConfiguration partitionConfiguration = new PartitionConfiguration(); + public List getPartitionsFromUserConfiguration(String configPath) throws IOException { + InputPartitionFields partitionConfiguration = new InputPartitionFields(); try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); objectReader.readValue(inputStream); diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java index 390a68ab3..f33a31d7e 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java @@ -272,11 +272,17 @@ public static class Table { String partitionSpec; String namespace; + @Data public class InputPartitionField { String partitionFieldName; List partitionFieldValues; PartitionTransformType transformType; } + + @Data + class InputPartitionFields { + private List partitions; + } } } From f7db318de1465e59aa08b00412582b2acea1a905 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 04:18:34 +0100 Subject: [PATCH 30/49] removed unacessary class --- .../parquet/PartitionConfiguration.java | 21 ------------------- 1 file changed, 21 deletions(-) delete mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java b/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java deleted file mode 100644 index abb79dc1d..000000000 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/PartitionConfiguration.java +++ /dev/null @@ -1,21 +0,0 @@ -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -public class PartitionConfiguration { - public String partition; - - - public PartitionConfiguration(String partition) { - this.partition = partition; - } - - public PartitionConfiguration() { - - } - - public String getPartition() { - return this.partition; - } - -} From 1323f630d94655e11322720421cb2522443a7c06 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 05:14:06 +0100 Subject: [PATCH 31/49] added alternative methods for ParquetSchemaExtractor: to test --- .../parquet/ParquetSchemaExtractror.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 55bbf858a..a79c9c9e5 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -50,6 +50,10 @@ import org.apache.xtable.model.schema.InternalType; import org.apache.xtable.schema.SchemaUtils; +import org.apache.xtable.avro.AvroSchemaConverter; +import org.apache.avro.Schema; +//import org.apache.parquet.avro.AvroSchemaConverter; + /** * 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 @@ -95,6 +99,13 @@ private static boolean groupTypeContainsNull(Type schema) { } } + // check which methods is best for the conversion + private InternalSchema toInternalSchema_bis( + MessageType schema, String parentPath, Map fieldNameToIdMapping) { + org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); + Schema avroSchema = avroParquetSchemaConverter.convert(schema); + return AvroSchemaConverter(avroSchema,parentPath,fieldNameToIdMapping); + } /** * Converts the parquet {@link Schema} to {@link InternalSchema}. @@ -303,6 +314,14 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { return fromInternalSchema(internalSchema, null); } + // check which methods is best for the conversion + private MessageType fromInternalSchema_bis( + InternalSchema internalSchema, String currentPath) { + org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); + Schema avroSchema = fromInternalSchema(internalSchema,currentPath); + MessageType parquetSchema = avroParquetSchemaConverter.convert(avroSchema); + return parquetSchema; + } /** * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. * @@ -312,7 +331,7 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { * records. * @return an parquet schema */ - private Schema fromInternalSchema(InternalSchema internalSchema, String currentPath) { + private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { /*case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); From 5c87799888e4361a6228e6ffba959d62eadcaf37 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 05:18:52 +0100 Subject: [PATCH 32/49] fixed small error in the previous commit --- .../org/apache/xtable/parquet/ParquetSchemaExtractror.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index a79c9c9e5..d246851ac 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -104,7 +104,8 @@ private InternalSchema toInternalSchema_bis( MessageType schema, String parentPath, Map fieldNameToIdMapping) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); Schema avroSchema = avroParquetSchemaConverter.convert(schema); - return AvroSchemaConverter(avroSchema,parentPath,fieldNameToIdMapping); + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + return avroSchemaConverter.toInternalSchema(avroSchema,parentPath,fieldNameToIdMapping); } /** @@ -318,7 +319,8 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { private MessageType fromInternalSchema_bis( InternalSchema internalSchema, String currentPath) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); - Schema avroSchema = fromInternalSchema(internalSchema,currentPath); + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + Schema avroSchema = avroSchemaConverter.fromInternalSchema(internalSchema,currentPath); MessageType parquetSchema = avroParquetSchemaConverter.convert(avroSchema); return parquetSchema; } From c53b7c53b7ca4d4175387e402e931f4c0856abf8 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 05:53:44 +0100 Subject: [PATCH 33/49] fixed small errors --- .../parquet/ParquetConversionSource.java | 28 +++++++++++++++---- .../parquet/ParquetSchemaExtractror.java | 26 ++++++++++++----- 2 files changed, 41 insertions(+), 13 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index ca70ae5e2..fd0692779 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -31,7 +31,7 @@ import org.apache.parquet.Type; import org.apache.parquet.SchemaBuilder; import org.apache.parquet.hadoop.metadata.ParquetMetadata; - +import org.apache.avro.Schema; import org.apache.xtable.model.*; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; @@ -46,8 +46,8 @@ public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); - private static final ParquetSchemaConverter parquetSchemaConverter = - ParquetSchemaConverter.getInstance(); +// private static final ParquetSchemaConverter parquetSchemaConverter = +// ParquetSchemaConverter.getInstance(); @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); @@ -94,15 +94,16 @@ public InternalTable getTable(Long modificationTime) { ParquetMetadata parquetMetadata = parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - Type tableSchema = - parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); + Schema tableSchema = + new org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); Set partitionKeys = initPartitionInfo().keySet(); // merge schema of partition into original as partition is not part of parquet fie if (!partitionKeys.isEmpty()) { - tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + //tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + tableSchema = mergeAvroSchema(tableSchema, partitionKeys); } InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); @@ -169,6 +170,21 @@ public InternalSnapshot getCurrentSnapshot() { .build(); } + private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { + + SchemaBuilder.FieldAssembler fieldAssembler = + SchemaBuilder.record(internalSchema.getName()).fields(); + for (Schema.Field field : internalSchema.getFields()) { + fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); + } + + for (String paritionKey : parititonFields) { + fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); + } + + return fieldAssembler.endRecord(); + } + private Type mergeParquetSchema(Type internalSchema, List parititonFields) { List listOfAllFields = internalSchema.getFields(); diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index d246851ac..72b050f15 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -33,7 +33,7 @@ import org.apache.parquet.LogicalType; import org.apache.parquet.LogicalTypes; -import org.apache.parquet.Schema; +//import org.apache.parquet.Schema; import org.apache.parquet.Schema.Type; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.format.NullType; @@ -98,13 +98,25 @@ private static boolean groupTypeContainsNull(Type schema) { return False; } -} + public InternalSchema toInternalSchema(Schema schema) { + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + Map fieldNameToIdMapping = + IdTracker.getInstance() + .getIdTracking(schema) + .map( + idTracking -> + idTracking.getIdMappings().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) + .orElse(Collections.emptyMap()); + return avroSchemaConverter.toInternalSchema(schema,null,fieldNameToIdMapping); + } + // check which methods is best for the conversion - private InternalSchema toInternalSchema_bis( + private InternalSchema toInternalSchema( MessageType schema, String parentPath, Map fieldNameToIdMapping) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); Schema avroSchema = avroParquetSchemaConverter.convert(schema); - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); return avroSchemaConverter.toInternalSchema(avroSchema,parentPath,fieldNameToIdMapping); } @@ -118,7 +130,7 @@ private InternalSchema toInternalSchema_bis( * source schema. If source schema does not contain IdMappings, map will be empty. * @return a converted schema */ - private InternalSchema toInternalSchema( + private InternalSchema toInternalSchema_bis( Type schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; @@ -316,7 +328,7 @@ public Schema fromInternalSchema(InternalSchema internalSchema) { } // check which methods is best for the conversion - private MessageType fromInternalSchema_bis( + private MessageType fromInternalSchema( InternalSchema internalSchema, String currentPath) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() @@ -333,7 +345,7 @@ private MessageType fromInternalSchema_bis( * records. * @return an parquet schema */ - private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { + private Type fromInternalSchema_bis(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { /*case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); From 80b930063c4902bc9bcd60a7374ce65aa1e8dbc8 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Fri, 14 Mar 2025 23:40:40 +0100 Subject: [PATCH 34/49] partitions are read from config --- .../parquet/ParquetConversionSource.java | 26 +++++------ .../parquet/ParquetPartitionExtractor.java | 6 +-- .../ParquetPartitionValueExtractor.java | 44 ++++++++++++------- .../xtable/parquet/ParquetTableExtractor.java | 6 +-- .../org/apache/xtable/utilities/RunSync.java | 5 ++- 5 files changed, 49 insertions(+), 38 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index fd0692779..93849f954 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -46,7 +46,7 @@ public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); -// private static final ParquetSchemaConverter parquetSchemaConverter = + // private static final ParquetSchemaConverter parquetSchemaConverter = // ParquetSchemaConverter.getInstance(); @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = @@ -54,6 +54,11 @@ public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetPartitionExtractor parquetPartitionExtractor = ParquetPartitionExtractor.getInstance(); + + @Builder.Default + private static final ParquetPartitionValueExtractor parquetPartitionValueExtractor = + ParquetPartitionValueExtractor.getInstance(); + @Builder.Default private static final ParquetStatsExtractor parquetStatsExtractor = ParquetStatsExtractor.getInstance(); @@ -64,12 +69,12 @@ public class ParquetConversionSource implements ConversionSource { @NonNull private final Configuration hadoopConf; - private Map> initPartitionInfo() { - return getPartitionFromConfiguration(); + private InputPartitionFields initPartitionInfo() { + return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); } public Map> getPartitionFromConfiguration() { - List partitionFields = parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + List partitionFields = initPartitionInfo().getPartitions(); Map> partitionsMap = new HashMap<>(); for (InputPartitionField partition : partitionFields) { partitionsMap @@ -126,11 +131,11 @@ public InternalTable getTable(Long modificationTime) { .build(); } - public List getInternalDataFiles(){ + public List getInternalDataFiles() { List parquetFiles = getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - Map> partitionInfo = initPartitionInfo(); - InternalTable table = getTable(-1L); + List partitionValuesFromConfig = parquetPartitionValueExtractor.createPartitionValues(parquetPartitionValueExtractor.extractPartitionValues(initPartitionInfo()) + InternalTable table = getTable(-1L); List internalDataFiles = parquetFiles.stream() .map( @@ -139,12 +144,7 @@ public List getInternalDataFiles(){ .physicalPath(file.getPath().toString()) .fileFormat(FileFormat.APACHE_PARQUET) .fileSizeBytes(file.getLen()) - .partitionValues( - parquetPartitionExtractor.getPartitionValue( - basePath, - file.getPath().toString(), - table.getReadSchema(), - partitionInfo)) + .partitionValues(partitionValuesFromConfig) .lastModified(file.getModificationTime()) .columnStats( parquetStatsExtractor diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 20fd8e4b8..b2c8e2924 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -51,7 +51,7 @@ public static ParquetPartitionExtractor getInstance() { public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - public List getPartitionsFromUserConfiguration(String configPath) throws IOException { + public InputPartitionFields getPartitionsFromUserConfiguration(String configPath) throws IOException { InputPartitionFields partitionConfiguration = new InputPartitionFields(); try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); @@ -61,7 +61,7 @@ public List getPartitionsFromUserConfiguration(String confi } // TODO logic is too complicated can be simplified - public List getPartitionValue( +/* public List getPartitionValue( String basePath, String filePath, InternalSchema schema, @@ -90,5 +90,5 @@ public List getPartitionValue( } } return partitionValues; - } + }*/ } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index d4251ba09..cbe3cef3f 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -48,47 +48,57 @@ public static ParquetPartitionValueExtractor getInstance() { } public List getInternalPartitionField( - Set partitionList, InternalSchema schema) { + InputPartitionFields partitions) { List partitionFields = new ArrayList<>(); - - for (String partitionKey : partitionList) { + String sourceField = partitions.getSourceField() + for (InputPartitionField partition : partitions) { partitionFields.add( InternalPartitionField.builder() - //TODO check if this still is true for parquet (get sourceField from shcema and partitionKey?) - .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, partitionKey)) - .transformType(PartitionTransformType.VALUE) + .sourceField(sourceField) + .transformType(partition.getTransformType()) .build()); } return partitionFields; } - /*public Map extractPartitionValues( - List partitionFields, Type schema) { + public List createPartitionValues(Map extractedPartitions) { + return extractedPartitions.entrySet() + .stream() + .map(internalPartitionField -> + PartitionValue.builder() + .InternalPartitionField(internalPartitionField.getKey()) + .Range(internalPartitionField.getValue()) + .collect(Collectors.toList()); + } + + public Map extractPartitionValues( + InternalPartitionFields partitionsConf) { Map partitionValues = new HashMap<>(); - for (int i = 0; i < partitionFields.size(); i++) { - InternalPartitionField partitionField = partitionFields.get(i); + List partitions = partitionsConf.getPartitions(); + for (int i = 0; i < partitions.size(); i++) { + InternalPartitionField partitionField = partitions.get(i); Object value; // Convert date based partitions into millis since epoch switch (partitionField.getTransformType()) { case YEAR: - value = EPOCH.plusYears(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + value = EPOCH.plusYears(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); break; case MONTH: - value = EPOCH.plusMonths(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + value = EPOCH.plusMonths(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); break; case DAY: - value = EPOCH.plusDays(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + value = EPOCH.plusDays(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); break; case HOUR: - value = EPOCH.plusHours(structLike.get(i, Integer.class)).toInstant().toEpochMilli(); + value = EPOCH.plusHours(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); break; default: - value = structLike.get(i, Object.class); + value = ((Object) partitionField.get().PartitionValue()); } - partitionValues.put(partitionFields.get(i), Range.scalar(value)); + partitionValues.put(partitionField, Range.scalar(value)); } return partitionValues; - }*/ + } } \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index e6014d5b4..34826eae2 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -53,9 +53,9 @@ public class ParquetTableExtractor { private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - private Map> initPartitionInfo() { + /* private Map> initPartitionInfo() { return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); - } + }*/ public String getBasePathFromLastModifiedTable(){ InternalTable table = parquetConversionSource.getTable(-1L); return table.getBasePath(); @@ -66,7 +66,7 @@ public InternalTable table(String tableName, Set partitionKeys) { MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema); List partitionFields = - partitionValueExtractor.getInternalPartitionField(partitionKeys, internalSchema); + parquetConversionSource.initPartitionInfo().getPartitions(); InternalTable snapshot = parquetConversionSource.getTable(-1L); // Assuming InternalTable.java has its getters Instant lastCommit = snapshot.latestCommitTime(); diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java index f33a31d7e..e9e1d890a 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java @@ -275,13 +275,14 @@ public static class Table { @Data public class InputPartitionField { String partitionFieldName; - List partitionFieldValues; + String partitionValue; PartitionTransformType transformType; } @Data class InputPartitionFields { - private List partitions; + String sourceField; + List partitions; } } } From c54d038d95dd4429563b9ea5e017d8b3638c8abf Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 15 Mar 2025 00:12:15 +0100 Subject: [PATCH 35/49] conversion source and schema extractor link fixed, TODO: split into two branches for review and tests --- .../parquet/ParquetConversionSource.java | 18 ++++++++++-------- .../parquet/ParquetSchemaExtractror.java | 14 +++++++------- 2 files changed, 17 insertions(+), 15 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 93849f954..dbb208ad9 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -46,8 +46,8 @@ public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); - // private static final ParquetSchemaConverter parquetSchemaConverter = -// ParquetSchemaConverter.getInstance(); +/* private static final ParquetSchemaConverter parquetSchemaConverter = + ParquetSchemaConverter.getInstance();*/ @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); @@ -99,16 +99,18 @@ public InternalTable getTable(Long modificationTime) { ParquetMetadata parquetMetadata = parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - Schema tableSchema = - new org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - + //Schema tableSchema = + // new org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); +// Type tableSchema = +// parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); + MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); Set partitionKeys = initPartitionInfo().keySet(); // merge schema of partition into original as partition is not part of parquet fie if (!partitionKeys.isEmpty()) { - //tableSchema = mergeParquetSchema(tableSchema, partitionKeys); - tableSchema = mergeAvroSchema(tableSchema, partitionKeys); + tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + //tableSchema = mergeAvroSchema(tableSchema, partitionKeys); } InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); @@ -185,7 +187,7 @@ private Schema mergeAvroSchema(Schema internalSchema, Set parititonField return fieldAssembler.endRecord(); } - private Type mergeParquetSchema(Type internalSchema, List parititonFields) { + private Type mergeParquetSchema(MessageType internalSchema, List parititonFields) { List listOfAllFields = internalSchema.getFields(); Type fieldsToMerge = listOfAllFields.get(0); diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 72b050f15..722300f96 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -82,7 +82,7 @@ public static ParquetSchemaExtractor getInstance() { return INSTANCE; } - private static Type finalizeSchema(Type targetSchema, InternalSchema inputSchema) { + private static Type finalizeSchema(MessageType targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { return targetSchema.union(LogicalTypeAnnotation.unknownType()) } @@ -98,7 +98,7 @@ private static boolean groupTypeContainsNull(Type schema) { return False; } - public InternalSchema toInternalSchema(Schema schema) { + public InternalSchema _toInternalSchema(Schema schema) { AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() Map fieldNameToIdMapping = IdTracker.getInstance() @@ -112,7 +112,7 @@ public InternalSchema toInternalSchema(Schema schema) { } // check which methods is best for the conversion - private InternalSchema toInternalSchema( + private InternalSchema _toInternalSchema( MessageType schema, String parentPath, Map fieldNameToIdMapping) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); Schema avroSchema = avroParquetSchemaConverter.convert(schema); @@ -130,8 +130,8 @@ private InternalSchema toInternalSchema( * source schema. If source schema does not contain IdMappings, map will be empty. * @return a converted schema */ - private InternalSchema toInternalSchema_bis( - Type schema, String parentPath, Map fieldNameToIdMapping) { + private InternalSchema toInternalSchema( + MessageType schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; PrimitiveType typeName; @@ -323,7 +323,7 @@ private Map getChildIdMap(IdMapping idMapping) { * @param internalSchema internal schema representation * @return an parquet schema */ - public Schema fromInternalSchema(InternalSchema internalSchema) { + public Schema _fromInternalSchema(InternalSchema internalSchema) { return fromInternalSchema(internalSchema, null); } @@ -345,7 +345,7 @@ private MessageType fromInternalSchema( * records. * @return an parquet schema */ - private Type fromInternalSchema_bis(InternalSchema internalSchema, String currentPath) { + private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { /*case BYTES: return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); From c49dbaa419c5e10c3713d6a1b5ab16588358fcc0 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 15 Mar 2025 00:47:07 +0100 Subject: [PATCH 36/49] Schema Extractor: List and Map and Fixed are converted Avro Types --- .../apache/xtable/parquet/ParquetSchemaExtractror.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 722300f96..0c1ac64e6 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -368,13 +368,13 @@ private Type fromInternalSchema(InternalSchema internalSchema, String currentPat // TODO check how to create ENUM case ENUM: return finalizeSchema( - Schema.createEnum( + new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( internalSchema.getName(), internalSchema.getComment(), null, (List) internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null), + null)), internalSchema); case DATE: return finalizeSchema( @@ -417,8 +417,8 @@ private Type fromInternalSchema(InternalSchema internalSchema, String currentPat .findFirst() .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); return finalizeSchema( - Schema.createArray( - fromInternalSchema(elementField.getSchema(), elementField.getPath())), + new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createArray( + fromInternalSchema(elementField.getSchema(), elementField.getPath()))), internalSchema); case MAP: InternalField valueField = @@ -428,7 +428,7 @@ private Type fromInternalSchema(InternalSchema internalSchema, String currentPat .findFirst() .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); return finalizeSchema( - Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath())), + new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath()))), internalSchema); case DECIMAL: int precision = From f95f87a5e068bf833786ba4ba57838127b79c134 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 15 Mar 2025 01:02:06 +0100 Subject: [PATCH 37/49] read config source bug fix --- .../org/apache/xtable/parquet/ParquetConversionSource.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index dbb208ad9..1134b68ef 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -105,7 +105,9 @@ public InternalTable getTable(Long modificationTime) { // parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); - Set partitionKeys = initPartitionInfo().keySet(); + List partitionKeys = initPartitionInfo().getPartitions().stream() + .map(InputPartitionField::getPartitionFieldName) + .collect(Collectors.toList()); // merge schema of partition into original as partition is not part of parquet fie if (!partitionKeys.isEmpty()) { From 9df1c42eed428afb5d5f518097d25bb55768bea6 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 15 Mar 2025 14:10:04 +0100 Subject: [PATCH 38/49] FIXED type conversion ok --- .../org/apache/xtable/parquet/ParquetPartitionExtractor.java | 2 +- .../org/apache/xtable/parquet/ParquetSchemaExtractror.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index b2c8e2924..8a4d106b4 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -50,7 +50,7 @@ public static ParquetPartitionExtractor getInstance() { } public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - +// todo this is to be put inside RunSync.java public InputPartitionFields getPartitionsFromUserConfiguration(String configPath) throws IOException { InputPartitionFields partitionConfiguration = new InputPartitionFields(); try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 0c1ac64e6..03b393ff1 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -453,8 +453,8 @@ private Type fromInternalSchema(InternalSchema internalSchema, String currentPat Integer fixedSize = (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); return finalizeSchema( - Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, fixedSize), + new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createFixed( + internalSchema.getName(), internalSchema.getComment(), null, fixedSize)), internalSchema); case UUID: /*Schema uuidSchema = From 60fdc8a14d84ce75677558fa34584a5c85152845 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 16 Mar 2025 22:35:01 +0100 Subject: [PATCH 39/49] fixed few compilation errors --- pom.xml | 34 ++++++ .../parquet/ParquetConversionSource.java | 11 +- .../parquet/ParquetMetadataExtractor.java | 7 +- .../parquet/ParquetPartitionExtractor.java | 1 + .../ParquetPartitionValueExtractor.java | 9 +- .../parquet/ParquetSchemaExtractror.java | 27 +++-- .../xtable/parquet/ParquetStatsExtractor.java | 13 ++- .../xtable/parquet/ParquetTableExtractor.java | 4 +- .../parquet/TestParquetSchemaExtractor.java | 106 ++++++++++-------- 9 files changed, 139 insertions(+), 73 deletions(-) diff --git a/pom.xml b/pom.xml index 3184a4be2..cdea79e0f 100644 --- a/pom.xml +++ b/pom.xml @@ -304,6 +304,40 @@ ${delta.hive.version} + + + org.apache.parquet + parquet + 1.13.0 + pom + + + + + org.apache.parquet + parquet-format + 2.10.0 + + + + + org.apache.parquet + parquet-hadoop + 1.13.0 + + + + org.apache.parquet + parquet-column + 1.13.0 + + + + org.apache.parquet + parquet-common + 1.13.0 + + org.apache.spark diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index 1134b68ef..d26b3634d 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -28,8 +28,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.parquet.Type; -import org.apache.parquet.SchemaBuilder; +import org.apache.parquet.schema.Type; +//import org.apache.parquet.SchemaBuilder; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.avro.Schema; import org.apache.xtable.model.*; @@ -39,7 +39,8 @@ import org.apache.xtable.spi.extractor.ConversionSource; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.Type.Repetition; - +import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; +import org.apache.parquet.schema.MessageType; @Builder public class ParquetConversionSource implements ConversionSource { @@ -138,7 +139,7 @@ public InternalTable getTable(Long modificationTime) { public List getInternalDataFiles() { List parquetFiles = getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - List partitionValuesFromConfig = parquetPartitionValueExtractor.createPartitionValues(parquetPartitionValueExtractor.extractPartitionValues(initPartitionInfo()) + List partitionValuesFromConfig = parquetPartitionValueExtractor.createPartitionValues(parquetPartitionValueExtractor.extractPartitionValues(initPartitionInfo())); InternalTable table = getTable(-1L); List internalDataFiles = parquetFiles.stream() @@ -200,7 +201,7 @@ private Type mergeParquetSchema(MessageType internalSchema, List paritit } for (String parition : parititonFields) { //create Type from partiton, TODO: check further... - fieldsToMerge = fieldsToMerge.union(Type(partition, Repetition.REQUIRED)) + fieldsToMerge = fieldsToMerge.union(Type(partition, Repetition.REQUIRED)); } return fieldsToMerge; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java index c5790d642..03796501e 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java @@ -15,11 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + +package org.apache.xtable.parquet; import org.apache.hadoop.fs.*; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; - +import org.apache.hadoop.conf.Configuration; public class ParquetMetadataExtractor { private static MessageType getSchema(ParquetMetadata footer) { @@ -27,7 +28,7 @@ private static MessageType getSchema(ParquetMetadata footer) { return schema; } - private static ParquetMetadata readParquetMetadata(HadoopConf conf, BasePath path) { + private static ParquetMetadata readParquetMetadata(Configuration conf, String path) { ParquetMetadata footer = ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); return footer; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java index 8a4d106b4..7a1b835ed 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; +import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.schema.PartitionTransformType; diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index cbe3cef3f..fad781153 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.xtable.iceberg; +package org.apache.xtable.parquet; import java.time.Instant; import java.time.OffsetDateTime; @@ -33,7 +33,8 @@ import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.stat.Range; import org.apache.xtable.schema.SchemaFieldFinder; - +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; /** * Partition value extractor for Parquet. */ @@ -50,7 +51,7 @@ public static ParquetPartitionValueExtractor getInstance() { public List getInternalPartitionField( InputPartitionFields partitions) { List partitionFields = new ArrayList<>(); - String sourceField = partitions.getSourceField() + String sourceField = partitions.getSourceField(); for (InputPartitionField partition : partitions) { partitionFields.add( InternalPartitionField.builder() @@ -68,7 +69,7 @@ public List createPartitionValues(Map PartitionValue.builder() .InternalPartitionField(internalPartitionField.getKey()) - .Range(internalPartitionField.getValue()) + .Range(internalPartitionField.getValue())) .collect(Collectors.toList()); } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index 03b393ff1..c4c290034 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -31,13 +31,16 @@ import lombok.AccessLevel; import lombok.NoArgsConstructor; -import org.apache.parquet.LogicalType; -import org.apache.parquet.LogicalTypes; +import org.apache.parquet.schema.LogicalType; +//import org.apache.parquet.LogicalTypes; //import org.apache.parquet.Schema; -import org.apache.parquet.Schema.Type; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.format.NullType; import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.Type.Repetition; + import org.apache.xtable.collectors.CustomCollectors; @@ -84,13 +87,13 @@ public static ParquetSchemaExtractor getInstance() { private static Type finalizeSchema(MessageType targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { - return targetSchema.union(LogicalTypeAnnotation.unknownType()) + return targetSchema.union(LogicalTypeAnnotation.unknownType()); } return targetSchema; } private static boolean groupTypeContainsNull(Type schema) { - for (Type field in schema.getFields()){ + for (Type field : schema.getFields()){ if (field == null) { return True; } @@ -99,7 +102,7 @@ private static boolean groupTypeContainsNull(Type schema) { } public InternalSchema _toInternalSchema(Schema schema) { - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); Map fieldNameToIdMapping = IdTracker.getInstance() .getIdTracking(schema) @@ -204,7 +207,7 @@ private InternalSchema toInternalSchema( } break; case FIXED_LEN_BYTE_ARRAY: - logicalType = schema.getLogicalTypeAnnotation() + logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.UUIDLogicalTypeAnnotation) { newDataType = InternalType.UUID; } else if (logicalType instanceof LogicalTypeAnnotation.IntervalLogicalTypeAnnotation) { @@ -215,7 +218,7 @@ private InternalSchema toInternalSchema( //TODO add other logicalTypes? case BINARY: //? Variant,GEOMETRY, GEOGRAPHY, - logicalType = schema.getLogicalTypeAnnotation() + logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.toOriginalType().values()); newDataType = InternalType.ENUM; @@ -331,7 +334,7 @@ public Schema _fromInternalSchema(InternalSchema internalSchema) { private MessageType fromInternalSchema( InternalSchema internalSchema, String currentPath) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance() + AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); Schema avroSchema = avroSchemaConverter.fromInternalSchema(internalSchema,currentPath); MessageType parquetSchema = avroParquetSchemaConverter.convert(avroSchema); return parquetSchema; @@ -383,17 +386,17 @@ private Type fromInternalSchema(InternalSchema internalSchema, String currentPat if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { return finalizeSchema( - , LogicalTypeAnnotation.timestampType(True, MICROS) + LogicalTypeAnnotation.timestampType(True, Repetition.MICROS), internalSchema); } if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MILLIS) { return finalizeSchema( - , LogicalTypeAnnotation.timestampType(True, MILLIS) + LogicalTypeAnnotation.timestampType(True, Repetition.MILLIS), internalSchema); } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.NANOS) { return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, NANOS), + LogicalTypeAnnotation.timestampType(True, Repetition.NANOS), internalSchema); } case TIMESTAMP_NTZ: diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index d9ba1e842..e72d7cc38 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - +package org.apache.xtable.parquet; import java.util.stream.Collectors; import org.apache.hadoop.fs.*; @@ -26,7 +26,18 @@ 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.storage.InternalDataFile; +import java.util.Map; +import java.util.Collection; +import java.util.Set; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Stream; + +@Value +@Builder public class ParquetStatsExtractor { @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 34826eae2..5f2677678 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -16,13 +16,13 @@ * limitations under the License. */ -package org.apache.xtable.delta; +package org.apache.xtable.parquet; import java.time.Instant; import java.util.List; import lombok.Builder; - +import java.util.Set; import org.apache.xtable.model.InternalTable; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java index 46b8ee809..da22bff08 100644 --- a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -8,7 +8,8 @@ import org.apache.parquet.schema.*; import org.apache.parquet.schema.Type.Repetition; - +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; @@ -25,49 +26,10 @@ public class TestParquetSchemaExtractor { static final MessageType messageType = new MessageType("schema", groupType2); private static final ParquetSchemaExtractor SCHEMA_EXTRACTOR = ParquetSchemaExtractor.getInstance(); - public static void main(String[] args) { - generateParquetFileFor(); - } - - @Test - public void testPrimitiveTypes() { - /* Map requiredEnumMetadata = - Collections.singletonMap( - InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("ONE", "TWO")); - Map optionalEnumMetadata = - Collections.singletonMap( - InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("THREE", "FOUR"));*/ - InternalSchema schemaWithPrimitiveTypes = - InternalSchema.builder() - .dataType(InternalType.RECORD) - .fields( - Arrays.asList( - InternalField.builder() - .name("int") - .schema( - InternalSchema.builder() - .name("REQUIRED_int") - .dataType(InternalType.INT) - .isNullable(false) - .metadata(null) - .build()) - .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) - .build(), - InternalField.builder() - .name("float") - .schema( - InternalSchema.builder() - .name("REQUIRED_double") - .dataType(InternalType.FLOAT) - .isNullable(true) - .metadata(null) - .build()) - .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) - .build())) - .build(); - //Type expectedSchema = mapGroupType; - assertTrue(TestParquetSchemaExtractor.mapGroupType.equals(SCHEMA_EXTRACTOR.toInternalSchema(schemaWithPrimitiveTypes))); + public static void main(String[] args) { + //generateParquetFileFor(); + testPrimitiveTypes(); } private static void generateParquetFileFor() { @@ -96,10 +58,10 @@ private static List generateRecords() { List recordList = new ArrayList<>(); - for(int i = 1; i <= 4; i++) { + for (int i = 1; i <= 4; i++) { Group mapGroup = new SimpleGroup(mapGroupType); - mapGroup.add("fakekey", i*i); - mapGroup.add("fakevalue", i*i*i); + mapGroup.add("fakekey", i * i); + mapGroup.add("fakevalue", i * i * i); Group group = new SimpleGroup(groupType); group.add("key_value", mapGroup); Group mapGroup2 = new SimpleGroup(mapGroupType2); @@ -114,4 +76,56 @@ private static List generateRecords() { return recordList; } + + @Test + public void testPrimitiveTypes() { + /* Map requiredEnumMetadata = + Collections.singletonMap( + InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("ONE", "TWO")); + Map optionalEnumMetadata = + Collections.singletonMap( + InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("THREE", "FOUR"));*/ + InternalSchema primitive1 = InternalSchema.builder() + .name("integer"); + .dataType(InternalType.INT); + InternalSchema primitive2 = InternalSchema.builder() + .name("string"); + .dataType(InternalType.STRING); + + +/* InternalSchema schemaWithPrimitiveTypes = + InternalSchema.builder() + .dataType(InternalType.RECORD) + .fields( + Arrays.asList( + InternalField.builder() + .name("int") + .schema( + InternalSchema.builder() + .name("REQUIRED_int") + .dataType(InternalType.INT) + .isNullable(false) + .metadata(null) + .build()) + .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) + .build(), + InternalField.builder() + .name("float") + .schema( + InternalSchema.builder() + .name("REQUIRED_double") + .dataType(InternalType.FLOAT) + .isNullable(true) + .metadata(null) + .build()) + .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) + .build() + )) + .build();*/ + //Type expectedSchema = mapGroupType; + MessageType integerPrimitiveType = MessageType(REQUIRED,PrimitiveType(Type.Repetition repetition, INT32, "integer") ); + Assertions.assertEquals( + primitive1, SCHEMA_EXTRACTOR.toInternalSchema(integerPrimitiveType, null, null)); + //assertTrue(TestParquetSchemaExtractor.mapGroupType.equals(SCHEMA_EXTRACTOR.toInternalSchema(schemaWithPrimitiveTypes))); + } } \ No newline at end of file From eb7f60f52f442510ba715dcdc0399c1a7fd8305e Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 16 Mar 2025 23:16:16 +0100 Subject: [PATCH 40/49] few other compilation errors fixed --- .../org/apache/xtable/parquet/ParquetStatsExtractor.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index e72d7cc38..5f0334e95 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -18,6 +18,7 @@ package org.apache.xtable.parquet; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Encoding; @@ -27,7 +28,9 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.apache.xtable.model.storage.InternalDataFile; - +import lombok.Builder; +import lombok.NonNull; +import lombok.Value; import java.util.Map; import java.util.Collection; import java.util.Set; From 96e91cd93b8efa51e3dd60cdc865dc9e7cf26e27 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sun, 16 Mar 2025 23:30:39 +0100 Subject: [PATCH 41/49] few other compilation errors fixed 2 --- .../apache/xtable/parquet/ParquetSchemaExtractror.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java index c4c290034..0d892e78a 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java @@ -31,7 +31,7 @@ import lombok.AccessLevel; import lombok.NoArgsConstructor; -import org.apache.parquet.schema.LogicalType; +//import org.apache.parquet.schema.LogicalType; //import org.apache.parquet.LogicalTypes; //import org.apache.parquet.Schema; import org.apache.parquet.schema.Type; @@ -101,7 +101,7 @@ private static boolean groupTypeContainsNull(Type schema) { return False; } - public InternalSchema _toInternalSchema(Schema schema) { + /* public InternalSchema _toInternalSchema(Schema schema) { AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); Map fieldNameToIdMapping = IdTracker.getInstance() @@ -121,7 +121,7 @@ private InternalSchema _toInternalSchema( Schema avroSchema = avroParquetSchemaConverter.convert(schema); AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); return avroSchemaConverter.toInternalSchema(avroSchema,parentPath,fieldNameToIdMapping); - } + }*/ /** * Converts the parquet {@link Schema} to {@link InternalSchema}. @@ -331,14 +331,14 @@ public Schema _fromInternalSchema(InternalSchema internalSchema) { } // check which methods is best for the conversion - private MessageType fromInternalSchema( + /*private MessageType fromInternalSchema( InternalSchema internalSchema, String currentPath) { org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); Schema avroSchema = avroSchemaConverter.fromInternalSchema(internalSchema,currentPath); MessageType parquetSchema = avroParquetSchemaConverter.convert(avroSchema); return parquetSchema; - } + }*/ /** * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. * From f1b75249bf34966293d3079957d024d3a829721c Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Mon, 17 Mar 2025 07:07:19 +0100 Subject: [PATCH 42/49] code compiling --- .../xtable/conversion/ConversionConfig.java | 7 +- .../model/config/InputPartitionField.java | 32 ++ .../model/config/InputPartitionFields.java | 30 ++ .../model/storage/InternalDataFile.java | 2 +- .../xtable/model/storage/TableFormat.java | 15 +- .../parquet/ParquetConversionSource.java | 107 ++-- .../ParquetConversionSourceProvider.java | 4 +- .../parquet/ParquetMetadataExtractor.java | 16 +- .../parquet/ParquetPartitionExtractor.java | 95 ---- .../ParquetPartitionValueExtractor.java | 44 +- ...ctror.java => ParquetSchemaExtractor.java} | 308 ++++++------ .../xtable/parquet/ParquetStatsExtractor.java | 218 ++++---- .../xtable/parquet/ParquetTableExtractor.java | 47 +- .../parquet/TestParquetSchemaExtractor.java | 17 + .../org/apache/xtable/utilities/RunSync.java | 476 +++++++++--------- 15 files changed, 740 insertions(+), 678 deletions(-) create mode 100644 xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java create mode 100644 xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java delete mode 100644 xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java rename xtable-core/src/main/java/org/apache/xtable/parquet/{ParquetSchemaExtractror.java => ParquetSchemaExtractor.java} (65%) diff --git a/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java b/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java index 63e9d6733..9a5d9f47a 100644 --- a/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java +++ b/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java @@ -29,7 +29,7 @@ import com.google.common.base.Preconditions; import org.apache.xtable.model.sync.SyncMode; - +import org.apache.xtable.model.config.InputPartitionFields; @Value @Builder public class ConversionConfig { @@ -42,13 +42,15 @@ public class ConversionConfig { Map> targetCatalogs; // The mode, incremental or snapshot SyncMode syncMode; + // Input partition config for parquet + InputPartitionFields partitions; @Builder ConversionConfig( @NonNull SourceTable sourceTable, List targetTables, Map> targetCatalogs, - SyncMode syncMode) { + SyncMode syncMode, InputPartitionFields partitions) { this.sourceTable = sourceTable; this.targetTables = targetTables; Preconditions.checkArgument( @@ -56,5 +58,6 @@ public class ConversionConfig { "Please provide at-least one format to sync"); this.targetCatalogs = targetCatalogs == null ? Collections.emptyMap() : targetCatalogs; this.syncMode = syncMode == null ? SyncMode.INCREMENTAL : syncMode; + this.partitions = partitions; } } diff --git a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java new file mode 100644 index 000000000..aa9e0fc80 --- /dev/null +++ b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java @@ -0,0 +1,32 @@ +/* + * 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.model.config; +import lombok.Data; +import lombok.Builder; +import lombok.Getter; +import lombok.Value; +import org.apache.xtable.model.schema.PartitionTransformType; +@Data +@Value +@Builder(toBuilder = true) +public class InputPartitionField { + String partitionFieldName; + String partitionValue; + PartitionTransformType transformType; +} \ No newline at end of file diff --git a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java new file mode 100644 index 000000000..1c656d68c --- /dev/null +++ b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java @@ -0,0 +1,30 @@ +/* + * 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.model.config; +import java.util.List; +import lombok.Data; +import lombok.Builder; +import lombok.Getter; +import lombok.Value; +@Data +@Value +@Builder(toBuilder = true) +public class InputPartitionFields { + String sourceField; + List partitions; +} \ No newline at end of file diff --git a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java index de6e98fb9..d7d0ac661 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java @@ -24,7 +24,7 @@ import lombok.Builder; import lombok.NonNull; import lombok.Value; - +import org.apache.xtable.model.storage.FileFormat; import org.apache.xtable.model.stat.ColumnStat; import org.apache.xtable.model.stat.PartitionValue; diff --git a/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java b/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java index bea0b4774..e98ad4874 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.model.storage; /** @@ -24,11 +24,12 @@ * @since 0.1 */ public class TableFormat { - public static final String HUDI = "HUDI"; - public static final String ICEBERG = "ICEBERG"; - public static final String DELTA = "DELTA"; + public static final String HUDI = "HUDI"; + public static final String ICEBERG = "ICEBERG"; + public static final String DELTA = "DELTA"; + public static final String PARQUET = "PARQUET"; - public static String[] values() { - return new String[] {"HUDI", "ICEBERG", "DELTA"}; - } + public static String[] values() { + return new String[]{"HUDI", "ICEBERG", "DELTA", "PARQUET"}; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index d26b3634d..fa366aef0 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -22,6 +22,7 @@ import java.time.Instant; import java.util.*; import java.util.stream.Collectors; +import java.util.stream.Stream; import lombok.Builder; import lombok.NonNull; @@ -39,25 +40,34 @@ import org.apache.xtable.spi.extractor.ConversionSource; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.Type.Repetition; -import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.config.InputPartitionField; import org.apache.parquet.schema.MessageType; +import org.apache.xtable.model.stat.PartitionValue; +import lombok.AccessLevel; +import lombok.NoArgsConstructor; @Builder -public class ParquetConversionSource implements ConversionSource { +//@NoArgsConstructor(access = AccessLevel.PRIVATE) +public class ParquetConversionSource {// implements ConversionSource { + + private final InputPartitionFields partitions; @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); + + /*private static final ParquetConversionSource INSTANCE = new ParquetConversionSource(); + public static ParquetConversionSource getInstance() { + return INSTANCE; + }*/ /* private static final ParquetSchemaConverter parquetSchemaConverter = ParquetSchemaConverter.getInstance();*/ @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - @Builder.Default - private static final ParquetPartitionExtractor parquetPartitionExtractor = - ParquetPartitionExtractor.getInstance(); @Builder.Default - private static final ParquetPartitionValueExtractor parquetPartitionValueExtractor = + private static final ParquetPartitionValueExtractor partitionValueExtractor = ParquetPartitionValueExtractor.getInstance(); @Builder.Default @@ -71,19 +81,20 @@ public class ParquetConversionSource implements ConversionSource { private final Configuration hadoopConf; private InputPartitionFields initPartitionInfo() { - return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + //return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + return partitions; } - public Map> getPartitionFromConfiguration() { - List partitionFields = initPartitionInfo().getPartitions(); - Map> partitionsMap = new HashMap<>(); - for (InputPartitionField partition : partitionFields) { - partitionsMap - .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) - .addAll(partition.partitionFieldValues()); - } - return partitionsMap; - } +// public Map> getPartitionFromConfiguration() { +// List partitionFields = initPartitionInfo().getPartitions(); +// Map> partitionsMap = new HashMap<>(); +// for (InputPartitionField partition : partitionFields) { +// partitionsMap +// .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) +// .addAll(partition.partitionFieldValues()); +// } +// return partitionsMap; +// } /** * To infer schema getting the latest file assumption is that latest file will have new fields @@ -91,7 +102,7 @@ public Map> getPartitionFromConfiguration() { * @param modificationTime the commit to consider for reading the table state * @return */ - @Override + //@Override public InternalTable getTable(Long modificationTime) { Optional latestFile = @@ -110,17 +121,17 @@ public InternalTable getTable(Long modificationTime) { .map(InputPartitionField::getPartitionFieldName) .collect(Collectors.toList()); - // merge schema of partition into original as partition is not part of parquet fie + // merge schema of partition into original as partition is not part of parquet file if (!partitionKeys.isEmpty()) { - tableSchema = mergeParquetSchema(tableSchema, partitionKeys); - //tableSchema = mergeAvroSchema(tableSchema, partitionKeys); + //TODO compilation error + // tableSchema = mergeParquetSchema(tableSchema, partitionKeys); } - InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema); + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema,null,null); List partitionFields = partitionKeys.isEmpty() ? Collections.emptyList() - : parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + : partitionValueExtractor.getInternalPartitionFields(partitions); DataLayoutStrategy dataLayoutStrategy = partitionFields.isEmpty() ? DataLayoutStrategy.FLAT @@ -137,9 +148,10 @@ public InternalTable getTable(Long modificationTime) { } public List getInternalDataFiles() { - List parquetFiles = + List internalDataFiles = null; + /* List parquetFiles = getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - List partitionValuesFromConfig = parquetPartitionValueExtractor.createPartitionValues(parquetPartitionValueExtractor.extractPartitionValues(initPartitionInfo())); + List partitionValuesFromConfig = partitionValueExtractor.createPartitionValues(partitionValueExtractor.extractPartitionValues(partitions)); InternalTable table = getTable(-1L); List internalDataFiles = parquetFiles.stream() @@ -155,27 +167,35 @@ public List getInternalDataFiles() { parquetStatsExtractor .getColumnStatsForaFile( parquetMetadataExtractor.readParquetMetadata( - hadoopConf, file.getPath().toString())) + hadoopConf, file.getPath())) .build()) - .collect(Collectors.toList())); + .collect(Collectors.toList()));*/ return internalDataFiles; } +/* + @Override + public CommitsBacklog getCommitsBacklog(){ + + } +*/ + /** * Here to get current snapshot listing all files hence the -1 is being passed * * @return */ - @Override + //@Override public InternalSnapshot getCurrentSnapshot() { - List internalDataFiles = getInternalDataFiles(); + /*List internalDataFiles = getInternalDataFiles(); return InternalSnapshot.builder() .table(table) .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) - .build(); + .build();*/ + return null; } - private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { + /* private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { SchemaBuilder.FieldAssembler fieldAssembler = SchemaBuilder.record(internalSchema.getName()).fields(); @@ -188,32 +208,33 @@ private Schema mergeAvroSchema(Schema internalSchema, Set parititonField } return fieldAssembler.endRecord(); - } + }*/ - private Type mergeParquetSchema(MessageType internalSchema, List parititonFields) { + /* private Type mergeParquetSchema(MessageType internalSchema, List parititonFields) { List listOfAllFields = internalSchema.getFields(); Type fieldsToMerge = listOfAllFields.get(0); listOfAllFields.remove(0); // start the merge for (Type field : internalSchema.getFields()) { - fieldsToMerge = fieldsToMerge.union(field); + fieldsToMerge = fieldsToMerge.union(field,false); } - for (String parition : parititonFields) { + *//* for (String partition : parititonFields) { //create Type from partiton, TODO: check further... - fieldsToMerge = fieldsToMerge.union(Type(partition, Repetition.REQUIRED)); - } + fieldsToMerge = fieldsToMerge.union(new Type(partition, Repetition.REQUIRED),false); + }*//* return fieldsToMerge; - } + }*/ public Stream getParquetFiles(Configuration hadoopConf, String basePath) { try { FileSystem fs = FileSystem.get(hadoopConf); RemoteIterator iterator = fs.listFiles(new Path(basePath), true); - return remoteIteratorToStream(iterator) - .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException | FileNotFoundException e) { + return null; + //remoteIteratorToStream(iterator) + // .filter(file -> file.getPath().getName().endsWith("parquet")); + } catch (IOException e) {//| FileNotFoundException e throw new RuntimeException(e); } } @@ -245,4 +266,8 @@ public Map> getPartitionFromDirectoryStructure( throw new RuntimeException(e); } } + //@Override + public boolean isIncrementalSyncSafeFrom(Instant instant) { + return false; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java index e3bfd20a2..1c41a7144 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java @@ -24,8 +24,8 @@ import org.apache.xtable.conversion.SourceTable; /** A concrete implementation of {@link ConversionSourceProvider} for Delta Lake table format. */ -public class ParquetConversionSourceProvider extends ConversionSourceProvider { - @Override +public class ParquetConversionSourceProvider {//extends ConversionSourceProvider { + //@Override public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { return ParquetConversionSource.builder() diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java index 03796501e..8e3cc82db 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java @@ -21,16 +21,24 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.apache.hadoop.conf.Configuration; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.format.converter.ParquetMetadataConverter; +import org.apache.hadoop.fs.Path; public class ParquetMetadataExtractor { - private static MessageType getSchema(ParquetMetadata footer) { + private static final ParquetMetadataExtractor INSTANCE = new ParquetMetadataExtractor(); + public static ParquetMetadataExtractor getInstance() { + return INSTANCE; + } + + public static MessageType getSchema(ParquetMetadata footer) { MessageType schema = footer.getFileMetaData().getSchema(); return schema; } - private static ParquetMetadata readParquetMetadata(Configuration conf, String path) { - ParquetMetadata footer = - ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + public static ParquetMetadata readParquetMetadata(Configuration conf, Path path) { + ParquetMetadata footer =null; + //ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); return footer; } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java deleted file mode 100644 index 7a1b835ed..000000000 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionExtractor.java +++ /dev/null @@ -1,95 +0,0 @@ -/* - * 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.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; -import org.apache.xtable.model.schema.InternalPartitionField; -import org.apache.xtable.model.schema.InternalSchema; -import org.apache.xtable.model.schema.PartitionTransformType; -import org.apache.xtable.model.stat.PartitionValue; -import org.apache.xtable.model.stat.Range; -import org.apache.xtable.schema.SchemaFieldFinder; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectReader; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; - -public class ParquetPartitionExtractor { - private static final ParquetPartitionExtractor INSTANCE = new ParquetPartitionExtractor(); - - public static ParquetPartitionExtractor getInstance() { - return INSTANCE; - } - public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - -// todo this is to be put inside RunSync.java - public InputPartitionFields getPartitionsFromUserConfiguration(String configPath) throws IOException { - InputPartitionFields partitionConfiguration = new InputPartitionFields(); - try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { - ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); - objectReader.readValue(inputStream); - return partitionConfiguration; - } - } - - // TODO logic is too complicated can be simplified -/* public List getPartitionValue( - String basePath, - String filePath, - InternalSchema schema, - Map> partitionInfo) { - List partitionValues = new ArrayList<>(); - java.nio.file.Path base = Paths.get(basePath).normalize(); - java.nio.file.Path file = Paths.get(filePath).normalize(); - java.nio.file.Path relative = base.relativize(file); - for (Map.Entry> entry : partitionInfo.entrySet()) { - String key = entry.getKey(); - List values = entry.getValue(); - for (String value : values) { - String pathCheck = key + "=" + value; - if (relative.startsWith(pathCheck)) { - System.out.println("Relative " + relative + " " + pathCheck); - partitionValues.add( - PartitionValue.builder() - .partitionField( - InternalPartitionField.builder() - .sourceField(SchemaFieldFinder.getInstance().findFieldByPath(schema, key)) - .transformType(PartitionTransformType.VALUE) - .build()) - .range(Range.scalar(value)) - .build()); - } - } - } - return partitionValues; - }*/ -} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index fad781153..3d99bf457 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -24,17 +24,21 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.NoArgsConstructor; - +import org.apache.xtable.model.schema.PartitionTransformType; //import org.apache.iceberg.StructLike; import org.apache.xtable.model.schema.InternalPartitionField; +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.config.InputPartitionField; import org.apache.xtable.model.stat.Range; import org.apache.xtable.schema.SchemaFieldFinder; import org.apache.xtable.model.stat.PartitionValue; -import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; +import java.util.ArrayList; +//import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; /** * Partition value extractor for Parquet. */ @@ -48,14 +52,15 @@ public static ParquetPartitionValueExtractor getInstance() { return INSTANCE; } - public List getInternalPartitionField( + public List getInternalPartitionFields( InputPartitionFields partitions) { List partitionFields = new ArrayList<>(); String sourceField = partitions.getSourceField(); - for (InputPartitionField partition : partitions) { + for (InputPartitionField partition : partitions.getPartitions()) { partitionFields.add( InternalPartitionField.builder() - .sourceField(sourceField) + // TODO convert sourceField type + .sourceField(null) .transformType(partition.getTransformType()) .build()); } @@ -63,43 +68,44 @@ public List getInternalPartitionField( return partitionFields; } + public List createPartitionValues(Map extractedPartitions) { - return extractedPartitions.entrySet() + return null;/*extractedPartitions.entrySet() .stream() .map(internalPartitionField -> PartitionValue.builder() .InternalPartitionField(internalPartitionField.getKey()) - .Range(internalPartitionField.getValue())) - .collect(Collectors.toList()); + .Range(null))//internalPartitionField.getValue()) + .collect(Collectors.toList());*/ } public Map extractPartitionValues( - InternalPartitionFields partitionsConf) { + InputPartitionFields partitionsConf) { Map partitionValues = new HashMap<>(); - List partitions = partitionsConf.getPartitions(); + /* List partitions = partitionsConf.getPartitions(); for (int i = 0; i < partitions.size(); i++) { - InternalPartitionField partitionField = partitions.get(i); + InputPartitionField partitionField = partitions.get(i); Object value; // Convert date based partitions into millis since epoch switch (partitionField.getTransformType()) { case YEAR: - value = EPOCH.plusYears(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); + value = EPOCH.plusYears(Integer.parseInt( partitionField.getPartitionValue())).toInstant().toEpochMilli(); break; case MONTH: - value = EPOCH.plusMonths(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); + value = EPOCH.plusMonths(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); break; case DAY: - value = EPOCH.plusDays(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); + value = EPOCH.plusDays(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); break; case HOUR: - value = EPOCH.plusHours(partitionField.get().PartitionValue()).toInstant().toEpochMilli(); + value = EPOCH.plusHours(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); break; default: - value = ((Object) partitionField.get().PartitionValue()); - } + value = ((Object) partitionField.getPartitionValue()); + }*/ + + // partitionValues.put(partitionField, Range.scalar(value)); - partitionValues.put(partitionField, Range.scalar(value)); - } return partitionValues; } } \ No newline at end of file diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java similarity index 65% rename from xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java rename to xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java index 0d892e78a..ada976c86 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractror.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java @@ -37,9 +37,11 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.GroupType; import org.apache.parquet.format.NullType; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.Type.Repetition; +import org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit; @@ -54,6 +56,7 @@ import org.apache.xtable.schema.SchemaUtils; import org.apache.xtable.avro.AvroSchemaConverter; +//import org.apache.parquet.avro.AvroSchemaConverter; import org.apache.avro.Schema; //import org.apache.parquet.avro.AvroSchemaConverter; @@ -77,28 +80,29 @@ public class ParquetSchemaExtractor { .defaultValue("") .build(); private static final ParquetSchemaExtractor INSTANCE = new ParquetSchemaExtractor(); + public static ParquetSchemaExtractor getInstance() { + return INSTANCE; + } 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 Type finalizeSchema(MessageType targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { - return targetSchema.union(LogicalTypeAnnotation.unknownType()); + return targetSchema.union(null);//LogicalTypeAnnotation.unknownType() } return targetSchema; } - private static boolean groupTypeContainsNull(Type schema) { + private static boolean groupTypeContainsNull(GroupType schema) { for (Type field : schema.getFields()){ if (field == null) { - return True; + return true; } } - return False; + return false; } /* public InternalSchema _toInternalSchema(Schema schema) { @@ -133,7 +137,7 @@ private InternalSchema _toInternalSchema( * source schema. If source schema does not contain IdMappings, map will be empty. * @return a converted schema */ - private InternalSchema toInternalSchema( + public InternalSchema toInternalSchema( MessageType schema, String parentPath, Map fieldNameToIdMapping) { // TODO - Does not handle recursion in parquet schema InternalType newDataType; @@ -147,7 +151,7 @@ private InternalSchema toInternalSchema( case INT64: logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation)logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { newDataType = InternalType.TIMESTAMP; metadata.put( @@ -163,13 +167,13 @@ private InternalSchema toInternalSchema( } } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; - } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + }/* else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation)logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS || timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { // check if INT is the InternalType needed here newDataType = InternalType.INT; } - } else { + }*/ else { newDataType = InternalType.INT; } break; @@ -177,13 +181,13 @@ private InternalSchema toInternalSchema( logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; - } else if (logicalType instanceof TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = logicalType.getUnit(); + } /*else if (logicalType instanceof TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimeLogicalTypeAnnotation)logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { // check if INT is the InternalType needed here newDataType = InternalType.INT; } - } else { + }*/ else { newDataType = InternalType.INT; } break; @@ -192,15 +196,15 @@ private InternalSchema toInternalSchema( break; case FLOAT: logicalType = schema.getLogicalTypeAnnotation(); - if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { + /* if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { newDataType = InternalType.FLOAT; - } else if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + } else*/ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { metadata.put( InternalSchema.MetadataKey.DECIMAL_PRECISION, - logicalType.getPrecision()); + ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation)logicalType).getPrecision()); metadata.put( InternalSchema.MetadataKey.DECIMAL_SCALE, - logicalType.getScale()); + ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation)logicalType).getScale()); newDataType = InternalType.DECIMAL; } else { newDataType = InternalType.FLOAT; @@ -220,7 +224,7 @@ private InternalSchema toInternalSchema( //? Variant,GEOMETRY, GEOGRAPHY, logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, schema.toOriginalType().values()); + metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, logicalType.toOriginalType().values()); newDataType = InternalType.ENUM; } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; @@ -235,14 +239,14 @@ private InternalSchema toInternalSchema( case BOOLEAN: newDataType = InternalType.BOOLEAN; break; - case UNKNOWN: + /* case UNKNOWN: newDataType = InternalType.NULL; - break; + break;*/ default: throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); } - } else { + } /*else { //GroupTypes typeName = schema.asGroupType(); switch (typeName.getOriginalType()) { @@ -302,33 +306,25 @@ private InternalSchema toInternalSchema( throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); } - } + }*/ + newDataType = null; return InternalSchema.builder() .name(schema.getName()) .dataType(newDataType) - .comment(schema.getDoc()) - .isNullable(schema.isNullable()) + .comment(null) + .isNullable(false)// to check .metadata(metadata.isEmpty() ? null : metadata) .build(); } - private Map getChildIdMap(IdMapping idMapping) { + /* private Map getChildIdMap(IdMapping idMapping) { if (idMapping == null) { return Collections.emptyMap(); } return idMapping.getFields().stream() .collect(Collectors.toMap(IdMapping::getName, Function.identity())); - } + }*/ - /** - * Converts the {@link InternalSchema} to parquet {@link Schema}. - * - * @param internalSchema internal schema representation - * @return an parquet schema - */ - public Schema _fromInternalSchema(InternalSchema internalSchema) { - return fromInternalSchema(internalSchema, null); - } // check which methods is best for the conversion /*private MessageType fromInternalSchema( @@ -348,128 +344,128 @@ public Schema _fromInternalSchema(InternalSchema internalSchema) { * records. * @return an parquet schema */ - private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { - switch (internalSchema.getDataType()) { - /*case BYTES: - return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); - case BOOLEAN: - return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ - case INT: - return finalizeSchema(LogicalTypeAnnotation.intType(32), internalSchema); - case LONG: - return finalizeSchema(LogicalTypeAnnotation.intType(64), internalSchema); - case STRING: - return finalizeSchema(LogicalTypeAnnotation.stringType(), internalSchema); - case FLOAT: - return finalizeSchema(LogicalTypeAnnotation.float16Type(), internalSchema); - case DOUBLE: - int precision = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); - int scale = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); - return finalizeSchema(LogicalTypeAnnotation.decimalType(scale, precision), internalSchema); - // TODO check how to create ENUM - case ENUM: - return finalizeSchema( - new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( - internalSchema.getName(), - internalSchema.getComment(), - null, - (List) - internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null)), - internalSchema); - case DATE: - return finalizeSchema( - LogicalTypeAnnotation.dateType(), internalSchema); - case TIMESTAMP: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, Repetition.MICROS), - internalSchema); - } if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MILLIS) { - return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, Repetition.MILLIS), - internalSchema); - } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.NANOS) { - return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, Repetition.NANOS), - internalSchema); - } - case TIMESTAMP_NTZ: - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) - == InternalSchema.MetadataValue.MICROS) { - return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, MICROS), - internalSchema); - } else { - return finalizeSchema( - LogicalTypeAnnotation.timestampType(True, MILLIS), - internalSchema); - } - // TODO check from here FIXED, LIST and MAP types (still to todo) - case LIST: - InternalField elementField = - internalSchema.getFields().stream() - .filter( - field -> - InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); - return finalizeSchema( - new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createArray( - fromInternalSchema(elementField.getSchema(), elementField.getPath()))), - internalSchema); - case MAP: - InternalField valueField = - internalSchema.getFields().stream() - .filter( - field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) - .findFirst() - .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); - return finalizeSchema( - new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath()))), - internalSchema); - case DECIMAL: - int precision = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); - int scale = - (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); -// Integer size = -// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); -// if (size == null) { - return finalizeSchema( - LogicalTypeAnnotation.decimalType(scale, precision), - internalSchema); +// private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { +// switch (internalSchema.getDataType()) { +// /*case BYTES: +// return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); +// case BOOLEAN: +// return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ +// case INT: +// return finalizeSchema(LogicalTypeAnnotation.intType(32,false), internalSchema); +// case LONG: +// return finalizeSchema(LogicalTypeAnnotation.intType(64,false), internalSchema); +// case STRING: +// return finalizeSchema(LogicalTypeAnnotation.stringType(), internalSchema); +// //case FLOAT: +// //return finalizeSchema(LogicalTypeAnnotation.float16Type(), internalSchema); +// /*case DOUBLE: +// int precision = +// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); +// int scale = +// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); +// return finalizeSchema(LogicalTypeAnnotation.decimalType(scale, precision), internalSchema); +// // TODO check how to create ENUM +// case ENUM: +// return finalizeSchema( +// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( +// internalSchema.getName(), +// internalSchema.getComment(), +// null, +// (List) +// internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), +// null)), +// internalSchema);*/ +// case DATE: +// return finalizeSchema( +// LogicalTypeAnnotation.dateType(), internalSchema); +// /*case TIMESTAMP: +// if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) +// == InternalSchema.MetadataValue.MICROS) { +// return finalizeSchema( +// LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), +// internalSchema); +// } if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) +// == InternalSchema.MetadataValue.MILLIS) { +// return finalizeSchema( +// LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), +// internalSchema); +// } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) +// == InternalSchema.MetadataValue.NANOS) { +// return finalizeSchema( +// LogicalTypeAnnotation.timestampType(true, TimeUnit.NANOS), +// internalSchema); +// } +// case TIMESTAMP_NTZ: +// if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) +// == InternalSchema.MetadataValue.MICROS) { +// return finalizeSchema( +// LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), +// internalSchema); // } else { // return finalizeSchema( -// LogicalTypes.decimal(precision, scale) -// .addToSchema( -// Schema.createFixed( -// internalSchema.getName(), internalSchema.getComment(), null, size)), +// LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), // internalSchema); -// } - case FIXED: - Integer fixedSize = - (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); - return finalizeSchema( - new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createFixed( - internalSchema.getName(), internalSchema.getComment(), null, fixedSize)), - internalSchema); - case UUID: - /*Schema uuidSchema = - Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); - uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid");*/ - return finalizeSchema(LogicalTypeAnnotation.uuidType(), internalSchema); - default: - throw new UnsupportedSchemaTypeException( - "Encountered unhandled type during InternalSchema to parquet conversion: " - + internalSchema.getDataType()); - } - } +// }*/ +// // TODO check from here FIXED, LIST and MAP types (still to todo) +// /*case LIST: +// InternalField elementField = +// internalSchema.getFields().stream() +// .filter( +// field -> +// InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) +// .findFirst() +// .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); +// return finalizeSchema( +// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createArray( +// fromInternalSchema(elementField.getSchema(), elementField.getPath()))), +// internalSchema); +// case MAP: +// InternalField valueField = +// internalSchema.getFields().stream() +// .filter( +// field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) +// .findFirst() +// .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); +// return finalizeSchema( +// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath()))), +// internalSchema);*/ +//// case DECIMAL: +//// int precision = +//// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); +//// int scale = +//// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); +////// Integer size = +////// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); +////// if (size == null) { +//// return finalizeSchema( +//// LogicalTypeAnnotation.decimalType(scale, precision), +//// internalSchema); +//// } else { +//// return finalizeSchema( +//// LogicalTypes.decimal(precision, scale) +//// .addToSchema( +//// Schema.createFixed( +//// internalSchema.getName(), internalSchema.getComment(), null, size)), +//// internalSchema); +//// } +// /*case FIXED: +// Integer fixedSize = +// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); +// return finalizeSchema( +// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createFixed( +// internalSchema.getName(), internalSchema.getComment(), null, fixedSize)), +// internalSchema); +// case UUID: +// *//*Schema uuidSchema = +// Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); +// uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid");*//* +// return finalizeSchema(LogicalTypeAnnotation.uuidType(), internalSchema);*/ +// default: +// throw new UnsupportedSchemaTypeException( +// "Encountered unhandled type during InternalSchema to parquet conversion: " +// + internalSchema.getDataType()); +// } +// } private String buildCurrentPath(InternalField field, String parentPath) { return Optional.ofNullable(parentPath) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index 5f0334e95..d7f11719b 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -16,8 +16,9 @@ * limitations under the License. */ package org.apache.xtable.parquet; -import java.util.stream.Collectors; +import java.util.stream.Collectors; +import org.apache.xtable.model.storage.FileFormat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.parquet.column.ColumnDescriptor; @@ -28,126 +29,139 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; import org.apache.xtable.model.storage.InternalDataFile; +import org.apache.xtable.model.stat.PartitionValue; import lombok.Builder; import lombok.NonNull; import lombok.Value; + import java.util.Map; +import java.util.TreeSet; import java.util.Collection; import java.util.Set; +import java.util.LinkedHashMap; import java.nio.file.Paths; import java.util.ArrayList; import java.util.List; import java.util.stream.Stream; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.config.InputPartitionField; + @Value @Builder public class ParquetStatsExtractor { - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - - @Builder.Default - private static final ParquetPartitionExtractor partitionExtractor = - ParquetPartitionExtractor.getInstance(); - - private static Map stats = - new LinkedHashMap(); - private static long recordCount = 0; - - private Map> initPartitionInfo() { - return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); - } - - private InternalDataFile toInternalDataFile( - Configuration hadoopConf, String parentPath, Map stats) { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus file = fs.getFileStatus(new Path(parentPath)); - Map> partitionInfo = initPartitionInfo(); - - ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); - MessageType schema = parquetMetadataExtractor.getSchema(footer); - InternalSchema schema = schemaExtractor.toInternalSchema(schema); - List partitionValues = - partitionExtractor.getPartitionValue( - parentPath, file.getPath().toString(), schema, partitionInfo); - return InternalDataFile.builder() - .physicalPath(parentPath) - .fileFormat(FileFormat.APACHE_PARQUET) - .partitionValues(partitionValues) - .fileSizeBytes(file.getLen()) - .recordCount(recordCount) - .columnStats(stats.values().stream().collect(Collectors.toList())) - .lastModified(file.getModificationTime()) - .build(); - } - - private static void getColumnStatsForaFile(ParquetMetadata footer) { - for (BlockMetaData blockMetaData : footer.getBlocks()) { - - MessageType schema = parquetMetadataExtractor.getSchema(footer); - recordCount += blockMetaData.getRowCount(); - List columns = blockMetaData.getColumns(); - for (ColumnChunkMetaData columnMetaData : columns) { - ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); - ColStats.add( - desc, - columnMetaData.getValueCount(), - columnMetaData.getTotalSize(), - columnMetaData.getTotalUncompressedSize(), - columnMetaData.getEncodings(), - columnMetaData.getStatistics()); - } + + private static final ParquetStatsExtractor INSTANCE = null;//new ParquetStatsExtractor(); + public static ParquetStatsExtractor getInstance() { + return INSTANCE; } - } + @Builder.Default + private static final ParquetPartitionValueExtractor partitionExtractor = ParquetPartitionValueExtractor.getInstance(); + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + private static Map stats = + new LinkedHashMap(); + private static long recordCount = 0; + private final InputPartitionFields partitions; + + public static void getColumnStatsForaFile(ParquetMetadata footer) { + for (BlockMetaData blockMetaData : footer.getBlocks()) { - private static class Stats { - long min = Long.MAX_VALUE; - long max = Long.MIN_VALUE; - long total = 0; + MessageType schema = parquetMetadataExtractor.getSchema(footer); + recordCount += blockMetaData.getRowCount(); + List columns = blockMetaData.getColumns(); + for (ColumnChunkMetaData columnMetaData : columns) { + ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); + ColStats.add( + desc, + columnMetaData.getValueCount(), + columnMetaData.getTotalSize(), + columnMetaData.getTotalUncompressedSize(), + columnMetaData.getEncodings(), + columnMetaData.getStatistics()); + } + } + } - public void add(long length) { - min = Math.min(length, min); - max = Math.max(length, max); - total += length; + private InputPartitionFields initPartitionInfo() { + return partitions; } - } - - private static class ColStats { - - Stats valueCountStats = new Stats(); - Stats allStats = new Stats(); - Stats uncStats = new Stats(); - Set encodings = new TreeSet(); - Statistics colValuesStats = null; - int blocks = 0; - - public void add( - long valueCount, - long size, - long uncSize, - Collection encodings, - Statistics colValuesStats) { - ++blocks; - valueCountStats.add(valueCount); - allStats.add(size); - uncStats.add(uncSize); - this.encodings.addAll(encodings); - this.colValuesStats = colValuesStats; + +/* private InternalDataFile toInternalDataFile( + Configuration hadoopConf, Path parentPath, Map stats) { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus file = fs.getFileStatus(new Path(parentPath)); + InputPartitionFields partitionInfo = initPartitionInfo(); + + ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); + MessageType schema = parquetMetadataExtractor.getSchema(footer); + InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema); + List partitionValues = partitionExtractor.createPartitionValues( + partitionExtractor.extractPartitionValues( + partitionInfo)); + return InternalDataFile.builder() + .physicalPath(parentPath.toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .partitionValues(partitionValues) + .fileSizeBytes(file.getLen()) + .recordCount(recordCount) + .columnStats(stats.values().stream().collect(Collectors.toList())) + .lastModified(file.getModificationTime()) + .build(); + }*/ + + private static class Stats { + long min = Long.MAX_VALUE; + long max = Long.MIN_VALUE; + long total = 0; + + public void add(long length) { + min = Math.min(length, min); + max = Math.max(length, max); + total += length; + } } - private static void add( - ColumnDescriptor desc, - long valueCount, - long size, - long uncSize, - Collection encodings, - Statistics colValuesStats) { - ColStats colStats = stats.get(desc); - if (colStats == null) { - colStats = new ColStats(); - stats.put(desc, colStats); - } - colStats.add(valueCount, size, uncSize, encodings, colValuesStats); + private static class ColStats { + + Stats valueCountStats = new Stats(); + Stats allStats = new Stats(); + Stats uncStats = new Stats(); + Set encodings = new TreeSet(); + Statistics colValuesStats = null; + int blocks = 0; + + private static void add( + ColumnDescriptor desc, + long valueCount, + long size, + long uncSize, + Collection encodings, + Statistics colValuesStats) { + ColStats colStats = stats.get(desc); + if (colStats == null) { + colStats = new ColStats(); + stats.put(desc, colStats); + } + colStats.add(valueCount, size, uncSize, encodings, colValuesStats); + } + + public void add( + long valueCount, + long size, + long uncSize, + Collection encodings, + Statistics colValuesStats) { + ++blocks; + valueCountStats.add(valueCount); + allStats.add(size); + uncStats.add(uncSize); + this.encodings.addAll(encodings); + this.colValuesStats = colValuesStats; + } } - } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index 5f2677678..a5bfbcbaa 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -23,50 +23,59 @@ import lombok.Builder; import java.util.Set; +import java.util.Map; import org.apache.xtable.model.InternalTable; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.storage.DataLayoutStrategy; import org.apache.xtable.model.storage.TableFormat; +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.config.InputPartitionField; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.schema.MessageType; /** * Extracts {@link InternalTable} canonical representation of a table at a point in time for Delta. */ @Builder public class ParquetTableExtractor { - @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = ParquetTableExtractor.getInstance(); + private static final InputPartitionFields partitions=null; + private static final ParquetTableExtractor INSTANCE = + new ParquetTableExtractor(); + public static ParquetTableExtractor getInstance() { + return INSTANCE; + } @Builder.Default - private static final ParquetPartitionExtractor partitionExtractor = - ParquetPartitionExtractor.getInstance(); + private static final ParquetTableExtractor tableExtractor = ParquetTableExtractor.getInstance(); + private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); + @Builder.Default private static final ParquetPartitionValueExtractor partitionValueExtractor = ParquetPartitionValueExtractor.getInstance(); - @Builder.Default + /* @Builder.Default private static final ParquetConversionSource parquetConversionSource = - ParquetConversionSource.getInstance(); + ParquetConversionSource.getInstance();*/ @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - /* private Map> initPartitionInfo() { - return getPartitionFromDirectoryStructure(hadoopConf, basePath, Collections.emptyMap()); - }*/ - public String getBasePathFromLastModifiedTable(){ + private InputPartitionFields initPartitionInfo() { + return partitions; + } + /* public String getBasePathFromLastModifiedTable(){ InternalTable table = parquetConversionSource.getTable(-1L); return table.getBasePath(); - } + }*/ - public InternalTable table(String tableName, Set partitionKeys) { - ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(conf, path); - MessageType schema = parquetMetadataExtractor.getSchema(footer); + /*public InternalTable table(String tableName, Set partitionKeys,MessageType schema) { InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema); - List partitionFields = + List partitionFields = parquetConversionSource.initPartitionInfo().getPartitions(); + List convertedPartitionFields = partitionValueExtractor.getInternalPartitionFields(partitionFields); InternalTable snapshot = parquetConversionSource.getTable(-1L); // Assuming InternalTable.java has its getters Instant lastCommit = snapshot.latestCommitTime(); @@ -75,13 +84,13 @@ public InternalTable table(String tableName, Set partitionKeys) { ? DataLayoutStrategy.HIVE_STYLE_PARTITION : DataLayoutStrategy.FLAT; return InternalTable.builder() - .tableFormat(TableFormat.APACHE_PARQUET) + .tableFormat(TableFormat.PARQUET) .basePath(getBasePathFromLastModifiedTable()) .name(tableName) .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) + .partitioningFields(convertedPartitionFields) .readSchema(internalSchema) - .latestCommitTime(Instant.ofEpochMilli(lastCommit)) + .latestCommitTime(lastCommit) .build(); - } + }*/ } diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java index da22bff08..6c7de326b 100644 --- a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -1,3 +1,20 @@ +/* + * 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 org.apache.hadoop.fs.Path; diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java index e9e1d890a..82220d362 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.utilities; import java.io.ByteArrayInputStream; @@ -57,6 +57,8 @@ import org.apache.xtable.model.sync.SyncMode; import org.apache.xtable.reflection.ReflectionUtils; +import org.apache.xtable.model.config.InputPartitionFields; + /** * Provides a standalone runner for the sync process. See README.md for more details on how to run * this. @@ -64,249 +66,263 @@ @Log4j2 public class RunSync { - public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - private static final String DATASET_CONFIG_OPTION = "d"; - private static final String HADOOP_CONFIG_PATH = "p"; - private static final String CONVERTERS_CONFIG_PATH = "c"; - private static final String ICEBERG_CATALOG_CONFIG_PATH = "i"; - private static final String HELP_OPTION = "h"; - - private static final Options OPTIONS = - new Options() - .addRequiredOption( - DATASET_CONFIG_OPTION, - "datasetConfig", - true, - "The path to a yaml file containing dataset configuration") - .addOption( - HADOOP_CONFIG_PATH, - "hadoopConfig", - true, - "Hadoop config xml file path containing configs necessary to access the " - + "file system. These configs will override the default configs.") - .addOption( - CONVERTERS_CONFIG_PATH, - "convertersConfig", - true, - "The path to a yaml file containing InternalTable converter configurations. " - + "These configs will override the default") - .addOption( - ICEBERG_CATALOG_CONFIG_PATH, - "icebergCatalogConfig", - true, - "The path to a yaml file containing Iceberg catalog configuration. The configuration will be " - + "used for any Iceberg source or target.") - .addOption(HELP_OPTION, "help", false, "Displays help information to run this utility"); - - public static void main(String[] args) throws IOException { - CommandLineParser parser = new DefaultParser(); - - CommandLine cmd; - try { - cmd = parser.parse(OPTIONS, args); - } catch (ParseException e) { - new HelpFormatter().printHelp("xtable.jar", OPTIONS, true); - return; - } - - if (cmd.hasOption(HELP_OPTION)) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("RunSync", OPTIONS); - return; + public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); + private static final String DATASET_CONFIG_OPTION = "d"; + private static final String HADOOP_CONFIG_PATH = "p"; + private static final String CONVERTERS_CONFIG_PATH = "c"; + private static final String ICEBERG_CATALOG_CONFIG_PATH = "i"; + private static final String HELP_OPTION = "h"; + + private static final Options OPTIONS = + new Options() + .addRequiredOption( + DATASET_CONFIG_OPTION, + "datasetConfig", + true, + "The path to a yaml file containing dataset configuration") + .addOption( + HADOOP_CONFIG_PATH, + "hadoopConfig", + true, + "Hadoop config xml file path containing configs necessary to access the " + + "file system. These configs will override the default configs.") + .addOption( + CONVERTERS_CONFIG_PATH, + "convertersConfig", + true, + "The path to a yaml file containing InternalTable converter configurations. " + + "These configs will override the default") + .addOption( + ICEBERG_CATALOG_CONFIG_PATH, + "icebergCatalogConfig", + true, + "The path to a yaml file containing Iceberg catalog configuration. The configuration will be " + + "used for any Iceberg source or target.") + .addOption(HELP_OPTION, "help", false, "Displays help information to run this utility"); + + public static void main(String[] args) throws IOException { + CommandLineParser parser = new DefaultParser(); + + CommandLine cmd; + try { + cmd = parser.parse(OPTIONS, args); + } catch (ParseException e) { + new HelpFormatter().printHelp("xtable.jar", OPTIONS, true); + return; + } + + if (cmd.hasOption(HELP_OPTION)) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp("RunSync", OPTIONS); + return; + } + + DatasetConfig datasetConfig = new DatasetConfig(); + try (InputStream inputStream = + Files.newInputStream(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION)))) { + ObjectReader objectReader = YAML_MAPPER.readerForUpdating(datasetConfig); + objectReader.readValue(inputStream); + } + + byte[] customConfig = getCustomConfigurations(cmd, HADOOP_CONFIG_PATH); + Configuration hadoopConf = loadHadoopConf(customConfig); + byte[] icebergCatalogConfigInput = getCustomConfigurations(cmd, ICEBERG_CATALOG_CONFIG_PATH); + IcebergCatalogConfig icebergCatalogConfig = loadIcebergCatalogConfig(icebergCatalogConfigInput); + + String sourceFormat = datasetConfig.sourceFormat; + customConfig = getCustomConfigurations(cmd, CONVERTERS_CONFIG_PATH); + TableFormatConverters tableFormatConverters = loadTableFormatConversionConfigs(customConfig); + TableFormatConverters.ConversionConfig sourceConversionConfig = + tableFormatConverters.getTableFormatConverters().get(sourceFormat); + if (sourceConversionConfig == null) { + throw new IllegalArgumentException( + String.format( + "Source format %s is not supported. Known source and target formats are %s", + sourceFormat, tableFormatConverters.getTableFormatConverters().keySet())); + } + String sourceProviderClass = sourceConversionConfig.conversionSourceProviderClass; + // get the right config for parquet + if (sourceProviderClass=="Parquet"){ + InputPartitionFields partitions = getPartitionsFromUserConfiguration(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION))); + } + ConversionSourceProvider conversionSourceProvider = + ReflectionUtils.createInstanceOfClass(sourceProviderClass); + conversionSourceProvider.init(hadoopConf); + + List tableFormatList = datasetConfig.getTargetFormats(); + ConversionController conversionController = new ConversionController(hadoopConf); + for (DatasetConfig.Table table : datasetConfig.getDatasets()) { + log.info( + "Running sync for basePath {} for following table formats {}", + table.getTableBasePath(), + tableFormatList); + Properties sourceProperties = new Properties(); + if (table.getPartitionSpec() != null) { + sourceProperties.put( + HudiSourceConfig.PARTITION_FIELD_SPEC_CONFIG, table.getPartitionSpec()); + } + SourceTable sourceTable = + SourceTable.builder() + .name(table.getTableName()) + .basePath(table.getTableBasePath()) + .namespace(table.getNamespace() == null ? null : table.getNamespace().split("\\.")) + .dataPath(table.getTableDataPath()) + .catalogConfig(icebergCatalogConfig) + .additionalProperties(sourceProperties) + .formatName(sourceFormat) + .build(); + List targetTables = + tableFormatList.stream() + .map( + tableFormat -> + TargetTable.builder() + .name(table.getTableName()) + .basePath(table.getTableBasePath()) + .namespace( + table.getNamespace() == null + ? null + : table.getNamespace().split("\\.")) + .catalogConfig(icebergCatalogConfig) + .formatName(tableFormat) + .build()) + .collect(Collectors.toList()); + + ConversionConfig conversionConfig = + ConversionConfig.builder() + .sourceTable(sourceTable) + .targetTables(targetTables) + .syncMode(SyncMode.INCREMENTAL) + .partitions(partitions) + .build(); + try { + conversionController.sync(conversionConfig, conversionSourceProvider); + } catch (Exception e) { + log.error("Error running sync for {}", table.getTableBasePath(), e); + } + } } - DatasetConfig datasetConfig = new DatasetConfig(); - try (InputStream inputStream = - Files.newInputStream(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION)))) { - ObjectReader objectReader = YAML_MAPPER.readerForUpdating(datasetConfig); - objectReader.readValue(inputStream); + static byte[] getCustomConfigurations(CommandLine cmd, String option) throws IOException { + byte[] customConfig = null; + if (cmd.hasOption(option)) { + customConfig = Files.readAllBytes(Paths.get(cmd.getOptionValue(option))); + } + return customConfig; } - byte[] customConfig = getCustomConfigurations(cmd, HADOOP_CONFIG_PATH); - Configuration hadoopConf = loadHadoopConf(customConfig); - byte[] icebergCatalogConfigInput = getCustomConfigurations(cmd, ICEBERG_CATALOG_CONFIG_PATH); - IcebergCatalogConfig icebergCatalogConfig = loadIcebergCatalogConfig(icebergCatalogConfigInput); - - String sourceFormat = datasetConfig.sourceFormat; - customConfig = getCustomConfigurations(cmd, CONVERTERS_CONFIG_PATH); - TableFormatConverters tableFormatConverters = loadTableFormatConversionConfigs(customConfig); - TableFormatConverters.ConversionConfig sourceConversionConfig = - tableFormatConverters.getTableFormatConverters().get(sourceFormat); - if (sourceConversionConfig == null) { - throw new IllegalArgumentException( - String.format( - "Source format %s is not supported. Known source and target formats are %s", - sourceFormat, tableFormatConverters.getTableFormatConverters().keySet())); + @VisibleForTesting + static Configuration loadHadoopConf(byte[] customConfig) { + Configuration conf = new Configuration(); + conf.addResource("xtable-hadoop-defaults.xml"); + if (customConfig != null) { + conf.addResource(new ByteArrayInputStream(customConfig), "customConfigStream"); + } + return conf; } - String sourceProviderClass = sourceConversionConfig.conversionSourceProviderClass; - ConversionSourceProvider conversionSourceProvider = - ReflectionUtils.createInstanceOfClass(sourceProviderClass); - conversionSourceProvider.init(hadoopConf); - - List tableFormatList = datasetConfig.getTargetFormats(); - ConversionController conversionController = new ConversionController(hadoopConf); - for (DatasetConfig.Table table : datasetConfig.getDatasets()) { - log.info( - "Running sync for basePath {} for following table formats {}", - table.getTableBasePath(), - tableFormatList); - Properties sourceProperties = new Properties(); - if (table.getPartitionSpec() != null) { - sourceProperties.put( - HudiSourceConfig.PARTITION_FIELD_SPEC_CONFIG, table.getPartitionSpec()); - } - SourceTable sourceTable = - SourceTable.builder() - .name(table.getTableName()) - .basePath(table.getTableBasePath()) - .namespace(table.getNamespace() == null ? null : table.getNamespace().split("\\.")) - .dataPath(table.getTableDataPath()) - .catalogConfig(icebergCatalogConfig) - .additionalProperties(sourceProperties) - .formatName(sourceFormat) - .build(); - List targetTables = - tableFormatList.stream() - .map( - tableFormat -> - TargetTable.builder() - .name(table.getTableName()) - .basePath(table.getTableBasePath()) - .namespace( - table.getNamespace() == null - ? null - : table.getNamespace().split("\\.")) - .catalogConfig(icebergCatalogConfig) - .formatName(tableFormat) - .build()) - .collect(Collectors.toList()); - - ConversionConfig conversionConfig = - ConversionConfig.builder() - .sourceTable(sourceTable) - .targetTables(targetTables) - .syncMode(SyncMode.INCREMENTAL) - .build(); - try { - conversionController.sync(conversionConfig, conversionSourceProvider); - } catch (Exception e) { - log.error("Error running sync for {}", table.getTableBasePath(), e); - } - } - } - - static byte[] getCustomConfigurations(CommandLine cmd, String option) throws IOException { - byte[] customConfig = null; - if (cmd.hasOption(option)) { - customConfig = Files.readAllBytes(Paths.get(cmd.getOptionValue(option))); - } - return customConfig; - } - - @VisibleForTesting - static Configuration loadHadoopConf(byte[] customConfig) { - Configuration conf = new Configuration(); - conf.addResource("xtable-hadoop-defaults.xml"); - if (customConfig != null) { - conf.addResource(new ByteArrayInputStream(customConfig), "customConfigStream"); - } - return conf; - } - - /** - * Loads the conversion configs. The method first loads the default configs and then merges any - * custom configs provided by the user. - * - * @param customConfigs the custom configs provided by the user - * @return available tableFormatConverters and their configs - */ - @VisibleForTesting - static TableFormatConverters loadTableFormatConversionConfigs(byte[] customConfigs) - throws IOException { - // get resource stream from default converter config yaml file - try (InputStream inputStream = - RunSync.class.getClassLoader().getResourceAsStream("xtable-conversion-defaults.yaml")) { - TableFormatConverters converters = - YAML_MAPPER.readValue(inputStream, TableFormatConverters.class); - if (customConfigs != null) { - YAML_MAPPER.readerForUpdating(converters).readValue(customConfigs); - } - return converters; - } - } - - @VisibleForTesting - static IcebergCatalogConfig loadIcebergCatalogConfig(byte[] customConfigs) throws IOException { - return customConfigs == null - ? null - : YAML_MAPPER.readValue(customConfigs, IcebergCatalogConfig.class); - } - - @Data - public static class DatasetConfig { /** - * Table format of the source table. This is a {@link TableFormat} value. Although the format of - * the source can be auto-detected, it is recommended to specify it explicitly for cases where - * the directory contains metadata of multiple formats. + * Loads the conversion configs. The method first loads the default configs and then merges any + * custom configs provided by the user. + * + * @param customConfigs the custom configs provided by the user + * @return available tableFormatConverters and their configs */ - String sourceFormat; + @VisibleForTesting + static TableFormatConverters loadTableFormatConversionConfigs(byte[] customConfigs) + throws IOException { + // get resource stream from default converter config yaml file + try (InputStream inputStream = + RunSync.class.getClassLoader().getResourceAsStream("xtable-conversion-defaults.yaml")) { + TableFormatConverters converters = + YAML_MAPPER.readValue(inputStream, TableFormatConverters.class); + if (customConfigs != null) { + YAML_MAPPER.readerForUpdating(converters).readValue(customConfigs); + } + return converters; + } + } - /** The target formats to sync to. This is a list of {@link TableFormat} values. */ - List targetFormats; + @VisibleForTesting + static IcebergCatalogConfig loadIcebergCatalogConfig(byte[] customConfigs) throws IOException { + return customConfigs == null + ? null + : YAML_MAPPER.readValue(customConfigs, IcebergCatalogConfig.class); + } - /** Configuration of the dataset to sync, path, table name, etc. */ - List datasets; + @VisibleForTesting + public InputPartitionFields getPartitionsFromUserConfiguration(String configPath) throws IOException { + InputPartitionFields partitionConfiguration = new InputPartitionFields(); + try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { + ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); + objectReader.readValue(inputStream); + return partitionConfiguration; + } + } @Data - public static class Table { - /** - * The base path of the table to sync. Any authentication configuration needed by HDFS client - * can be provided using hadoop config file - */ - String tableBasePath; - - String tableDataPath; - - String tableName; - String partitionSpec; - String namespace; - - @Data - public class InputPartitionField { - String partitionFieldName; - String partitionValue; - PartitionTransformType transformType; - } - - @Data - class InputPartitionFields { - String sourceField; - List partitions; - } + public static class DatasetConfig { + + /** + * Table format of the source table. This is a {@link TableFormat} value. Although the format of + * the source can be auto-detected, it is recommended to specify it explicitly for cases where + * the directory contains metadata of multiple formats. + */ + String sourceFormat; + + /** + * The target formats to sync to. This is a list of {@link TableFormat} values. + */ + List targetFormats; + + /** + * Configuration of the dataset to sync, path, table name, etc. + */ + List
datasets; + + @Data + public static class Table { + /** + * The base path of the table to sync. Any authentication configuration needed by HDFS client + * can be provided using hadoop config file + */ + String tableBasePath; + + String tableDataPath; + + String tableName; + String partitionSpec; + String namespace; + + } } - } - - @Data - public static class TableFormatConverters { - /** Map of table format name to the conversion configs. */ - @JsonProperty("tableFormatConverters") - @JsonMerge - Map tableFormatConverters; @Data - public static class ConversionConfig { - /** - * The class name of the {@link ConversionSourceProvider} that will generate the {@link - * org.apache.xtable.spi.extractor.ConversionSource}. - */ - String conversionSourceProviderClass; - - /** The class name of the target converter which writes the table metadata. */ - String conversionTargetProviderClass; - - /** the configuration specific to the table format. */ - @JsonMerge Map configuration; + public static class TableFormatConverters { + /** + * Map of table format name to the conversion configs. + */ + @JsonProperty("tableFormatConverters") + @JsonMerge + Map tableFormatConverters; + + @Data + public static class ConversionConfig { + /** + * The class name of the {@link ConversionSourceProvider} that will generate the {@link + * org.apache.xtable.spi.extractor.ConversionSource}. + */ + String conversionSourceProviderClass; + + /** + * The class name of the target converter which writes the table metadata. + */ + String conversionTargetProviderClass; + + /** + * the configuration specific to the table format. + */ + @JsonMerge + Map configuration; + } } - } } From a79af62b1ea7a13307309156f4e8179d9c035510 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 18 Mar 2025 01:03:01 +0100 Subject: [PATCH 43/49] cleanups for ParquetStatsExtractor and ParquetSchemaExtractor: compiling but not tested --- pom.xml | 12 +- .../xtable/conversion/ConversionConfig.java | 6 +- .../model/config/InputPartitionField.java | 16 +- .../model/config/InputPartitionFields.java | 13 +- .../model/storage/InternalDataFile.java | 2 +- .../xtable/model/storage/TableFormat.java | 16 +- .../parquet/ParquetConversionSource.java | 459 +++++++++-------- .../ParquetConversionSourceProvider.java | 4 +- .../parquet/ParquetMetadataExtractor.java | 15 +- .../ParquetPartitionValueExtractor.java | 133 +++-- .../parquet/ParquetSchemaExtractor.java | 433 ++++++---------- .../xtable/parquet/ParquetStatsExtractor.java | 94 ++-- .../xtable/parquet/ParquetTableExtractor.java | 31 +- .../parquet/TestParquetSchemaExtractor.java | 151 +----- .../org/apache/xtable/utilities/RunSync.java | 478 +++++++++--------- 15 files changed, 811 insertions(+), 1052 deletions(-) diff --git a/pom.xml b/pom.xml index cdea79e0f..7acb6e05c 100644 --- a/pom.xml +++ b/pom.xml @@ -47,12 +47,12 @@ - xtable-api - xtable-hudi-support + xtable-core - xtable-utilities - xtable-aws - xtable-hive-metastore + @@ -828,7 +828,7 @@ - ${google.java.format.version} + 1.19.2 com.google.googlejavaformat:google-java-format diff --git a/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java b/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java index 9a5d9f47a..b6145032d 100644 --- a/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java +++ b/xtable-api/src/main/java/org/apache/xtable/conversion/ConversionConfig.java @@ -28,8 +28,9 @@ import com.google.common.base.Preconditions; -import org.apache.xtable.model.sync.SyncMode; import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.sync.SyncMode; + @Value @Builder public class ConversionConfig { @@ -50,7 +51,8 @@ public class ConversionConfig { @NonNull SourceTable sourceTable, List targetTables, Map> targetCatalogs, - SyncMode syncMode, InputPartitionFields partitions) { + SyncMode syncMode, + InputPartitionFields partitions) { this.sourceTable = sourceTable; this.targetTables = targetTables; Preconditions.checkArgument( diff --git a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java index aa9e0fc80..2d1ce83b2 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionField.java @@ -15,18 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.model.config; -import lombok.Data; + import lombok.Builder; -import lombok.Getter; +import lombok.Data; import lombok.Value; + import org.apache.xtable.model.schema.PartitionTransformType; + @Data @Value @Builder(toBuilder = true) public class InputPartitionField { - String partitionFieldName; - String partitionValue; - PartitionTransformType transformType; -} \ No newline at end of file + String partitionFieldName; + String partitionValue; + PartitionTransformType transformType; +} diff --git a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java index 1c656d68c..10eb27369 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/config/InputPartitionFields.java @@ -15,16 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.xtable.model.config; + import java.util.List; -import lombok.Data; + import lombok.Builder; -import lombok.Getter; +import lombok.Data; import lombok.Value; + @Data @Value @Builder(toBuilder = true) public class InputPartitionFields { - String sourceField; - List partitions; -} \ No newline at end of file + String sourceField; + List partitions; +} diff --git a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java index d7d0ac661..de6e98fb9 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/storage/InternalDataFile.java @@ -24,7 +24,7 @@ import lombok.Builder; import lombok.NonNull; import lombok.Value; -import org.apache.xtable.model.storage.FileFormat; + import org.apache.xtable.model.stat.ColumnStat; import org.apache.xtable.model.stat.PartitionValue; diff --git a/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java b/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java index e98ad4874..ad727210e 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.model.storage; /** @@ -24,12 +24,12 @@ * @since 0.1 */ public class TableFormat { - public static final String HUDI = "HUDI"; - public static final String ICEBERG = "ICEBERG"; - public static final String DELTA = "DELTA"; - public static final String PARQUET = "PARQUET"; + public static final String HUDI = "HUDI"; + public static final String ICEBERG = "ICEBERG"; + public static final String DELTA = "DELTA"; + public static final String PARQUET = "PARQUET"; - public static String[] values() { - return new String[]{"HUDI", "ICEBERG", "DELTA", "PARQUET"}; - } + public static String[] values() { + return new String[] {"HUDI", "ICEBERG", "DELTA", "PARQUET"}; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index fa366aef0..f3918087d 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.io.IOException; @@ -26,248 +26,247 @@ import lombok.Builder; import lombok.NonNull; - +import org.apache.xtable.model.storage.InternalDataFile; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; -import org.apache.parquet.schema.Type; -//import org.apache.parquet.SchemaBuilder; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.avro.Schema; +import org.apache.parquet.schema.MessageType; + import org.apache.xtable.model.*; +import org.apache.xtable.model.config.InputPartitionField; +import org.apache.xtable.model.config.InputPartitionFields; import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.storage.*; -import org.apache.xtable.spi.extractor.ConversionSource; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.Type.Repetition; -import org.apache.xtable.model.config.InputPartitionFields; -import org.apache.xtable.model.config.InputPartitionField; -import org.apache.parquet.schema.MessageType; -import org.apache.xtable.model.stat.PartitionValue; -import lombok.AccessLevel; -import lombok.NoArgsConstructor; + @Builder -//@NoArgsConstructor(access = AccessLevel.PRIVATE) -public class ParquetConversionSource {// implements ConversionSource { - - private final InputPartitionFields partitions; - - @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = - ParquetSchemaExtractor.getInstance(); - - /*private static final ParquetConversionSource INSTANCE = new ParquetConversionSource(); - public static ParquetConversionSource getInstance() { - return INSTANCE; - }*/ -/* private static final ParquetSchemaConverter parquetSchemaConverter = - ParquetSchemaConverter.getInstance();*/ - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - - @Builder.Default - private static final ParquetPartitionValueExtractor partitionValueExtractor = - ParquetPartitionValueExtractor.getInstance(); - - @Builder.Default - private static final ParquetStatsExtractor parquetStatsExtractor = - ParquetStatsExtractor.getInstance(); - private final String tableName; - private final String basePath; - // user config path of the parquet file (partitions) - private final String configPath; - @NonNull - private final Configuration hadoopConf; - - private InputPartitionFields initPartitionInfo() { - //return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); - return partitions; +// @NoArgsConstructor(access = AccessLevel.PRIVATE) +public class ParquetConversionSource { // implements ConversionSource { + + private final InputPartitionFields partitions; + + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + + /*private static final ParquetConversionSource INSTANCE = new ParquetConversionSource(); + public static ParquetConversionSource getInstance() { + return INSTANCE; + }*/ + /* private static final ParquetSchemaConverter parquetSchemaConverter = + ParquetSchemaConverter.getInstance();*/ + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + + @Builder.Default + private static final ParquetPartitionValueExtractor partitionValueExtractor = + ParquetPartitionValueExtractor.getInstance(); + + @Builder.Default + private static final ParquetStatsExtractor parquetStatsExtractor = + ParquetStatsExtractor.getInstance(); + + private final String tableName; + private final String basePath; + // user config path of the parquet file (partitions) + private final String configPath; + @NonNull private final Configuration hadoopConf; + + private InputPartitionFields initPartitionInfo() { + // return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + return partitions; + } + + // public Map> getPartitionFromConfiguration() { + // List partitionFields = initPartitionInfo().getPartitions(); + // Map> partitionsMap = new HashMap<>(); + // for (InputPartitionField partition : partitionFields) { + // partitionsMap + // .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) + // .addAll(partition.partitionFieldValues()); + // } + // return partitionsMap; + // } + + /** + * To infer schema getting the latest file assumption is that latest file will have new fields + * + * @param modificationTime the commit to consider for reading the table state + * @return + */ + // @Override + public InternalTable getTable(Long modificationTime) { + + Optional latestFile = + getParquetFiles(hadoopConf, basePath) + .max(Comparator.comparing(FileStatus::getModificationTime)); + + ParquetMetadata parquetMetadata = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); + // Schema tableSchema = + // new + // org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); + // Type tableSchema = + // + // parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); + MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); + + List partitionKeys = + initPartitionInfo().getPartitions().stream() + .map(InputPartitionField::getPartitionFieldName) + .collect(Collectors.toList()); + + // merge schema of partition into original as partition is not part of parquet file + if (!partitionKeys.isEmpty()) { + // TODO compilation error + // tableSchema = mergeParquetSchema(tableSchema, partitionKeys); } - -// public Map> getPartitionFromConfiguration() { -// List partitionFields = initPartitionInfo().getPartitions(); -// Map> partitionsMap = new HashMap<>(); -// for (InputPartitionField partition : partitionFields) { -// partitionsMap -// .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) -// .addAll(partition.partitionFieldValues()); -// } -// return partitionsMap; -// } - - /** - * To infer schema getting the latest file assumption is that latest file will have new fields - * - * @param modificationTime the commit to consider for reading the table state - * @return - */ - //@Override - public InternalTable getTable(Long modificationTime) { - - Optional latestFile = - getParquetFiles(hadoopConf, basePath) - .max(Comparator.comparing(FileStatus::getModificationTime)); - - ParquetMetadata parquetMetadata = - parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - //Schema tableSchema = - // new org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); -// Type tableSchema = -// parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); - - List partitionKeys = initPartitionInfo().getPartitions().stream() - .map(InputPartitionField::getPartitionFieldName) - .collect(Collectors.toList()); - - // merge schema of partition into original as partition is not part of parquet file - if (!partitionKeys.isEmpty()) { - //TODO compilation error - // tableSchema = mergeParquetSchema(tableSchema, partitionKeys); - } - InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema,null,null); - - List partitionFields = - partitionKeys.isEmpty() - ? Collections.emptyList() - : partitionValueExtractor.getInternalPartitionFields(partitions); - DataLayoutStrategy dataLayoutStrategy = - partitionFields.isEmpty() - ? DataLayoutStrategy.FLAT - : DataLayoutStrategy.HIVE_STYLE_PARTITION; - return InternalTable.builder() - .tableFormat(TableFormat.PARQUET) - .basePath(basePath) - .name(tableName) - .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) - .build(); + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema, null, null); + + List partitionFields = + partitionKeys.isEmpty() + ? Collections.emptyList() + : partitionValueExtractor.getInternalPartitionFields(partitions); + DataLayoutStrategy dataLayoutStrategy = + partitionFields.isEmpty() + ? DataLayoutStrategy.FLAT + : DataLayoutStrategy.HIVE_STYLE_PARTITION; + return InternalTable.builder() + .tableFormat(TableFormat.PARQUET) + .basePath(basePath) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) + .build(); + } + + public List getInternalDataFiles() { + List internalDataFiles = null; + /* List parquetFiles = + getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); + List partitionValuesFromConfig = partitionValueExtractor.createPartitionValues(partitionValueExtractor.extractPartitionValues(partitions)); + InternalTable table = getTable(-1L); + List internalDataFiles = + parquetFiles.stream() + .map( + file -> + InternalDataFile.builder() + .physicalPath(file.getPath().toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .fileSizeBytes(file.getLen()) + .partitionValues(partitionValuesFromConfig) + .lastModified(file.getModificationTime()) + .columnStats( + parquetStatsExtractor + .getColumnStatsForaFile( + parquetMetadataExtractor.readParquetMetadata( + hadoopConf, file.getPath())) + .build()) + .collect(Collectors.toList()));*/ + return internalDataFiles; + } + + /* + @Override + public CommitsBacklog getCommitsBacklog(){ + + } + */ + + /** + * Here to get current snapshot listing all files hence the -1 is being passed + * + * @return + */ + // @Override + public InternalSnapshot getCurrentSnapshot() { + /*List internalDataFiles = getInternalDataFiles(); + return InternalSnapshot.builder() + .table(table) + .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) + .build();*/ + return null; + } + + /* private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { + + SchemaBuilder.FieldAssembler fieldAssembler = + SchemaBuilder.record(internalSchema.getName()).fields(); + for (Schema.Field field : internalSchema.getFields()) { + fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); + } + + for (String paritionKey : parititonFields) { + fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); + } + + return fieldAssembler.endRecord(); + }*/ + + /* private Type mergeParquetSchema(MessageType internalSchema, List parititonFields) { + + List listOfAllFields = internalSchema.getFields(); + Type fieldsToMerge = listOfAllFields.get(0); + listOfAllFields.remove(0); + // start the merge + for (Type field : internalSchema.getFields()) { + fieldsToMerge = fieldsToMerge.union(field,false); } - - public List getInternalDataFiles() { - List internalDataFiles = null; - /* List parquetFiles = - getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - List partitionValuesFromConfig = partitionValueExtractor.createPartitionValues(partitionValueExtractor.extractPartitionValues(partitions)); - InternalTable table = getTable(-1L); - List internalDataFiles = - parquetFiles.stream() - .map( - file -> - InternalDataFile.builder() - .physicalPath(file.getPath().toString()) - .fileFormat(FileFormat.APACHE_PARQUET) - .fileSizeBytes(file.getLen()) - .partitionValues(partitionValuesFromConfig) - .lastModified(file.getModificationTime()) - .columnStats( - parquetStatsExtractor - .getColumnStatsForaFile( - parquetMetadataExtractor.readParquetMetadata( - hadoopConf, file.getPath())) - .build()) - .collect(Collectors.toList()));*/ - return internalDataFiles; + */ + /* for (String partition : parititonFields) { + //create Type from partiton, TODO: check further... + fieldsToMerge = fieldsToMerge.union(new Type(partition, Repetition.REQUIRED),false); + }*/ + /* + + return fieldsToMerge; + }*/ + + public Stream getParquetFiles(Configuration hadoopConf, String basePath) { + try { + FileSystem fs = FileSystem.get(hadoopConf); + RemoteIterator iterator = fs.listFiles(new Path(basePath), true); + return null; + // remoteIteratorToStream(iterator) + // .filter(file -> file.getPath().getName().endsWith("parquet")); + } catch (IOException e) { // | FileNotFoundException e + throw new RuntimeException(e); } - -/* - @Override - public CommitsBacklog getCommitsBacklog(){ - - } -*/ - - /** - * Here to get current snapshot listing all files hence the -1 is being passed - * - * @return - */ - //@Override - public InternalSnapshot getCurrentSnapshot() { - /*List internalDataFiles = getInternalDataFiles(); - return InternalSnapshot.builder() - .table(table) - .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) - .build();*/ - return null; - } - - /* private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { - - SchemaBuilder.FieldAssembler fieldAssembler = - SchemaBuilder.record(internalSchema.getName()).fields(); - for (Schema.Field field : internalSchema.getFields()) { - fieldAssembler = fieldAssembler.name(field.name()).type(field.schema()).noDefault(); - } - - for (String paritionKey : parititonFields) { - fieldAssembler = fieldAssembler.name(paritionKey).type().stringType().noDefault(); + } + + public Map> getPartitionFromDirectoryStructure( + Configuration hadoopConf, String basePath, Map> partitionMap) { + + try { + FileSystem fs = FileSystem.get(hadoopConf); + FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); + Map> currentPartitionMap = new HashMap<>(partitionMap); + + for (FileStatus dirStatus : baseFileStatus) { + if (dirStatus.isDirectory()) { + String partitionPath = dirStatus.getPath().getName(); + if (partitionPath.contains("=")) { + String[] partitionKeyValue = partitionPath.split("="); + currentPartitionMap + .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) + .add(partitionKeyValue[1]); + getPartitionFromDirectoryStructure( + hadoopConf, dirStatus.getPath().toString(), partitionMap); + } } + } + return currentPartitionMap; - return fieldAssembler.endRecord(); - }*/ - - /* private Type mergeParquetSchema(MessageType internalSchema, List parititonFields) { - - List listOfAllFields = internalSchema.getFields(); - Type fieldsToMerge = listOfAllFields.get(0); - listOfAllFields.remove(0); - // start the merge - for (Type field : internalSchema.getFields()) { - fieldsToMerge = fieldsToMerge.union(field,false); - } - *//* for (String partition : parititonFields) { - //create Type from partiton, TODO: check further... - fieldsToMerge = fieldsToMerge.union(new Type(partition, Repetition.REQUIRED),false); - }*//* - - return fieldsToMerge; - }*/ - - public Stream getParquetFiles(Configuration hadoopConf, String basePath) { - try { - FileSystem fs = FileSystem.get(hadoopConf); - RemoteIterator iterator = fs.listFiles(new Path(basePath), true); - return null; - //remoteIteratorToStream(iterator) - // .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException e) {//| FileNotFoundException e - throw new RuntimeException(e); - } + } catch (IOException e) { + throw new RuntimeException(e); } + } - public Map> getPartitionFromDirectoryStructure( - Configuration hadoopConf, String basePath, Map> partitionMap) { - - try { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); - Map> currentPartitionMap = new HashMap<>(partitionMap); - - for (FileStatus dirStatus : baseFileStatus) { - if (dirStatus.isDirectory()) { - String partitionPath = dirStatus.getPath().getName(); - if (partitionPath.contains("=")) { - String[] partitionKeyValue = partitionPath.split("="); - currentPartitionMap - .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) - .add(partitionKeyValue[1]); - getPartitionFromDirectoryStructure( - hadoopConf, dirStatus.getPath().toString(), partitionMap); - } - } - } - return currentPartitionMap; - - } catch (IOException e) { - throw new RuntimeException(e); - } - } - //@Override - public boolean isIncrementalSyncSafeFrom(Instant instant) { - return false; - } + // @Override + public boolean isIncrementalSyncSafeFrom(Instant instant) { + return false; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java index 1c41a7144..01b90b862 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java @@ -24,8 +24,8 @@ import org.apache.xtable.conversion.SourceTable; /** A concrete implementation of {@link ConversionSourceProvider} for Delta Lake table format. */ -public class ParquetConversionSourceProvider {//extends ConversionSourceProvider { - //@Override +public class ParquetConversionSourceProvider { // extends ConversionSourceProvider { + // @Override public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { return ParquetConversionSource.builder() diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java index 8e3cc82db..a0cfc4863 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetMetadataExtractor.java @@ -15,18 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; -import org.apache.hadoop.conf.Configuration; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.format.converter.ParquetMetadataConverter; -import org.apache.hadoop.fs.Path; + public class ParquetMetadataExtractor { private static final ParquetMetadataExtractor INSTANCE = new ParquetMetadataExtractor(); + public static ParquetMetadataExtractor getInstance() { return INSTANCE; } @@ -37,8 +38,8 @@ public static MessageType getSchema(ParquetMetadata footer) { } public static ParquetMetadata readParquetMetadata(Configuration conf, Path path) { - ParquetMetadata footer =null; - //ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); + ParquetMetadata footer = null; + // ParquetFileReader.readFooter(conf, path, ParquetMetadataConverter.NO_FILTER); return footer; } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index 3d99bf457..02a127c8f 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -15,97 +15,90 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.time.Instant; import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.NoArgsConstructor; -import org.apache.xtable.model.schema.PartitionTransformType; -//import org.apache.iceberg.StructLike; -import org.apache.xtable.model.schema.InternalPartitionField; -import org.apache.xtable.model.config.InputPartitionFields; import org.apache.xtable.model.config.InputPartitionField; -import org.apache.xtable.model.stat.Range; -import org.apache.xtable.schema.SchemaFieldFinder; +import org.apache.xtable.model.config.InputPartitionFields; +import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.stat.PartitionValue; -import java.util.ArrayList; -//import org.apache.xtable.utilities.RunSync.DatasetConfig.Table.InputPartitionFields; -/** - * Partition value extractor for Parquet. - */ +import org.apache.xtable.model.stat.Range; + +/** Partition value extractor for Parquet. */ @NoArgsConstructor(access = AccessLevel.PRIVATE) public class ParquetPartitionValueExtractor { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final ParquetPartitionValueExtractor INSTANCE = - new ParquetPartitionValueExtractor(); + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final ParquetPartitionValueExtractor INSTANCE = + new ParquetPartitionValueExtractor(); - public static ParquetPartitionValueExtractor getInstance() { - return INSTANCE; - } + public static ParquetPartitionValueExtractor getInstance() { + return INSTANCE; + } - public List getInternalPartitionFields( - InputPartitionFields partitions) { - List partitionFields = new ArrayList<>(); - String sourceField = partitions.getSourceField(); - for (InputPartitionField partition : partitions.getPartitions()) { - partitionFields.add( - InternalPartitionField.builder() - // TODO convert sourceField type - .sourceField(null) - .transformType(partition.getTransformType()) - .build()); - } - - return partitionFields; + public List getInternalPartitionFields(InputPartitionFields partitions) { + List partitionFields = new ArrayList<>(); + String sourceField = partitions.getSourceField(); + for (InputPartitionField partition : partitions.getPartitions()) { + partitionFields.add( + InternalPartitionField.builder() + // TODO convert sourceField type + .sourceField(null) + .transformType(partition.getTransformType()) + .build()); } + return partitionFields; + } - public List createPartitionValues(Map extractedPartitions) { - return null;/*extractedPartitions.entrySet() - .stream() - .map(internalPartitionField -> - PartitionValue.builder() - .InternalPartitionField(internalPartitionField.getKey()) - .Range(null))//internalPartitionField.getValue()) - .collect(Collectors.toList());*/ - } + public List createPartitionValues( + Map extractedPartitions) { + return null; /*extractedPartitions.entrySet() + .stream() + .map(internalPartitionField -> + PartitionValue.builder() + .InternalPartitionField(internalPartitionField.getKey()) + .Range(null))//internalPartitionField.getValue()) + .collect(Collectors.toList());*/ + } - public Map extractPartitionValues( - InputPartitionFields partitionsConf) { - Map partitionValues = new HashMap<>(); - /* List partitions = partitionsConf.getPartitions(); - for (int i = 0; i < partitions.size(); i++) { - InputPartitionField partitionField = partitions.get(i); - Object value; - // Convert date based partitions into millis since epoch - switch (partitionField.getTransformType()) { - case YEAR: - value = EPOCH.plusYears(Integer.parseInt( partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case MONTH: - value = EPOCH.plusMonths(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case DAY: - value = EPOCH.plusDays(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case HOUR: - value = EPOCH.plusHours(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - default: - value = ((Object) partitionField.getPartitionValue()); - }*/ + public Map extractPartitionValues( + InputPartitionFields partitionsConf) { + Map partitionValues = new HashMap<>(); + /* List partitions = partitionsConf.getPartitions(); + for (int i = 0; i < partitions.size(); i++) { + InputPartitionField partitionField = partitions.get(i); + Object value; + // Convert date based partitions into millis since epoch + switch (partitionField.getTransformType()) { + case YEAR: + value = EPOCH.plusYears(Integer.parseInt( partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case MONTH: + value = EPOCH.plusMonths(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case DAY: + value = EPOCH.plusDays(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case HOUR: + value = EPOCH.plusHours(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + default: + value = ((Object) partitionField.getPartitionValue()); + }*/ - // partitionValues.put(partitionField, Range.scalar(value)); + // partitionValues.put(partitionField, Range.scalar(value)); - return partitionValues; - } -} \ No newline at end of file + return partitionValues; + } +} diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java index ada976c86..d9e9251d9 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java @@ -18,48 +18,29 @@ package org.apache.xtable.parquet; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; +import java.util.List; import java.util.Optional; -import java.util.function.Function; -import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.NoArgsConstructor; -//import org.apache.parquet.schema.LogicalType; -//import org.apache.parquet.LogicalTypes; -//import org.apache.parquet.Schema; -import org.apache.parquet.schema.Type; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; +import org.apache.avro.Schema; import org.apache.parquet.schema.GroupType; -import org.apache.parquet.format.NullType; import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.Type.Repetition; -import org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit; - - +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; -import org.apache.xtable.collectors.CustomCollectors; -import org.apache.xtable.exception.SchemaExtractorException; import org.apache.xtable.exception.UnsupportedSchemaTypeException; -import org.apache.xtable.hudi.idtracking.IdTracker; import org.apache.xtable.hudi.idtracking.models.IdMapping; 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; - -import org.apache.xtable.avro.AvroSchemaConverter; -//import org.apache.parquet.avro.AvroSchemaConverter; -import org.apache.avro.Schema; //import org.apache.parquet.avro.AvroSchemaConverter; + /** * 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 @@ -80,24 +61,23 @@ public class ParquetSchemaExtractor { .defaultValue("") .build(); private static final ParquetSchemaExtractor INSTANCE = new ParquetSchemaExtractor(); - public static ParquetSchemaExtractor getInstance() { - return INSTANCE; - } 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 Type finalizeSchema(MessageType targetSchema, InternalSchema inputSchema) { +/* private static LogicalTypeAnnotation finalizeSchema(LogicalTypeAnnotation targetSchema, InternalSchema inputSchema) { if (inputSchema.isNullable()) { - return targetSchema.union(null);//LogicalTypeAnnotation.unknownType() + return targetSchema.union(null); // LogicalTypeAnnotation.unknownType() } return targetSchema; - } + }*/ private static boolean groupTypeContainsNull(GroupType schema) { - for (Type field : schema.getFields()){ + for (Type field : schema.getFields()) { if (field == null) { return true; } @@ -105,27 +85,6 @@ private static boolean groupTypeContainsNull(GroupType schema) { return false; } - /* public InternalSchema _toInternalSchema(Schema schema) { - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); - Map fieldNameToIdMapping = - IdTracker.getInstance() - .getIdTracking(schema) - .map( - idTracking -> - idTracking.getIdMappings().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity()))) - .orElse(Collections.emptyMap()); - return avroSchemaConverter.toInternalSchema(schema,null,fieldNameToIdMapping); - } - - // check which methods is best for the conversion - private InternalSchema _toInternalSchema( - MessageType schema, String parentPath, Map fieldNameToIdMapping) { - org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); - Schema avroSchema = avroParquetSchemaConverter.convert(schema); - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); - return avroSchemaConverter.toInternalSchema(avroSchema,parentPath,fieldNameToIdMapping); - }*/ /** * Converts the parquet {@link Schema} to {@link InternalSchema}. @@ -147,33 +106,37 @@ public InternalSchema toInternalSchema( if (schema.isPrimitive()) { typeName = schema.asPrimitiveType(); switch (typeName.getPrimitiveTypeName()) { - //PrimitiveTypes + // PrimitiveTypes case INT64: logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation)logicalType).getUnit(); + LogicalTypeAnnotation.TimeUnit timeUnit = + ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS) { newDataType = InternalType.TIMESTAMP; metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MICROS); + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, + InternalSchema.MetadataValue.MICROS); } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { newDataType = InternalType.TIMESTAMP_NTZ; metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.MILLIS); + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, + InternalSchema.MetadataValue.MILLIS); } else if (timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { newDataType = InternalType.TIMESTAMP_NTZ; metadata.put( - InternalSchema.MetadataKey.TIMESTAMP_PRECISION, InternalSchema.MetadataValue.NANOS); + InternalSchema.MetadataKey.TIMESTAMP_PRECISION, + InternalSchema.MetadataValue.NANOS); } } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { newDataType = InternalType.INT; - }/* else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimestampLogicalTypeAnnotation)logicalType).getUnit(); + } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MICROS || timeUnit == LogicalTypeAnnotation.TimeUnit.NANOS) { // check if INT is the InternalType needed here newDataType = InternalType.INT; } - }*/ else { + } else { newDataType = InternalType.INT; } break; @@ -181,13 +144,13 @@ public InternalSchema toInternalSchema( logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.DateLogicalTypeAnnotation) { newDataType = InternalType.DATE; - } /*else if (logicalType instanceof TimeLogicalTypeAnnotation) { - LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimeLogicalTypeAnnotation)logicalType).getUnit(); + } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { + LogicalTypeAnnotation.TimeUnit timeUnit = ((LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType).getUnit(); if (timeUnit == LogicalTypeAnnotation.TimeUnit.MILLIS) { // check if INT is the InternalType needed here newDataType = InternalType.INT; } - }*/ else { + } else { newDataType = InternalType.INT; } break; @@ -196,15 +159,17 @@ public InternalSchema toInternalSchema( break; case FLOAT: logicalType = schema.getLogicalTypeAnnotation(); - /* if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { - newDataType = InternalType.FLOAT; - } else*/ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { + /* if (logicalType instanceof LogicalTypeAnnotation.Float16LogicalTypeAnnotation) { + newDataType = InternalType.FLOAT; + } else*/ + if (logicalType + instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { metadata.put( InternalSchema.MetadataKey.DECIMAL_PRECISION, - ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation)logicalType).getPrecision()); + ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType).getPrecision()); metadata.put( InternalSchema.MetadataKey.DECIMAL_SCALE, - ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation)logicalType).getScale()); + ((LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType).getScale()); newDataType = InternalType.DECIMAL; } else { newDataType = InternalType.FLOAT; @@ -219,12 +184,13 @@ public InternalSchema toInternalSchema( newDataType = InternalType.FIXED; } break; - //TODO add other logicalTypes? + // TODO add other logicalTypes? case BINARY: - //? Variant,GEOMETRY, GEOGRAPHY, + // ? Variant,GEOMETRY, GEOGRAPHY, logicalType = schema.getLogicalTypeAnnotation(); if (logicalType instanceof LogicalTypeAnnotation.EnumLogicalTypeAnnotation) { - metadata.put(InternalSchema.MetadataKey.ENUM_VALUES, logicalType.toOriginalType().values()); + metadata.put( + InternalSchema.MetadataKey.ENUM_VALUES, logicalType.toOriginalType().values()); newDataType = InternalType.ENUM; } else if (logicalType instanceof LogicalTypeAnnotation.JsonLogicalTypeAnnotation) { newDataType = InternalType.BYTES; @@ -239,102 +205,85 @@ public InternalSchema toInternalSchema( case BOOLEAN: newDataType = InternalType.BOOLEAN; break; - /* case UNKNOWN: - newDataType = InternalType.NULL; - break;*/ + /* case UNKNOWN: + newDataType = InternalType.NULL; + break;*/ default: throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); } } /*else { - //GroupTypes - typeName = schema.asGroupType(); - switch (typeName.getOriginalType()) { - case LIST: - IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); - InternalSchema elementSchema = - toInternalSchema( - schema.getName(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), - getChildIdMap(elementMapping)); - InternalField elementField = - InternalField.builder() - .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) - .parentPath(parentPath) - .schema(elementSchema) - .fieldId(elementMapping == null ? null : elementMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.LIST) - .comment(schema.toString()) - .isNullable(groupTypeContainsNull(schema)) - .fields(Collections.singletonList(elementField)) - .build(); + //GroupTypes + typeName = schema.asGroupType(); + switch (typeName.getOriginalType()) { + case LIST: + IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + InternalSchema elementSchema = + toInternalSchema( + schema.getName(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), + getChildIdMap(elementMapping)); + InternalField elementField = + InternalField.builder() + .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) + .parentPath(parentPath) + .schema(elementSchema) + .fieldId(elementMapping == null ? null : elementMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.LIST) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) + .fields(Collections.singletonList(elementField)) + .build(); - case MAP: - IdMapping keyMapping = fieldNameToIdMapping.get(KEY); - IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); - InternalSchema valueSchema = - toInternalSchema( - schema.getName(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), - getChildIdMap(valueMapping)); - InternalField valueField = - InternalField.builder() - .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) - .parentPath(parentPath) - .schema(valueSchema) - .fieldId(valueMapping == null ? null : valueMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.MAP) - .comment(schema.toString()) - .isNullable(groupTypeContainsNull(schema)) - .fields( - Arrays.asList( - MAP_KEY_FIELD.toBuilder() - .parentPath(parentPath) - .fieldId(keyMapping == null ? null : keyMapping.getId()) - .build(), - valueField)) - .build(); - default: - throw new UnsupportedSchemaTypeException( - String.format("Unsupported schema type %s", schema)); - } - }*/ - newDataType = null; + case MAP: + IdMapping keyMapping = fieldNameToIdMapping.get(KEY); + IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); + InternalSchema valueSchema = + toInternalSchema( + schema.getName(), + SchemaUtils.getFullyQualifiedPath( + parentPath, InternalField.Constants.MAP_VALUE_FIELD_NAME), + getChildIdMap(valueMapping)); + InternalField valueField = + InternalField.builder() + .name(InternalField.Constants.MAP_VALUE_FIELD_NAME) + .parentPath(parentPath) + .schema(valueSchema) + .fieldId(valueMapping == null ? null : valueMapping.getId()) + .build(); + return InternalSchema.builder() + .name(schema.getName()) + .dataType(InternalType.MAP) + .comment(schema.toString()) + .isNullable(groupTypeContainsNull(schema)) + .fields( + Arrays.asList( + MAP_KEY_FIELD.toBuilder() + .parentPath(parentPath) + .fieldId(keyMapping == null ? null : keyMapping.getId()) + .build(), + valueField)) + .build(); + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); + } + }*/ + newDataType = null; return InternalSchema.builder() .name(schema.getName()) .dataType(newDataType) .comment(null) - .isNullable(false)// to check + .isNullable(false) // to check .metadata(metadata.isEmpty() ? null : metadata) .build(); } - /* private Map getChildIdMap(IdMapping idMapping) { - if (idMapping == null) { - return Collections.emptyMap(); - } - return idMapping.getFields().stream() - .collect(Collectors.toMap(IdMapping::getName, Function.identity())); - }*/ - - // check which methods is best for the conversion - /*private MessageType fromInternalSchema( - InternalSchema internalSchema, String currentPath) { - org.apache.parquet.avro.AvroSchemaConverter avroParquetSchemaConverter = new org.apache.parquet.avro.AvroSchemaConverter(); - AvroSchemaConverter avroSchemaConverter = AvroSchemaConverter.getInstance(); - Schema avroSchema = avroSchemaConverter.fromInternalSchema(internalSchema,currentPath); - MessageType parquetSchema = avroParquetSchemaConverter.convert(avroSchema); - return parquetSchema; - }*/ /** * Internal method for converting the {@link InternalSchema} to parquet {@link Schema}. * @@ -344,132 +293,56 @@ public InternalSchema toInternalSchema( * records. * @return an parquet schema */ -// private Type fromInternalSchema(InternalSchema internalSchema, String currentPath) { -// switch (internalSchema.getDataType()) { -// /*case BYTES: -// return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); -// case BOOLEAN: -// return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ -// case INT: -// return finalizeSchema(LogicalTypeAnnotation.intType(32,false), internalSchema); -// case LONG: -// return finalizeSchema(LogicalTypeAnnotation.intType(64,false), internalSchema); -// case STRING: -// return finalizeSchema(LogicalTypeAnnotation.stringType(), internalSchema); -// //case FLOAT: -// //return finalizeSchema(LogicalTypeAnnotation.float16Type(), internalSchema); -// /*case DOUBLE: -// int precision = -// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); -// int scale = -// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); -// return finalizeSchema(LogicalTypeAnnotation.decimalType(scale, precision), internalSchema); -// // TODO check how to create ENUM -// case ENUM: -// return finalizeSchema( -// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( -// internalSchema.getName(), -// internalSchema.getComment(), -// null, -// (List) -// internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), -// null)), -// internalSchema);*/ -// case DATE: -// return finalizeSchema( -// LogicalTypeAnnotation.dateType(), internalSchema); -// /*case TIMESTAMP: -// if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) -// == InternalSchema.MetadataValue.MICROS) { -// return finalizeSchema( -// LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), -// internalSchema); -// } if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) -// == InternalSchema.MetadataValue.MILLIS) { -// return finalizeSchema( -// LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), -// internalSchema); -// } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) -// == InternalSchema.MetadataValue.NANOS) { -// return finalizeSchema( -// LogicalTypeAnnotation.timestampType(true, TimeUnit.NANOS), -// internalSchema); -// } -// case TIMESTAMP_NTZ: -// if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) -// == InternalSchema.MetadataValue.MICROS) { -// return finalizeSchema( -// LogicalTypeAnnotation.timestampType(true, TimeUnit.MICROS), -// internalSchema); -// } else { -// return finalizeSchema( -// LogicalTypeAnnotation.timestampType(true, TimeUnit.MILLIS), -// internalSchema); -// }*/ -// // TODO check from here FIXED, LIST and MAP types (still to todo) -// /*case LIST: -// InternalField elementField = -// internalSchema.getFields().stream() -// .filter( -// field -> -// InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME.equals(field.getName())) -// .findFirst() -// .orElseThrow(() -> new SchemaExtractorException("Invalid array schema")); -// return finalizeSchema( -// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createArray( -// fromInternalSchema(elementField.getSchema(), elementField.getPath()))), -// internalSchema); -// case MAP: -// InternalField valueField = -// internalSchema.getFields().stream() -// .filter( -// field -> InternalField.Constants.MAP_VALUE_FIELD_NAME.equals(field.getName())) -// .findFirst() -// .orElseThrow(() -> new SchemaExtractorException("Invalid map schema")); -// return finalizeSchema( -// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createMap(fromInternalSchema(valueField.getSchema(), valueField.getPath()))), -// internalSchema);*/ -//// case DECIMAL: -//// int precision = -//// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); -//// int scale = -//// (int) internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); -////// Integer size = -////// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); -////// if (size == null) { -//// return finalizeSchema( -//// LogicalTypeAnnotation.decimalType(scale, precision), -//// internalSchema); -//// } else { -//// return finalizeSchema( -//// LogicalTypes.decimal(precision, scale) -//// .addToSchema( -//// Schema.createFixed( -//// internalSchema.getName(), internalSchema.getComment(), null, size)), -//// internalSchema); -//// } -// /*case FIXED: -// Integer fixedSize = -// (Integer) internalSchema.getMetadata().get(InternalSchema.MetadataKey.FIXED_BYTES_SIZE); -// return finalizeSchema( -// new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createFixed( -// internalSchema.getName(), internalSchema.getComment(), null, fixedSize)), -// internalSchema); -// case UUID: -// *//*Schema uuidSchema = -// Schema.createFixed(internalSchema.getName(), internalSchema.getComment(), null, 16); -// uuidSchema.addProp(InternalSchema.XTABLE_LOGICAL_TYPE, "uuid");*//* -// return finalizeSchema(LogicalTypeAnnotation.uuidType(), internalSchema);*/ -// default: -// throw new UnsupportedSchemaTypeException( -// "Encountered unhandled type during InternalSchema to parquet conversion: " -// + internalSchema.getDataType()); -// } -// } + private LogicalTypeAnnotation fromInternalSchema(InternalSchema internalSchema, String currentPath) { + switch (internalSchema.getDataType()) { + /*case BYTES: + return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); + case BOOLEAN: + return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ + case INT: + return LogicalTypeAnnotation.intType(32, false); + case LONG: + LogicalTypeAnnotation.intType(64, false); + case STRING: + return LogicalTypeAnnotation.stringType(); + case FLOAT: + int precision = + (int) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_PRECISION); + int scale = + (int) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); + return LogicalTypeAnnotation.decimalType(scale, precision); + case ENUM: + /* return AvroSchemaConverter().convert(Schema.createEnum( + internalSchema.getName(), + internalSchema.getComment(), + null, + (List) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), + null));*/ + case DATE: + return LogicalTypeAnnotation.dateType(); + case TIMESTAMP: + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS); + } + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION)== InternalSchema.MetadataValue.MILLIS) { + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS); + } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION)== InternalSchema.MetadataValue.NANOS) { + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.NANOS); + } + case TIMESTAMP_NTZ: + if(internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + == InternalSchema.MetadataValue.MICROS) { + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS); - private String buildCurrentPath(InternalField field, String parentPath) { - return Optional.ofNullable(parentPath) - .map(path -> path + "." + field.getName()) - .orElse(field.getName()); + } else { + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS); + + } + default: + throw new UnsupportedSchemaTypeException("Encountered unhandled type during InternalSchema to parquet conversion:" + internalSchema.getDataType()); + } } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index d7f11719b..7edb67c26 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -15,11 +15,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.xtable.parquet; +import org.apache.xtable.model.schema.InternalSchema; + +import java.util.Collection; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; -import org.apache.xtable.model.storage.FileFormat; -import org.apache.hadoop.conf.Configuration; +import java.util.TreeSet; + +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.xtable.model.stat.ColumnStat; +import lombok.Builder; +import lombok.Value; +import org.apache.xtable.model.storage.InternalDataFile; import org.apache.hadoop.fs.*; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Encoding; @@ -28,38 +41,21 @@ 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.storage.InternalDataFile; -import org.apache.xtable.model.stat.PartitionValue; -import lombok.Builder; -import lombok.NonNull; -import lombok.Value; - -import java.util.Map; -import java.util.TreeSet; -import java.util.Collection; -import java.util.Set; -import java.util.LinkedHashMap; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Stream; - -import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.storage.FileFormat; import org.apache.xtable.model.config.InputPartitionFields; -import org.apache.xtable.model.config.InputPartitionField; +import org.apache.hadoop.conf.Configuration; @Value @Builder public class ParquetStatsExtractor { - private static final ParquetStatsExtractor INSTANCE = null;//new ParquetStatsExtractor(); - public static ParquetStatsExtractor getInstance() { - return INSTANCE; - } + private static final ParquetStatsExtractor INSTANCE = null; // new ParquetStatsExtractor(); @Builder.Default - private static final ParquetPartitionValueExtractor partitionExtractor = ParquetPartitionValueExtractor.getInstance(); + private static final ParquetPartitionValueExtractor partitionExtractor = + ParquetPartitionValueExtractor.getInstance(); @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); @Builder.Default private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); @@ -68,6 +64,10 @@ public static ParquetStatsExtractor getInstance() { private static long recordCount = 0; private final InputPartitionFields partitions; + public static ParquetStatsExtractor getInstance() { + return INSTANCE; + } + public static void getColumnStatsForaFile(ParquetMetadata footer) { for (BlockMetaData blockMetaData : footer.getBlocks()) { @@ -91,18 +91,24 @@ private InputPartitionFields initPartitionInfo() { return partitions; } -/* private InternalDataFile toInternalDataFile( - Configuration hadoopConf, Path parentPath, Map stats) { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus file = fs.getFileStatus(new Path(parentPath)); - InputPartitionFields partitionInfo = initPartitionInfo(); - - ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); - MessageType schema = parquetMetadataExtractor.getSchema(footer); - InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema); - List partitionValues = partitionExtractor.createPartitionValues( - partitionExtractor.extractPartitionValues( - partitionInfo)); + private InternalDataFile toInternalDataFile( + Configuration hadoopConf, Path parentPath, Map stats) throws java.io.IOException { + FileStatus file = null; + List partitionValues =null; + try { + FileSystem fs = FileSystem.get(hadoopConf); + file = fs.getFileStatus(parentPath); + InputPartitionFields partitionInfo = initPartitionInfo(); + + ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); + MessageType schema = parquetMetadataExtractor.getSchema(footer); + InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema, null, null); + partitionValues = partitionExtractor.createPartitionValues( + partitionExtractor.extractPartitionValues( + partitionInfo)); + } catch (java.io.IOException e) { + + } return InternalDataFile.builder() .physicalPath(parentPath.toString()) .fileFormat(FileFormat.APACHE_PARQUET) @@ -112,7 +118,7 @@ private InputPartitionFields initPartitionInfo() { .columnStats(stats.values().stream().collect(Collectors.toList())) .lastModified(file.getModificationTime()) .build(); - }*/ + } private static class Stats { long min = Long.MAX_VALUE; @@ -130,7 +136,7 @@ private static class ColStats { Stats valueCountStats = new Stats(); Stats allStats = new Stats(); - Stats uncStats = new Stats(); + Stats uncompressedStats = new Stats(); Set encodings = new TreeSet(); Statistics colValuesStats = null; int blocks = 0; @@ -139,7 +145,7 @@ private static void add( ColumnDescriptor desc, long valueCount, long size, - long uncSize, + long uncompressedSize, Collection encodings, Statistics colValuesStats) { ColStats colStats = stats.get(desc); @@ -147,19 +153,19 @@ private static void add( colStats = new ColStats(); stats.put(desc, colStats); } - colStats.add(valueCount, size, uncSize, encodings, colValuesStats); + colStats.add(valueCount, size, uncompressedSize, encodings, colValuesStats); } public void add( long valueCount, long size, - long uncSize, + long uncompressedSize, Collection encodings, Statistics colValuesStats) { ++blocks; valueCountStats.add(valueCount); allStats.add(size); - uncStats.add(uncSize); + uncompressedStats.add(uncompressedSize); this.encodings.addAll(encodings); this.colValuesStats = colValuesStats; } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java index a5bfbcbaa..cc62b000a 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetTableExtractor.java @@ -18,44 +18,35 @@ package org.apache.xtable.parquet; -import java.time.Instant; -import java.util.List; - import lombok.Builder; -import java.util.Set; -import java.util.Map; +import org.apache.xtable.model.storage.InternalDataFile; import org.apache.xtable.model.InternalTable; -import org.apache.xtable.model.schema.InternalPartitionField; -import org.apache.xtable.model.schema.InternalSchema; -import org.apache.xtable.model.storage.DataLayoutStrategy; -import org.apache.xtable.model.storage.TableFormat; import org.apache.xtable.model.config.InputPartitionFields; -import org.apache.xtable.model.config.InputPartitionField; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.schema.MessageType; /** * Extracts {@link InternalTable} canonical representation of a table at a point in time for Delta. */ @Builder public class ParquetTableExtractor { - private static final InputPartitionFields partitions=null; + private static final InputPartitionFields partitions = null; + + private static final ParquetTableExtractor INSTANCE = new ParquetTableExtractor(); - private static final ParquetTableExtractor INSTANCE = - new ParquetTableExtractor(); public static ParquetTableExtractor getInstance() { return INSTANCE; } + @Builder.Default private static final ParquetTableExtractor tableExtractor = ParquetTableExtractor.getInstance(); - private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); @Builder.Default private static final ParquetPartitionValueExtractor partitionValueExtractor = - ParquetPartitionValueExtractor.getInstance(); + ParquetPartitionValueExtractor.getInstance(); - /* @Builder.Default + /* @Builder.Default private static final ParquetConversionSource parquetConversionSource = ParquetConversionSource.getInstance();*/ @@ -63,10 +54,10 @@ public static ParquetTableExtractor getInstance() { private static final ParquetMetadataExtractor parquetMetadataExtractor = ParquetMetadataExtractor.getInstance(); - private InputPartitionFields initPartitionInfo() { + private InputPartitionFields initPartitionInfo() { return partitions; } - /* public String getBasePathFromLastModifiedTable(){ + /* public String getBasePathFromLastModifiedTable(){ InternalTable table = parquetConversionSource.getTable(-1L); return table.getBasePath(); }*/ diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java index 6c7de326b..2d5acd8ee 100644 --- a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -15,134 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.xtable.parquet; - -import org.apache.hadoop.fs.Path; -import org.apache.parquet.example.data.Group; -import org.apache.parquet.example.data.simple.SimpleGroup; -import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.*; +import org.junit.jupiter.api.Test; import org.apache.parquet.schema.*; -import org.apache.parquet.schema.Type.Repetition; - import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import java.util.ArrayList; -import java.util.List; - +// Test class added ONLY to cover main() invocation not covered by application tests. public class TestParquetSchemaExtractor { - static final GroupType mapGroupType = new GroupType(Type.Repetition.REPEATED, "key_value", - new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), - new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.FLOAT, "fakevalue")); - static final GroupType groupType = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType); - static final GroupType mapGroupType2 = new GroupType(Type.Repetition.REPEATED, "key_value", - new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, "fakekey"), - groupType); - static final GroupType groupType2 = new GroupType(Type.Repetition.REQUIRED, "my_map", OriginalType.MAP, mapGroupType2); - static final MessageType messageType = new MessageType("schema", groupType2); - private static final ParquetSchemaExtractor SCHEMA_EXTRACTOR = - ParquetSchemaExtractor.getInstance(); - - public static void main(String[] args) { - //generateParquetFileFor(); - testPrimitiveTypes(); - } - - private static void generateParquetFileFor() { - try { - MessageType schema = parseSchema(); - Path path = new Path("test.parquet"); - - List recordList = generateRecords(); - - ExampleParquetWriter.Builder builder = ExampleParquetWriter.builder(path); - try (ParquetWriter writer = builder.withType(schema).build()) { - for (Group record : recordList) { - writer.write(record); - } - } - } catch (Exception ex) { - ex.printStackTrace(System.out); - } - } - - private static MessageType parseSchema() { - return messageType; - } - - private static List generateRecords() { - - List recordList = new ArrayList<>(); - - for (int i = 1; i <= 4; i++) { - Group mapGroup = new SimpleGroup(mapGroupType); - mapGroup.add("fakekey", i * i); - mapGroup.add("fakevalue", i * i * i); - Group group = new SimpleGroup(groupType); - group.add("key_value", mapGroup); - Group mapGroup2 = new SimpleGroup(mapGroupType2); - mapGroup2.add("fakekey", i); - mapGroup2.add("my_map", group); - Group group2 = new SimpleGroup(groupType2); - group2.add("key_value", mapGroup2); - Group mType = new SimpleGroup(messageType); - mType.add("my_map", group2); - recordList.add(mType); - } - - return recordList; - } - - @Test - public void testPrimitiveTypes() { - /* Map requiredEnumMetadata = - Collections.singletonMap( - InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("ONE", "TWO")); - Map optionalEnumMetadata = - Collections.singletonMap( - InternalSchema.MetadataKey.ENUM_VALUES, Arrays.asList("THREE", "FOUR"));*/ - InternalSchema primitive1 = InternalSchema.builder() - .name("integer"); - .dataType(InternalType.INT); - InternalSchema primitive2 = InternalSchema.builder() - .name("string"); - .dataType(InternalType.STRING); - - -/* InternalSchema schemaWithPrimitiveTypes = - InternalSchema.builder() - .dataType(InternalType.RECORD) - .fields( - Arrays.asList( - InternalField.builder() - .name("int") - .schema( - InternalSchema.builder() - .name("REQUIRED_int") - .dataType(InternalType.INT) - .isNullable(false) - .metadata(null) - .build()) - .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) - .build(), - InternalField.builder() - .name("float") - .schema( - InternalSchema.builder() - .name("REQUIRED_double") - .dataType(InternalType.FLOAT) - .isNullable(true) - .metadata(null) - .build()) - .defaultValue(InternalField.Constants.NULL_DEFAULT_VALUE) - .build() - )) - .build();*/ - //Type expectedSchema = mapGroupType; - MessageType integerPrimitiveType = MessageType(REQUIRED,PrimitiveType(Type.Repetition repetition, INT32, "integer") ); - Assertions.assertEquals( - primitive1, SCHEMA_EXTRACTOR.toInternalSchema(integerPrimitiveType, null, null)); - //assertTrue(TestParquetSchemaExtractor.mapGroupType.equals(SCHEMA_EXTRACTOR.toInternalSchema(schemaWithPrimitiveTypes))); - } -} \ No newline at end of file + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + + @Test + public void testPrimitiveTypes() { + // InternalSchema primitive1 = + // InternalSchema.builder().name("integer").dataType(InternalType.INT); + /* InternalSchema primitive2 = + InternalSchema.builder().name("string").dataType(InternalType.STRING);*/ + MessageType integerPrimitiveType = null; + + Assertions.assertEquals( + null, schemaExtractor.toInternalSchema(integerPrimitiveType, null, null)); + } + + @Test + public void main() { + testPrimitiveTypes(); + } +} diff --git a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java index 82220d362..043570f39 100644 --- a/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java +++ b/xtable-utilities/src/main/java/org/apache/xtable/utilities/RunSync.java @@ -15,13 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.utilities; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -53,12 +54,11 @@ import org.apache.xtable.conversion.TargetTable; import org.apache.xtable.hudi.HudiSourceConfig; import org.apache.xtable.iceberg.IcebergCatalogConfig; +import org.apache.xtable.model.config.InputPartitionFields; import org.apache.xtable.model.storage.TableFormat; import org.apache.xtable.model.sync.SyncMode; import org.apache.xtable.reflection.ReflectionUtils; -import org.apache.xtable.model.config.InputPartitionFields; - /** * Provides a standalone runner for the sync process. See README.md for more details on how to run * this. @@ -66,263 +66,253 @@ @Log4j2 public class RunSync { - public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); - private static final String DATASET_CONFIG_OPTION = "d"; - private static final String HADOOP_CONFIG_PATH = "p"; - private static final String CONVERTERS_CONFIG_PATH = "c"; - private static final String ICEBERG_CATALOG_CONFIG_PATH = "i"; - private static final String HELP_OPTION = "h"; - - private static final Options OPTIONS = - new Options() - .addRequiredOption( - DATASET_CONFIG_OPTION, - "datasetConfig", - true, - "The path to a yaml file containing dataset configuration") - .addOption( - HADOOP_CONFIG_PATH, - "hadoopConfig", - true, - "Hadoop config xml file path containing configs necessary to access the " - + "file system. These configs will override the default configs.") - .addOption( - CONVERTERS_CONFIG_PATH, - "convertersConfig", - true, - "The path to a yaml file containing InternalTable converter configurations. " - + "These configs will override the default") - .addOption( - ICEBERG_CATALOG_CONFIG_PATH, - "icebergCatalogConfig", - true, - "The path to a yaml file containing Iceberg catalog configuration. The configuration will be " - + "used for any Iceberg source or target.") - .addOption(HELP_OPTION, "help", false, "Displays help information to run this utility"); - - public static void main(String[] args) throws IOException { - CommandLineParser parser = new DefaultParser(); - - CommandLine cmd; - try { - cmd = parser.parse(OPTIONS, args); - } catch (ParseException e) { - new HelpFormatter().printHelp("xtable.jar", OPTIONS, true); - return; - } - - if (cmd.hasOption(HELP_OPTION)) { - HelpFormatter formatter = new HelpFormatter(); - formatter.printHelp("RunSync", OPTIONS); - return; - } - - DatasetConfig datasetConfig = new DatasetConfig(); - try (InputStream inputStream = - Files.newInputStream(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION)))) { - ObjectReader objectReader = YAML_MAPPER.readerForUpdating(datasetConfig); - objectReader.readValue(inputStream); - } - - byte[] customConfig = getCustomConfigurations(cmd, HADOOP_CONFIG_PATH); - Configuration hadoopConf = loadHadoopConf(customConfig); - byte[] icebergCatalogConfigInput = getCustomConfigurations(cmd, ICEBERG_CATALOG_CONFIG_PATH); - IcebergCatalogConfig icebergCatalogConfig = loadIcebergCatalogConfig(icebergCatalogConfigInput); - - String sourceFormat = datasetConfig.sourceFormat; - customConfig = getCustomConfigurations(cmd, CONVERTERS_CONFIG_PATH); - TableFormatConverters tableFormatConverters = loadTableFormatConversionConfigs(customConfig); - TableFormatConverters.ConversionConfig sourceConversionConfig = - tableFormatConverters.getTableFormatConverters().get(sourceFormat); - if (sourceConversionConfig == null) { - throw new IllegalArgumentException( - String.format( - "Source format %s is not supported. Known source and target formats are %s", - sourceFormat, tableFormatConverters.getTableFormatConverters().keySet())); - } - String sourceProviderClass = sourceConversionConfig.conversionSourceProviderClass; - // get the right config for parquet - if (sourceProviderClass=="Parquet"){ - InputPartitionFields partitions = getPartitionsFromUserConfiguration(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION))); - } - ConversionSourceProvider conversionSourceProvider = - ReflectionUtils.createInstanceOfClass(sourceProviderClass); - conversionSourceProvider.init(hadoopConf); - - List tableFormatList = datasetConfig.getTargetFormats(); - ConversionController conversionController = new ConversionController(hadoopConf); - for (DatasetConfig.Table table : datasetConfig.getDatasets()) { - log.info( - "Running sync for basePath {} for following table formats {}", - table.getTableBasePath(), - tableFormatList); - Properties sourceProperties = new Properties(); - if (table.getPartitionSpec() != null) { - sourceProperties.put( - HudiSourceConfig.PARTITION_FIELD_SPEC_CONFIG, table.getPartitionSpec()); - } - SourceTable sourceTable = - SourceTable.builder() - .name(table.getTableName()) - .basePath(table.getTableBasePath()) - .namespace(table.getNamespace() == null ? null : table.getNamespace().split("\\.")) - .dataPath(table.getTableDataPath()) - .catalogConfig(icebergCatalogConfig) - .additionalProperties(sourceProperties) - .formatName(sourceFormat) - .build(); - List targetTables = - tableFormatList.stream() - .map( - tableFormat -> - TargetTable.builder() - .name(table.getTableName()) - .basePath(table.getTableBasePath()) - .namespace( - table.getNamespace() == null - ? null - : table.getNamespace().split("\\.")) - .catalogConfig(icebergCatalogConfig) - .formatName(tableFormat) - .build()) - .collect(Collectors.toList()); - - ConversionConfig conversionConfig = - ConversionConfig.builder() - .sourceTable(sourceTable) - .targetTables(targetTables) - .syncMode(SyncMode.INCREMENTAL) - .partitions(partitions) - .build(); - try { - conversionController.sync(conversionConfig, conversionSourceProvider); - } catch (Exception e) { - log.error("Error running sync for {}", table.getTableBasePath(), e); - } - } + public static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); + private static final String DATASET_CONFIG_OPTION = "d"; + private static final String HADOOP_CONFIG_PATH = "p"; + private static final String CONVERTERS_CONFIG_PATH = "c"; + private static final String ICEBERG_CATALOG_CONFIG_PATH = "i"; + private static final String HELP_OPTION = "h"; + + private static final Options OPTIONS = + new Options() + .addRequiredOption( + DATASET_CONFIG_OPTION, + "datasetConfig", + true, + "The path to a yaml file containing dataset configuration") + .addOption( + HADOOP_CONFIG_PATH, + "hadoopConfig", + true, + "Hadoop config xml file path containing configs necessary to access the " + + "file system. These configs will override the default configs.") + .addOption( + CONVERTERS_CONFIG_PATH, + "convertersConfig", + true, + "The path to a yaml file containing InternalTable converter configurations. " + + "These configs will override the default") + .addOption( + ICEBERG_CATALOG_CONFIG_PATH, + "icebergCatalogConfig", + true, + "The path to a yaml file containing Iceberg catalog configuration. The configuration will be " + + "used for any Iceberg source or target.") + .addOption(HELP_OPTION, "help", false, "Displays help information to run this utility"); + + public static void main(String[] args) throws IOException { + CommandLineParser parser = new DefaultParser(); + + CommandLine cmd; + try { + cmd = parser.parse(OPTIONS, args); + } catch (ParseException e) { + new HelpFormatter().printHelp("xtable.jar", OPTIONS, true); + return; } - static byte[] getCustomConfigurations(CommandLine cmd, String option) throws IOException { - byte[] customConfig = null; - if (cmd.hasOption(option)) { - customConfig = Files.readAllBytes(Paths.get(cmd.getOptionValue(option))); - } - return customConfig; + if (cmd.hasOption(HELP_OPTION)) { + HelpFormatter formatter = new HelpFormatter(); + formatter.printHelp("RunSync", OPTIONS); + return; } - - @VisibleForTesting - static Configuration loadHadoopConf(byte[] customConfig) { - Configuration conf = new Configuration(); - conf.addResource("xtable-hadoop-defaults.xml"); - if (customConfig != null) { - conf.addResource(new ByteArrayInputStream(customConfig), "customConfigStream"); - } - return conf; + InputPartitionFields partitions = null; + DatasetConfig datasetConfig = new DatasetConfig(); + try (InputStream inputStream = + Files.newInputStream(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION)))) { + ObjectReader objectReader = YAML_MAPPER.readerForUpdating(datasetConfig); + objectReader.readValue(inputStream); } - /** - * Loads the conversion configs. The method first loads the default configs and then merges any - * custom configs provided by the user. - * - * @param customConfigs the custom configs provided by the user - * @return available tableFormatConverters and their configs - */ - @VisibleForTesting - static TableFormatConverters loadTableFormatConversionConfigs(byte[] customConfigs) - throws IOException { - // get resource stream from default converter config yaml file - try (InputStream inputStream = - RunSync.class.getClassLoader().getResourceAsStream("xtable-conversion-defaults.yaml")) { - TableFormatConverters converters = - YAML_MAPPER.readValue(inputStream, TableFormatConverters.class); - if (customConfigs != null) { - YAML_MAPPER.readerForUpdating(converters).readValue(customConfigs); - } - return converters; - } + byte[] customConfig = getCustomConfigurations(cmd, HADOOP_CONFIG_PATH); + Configuration hadoopConf = loadHadoopConf(customConfig); + byte[] icebergCatalogConfigInput = getCustomConfigurations(cmd, ICEBERG_CATALOG_CONFIG_PATH); + IcebergCatalogConfig icebergCatalogConfig = loadIcebergCatalogConfig(icebergCatalogConfigInput); + + String sourceFormat = datasetConfig.sourceFormat; + customConfig = getCustomConfigurations(cmd, CONVERTERS_CONFIG_PATH); + TableFormatConverters tableFormatConverters = loadTableFormatConversionConfigs(customConfig); + TableFormatConverters.ConversionConfig sourceConversionConfig = + tableFormatConverters.getTableFormatConverters().get(sourceFormat); + if (sourceConversionConfig == null) { + throw new IllegalArgumentException( + String.format( + "Source format %s is not supported. Known source and target formats are %s", + sourceFormat, tableFormatConverters.getTableFormatConverters().keySet())); } - - @VisibleForTesting - static IcebergCatalogConfig loadIcebergCatalogConfig(byte[] customConfigs) throws IOException { - return customConfigs == null - ? null - : YAML_MAPPER.readValue(customConfigs, IcebergCatalogConfig.class); + String sourceProviderClass = sourceConversionConfig.conversionSourceProviderClass; + // get the right config for parquet + if (sourceProviderClass == "Parquet") { + partitions = + getPartitionsFromUserConfiguration(Paths.get(cmd.getOptionValue(DATASET_CONFIG_OPTION))); } - - @VisibleForTesting - public InputPartitionFields getPartitionsFromUserConfiguration(String configPath) throws IOException { - InputPartitionFields partitionConfiguration = new InputPartitionFields(); - try (InputStream inputStream = Files.newInputStream(Paths.get(configPath))) { - ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); - objectReader.readValue(inputStream); - return partitionConfiguration; - } + ConversionSourceProvider conversionSourceProvider = + ReflectionUtils.createInstanceOfClass(sourceProviderClass); + conversionSourceProvider.init(hadoopConf); + + List tableFormatList = datasetConfig.getTargetFormats(); + ConversionController conversionController = new ConversionController(hadoopConf); + for (DatasetConfig.Table table : datasetConfig.getDatasets()) { + log.info( + "Running sync for basePath {} for following table formats {}", + table.getTableBasePath(), + tableFormatList); + Properties sourceProperties = new Properties(); + if (table.getPartitionSpec() != null) { + sourceProperties.put( + HudiSourceConfig.PARTITION_FIELD_SPEC_CONFIG, table.getPartitionSpec()); + } + SourceTable sourceTable = + SourceTable.builder() + .name(table.getTableName()) + .basePath(table.getTableBasePath()) + .namespace(table.getNamespace() == null ? null : table.getNamespace().split("\\.")) + .dataPath(table.getTableDataPath()) + .catalogConfig(icebergCatalogConfig) + .additionalProperties(sourceProperties) + .formatName(sourceFormat) + .build(); + List targetTables = + tableFormatList.stream() + .map( + tableFormat -> + TargetTable.builder() + .name(table.getTableName()) + .basePath(table.getTableBasePath()) + .namespace( + table.getNamespace() == null + ? null + : table.getNamespace().split("\\.")) + .catalogConfig(icebergCatalogConfig) + .formatName(tableFormat) + .build()) + .collect(Collectors.toList()); + + ConversionConfig conversionConfig = + ConversionConfig.builder() + .sourceTable(sourceTable) + .targetTables(targetTables) + .syncMode(SyncMode.INCREMENTAL) + .partitions(partitions) + .build(); + try { + conversionController.sync(conversionConfig, conversionSourceProvider); + } catch (Exception e) { + log.error("Error running sync for {}", table.getTableBasePath(), e); + } } + } - @Data - public static class DatasetConfig { - - /** - * Table format of the source table. This is a {@link TableFormat} value. Although the format of - * the source can be auto-detected, it is recommended to specify it explicitly for cases where - * the directory contains metadata of multiple formats. - */ - String sourceFormat; - - /** - * The target formats to sync to. This is a list of {@link TableFormat} values. - */ - List targetFormats; - - /** - * Configuration of the dataset to sync, path, table name, etc. - */ - List
datasets; + static byte[] getCustomConfigurations(CommandLine cmd, String option) throws IOException { + byte[] customConfig = null; + if (cmd.hasOption(option)) { + customConfig = Files.readAllBytes(Paths.get(cmd.getOptionValue(option))); + } + return customConfig; + } + + @VisibleForTesting + static Configuration loadHadoopConf(byte[] customConfig) { + Configuration conf = new Configuration(); + conf.addResource("xtable-hadoop-defaults.xml"); + if (customConfig != null) { + conf.addResource(new ByteArrayInputStream(customConfig), "customConfigStream"); + } + return conf; + } + + /** + * Loads the conversion configs. The method first loads the default configs and then merges any + * custom configs provided by the user. + * + * @param customConfigs the custom configs provided by the user + * @return available tableFormatConverters and their configs + */ + @VisibleForTesting + static TableFormatConverters loadTableFormatConversionConfigs(byte[] customConfigs) + throws IOException { + // get resource stream from default converter config yaml file + try (InputStream inputStream = + RunSync.class.getClassLoader().getResourceAsStream("xtable-conversion-defaults.yaml")) { + TableFormatConverters converters = + YAML_MAPPER.readValue(inputStream, TableFormatConverters.class); + if (customConfigs != null) { + YAML_MAPPER.readerForUpdating(converters).readValue(customConfigs); + } + return converters; + } + } + + @VisibleForTesting + static IcebergCatalogConfig loadIcebergCatalogConfig(byte[] customConfigs) throws IOException { + return customConfigs == null + ? null + : YAML_MAPPER.readValue(customConfigs, IcebergCatalogConfig.class); + } + + @VisibleForTesting + public static InputPartitionFields getPartitionsFromUserConfiguration(Path configPath) + throws IOException { + InputPartitionFields partitionConfiguration = null; + try (InputStream inputStream = Files.newInputStream(null)) { + ObjectReader objectReader = YAML_MAPPER.readerForUpdating(partitionConfiguration); + objectReader.readValue(inputStream); + return partitionConfiguration; + } + } - @Data - public static class Table { - /** - * The base path of the table to sync. Any authentication configuration needed by HDFS client - * can be provided using hadoop config file - */ - String tableBasePath; + @Data + public static class DatasetConfig { - String tableDataPath; + /** + * Table format of the source table. This is a {@link TableFormat} value. Although the format of + * the source can be auto-detected, it is recommended to specify it explicitly for cases where + * the directory contains metadata of multiple formats. + */ + String sourceFormat; - String tableName; - String partitionSpec; - String namespace; + /** The target formats to sync to. This is a list of {@link TableFormat} values. */ + List targetFormats; - } - } + /** Configuration of the dataset to sync, path, table name, etc. */ + List
datasets; @Data - public static class TableFormatConverters { - /** - * Map of table format name to the conversion configs. - */ - @JsonProperty("tableFormatConverters") - @JsonMerge - Map tableFormatConverters; - - @Data - public static class ConversionConfig { - /** - * The class name of the {@link ConversionSourceProvider} that will generate the {@link - * org.apache.xtable.spi.extractor.ConversionSource}. - */ - String conversionSourceProviderClass; + public static class Table { + /** + * The base path of the table to sync. Any authentication configuration needed by HDFS client + * can be provided using hadoop config file + */ + String tableBasePath; + + String tableDataPath; + + String tableName; + String partitionSpec; + String namespace; + } + } - /** - * The class name of the target converter which writes the table metadata. - */ - String conversionTargetProviderClass; + @Data + public static class TableFormatConverters { + /** Map of table format name to the conversion configs. */ + @JsonProperty("tableFormatConverters") + @JsonMerge + Map tableFormatConverters; - /** - * the configuration specific to the table format. - */ - @JsonMerge - Map configuration; - } + @Data + public static class ConversionConfig { + /** + * The class name of the {@link ConversionSourceProvider} that will generate the {@link + * org.apache.xtable.spi.extractor.ConversionSource}. + */ + String conversionSourceProviderClass; + + /** The class name of the target converter which writes the table metadata. */ + String conversionTargetProviderClass; + + /** the configuration specific to the table format. */ + @JsonMerge Map configuration; } + } } From a27e1720f2e63bf4afffbc0376839800598630cf Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 18 Mar 2025 02:19:11 +0100 Subject: [PATCH 44/49] cleanups for Parquet partition methods: compiling but not tested --- .../ParquetPartitionValueExtractor.java | 149 +++++++++++------- 1 file changed, 89 insertions(+), 60 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index 02a127c8f..da6c28172 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.time.Instant; @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import lombok.AccessLevel; import lombok.NoArgsConstructor; @@ -34,71 +35,99 @@ import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.stat.PartitionValue; import org.apache.xtable.model.stat.Range; +import org.apache.xtable.model.schema.InternalType; +import org.apache.xtable.model.schema.InternalField; +import org.apache.xtable.model.schema.InternalSchema; -/** Partition value extractor for Parquet. */ +/** + * Partition value extractor for Parquet. + */ @NoArgsConstructor(access = AccessLevel.PRIVATE) public class ParquetPartitionValueExtractor { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final ParquetPartitionValueExtractor INSTANCE = - new ParquetPartitionValueExtractor(); + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final ParquetPartitionValueExtractor INSTANCE = + new ParquetPartitionValueExtractor(); - public static ParquetPartitionValueExtractor getInstance() { - return INSTANCE; - } - - public List getInternalPartitionFields(InputPartitionFields partitions) { - List partitionFields = new ArrayList<>(); - String sourceField = partitions.getSourceField(); - for (InputPartitionField partition : partitions.getPartitions()) { - partitionFields.add( - InternalPartitionField.builder() - // TODO convert sourceField type - .sourceField(null) - .transformType(partition.getTransformType()) - .build()); + public static ParquetPartitionValueExtractor getInstance() { + return INSTANCE; } - return partitionFields; - } - - public List createPartitionValues( - Map extractedPartitions) { - return null; /*extractedPartitions.entrySet() - .stream() - .map(internalPartitionField -> - PartitionValue.builder() - .InternalPartitionField(internalPartitionField.getKey()) - .Range(null))//internalPartitionField.getValue()) - .collect(Collectors.toList());*/ - } - - public Map extractPartitionValues( - InputPartitionFields partitionsConf) { - Map partitionValues = new HashMap<>(); - /* List partitions = partitionsConf.getPartitions(); - for (int i = 0; i < partitions.size(); i++) { - InputPartitionField partitionField = partitions.get(i); - Object value; - // Convert date based partitions into millis since epoch - switch (partitionField.getTransformType()) { - case YEAR: - value = EPOCH.plusYears(Integer.parseInt( partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case MONTH: - value = EPOCH.plusMonths(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case DAY: - value = EPOCH.plusDays(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - case HOUR: - value = EPOCH.plusHours(Integer.parseInt(partitionField.getPartitionValue())).toInstant().toEpochMilli(); - break; - default: - value = ((Object) partitionField.getPartitionValue()); - }*/ + public List getInternalPartitionFields(InputPartitionFields partitions) { + List partitionFields = new ArrayList<>(); + String sourceField = partitions.getSourceField(); + for (InputPartitionField partition : partitions.getPartitions()) { + partitionFields.add( + InternalPartitionField.builder() + // TODO check the sourceField dataType (from the user config of the partitions) + .sourceField( + InternalField.builder() + .name(sourceField) + .schema( + InternalSchema.builder() + .name(sourceField) + .dataType(InternalType.STRING) + .build()) + .build()) + .transformType(partition.getTransformType()) + .build()); + } + return partitionFields; + } - // partitionValues.put(partitionField, Range.scalar(value)); + public List createPartitionValues( + Map extractedPartitions) { + return extractedPartitions.entrySet() + .stream() + .map(internalPartitionField -> + PartitionValue.builder() + .partitionField( + internalPartitionField.getKey()) + .range(internalPartitionField.getValue()) + .build()) + .collect(Collectors.toList()); + } - return partitionValues; - } + public Map extractPartitionValues( + InputPartitionFields partitionsConf) { + Map partitionValues = new HashMap<>(); + List partitions = partitionsConf.getPartitions(); + InternalPartitionField internalPartitionField = null; + InputPartitionField partitionField = null; + Object value = null; + for (int i = 0; i < partitions.size(); i++) { + partitionField = partitions.get(i); + // Convert date based partitions into millis since epoch + switch (partitionField.getTransformType()) { + case YEAR: + value = EPOCH.plusYears(Long.parseLong(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case MONTH: + value = EPOCH.plusMonths(Long.parseLong(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case DAY: + value = EPOCH.plusDays(Long.parseLong(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + case HOUR: + value = EPOCH.plusHours(Long.parseLong(partitionField.getPartitionValue())).toInstant().toEpochMilli(); + break; + default: + value = ((Object) partitionField.getPartitionValue()); + } + } + internalPartitionField = InternalPartitionField.builder() + .sourceField( + InternalField.builder() + .name(partitionsConf.getSourceField()) + .schema( + InternalSchema.builder() + .name(partitionsConf.getSourceField()) + // TODO check type + .dataType(InternalType.STRING) + .build()) + .build()) + .transformType(partitionField.getTransformType()) + .build(); + partitionValues.put(internalPartitionField, Range.scalar(value)); + return partitionValues; + } } From c58fe53a11169912c3b8e2d48af5f782d9c3e3e3 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Tue, 18 Mar 2025 03:29:06 +0100 Subject: [PATCH 45/49] cleanups for Parquet Conversion Source and adjust StatsExtractor for InternalDataFile: code compiles but not tested --- pom.xml | 4 +- xtable-api/pom.xml | 6 + .../apache/xtable/model/stat/ColumnStat.java | 7 +- .../parquet/ParquetConversionSource.java | 291 ++++++++---------- .../xtable/parquet/ParquetStatsExtractor.java | 21 +- 5 files changed, 151 insertions(+), 178 deletions(-) diff --git a/pom.xml b/pom.xml index 7acb6e05c..3ebf3481d 100644 --- a/pom.xml +++ b/pom.xml @@ -47,8 +47,8 @@ - + xtable-api + xtable-core + + org.apache.parquet + parquet-column + 1.13.0 + com.fasterxml.jackson.core jackson-annotations diff --git a/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java b/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java index 4e579418a..eae87e011 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java @@ -17,11 +17,13 @@ */ package org.apache.xtable.model.stat; - +import java.util.Set; import lombok.Builder; import lombok.Value; import org.apache.xtable.model.schema.InternalField; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.statistics.Statistics; /** * Captures column level statistics for a field. @@ -36,4 +38,7 @@ public class ColumnStat { long numNulls; long numValues; long totalSize; + long uncompressedSize; + Set encodings; + Statistics statistics; } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index f3918087d..fa8c6293b 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - + package org.apache.xtable.parquet; import java.io.IOException; @@ -23,6 +23,7 @@ import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.io.FileNotFoundException; import lombok.Builder; import lombok.NonNull; @@ -38,136 +39,122 @@ import org.apache.xtable.model.schema.InternalPartitionField; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.storage.*; +import org.apache.xtable.model.storage.FileFormat; +import org.apache.xtable.model.stat.PartitionValue; +import org.apache.hadoop.util.functional.RemoteIterators; @Builder // @NoArgsConstructor(access = AccessLevel.PRIVATE) public class ParquetConversionSource { // implements ConversionSource { - private final InputPartitionFields partitions; - - @Builder.Default - private static final ParquetSchemaExtractor schemaExtractor = - ParquetSchemaExtractor.getInstance(); - - /*private static final ParquetConversionSource INSTANCE = new ParquetConversionSource(); - public static ParquetConversionSource getInstance() { - return INSTANCE; - }*/ + @Builder.Default + private static final ParquetSchemaExtractor schemaExtractor = + ParquetSchemaExtractor.getInstance(); + /*private static final ParquetConversionSource INSTANCE = new ParquetConversionSource(); + public static ParquetConversionSource getInstance() { + return INSTANCE; + }*/ /* private static final ParquetSchemaConverter parquetSchemaConverter = ParquetSchemaConverter.getInstance();*/ - @Builder.Default - private static final ParquetMetadataExtractor parquetMetadataExtractor = - ParquetMetadataExtractor.getInstance(); - - @Builder.Default - private static final ParquetPartitionValueExtractor partitionValueExtractor = - ParquetPartitionValueExtractor.getInstance(); - - @Builder.Default - private static final ParquetStatsExtractor parquetStatsExtractor = - ParquetStatsExtractor.getInstance(); - - private final String tableName; - private final String basePath; - // user config path of the parquet file (partitions) - private final String configPath; - @NonNull private final Configuration hadoopConf; + @Builder.Default + private static final ParquetMetadataExtractor parquetMetadataExtractor = + ParquetMetadataExtractor.getInstance(); + @Builder.Default + private static final ParquetPartitionValueExtractor partitionValueExtractor = + ParquetPartitionValueExtractor.getInstance(); + @Builder.Default + private static final ParquetStatsExtractor parquetStatsExtractor = + ParquetStatsExtractor.getInstance(); + private final InputPartitionFields partitions; + private final String tableName; + private final String basePath; + // user config path of the parquet file (partitions) + private final String configPath; + @NonNull + private final Configuration hadoopConf; + + private InputPartitionFields initPartitionInfo() { + // return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); + return partitions; + } - private InputPartitionFields initPartitionInfo() { - // return parquetPartitionExtractor.getPartitionsFromUserConfiguration(configPath); - return partitions; - } - // public Map> getPartitionFromConfiguration() { - // List partitionFields = initPartitionInfo().getPartitions(); - // Map> partitionsMap = new HashMap<>(); - // for (InputPartitionField partition : partitionFields) { - // partitionsMap - // .computeIfAbsent(partition.getPartitionFieldName(), k -> new ArrayList<>()) - // .addAll(partition.partitionFieldValues()); - // } - // return partitionsMap; - // } + /** + * To infer schema getting the latest file assumption is that latest file will have new fields + * + * @param modificationTime the commit to consider for reading the table state + * @return + */ + // @Override + public InternalTable getTable(Long modificationTime) { - /** - * To infer schema getting the latest file assumption is that latest file will have new fields - * - * @param modificationTime the commit to consider for reading the table state - * @return - */ - // @Override - public InternalTable getTable(Long modificationTime) { + List parquetFiles = + getParquetFiles(hadoopConf, basePath); + // TODO last file in terms of modifcation time instead + LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size()-1); - Optional latestFile = - getParquetFiles(hadoopConf, basePath) - .max(Comparator.comparing(FileStatus::getModificationTime)); + //.max(Comparator.comparing(FileStatus::getModificationTime)); - ParquetMetadata parquetMetadata = - parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.get().getPath()); - // Schema tableSchema = - // new - // org.apache.parquet.avro.AvroSchemaConverter().convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - // Type tableSchema = - // - // parquetSchemaConverter.convert(parquetMetadataExtractor.getSchema(parquetMetadata)); - MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); + ParquetMetadata parquetMetadata = + parquetMetadataExtractor.readParquetMetadata(hadoopConf, latestFile.getPath()); + MessageType tableSchema = parquetMetadataExtractor.getSchema(parquetMetadata); - List partitionKeys = - initPartitionInfo().getPartitions().stream() - .map(InputPartitionField::getPartitionFieldName) - .collect(Collectors.toList()); + List partitionKeys = + initPartitionInfo().getPartitions().stream() + .map(InputPartitionField::getPartitionFieldName) + .collect(Collectors.toList()); - // merge schema of partition into original as partition is not part of parquet file - if (!partitionKeys.isEmpty()) { - // TODO compilation error - // tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + // merge schema of partition into original as partition is not part of parquet file + if (!partitionKeys.isEmpty()) { + // TODO compilation error + // tableSchema = mergeParquetSchema(tableSchema, partitionKeys); + } + InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema, null, null); + + List partitionFields = + partitionKeys.isEmpty() + ? Collections.emptyList() + : partitionValueExtractor.getInternalPartitionFields(partitions); + DataLayoutStrategy dataLayoutStrategy = + partitionFields.isEmpty() + ? DataLayoutStrategy.FLAT + : DataLayoutStrategy.HIVE_STYLE_PARTITION; + return InternalTable.builder() + .tableFormat(TableFormat.PARQUET) + .basePath(basePath) + .name(tableName) + .layoutStrategy(dataLayoutStrategy) + .partitioningFields(partitionFields) + .readSchema(schema) + .latestCommitTime(Instant.ofEpochMilli(latestFile.getModificationTime())) + .build(); } - InternalSchema schema = schemaExtractor.toInternalSchema(tableSchema, null, null); - - List partitionFields = - partitionKeys.isEmpty() - ? Collections.emptyList() - : partitionValueExtractor.getInternalPartitionFields(partitions); - DataLayoutStrategy dataLayoutStrategy = - partitionFields.isEmpty() - ? DataLayoutStrategy.FLAT - : DataLayoutStrategy.HIVE_STYLE_PARTITION; - return InternalTable.builder() - .tableFormat(TableFormat.PARQUET) - .basePath(basePath) - .name(tableName) - .layoutStrategy(dataLayoutStrategy) - .partitioningFields(partitionFields) - .readSchema(schema) - .latestCommitTime(Instant.ofEpochMilli(latestFile.get().getModificationTime())) - .build(); - } - public List getInternalDataFiles() { - List internalDataFiles = null; - /* List parquetFiles = - getParquetFiles(hadoopConf, basePath).collect(Collectors.toList()); - List partitionValuesFromConfig = partitionValueExtractor.createPartitionValues(partitionValueExtractor.extractPartitionValues(partitions)); - InternalTable table = getTable(-1L); - List internalDataFiles = - parquetFiles.stream() - .map( - file -> - InternalDataFile.builder() - .physicalPath(file.getPath().toString()) - .fileFormat(FileFormat.APACHE_PARQUET) - .fileSizeBytes(file.getLen()) - .partitionValues(partitionValuesFromConfig) - .lastModified(file.getModificationTime()) - .columnStats( - parquetStatsExtractor - .getColumnStatsForaFile( - parquetMetadataExtractor.readParquetMetadata( - hadoopConf, file.getPath())) - .build()) - .collect(Collectors.toList()));*/ - return internalDataFiles; - } + public List getInternalDataFiles() { + List internalDataFiles = null; + List parquetFiles = + getParquetFiles(hadoopConf, basePath); + List partitionValuesFromConfig = partitionValueExtractor.createPartitionValues(partitionValueExtractor.extractPartitionValues(partitions)); + InternalTable table = getTable(-1L); + internalDataFiles = + parquetFiles.stream() + .map( + file -> + InternalDataFile.builder() + .physicalPath(file.getPath().toString()) + .fileFormat(FileFormat.APACHE_PARQUET) + .fileSizeBytes(file.getLen()) + .partitionValues(partitionValuesFromConfig) + .lastModified(file.getModificationTime()) + .columnStats( + parquetStatsExtractor + .getColumnStatsForaFile( + parquetMetadataExtractor.readParquetMetadata( + hadoopConf, file.getPath()))) + .build()) + .collect(Collectors.toList()); + return internalDataFiles; + } /* @Override @@ -176,20 +163,20 @@ public CommitsBacklog getCommitsBacklog(){ } */ - /** - * Here to get current snapshot listing all files hence the -1 is being passed - * - * @return - */ - // @Override - public InternalSnapshot getCurrentSnapshot() { + /** + * Here to get current snapshot listing all files hence the -1 is being passed + * + * @return + */ + // @Override + public InternalSnapshot getCurrentSnapshot() { /*List internalDataFiles = getInternalDataFiles(); return InternalSnapshot.builder() .table(table) .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) .build();*/ - return null; - } + return null; + } /* private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { @@ -224,49 +211,21 @@ public InternalSnapshot getCurrentSnapshot() { return fieldsToMerge; }*/ - - public Stream getParquetFiles(Configuration hadoopConf, String basePath) { - try { - FileSystem fs = FileSystem.get(hadoopConf); - RemoteIterator iterator = fs.listFiles(new Path(basePath), true); - return null; - // remoteIteratorToStream(iterator) - // .filter(file -> file.getPath().getName().endsWith("parquet")); - } catch (IOException e) { // | FileNotFoundException e - throw new RuntimeException(e); - } - } - - public Map> getPartitionFromDirectoryStructure( - Configuration hadoopConf, String basePath, Map> partitionMap) { - - try { - FileSystem fs = FileSystem.get(hadoopConf); - FileStatus[] baseFileStatus = fs.listStatus(new Path(basePath)); - Map> currentPartitionMap = new HashMap<>(partitionMap); - - for (FileStatus dirStatus : baseFileStatus) { - if (dirStatus.isDirectory()) { - String partitionPath = dirStatus.getPath().getName(); - if (partitionPath.contains("=")) { - String[] partitionKeyValue = partitionPath.split("="); - currentPartitionMap - .computeIfAbsent(partitionKeyValue[0], k -> new ArrayList<>()) - .add(partitionKeyValue[1]); - getPartitionFromDirectoryStructure( - hadoopConf, dirStatus.getPath().toString(), partitionMap); - } + // was returning Stream + public List getParquetFiles(Configuration hadoopConf, String basePath) { + try { + FileSystem fs = FileSystem.get(hadoopConf); + RemoteIterator iterator = fs.listFiles(new Path(basePath), true); + return RemoteIterators.toList(iterator).stream() + .filter(file -> file.getPath().getName().endsWith("parquet")) + .collect(Collectors.toList()); + } catch (IOException e) { // + throw new RuntimeException(e); } - } - return currentPartitionMap; - - } catch (IOException e) { - throw new RuntimeException(e); } - } - // @Override - public boolean isIncrementalSyncSafeFrom(Instant instant) { - return false; - } + // @Override + public boolean isIncrementalSyncSafeFrom(Instant instant) { + return false; + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index 7edb67c26..ceb7069ca 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -23,6 +23,7 @@ import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; +import java.util.ArrayList; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -68,23 +69,25 @@ public static ParquetStatsExtractor getInstance() { return INSTANCE; } - public static void getColumnStatsForaFile(ParquetMetadata footer) { + public static List getColumnStatsForaFile(ParquetMetadata footer) { + List colStat = new ArrayList(); for (BlockMetaData blockMetaData : footer.getBlocks()) { - MessageType schema = parquetMetadataExtractor.getSchema(footer); recordCount += blockMetaData.getRowCount(); List columns = blockMetaData.getColumns(); for (ColumnChunkMetaData columnMetaData : columns) { ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); - ColStats.add( - desc, - columnMetaData.getValueCount(), - columnMetaData.getTotalSize(), - columnMetaData.getTotalUncompressedSize(), - columnMetaData.getEncodings(), - columnMetaData.getStatistics()); + colStat.add(ColumnStat.builder() + .numValues(columnMetaData.getValueCount()) + .totalSize(columnMetaData.getTotalSize()) + .uncompressedSize(columnMetaData.getTotalUncompressedSize()) + .encodings(columnMetaData.getEncodings()) + .statistics(columnMetaData.getStatistics()) + .build() + ); } } + return colStat; } private InputPartitionFields initPartitionInfo() { From 429581a5bf5a23b2074615efc40ebec40dc1352e Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 19 Mar 2025 03:43:34 +0100 Subject: [PATCH 46/49] ENUM conversion using avro-parquet: compiling but not tested --- pom.xml | 1 + xtable-core/pom.xml | 6 ++++++ .../org/apache/xtable/parquet/ParquetSchemaExtractor.java | 4 ++-- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 3ebf3481d..7359e5a67 100644 --- a/pom.xml +++ b/pom.xml @@ -319,6 +319,7 @@ 2.10.0 + org.apache.parquet diff --git a/xtable-core/pom.xml b/xtable-core/pom.xml index 24bc31df5..9bf18bee9 100644 --- a/xtable-core/pom.xml +++ b/xtable-core/pom.xml @@ -57,6 +57,12 @@ + + + org.apache.parquet + parquet-avro + 1.15.0 + org.apache.avro avro diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java index d9e9251d9..24e45bfcf 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java @@ -314,13 +314,13 @@ private LogicalTypeAnnotation fromInternalSchema(InternalSchema internalSchema, internalSchema.getMetadata().get(InternalSchema.MetadataKey.DECIMAL_SCALE); return LogicalTypeAnnotation.decimalType(scale, precision); case ENUM: - /* return AvroSchemaConverter().convert(Schema.createEnum( + return new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( internalSchema.getName(), internalSchema.getComment(), null, (List) internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null));*/ + null)).getLogicalTypeAnnotation(); case DATE: return LogicalTypeAnnotation.dateType(); case TIMESTAMP: From 0f3c60b80cf57812bbc96500d30a641476a3f5c6 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Wed, 19 Mar 2025 21:02:20 +0100 Subject: [PATCH 47/49] LIST and RECORD for parquet data conversion --- .../ParquetPartitionValueExtractor.java | 8 ++ .../parquet/ParquetSchemaExtractor.java | 121 +++++++++++++----- .../xtable/parquet/ParquetStatsExtractor.java | 1 + 3 files changed, 96 insertions(+), 34 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java index da6c28172..c1b53b668 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Collections; import java.util.Map; import java.util.stream.Collectors; @@ -52,6 +53,13 @@ public static ParquetPartitionValueExtractor getInstance() { return INSTANCE; } + + public List convertFromParquertUserDefinedPartitionConfig(InputPartitionFields userDefinedPartitionSchema) { + if (userDefinedPartitionSchema.getPartitions().isEmpty()) { + return Collections.emptyList(); + } + return getInternalPartitionFields(userDefinedPartitionSchema); + } public List getInternalPartitionFields(InputPartitionFields partitions) { List partitionFields = new ArrayList<>(); String sourceField = partitions.getSourceField(); diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java index 24e45bfcf..a825a53b4 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java @@ -21,21 +21,32 @@ import java.util.HashMap; import java.util.Map; import java.util.List; +import java.util.ArrayList; + +import org.apache.xtable.schema.SchemaUtils; + +import java.util.Collections; import java.util.Optional; import lombok.AccessLevel; import lombok.NoArgsConstructor; +import java.util.Collections; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.xtable.hudi.idtracking.models.IdMapping; import org.apache.avro.Schema; import org.apache.parquet.schema.GroupType; 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.xtable.collectors.CustomCollectors; import org.apache.xtable.exception.UnsupportedSchemaTypeException; import org.apache.xtable.hudi.idtracking.models.IdMapping; import org.apache.xtable.model.schema.InternalField; +import org.apache.parquet.schema.Type.Repetition; import org.apache.xtable.model.schema.InternalSchema; import org.apache.xtable.model.schema.InternalType; //import org.apache.parquet.avro.AvroSchemaConverter; @@ -69,22 +80,28 @@ public static ParquetSchemaExtractor getInstance() { return INSTANCE; } -/* private static LogicalTypeAnnotation finalizeSchema(LogicalTypeAnnotation targetSchema, InternalSchema inputSchema) { - if (inputSchema.isNullable()) { - return targetSchema.union(null); // LogicalTypeAnnotation.unknownType() - } - return targetSchema; - }*/ - private static boolean groupTypeContainsNull(GroupType schema) { for (Type field : schema.getFields()) { - if (field == null) { + if (field.getLogicalTypeAnnotation().toOriginalType() == null) { return true; } } return false; } + /* private static LogicalTypeAnnotation finalizeSchema(LogicalTypeAnnotation targetSchema, InternalSchema inputSchema) { + if (inputSchema.isNullable()) { + return targetSchema.union(null); // LogicalTypeAnnotation.unknownType() + } + return targetSchema; + }*/ + private Map getChildIdMap(IdMapping idMapping) { + if (idMapping == null) { + return Collections.emptyMap(); + } + return idMapping.getFields().stream() + .collect(Collectors.toMap(IdMapping::getName, Function.identity())); + } /** * Converts the parquet {@link Schema} to {@link InternalSchema}. @@ -212,12 +229,37 @@ public InternalSchema toInternalSchema( throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); } - } /*else { - //GroupTypes - typeName = schema.asGroupType(); - switch (typeName.getOriginalType()) { - case LIST: - IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); + } else { + //GroupTypes + //typeName = schema.asGroupType(); + switch (schema.getOriginalType()) { + case LIST://or LIST? + List subFields = new ArrayList<>(schema.getFields().size()); + for (Type parquetField : schema.getFields()) { + IdMapping idMapping = fieldNameToIdMapping.get(parquetField.getName()); + InternalSchema subFieldSchema = + toInternalSchema( + new MessageType(parquetField.getName(), parquetField), + SchemaUtils.getFullyQualifiedPath(parentPath, parquetField.getName()), + getChildIdMap(idMapping)); + //Object defaultValue = getDefaultValue(parquetField); + subFields.add( + InternalField.builder() + .parentPath(parentPath) + .name(parquetField.getName()) + .schema(subFieldSchema) + //.defaultValue(defaultValue) + .fieldId(idMapping == null ? null : idMapping.getId()) + .build()); + } + return InternalSchema.builder() + .name(schema.getName()) + //.comment(schema.getDoc()) + .dataType(InternalType.RECORD) + .fields(subFields) + .isNullable(groupTypeContainsNull(schema)) + .build(); + /* IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); InternalSchema elementSchema = toInternalSchema( schema.getName(), @@ -238,8 +280,8 @@ public InternalSchema toInternalSchema( .isNullable(groupTypeContainsNull(schema)) .fields(Collections.singletonList(elementField)) .build(); - - case MAP: +*/ + /*case MAP: IdMapping keyMapping = fieldNameToIdMapping.get(KEY); IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); InternalSchema valueSchema = @@ -267,18 +309,18 @@ public InternalSchema toInternalSchema( .fieldId(keyMapping == null ? null : keyMapping.getId()) .build(), valueField)) - .build(); - default: - throw new UnsupportedSchemaTypeException( - String.format("Unsupported schema type %s", schema)); - } - }*/ + .build();*/ + default: + throw new UnsupportedSchemaTypeException( + String.format("Unsupported schema type %s", schema)); + } + } newDataType = null; return InternalSchema.builder() .name(schema.getName()) .dataType(newDataType) .comment(null) - .isNullable(false) // to check + .isNullable(groupTypeContainsNull(schema)) // to check .metadata(metadata.isEmpty() ? null : metadata) .build(); } @@ -315,32 +357,43 @@ private LogicalTypeAnnotation fromInternalSchema(InternalSchema internalSchema, return LogicalTypeAnnotation.decimalType(scale, precision); case ENUM: return new org.apache.parquet.avro.AvroSchemaConverter().convert(Schema.createEnum( - internalSchema.getName(), - internalSchema.getComment(), - null, - (List) - internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), - null)).getLogicalTypeAnnotation(); + internalSchema.getName(), + internalSchema.getComment(), + null, + (List) + internalSchema.getMetadata().get(InternalSchema.MetadataKey.ENUM_VALUES), + null)).getLogicalTypeAnnotation(); case DATE: return LogicalTypeAnnotation.dateType(); case TIMESTAMP: if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS); } - if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION)== InternalSchema.MetadataValue.MILLIS) { + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MILLIS) { return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS); - } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION)== InternalSchema.MetadataValue.NANOS) { + } else if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.NANOS) { return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.NANOS); } case TIMESTAMP_NTZ: - if(internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) + if (internalSchema.getMetadata().get(InternalSchema.MetadataKey.TIMESTAMP_PRECISION) == InternalSchema.MetadataValue.MICROS) { return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MICROS); } else { - return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS); + return LogicalTypeAnnotation.timestampType(true, LogicalTypeAnnotation.TimeUnit.MILLIS); } + case RECORD: + List fields = + internalSchema.getFields().stream() + .map( + field -> + LogicalTypeAnnotation.fromOriginalType( + fromInternalSchema( + field.getSchema(), + SchemaUtils.getFullyQualifiedPath(field.getName(), currentPath)).toOriginalType(),null + )) + .collect(CustomCollectors.toList(internalSchema.getFields().size())); default: throw new UnsupportedSchemaTypeException("Encountered unhandled type during InternalSchema to parquet conversion:" + internalSchema.getDataType()); } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index ceb7069ca..a04e18887 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -105,6 +105,7 @@ private InternalDataFile toInternalDataFile( ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); MessageType schema = parquetMetadataExtractor.getSchema(footer); + InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema, null, null); partitionValues = partitionExtractor.createPartitionValues( partitionExtractor.extractPartitionValues( From 60dced9fc12b21f1b3f08d38a76e0d3f39744318 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Thu, 20 Mar 2025 23:47:43 +0100 Subject: [PATCH 48/49] Boolean for parquet data conversion, code compiling, not tested --- .../parquet/ParquetConversionSource.java | 42 ++++++++++++------- .../ParquetConversionSourceProvider.java | 4 +- .../parquet/ParquetSchemaExtractor.java | 41 +++++------------- 3 files changed, 40 insertions(+), 47 deletions(-) diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java index fa8c6293b..bcb352ad6 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java @@ -24,15 +24,16 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import java.io.FileNotFoundException; - +import org.apache.xtable.model.CommitsBacklog; import lombok.Builder; import lombok.NonNull; +import org.apache.xtable.model.InstantsForIncrementalSync; import org.apache.xtable.model.storage.InternalDataFile; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.*; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import org.apache.parquet.schema.MessageType; - +import org.apache.xtable.model.TableChange; import org.apache.xtable.model.*; import org.apache.xtable.model.config.InputPartitionField; import org.apache.xtable.model.config.InputPartitionFields; @@ -42,11 +43,12 @@ import org.apache.xtable.model.storage.FileFormat; import org.apache.xtable.model.stat.PartitionValue; import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.xtable.spi.extractor.ConversionSource; +import org.apache.parquet.format.FileMetaData; @Builder // @NoArgsConstructor(access = AccessLevel.PRIVATE) -public class ParquetConversionSource { // implements ConversionSource { - +public class ParquetConversionSource implements ConversionSource { @Builder.Default private static final ParquetSchemaExtractor schemaExtractor = ParquetSchemaExtractor.getInstance(); @@ -85,7 +87,7 @@ private InputPartitionFields initPartitionInfo() { * @param modificationTime the commit to consider for reading the table state * @return */ - // @Override + @Override public InternalTable getTable(Long modificationTime) { List parquetFiles = @@ -156,26 +158,34 @@ public List getInternalDataFiles() { return internalDataFiles; } - /* + // since we are considering files instead of tables in parquet @Override - public CommitsBacklog getCommitsBacklog(){ - + public CommitsBacklog getCommitsBacklog(InstantsForIncrementalSync lastSyncInstant){ + long epochMilli = lastSyncInstant.getLastSyncInstant().toEpochMilli(); + return null; } - */ + @Override + public TableChange getTableChangeForCommit(java.lang.Long commit){ + return null; + } + @Override + public InternalTable getCurrentTable(){ + return null; + }; /** * Here to get current snapshot listing all files hence the -1 is being passed * * @return */ - // @Override + @Override public InternalSnapshot getCurrentSnapshot() { - /*List internalDataFiles = getInternalDataFiles(); + List internalDataFiles = getInternalDataFiles(); + InternalTable table = getTable(-1L); return InternalSnapshot.builder() .table(table) .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles)) - .build();*/ - return null; + .build(); } /* private Schema mergeAvroSchema(Schema internalSchema, Set parititonFields) { @@ -224,8 +234,12 @@ public List getParquetFiles(Configuration hadoopConf, String } } - // @Override + @Override public boolean isIncrementalSyncSafeFrom(Instant instant) { return false; } + @Override + public void close() { + + } } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java index 01b90b862..ba3bb3a07 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java @@ -24,8 +24,8 @@ import org.apache.xtable.conversion.SourceTable; /** A concrete implementation of {@link ConversionSourceProvider} for Delta Lake table format. */ -public class ParquetConversionSourceProvider { // extends ConversionSourceProvider { - // @Override +public class ParquetConversionSourceProvider extends ConversionSourceProvider { + @Override public ParquetConversionSource getConversionSourceInstance(SourceTable sourceTable) { return ParquetConversionSource.builder() diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java index a825a53b4..031b98df4 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSchemaExtractor.java @@ -222,9 +222,9 @@ public InternalSchema toInternalSchema( case BOOLEAN: newDataType = InternalType.BOOLEAN; break; - /* case UNKNOWN: - newDataType = InternalType.NULL; - break;*/ + /* case UNKNOWN: + newDataType = InternalType.NULL; + break;*/ default: throw new UnsupportedSchemaTypeException( String.format("Unsupported schema type %s", schema)); @@ -233,7 +233,7 @@ public InternalSchema toInternalSchema( //GroupTypes //typeName = schema.asGroupType(); switch (schema.getOriginalType()) { - case LIST://or LIST? + case LIST: List subFields = new ArrayList<>(schema.getFields().size()); for (Type parquetField : schema.getFields()) { IdMapping idMapping = fieldNameToIdMapping.get(parquetField.getName()); @@ -259,28 +259,6 @@ public InternalSchema toInternalSchema( .fields(subFields) .isNullable(groupTypeContainsNull(schema)) .build(); - /* IdMapping elementMapping = fieldNameToIdMapping.get(ELEMENT); - InternalSchema elementSchema = - toInternalSchema( - schema.getName(), - SchemaUtils.getFullyQualifiedPath( - parentPath, InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME), - getChildIdMap(elementMapping)); - InternalField elementField = - InternalField.builder() - .name(InternalField.Constants.ARRAY_ELEMENT_FIELD_NAME) - .parentPath(parentPath) - .schema(elementSchema) - .fieldId(elementMapping == null ? null : elementMapping.getId()) - .build(); - return InternalSchema.builder() - .name(schema.getName()) - .dataType(InternalType.LIST) - .comment(schema.toString()) - .isNullable(groupTypeContainsNull(schema)) - .fields(Collections.singletonList(elementField)) - .build(); -*/ /*case MAP: IdMapping keyMapping = fieldNameToIdMapping.get(KEY); IdMapping valueMapping = fieldNameToIdMapping.get(VALUE); @@ -338,9 +316,9 @@ public InternalSchema toInternalSchema( private LogicalTypeAnnotation fromInternalSchema(InternalSchema internalSchema, String currentPath) { switch (internalSchema.getDataType()) { /*case BYTES: - return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema); - case BOOLEAN: - return finalizeSchema(Schema.create(Schema.Type.BOOLEAN), internalSchema);*/ + return finalizeSchema(Schema.create(Schema.Type.BYTES), internalSchema);*/ + case BOOLEAN: + return LogicalTypeAnnotation.intType(8, false); case INT: return LogicalTypeAnnotation.intType(32, false); case LONG: @@ -386,13 +364,14 @@ private LogicalTypeAnnotation fromInternalSchema(InternalSchema internalSchema, case RECORD: List fields = internalSchema.getFields().stream() + // TODO check if field is decimal then its metadata should be not null (below) .map( field -> LogicalTypeAnnotation.fromOriginalType( fromInternalSchema( field.getSchema(), - SchemaUtils.getFullyQualifiedPath(field.getName(), currentPath)).toOriginalType(),null - )) + SchemaUtils.getFullyQualifiedPath(field.getName(), currentPath)).toOriginalType(), null + )) .collect(CustomCollectors.toList(internalSchema.getFields().size())); default: throw new UnsupportedSchemaTypeException("Encountered unhandled type during InternalSchema to parquet conversion:" + internalSchema.getDataType()); From 1b698bd055f05326e7e017e018657fe78a771b19 Mon Sep 17 00:00:00 2001 From: Selim Soufargi Date: Sat, 22 Mar 2025 01:22:53 +0100 Subject: [PATCH 49/49] StatsExtractor adjusted to remove encodings and add min max vals --- .../apache/xtable/model/stat/ColumnStat.java | 5 --- .../xtable/parquet/ParquetStatsExtractor.java | 39 +++++++++++-------- .../parquet/TestParquetSchemaExtractor.java | 2 + 3 files changed, 25 insertions(+), 21 deletions(-) diff --git a/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java b/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java index eae87e011..943d3b4f5 100644 --- a/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java +++ b/xtable-api/src/main/java/org/apache/xtable/model/stat/ColumnStat.java @@ -22,8 +22,6 @@ import lombok.Value; import org.apache.xtable.model.schema.InternalField; -import org.apache.parquet.column.Encoding; -import org.apache.parquet.column.statistics.Statistics; /** * Captures column level statistics for a field. @@ -38,7 +36,4 @@ public class ColumnStat { long numNulls; long numValues; long totalSize; - long uncompressedSize; - Set encodings; - Statistics statistics; } diff --git a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java index a04e18887..d82805276 100644 --- a/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java +++ b/xtable-core/src/main/java/org/apache/xtable/parquet/ParquetStatsExtractor.java @@ -25,12 +25,14 @@ import java.util.List; import java.util.ArrayList; import java.util.Map; +import java.util.HashMap; import java.util.Set; import java.util.stream.Collectors; import java.util.TreeSet; import org.apache.xtable.model.stat.PartitionValue; import org.apache.xtable.model.stat.ColumnStat; +import org.apache.xtable.model.stat.Range; import lombok.Builder; import lombok.Value; import org.apache.xtable.model.storage.InternalDataFile; @@ -70,24 +72,29 @@ public static ParquetStatsExtractor getInstance() { } public static List getColumnStatsForaFile(ParquetMetadata footer) { + return getStatsForaFile(footer).values().stream() + .flatMap(List::stream) + .collect(Collectors.toList()); + } + + + public static Map> getStatsForaFile(ParquetMetadata footer) { List colStat = new ArrayList(); + Map> columnDescStats = new HashMap<>(); for (BlockMetaData blockMetaData : footer.getBlocks()) { MessageType schema = parquetMetadataExtractor.getSchema(footer); - recordCount += blockMetaData.getRowCount(); List columns = blockMetaData.getColumns(); - for (ColumnChunkMetaData columnMetaData : columns) { - ColumnDescriptor desc = schema.getColumnDescription(columnMetaData.getPath().toArray()); - colStat.add(ColumnStat.builder() - .numValues(columnMetaData.getValueCount()) - .totalSize(columnMetaData.getTotalSize()) - .uncompressedSize(columnMetaData.getTotalUncompressedSize()) - .encodings(columnMetaData.getEncodings()) - .statistics(columnMetaData.getStatistics()) - .build() - ); - } + columnDescStats = + columns + .stream() + .collect(Collectors.groupingBy(columnMetaData -> schema.getColumnDescription(columnMetaData.getPath().toArray()), + Collectors.mapping(columnMetaData ->ColumnStat.builder() + .numValues(columnMetaData.getValueCount()) + .totalSize(columnMetaData.getTotalSize()) + .range(Range.vector(columnMetaData.getStatistics().genericGetMin(), columnMetaData.getStatistics().genericGetMax())) + .build(), Collectors.toList()))); } - return colStat; + return columnDescStats; } private InputPartitionFields initPartitionInfo() { @@ -97,17 +104,17 @@ private InputPartitionFields initPartitionInfo() { private InternalDataFile toInternalDataFile( Configuration hadoopConf, Path parentPath, Map stats) throws java.io.IOException { FileStatus file = null; - List partitionValues =null; + List partitionValues = null; try { FileSystem fs = FileSystem.get(hadoopConf); - file = fs.getFileStatus(parentPath); + file = fs.getFileStatus(parentPath); InputPartitionFields partitionInfo = initPartitionInfo(); ParquetMetadata footer = parquetMetadataExtractor.readParquetMetadata(hadoopConf, parentPath); MessageType schema = parquetMetadataExtractor.getSchema(footer); InternalSchema internalSchema = schemaExtractor.toInternalSchema(schema, null, null); - partitionValues = partitionExtractor.createPartitionValues( + partitionValues = partitionExtractor.createPartitionValues( partitionExtractor.extractPartitionValues( partitionInfo)); } catch (java.io.IOException e) { diff --git a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java index 2d5acd8ee..d8ed08072 100644 --- a/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java +++ b/xtable-core/src/test/java/org/apache/xtable/parquet/TestParquetSchemaExtractor.java @@ -22,6 +22,8 @@ import org.junit.jupiter.api.Test; import org.apache.parquet.schema.*; import org.junit.jupiter.api.Assertions; +import org.apache.xtable.model.schema.InternalSchema; +import org.apache.xtable.model.schema.InternalType; // Test class added ONLY to cover main() invocation not covered by application tests. public class TestParquetSchemaExtractor {