the-other-tim-brown commented on code in PR #728:
URL: https://github.com/apache/incubator-xtable/pull/728#discussion_r2160624331


##########
xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java:
##########
@@ -27,8 +27,9 @@ 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 String[] values() {
-    return new String[] {"HUDI", "ICEBERG", "DELTA"};
+    return new String[] {"HUDI", "ICEBERG", "DELTA","PARQUET"};

Review Comment:
   nitpick: add a space after the comma 



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSourceConfig.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Properties;
+
+import lombok.Value;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.xtable.model.schema.PartitionTransformType;
+import org.apache.xtable.reflection.ReflectionUtils;
+
+/** Configuration of Parquet source format for the sync process. */
+@Value
+public class ParquetSourceConfig {
+    public static final String PARTITION_SPEC_EXTRACTOR_CLASS =
+            "xtable.parquet.source.partition_spec_extractor_class";
+    public static final String PARTITION_FIELD_SPEC_CONFIG =
+            "xtable.parquet.source.partition_field_spec_config";
+
+    String partitionSpecExtractorClass;
+    List<PartitionFieldSpec> partitionFieldSpecs;
+
+    public static ParquetSourceConfig fromPartitionFieldSpecConfig(String 
partitionFieldSpecConfig) {
+        return new ParquetSourceConfig(
+                ParquetPartitionSpecExtractor.class.getName(),
+                parsePartitionFieldSpecs(partitionFieldSpecConfig));
+    }
+
+    public static ParquetSourceConfig fromProperties(Properties properties) {
+        String partitionSpecExtractorClass =
+                properties.getProperty(
+                        PARTITION_SPEC_EXTRACTOR_CLASS,
+                        ParquetPartitionSpecExtractor.class.getName());
+        String partitionFieldSpecString = 
properties.getProperty(PARTITION_FIELD_SPEC_CONFIG);
+        List<PartitionFieldSpec> partitionFieldSpecs =
+                parsePartitionFieldSpecs(partitionFieldSpecString);
+        return new ParquetSourceConfig(partitionSpecExtractorClass, 
partitionFieldSpecs);
+    }
+
+    @Value
+    static class PartitionFieldSpec {

Review Comment:
   Can this be moved into a class that can be shared with the HudiSourceConfig?



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSourceProvider.java:
##########
@@ -0,0 +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 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<Long> {
+  @Override
+  public ParquetConversionSource getConversionSourceInstance(SourceTable 
sourceTable) {
+
+    return ParquetConversionSource.builder()
+        .tableName(sourceTable.getName())
+        .basePath(sourceTable.getBasePath())
+        .hadoopConf(new Configuration())

Review Comment:
   Use the `hadoopConf` from the parent class to ensure the proper 
configurations are set



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);
+        internalDataFiles =
+                parquetFiles.stream()
+                        .map(
+                                file ->
+                                        InternalDataFile.builder()
+                                                
.physicalPath(file.getPath().toString())
+                                                
.fileFormat(FileFormat.APACHE_PARQUET)
+                                                .fileSizeBytes(file.getLen())
+                                                
.partitionValues(partitionValueExtractor.extractPartitionValues(
+                                                        
partitionValueExtractor.extractParquertPartitions(parquetMetadataExtractor.readParquetMetadata(
+                                                                hadoopConf, 
file.getPath()), file.getPath().toString()),
+                                                        basePath))
+                                                
.lastModified(file.getModificationTime())
+                                                .columnStats(
+                                                        
parquetStatsExtractor.getColumnStatsForaFile(
+                                                                
parquetMetadataExtractor.readParquetMetadata(
+                                                                        
hadoopConf, file.getPath())))
+                                                .build())
+                        .collect(Collectors.toList());
+        return internalDataFiles;

Review Comment:
   Nitpick: just return instead of assigning the output to a local variable



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);
+        internalDataFiles =
+                parquetFiles.stream()
+                        .map(
+                                file ->
+                                        InternalDataFile.builder()
+                                                
.physicalPath(file.getPath().toString())
+                                                
.fileFormat(FileFormat.APACHE_PARQUET)
+                                                .fileSizeBytes(file.getLen())
+                                                
.partitionValues(partitionValueExtractor.extractPartitionValues(
+                                                        
partitionValueExtractor.extractParquertPartitions(parquetMetadataExtractor.readParquetMetadata(
+                                                                hadoopConf, 
file.getPath()), file.getPath().toString()),
+                                                        basePath))
+                                                
.lastModified(file.getModificationTime())
+                                                .columnStats(
+                                                        
parquetStatsExtractor.getColumnStatsForaFile(
+                                                                
parquetMetadataExtractor.readParquetMetadata(
+                                                                        
hadoopConf, file.getPath())))
+                                                .build())
+                        .collect(Collectors.toList());
+        return internalDataFiles;
+    }
+
+    // since we are considering files instead of tables in parquet
+    @Override
+    public CommitsBacklog<java.lang.Long> 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
+    public InternalSnapshot getCurrentSnapshot() {
+        List<InternalDataFile> internalDataFiles = getInternalDataFiles();
+        InternalTable table = getTable(-1L);
+        return InternalSnapshot.builder()
+                .table(table)
+                
.partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles))
+                .build();
+    }
+
+
+    public List<LocatedFileStatus> getParquetFiles(Configuration hadoopConf, 
String basePath) {

Review Comment:
   Let's make this private as well



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetPartitionValueExtractor.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.Collections;
+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.parquet.schema.MessageType;
+import org.apache.xtable.model.schema.InternalField;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.schema.InternalType;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.xtable.model.stat.Range;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+
+import lombok.AllArgsConstructor;
+import lombok.NonNull;
+import lombok.Value;
+
+import org.apache.xtable.exception.PartitionValuesExtractorException;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalType;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.stat.Range;
+
+
+/** Partition value extractor for Parquet. */
+// Extracts the partitionFields and values and create InputParitionFields 
object (fields and types)
+//then convert those to InternalPartitionField for the ConversionSource
+//@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 ParquetSchemaExtractor schemaExtractor =
+          ParquetSchemaExtractor.getInstance();
+  private static final ParquetMetadataExtractor parquetMetadataExtractor =
+          ParquetMetadataExtractor.getInstance();
+
+  private static final ParquetPartitionSpecExtractor partitionsSpecExtractor =
+          ParquetPartitionSpecExtractor.getInstance();
+
+  public static ParquetPartitionValueExtractor getInstance() {
+    return INSTANCE;
+  }
+  private static final String HIVE_DEFAULT_PARTITION = 
"__HIVE_DEFAULT_PARTITION__";
+  private final Map<String, String> pathToPartitionFieldFormat = null;
+
+
+  public List<InternalPartitionField> extractParquertPartitions(
+          ParquetMetadata footer, String path) {
+    MessageType parquetSchema  = parquetMetadataExtractor.getSchema(footer);
+    InternalSchema internalSchema = 
schemaExtractor.toInternalSchema(parquetSchema, path);
+    List<InternalPartitionField> partitions = 
partitionsSpecExtractor.spec(internalSchema);
+    return partitions;
+  }
+
+
+
+  public List<PartitionValue> extractPartitionValues(

Review Comment:
   Can you take the existing `HudiPartitionValuesExtractor`, rename it to 
something like `PathBasedPartitionValuesExtractor`, and then use it here 
instead of repeating the code?



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetSourcePartitionSpecExtractor.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.Map;
+
+import org.apache.xtable.spi.extractor.SourcePartitionSpecExtractor;
+
+/**
+ * Partition spec extractor interface specifically designed for Parquet to 
parse partition values
+ * appropriately.
+ */
+public interface ParquetSourcePartitionSpecExtractor extends 
SourcePartitionSpecExtractor {

Review Comment:
   This is pretty much the same as the `HudiSourcePartitionSpecExtractor`, can 
we rename that class to `PathBasedPartitionSpecExtractor` and then reuse it for 
the Parquet source? 



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);
+        internalDataFiles =
+                parquetFiles.stream()
+                        .map(
+                                file ->
+                                        InternalDataFile.builder()
+                                                
.physicalPath(file.getPath().toString())
+                                                
.fileFormat(FileFormat.APACHE_PARQUET)
+                                                .fileSizeBytes(file.getLen())
+                                                
.partitionValues(partitionValueExtractor.extractPartitionValues(
+                                                        
partitionValueExtractor.extractParquertPartitions(parquetMetadataExtractor.readParquetMetadata(
+                                                                hadoopConf, 
file.getPath()), file.getPath().toString()),
+                                                        basePath))
+                                                
.lastModified(file.getModificationTime())
+                                                .columnStats(
+                                                        
parquetStatsExtractor.getColumnStatsForaFile(
+                                                                
parquetMetadataExtractor.readParquetMetadata(
+                                                                        
hadoopConf, file.getPath())))
+                                                .build())
+                        .collect(Collectors.toList());
+        return internalDataFiles;
+    }
+
+    // since we are considering files instead of tables in parquet
+    @Override
+    public CommitsBacklog<java.lang.Long> 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
+    public InternalSnapshot getCurrentSnapshot() {
+        List<InternalDataFile> internalDataFiles = getInternalDataFiles();
+        InternalTable table = getTable(-1L);

Review Comment:
   Since this call also requires listing the data files, let's make a new 
private method for `getTable` that takes in the list of files so it doesn't get 
read twice



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);

Review Comment:
   Remove this unused line



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {

Review Comment:
   Let's make this private



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);
+        internalDataFiles =
+                parquetFiles.stream()
+                        .map(
+                                file ->
+                                        InternalDataFile.builder()
+                                                
.physicalPath(file.getPath().toString())
+                                                
.fileFormat(FileFormat.APACHE_PARQUET)
+                                                .fileSizeBytes(file.getLen())
+                                                
.partitionValues(partitionValueExtractor.extractPartitionValues(
+                                                        
partitionValueExtractor.extractParquertPartitions(parquetMetadataExtractor.readParquetMetadata(
+                                                                hadoopConf, 
file.getPath()), file.getPath().toString()),
+                                                        basePath))
+                                                
.lastModified(file.getModificationTime())
+                                                .columnStats(
+                                                        
parquetStatsExtractor.getColumnStatsForaFile(
+                                                                
parquetMetadataExtractor.readParquetMetadata(
+                                                                        
hadoopConf, file.getPath())))
+                                                .build())
+                        .collect(Collectors.toList());
+        return internalDataFiles;
+    }
+
+    // since we are considering files instead of tables in parquet
+    @Override
+    public CommitsBacklog<java.lang.Long> 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
+    public InternalSnapshot getCurrentSnapshot() {
+        List<InternalDataFile> internalDataFiles = getInternalDataFiles();
+        InternalTable table = getTable(-1L);
+        return InternalSnapshot.builder()

Review Comment:
   We'll need to set the `version` here. I am guessing it should be the last 
modification time but I need to think it through more.



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {
+    @Builder.Default
+    private static final ParquetSchemaExtractor schemaExtractor =
+            ParquetSchemaExtractor.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;
+
+    /**
+     * 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<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        // TODO last file in terms of modifcation time instead
+        LocatedFileStatus latestFile = parquetFiles.get(parquetFiles.size() - 
1);
+
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields = 
partitionValueExtractor.extractParquertPartitions(parquetMetadata, 
latestFile.getPath().toString());
+        MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+        InternalSchema schema = 
schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+        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();
+    }
+
+    public List<InternalDataFile> getInternalDataFiles() {
+        List<InternalDataFile> internalDataFiles = null;
+        List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+        InternalTable table = getTable(-1L);
+        internalDataFiles =
+                parquetFiles.stream()
+                        .map(
+                                file ->
+                                        InternalDataFile.builder()
+                                                
.physicalPath(file.getPath().toString())
+                                                
.fileFormat(FileFormat.APACHE_PARQUET)
+                                                .fileSizeBytes(file.getLen())
+                                                
.partitionValues(partitionValueExtractor.extractPartitionValues(
+                                                        
partitionValueExtractor.extractParquertPartitions(parquetMetadataExtractor.readParquetMetadata(
+                                                                hadoopConf, 
file.getPath()), file.getPath().toString()),
+                                                        basePath))
+                                                
.lastModified(file.getModificationTime())
+                                                .columnStats(
+                                                        
parquetStatsExtractor.getColumnStatsForaFile(
+                                                                
parquetMetadataExtractor.readParquetMetadata(
+                                                                        
hadoopConf, file.getPath())))
+                                                .build())
+                        .collect(Collectors.toList());
+        return internalDataFiles;
+    }
+
+    // since we are considering files instead of tables in parquet
+    @Override
+    public CommitsBacklog<java.lang.Long> getCommitsBacklog(

Review Comment:
   For all usages of `java.lang.Long` you can shorten it to `Long`



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.util.functional.RemoteIterators;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.model.*;
+import org.apache.xtable.model.CommitsBacklog;
+import org.apache.xtable.model.InstantsForIncrementalSync;
+import org.apache.xtable.model.TableChange;
+import org.apache.xtable.model.schema.InternalPartitionField;
+import org.apache.xtable.model.schema.InternalSchema;
+import org.apache.xtable.model.stat.PartitionValue;
+import org.apache.xtable.model.storage.*;
+import org.apache.xtable.model.storage.FileFormat;
+import org.apache.xtable.model.storage.InternalDataFile;
+import org.apache.xtable.spi.extractor.ConversionSource;
+
+@Builder
+// @NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class ParquetConversionSource implements ConversionSource<Long> {

Review Comment:
   For us to provide support for incremental sync, I think this may need to be 
a range instead of a singular point. That way we can filter the files by a 
start and end time when performing the sync.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@xtable.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to