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


##########
xtable-api/src/main/java/org/apache/xtable/model/storage/TableFormat.java:
##########
@@ -27,6 +27,7 @@ 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"};

Review Comment:
   Should `PARQUET` be added here?



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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 ParquetStatsExtractor parquetStatsExtractor =
+      ParquetStatsExtractor.getInstance();
+
+  private final ParquetPartitionValueExtractor partitionValueExtractor;
+  private final ConfigurationBasedPartitionSpecExtractor 
partitionSpecExtractor;
+  private final String tableName;
+  private final String basePath;
+  @NonNull private final Configuration hadoopConf;
+
+  private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {
+    ParquetMetadata parquetMetadata =
+        parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+    MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+    InternalSchema schema =
+        schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+    List<InternalPartitionField> partitionFields = 
partitionSpecExtractor.spec(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.getModificationTime()))
+        .build();
+  }
+
+  private InternalTable getMostRecentTable(List<LocatedFileStatus> 
parquetFiles) {
+    LocatedFileStatus latestFile = getMostRecentParquetFile(parquetFiles);
+    return createInternalTableFromTable(latestFile);
+  }
+
+  @Override
+  public InternalTable getTable(Long modificationTime) {
+    // get parquetFile at specific time modificationTime
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    LocatedFileStatus file = getParquetFileAt(parquetFiles, modificationTime);
+    return createInternalTableFromTable(file);
+  }
+
+  private List<InternalDataFile> getInternalDataFiles(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .map(
+            file ->
+                InternalDataFile.builder()
+                    .physicalPath(file.getPath().toString())
+                    .fileFormat(FileFormat.APACHE_PARQUET)
+                    .fileSizeBytes(file.getLen())
+                    .partitionValues(
+                        partitionValueExtractor.extractPartitionValues(
+                            partitionValueExtractor.extractParquetPartitions(
+                                parquetMetadataExtractor.readParquetMetadata(
+                                    hadoopConf, file.getPath()),
+                                file.getPath().toString()),
+                            basePath))
+                    .lastModified(file.getModificationTime())
+                    .columnStats(
+                        parquetStatsExtractor.getColumnStatsForaFile(
+                            parquetMetadataExtractor.readParquetMetadata(
+                                hadoopConf, file.getPath())))
+                    .build())
+        .collect(Collectors.toList());
+  }
+
+  private InternalDataFile createInternalDataFileFromParquetFile(FileStatus 
parquetFile) {
+    return InternalDataFile.builder()
+        .physicalPath(parquetFile.getPath().toString())
+        .partitionValues(
+            partitionValueExtractor.extractPartitionValues(
+                partitionValueExtractor.extractParquetPartitions(
+                    parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath()),
+                    parquetFile.getPath().toString()),
+                basePath))
+        .lastModified(parquetFile.getModificationTime())
+        .fileSizeBytes(parquetFile.getLen())
+        .columnStats(
+            parquetStatsExtractor.getColumnStatsForaFile(
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath())))
+        .build();
+  }
+
+  @Override
+  public CommitsBacklog<Long> getCommitsBacklog(InstantsForIncrementalSync 
syncInstants) {
+    // based on either table formats?
+    List<Long> commitsToProcess =
+        
Collections.singletonList(syncInstants.getLastSyncInstant().toEpochMilli());
+    return 
CommitsBacklog.<Long>builder().commitsToProcess(commitsToProcess).build();
+  }
+
+  @Override
+  public TableChange getTableChangeForCommit(Long modificationTime) {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    Set<InternalDataFile> addedInternalDataFiles = new HashSet<>();
+
+    List<FileStatus> tableChangesAfter =
+        parquetFiles.stream()
+            .filter(fileStatus -> fileStatus.getModificationTime() > 
modificationTime)
+            .collect(Collectors.toList());
+    InternalTable internalTable = getMostRecentTable(parquetFiles);
+    for (FileStatus tableStatus : tableChangesAfter) {
+      InternalDataFile currentDataFile = 
createInternalDataFileFromParquetFile(tableStatus);
+      addedInternalDataFiles.add(currentDataFile);
+    }
+
+    return TableChange.builder()
+        .tableAsOfChange(internalTable)
+        
.filesDiff(InternalFilesDiff.builder().filesAdded(addedInternalDataFiles).build())
+        .build();
+  }
+
+  @Override
+  public InternalTable getCurrentTable() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    return getMostRecentTable(parquetFiles);
+  }
+
+  /**
+   * Here to get current snapshot listing all files hence the -1 is being 
passed
+   *
+   * @return
+   */
+  @Override
+  public InternalSnapshot getCurrentSnapshot() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    List<InternalDataFile> internalDataFiles = 
getInternalDataFiles(parquetFiles);
+    InternalTable table = getMostRecentTable(parquetFiles);
+    return InternalSnapshot.builder()
+        .table(table)
+        .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles))
+        .build();
+  }
+
+  private LocatedFileStatus getMostRecentParquetFile(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .max(Comparator.comparing(FileStatus::getModificationTime))
+        .orElseThrow(() -> new IllegalStateException("No files found"));
+  }
+
+  private LocatedFileStatus getParquetFileAt(
+      List<LocatedFileStatus> parquetFiles, long modificationTime) {
+    return parquetFiles.stream()
+        .filter(fileStatus -> fileStatus.getModificationTime() == 
modificationTime)
+        .findFirst()
+        .orElseThrow(
+            () -> new IllegalStateException("No file found at " + 
Long.valueOf(modificationTime)));
+  }
+
+  private List<LocatedFileStatus> getParquetFilesInRange(
+      List<LocatedFileStatus> parquetFiles, Long minModificationDate, Long 
maxModicationDate) {
+    return parquetFiles.stream()
+        .filter(
+            fileStatus ->
+                fileStatus.getModificationTime() > minModificationDate
+                    && fileStatus.getModificationTime() < maxModicationDate)
+        .collect(Collectors.toList());
+  }
+
+  private List<LocatedFileStatus> getParquetFiles(Configuration hadoopConf, 
String basePath) {
+    try {
+      FileSystem fs = FileSystem.get(hadoopConf);
+      RemoteIterator<LocatedFileStatus> 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);
+    }
+  }
+
+  @Override
+  public boolean isIncrementalSyncSafeFrom(Instant instant) {

Review Comment:
   This looks like we're just checking if there are files older than the 
provided instant. Is this because we are assuming the files may be removed from 
the source table?



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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 ParquetStatsExtractor parquetStatsExtractor =
+      ParquetStatsExtractor.getInstance();
+
+  private final ParquetPartitionValueExtractor partitionValueExtractor;
+  private final ConfigurationBasedPartitionSpecExtractor 
partitionSpecExtractor;
+  private final String tableName;
+  private final String basePath;
+  @NonNull private final Configuration hadoopConf;
+
+  private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {
+    ParquetMetadata parquetMetadata =
+        parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+    MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+    InternalSchema schema =
+        schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+    List<InternalPartitionField> partitionFields = 
partitionSpecExtractor.spec(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.getModificationTime()))
+        .build();
+  }
+
+  private InternalTable getMostRecentTable(List<LocatedFileStatus> 
parquetFiles) {
+    LocatedFileStatus latestFile = getMostRecentParquetFile(parquetFiles);
+    return createInternalTableFromTable(latestFile);
+  }
+
+  @Override
+  public InternalTable getTable(Long modificationTime) {
+    // get parquetFile at specific time modificationTime
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    LocatedFileStatus file = getParquetFileAt(parquetFiles, modificationTime);
+    return createInternalTableFromTable(file);
+  }
+
+  private List<InternalDataFile> getInternalDataFiles(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .map(
+            file ->
+                InternalDataFile.builder()
+                    .physicalPath(file.getPath().toString())
+                    .fileFormat(FileFormat.APACHE_PARQUET)
+                    .fileSizeBytes(file.getLen())
+                    .partitionValues(
+                        partitionValueExtractor.extractPartitionValues(
+                            partitionValueExtractor.extractParquetPartitions(
+                                parquetMetadataExtractor.readParquetMetadata(
+                                    hadoopConf, file.getPath()),
+                                file.getPath().toString()),
+                            basePath))
+                    .lastModified(file.getModificationTime())
+                    .columnStats(
+                        parquetStatsExtractor.getColumnStatsForaFile(
+                            parquetMetadataExtractor.readParquetMetadata(
+                                hadoopConf, file.getPath())))
+                    .build())
+        .collect(Collectors.toList());
+  }
+
+  private InternalDataFile createInternalDataFileFromParquetFile(FileStatus 
parquetFile) {
+    return InternalDataFile.builder()
+        .physicalPath(parquetFile.getPath().toString())
+        .partitionValues(
+            partitionValueExtractor.extractPartitionValues(
+                partitionValueExtractor.extractParquetPartitions(
+                    parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath()),
+                    parquetFile.getPath().toString()),
+                basePath))
+        .lastModified(parquetFile.getModificationTime())
+        .fileSizeBytes(parquetFile.getLen())
+        .columnStats(
+            parquetStatsExtractor.getColumnStatsForaFile(
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath())))
+        .build();
+  }
+
+  @Override
+  public CommitsBacklog<Long> getCommitsBacklog(InstantsForIncrementalSync 
syncInstants) {
+    // based on either table formats?
+    List<Long> commitsToProcess =
+        
Collections.singletonList(syncInstants.getLastSyncInstant().toEpochMilli());
+    return 
CommitsBacklog.<Long>builder().commitsToProcess(commitsToProcess).build();
+  }
+
+  @Override
+  public TableChange getTableChangeForCommit(Long modificationTime) {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    Set<InternalDataFile> addedInternalDataFiles = new HashSet<>();
+
+    List<FileStatus> tableChangesAfter =
+        parquetFiles.stream()
+            .filter(fileStatus -> fileStatus.getModificationTime() > 
modificationTime)
+            .collect(Collectors.toList());
+    InternalTable internalTable = getMostRecentTable(parquetFiles);
+    for (FileStatus tableStatus : tableChangesAfter) {
+      InternalDataFile currentDataFile = 
createInternalDataFileFromParquetFile(tableStatus);
+      addedInternalDataFiles.add(currentDataFile);
+    }
+
+    return TableChange.builder()
+        .tableAsOfChange(internalTable)
+        
.filesDiff(InternalFilesDiff.builder().filesAdded(addedInternalDataFiles).build())
+        .build();
+  }
+
+  @Override
+  public InternalTable getCurrentTable() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    return getMostRecentTable(parquetFiles);
+  }
+
+  /**
+   * Here to get current snapshot listing all files hence the -1 is being 
passed
+   *
+   * @return
+   */
+  @Override
+  public InternalSnapshot getCurrentSnapshot() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    List<InternalDataFile> internalDataFiles = 
getInternalDataFiles(parquetFiles);
+    InternalTable table = getMostRecentTable(parquetFiles);
+    return InternalSnapshot.builder()
+        .table(table)
+        .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles))
+        .build();
+  }
+
+  private LocatedFileStatus getMostRecentParquetFile(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .max(Comparator.comparing(FileStatus::getModificationTime))
+        .orElseThrow(() -> new IllegalStateException("No files found"));
+  }
+
+  private LocatedFileStatus getParquetFileAt(
+      List<LocatedFileStatus> parquetFiles, long modificationTime) {
+    return parquetFiles.stream()
+        .filter(fileStatus -> fileStatus.getModificationTime() == 
modificationTime)
+        .findFirst()
+        .orElseThrow(
+            () -> new IllegalStateException("No file found at " + 
Long.valueOf(modificationTime)));
+  }
+
+  private List<LocatedFileStatus> getParquetFilesInRange(
+      List<LocatedFileStatus> parquetFiles, Long minModificationDate, Long 
maxModicationDate) {
+    return parquetFiles.stream()
+        .filter(
+            fileStatus ->
+                fileStatus.getModificationTime() > minModificationDate
+                    && fileStatus.getModificationTime() < maxModicationDate)
+        .collect(Collectors.toList());
+  }

Review Comment:
   This looks unused now, let's remove it



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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 ParquetStatsExtractor parquetStatsExtractor =
+      ParquetStatsExtractor.getInstance();
+
+  private final ParquetPartitionValueExtractor partitionValueExtractor;
+  private final ConfigurationBasedPartitionSpecExtractor 
partitionSpecExtractor;
+  private final String tableName;
+  private final String basePath;
+  @NonNull private final Configuration hadoopConf;
+
+  private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {
+    ParquetMetadata parquetMetadata =
+        parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+    MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+    InternalSchema schema =
+        schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+    List<InternalPartitionField> partitionFields = 
partitionSpecExtractor.spec(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.getModificationTime()))
+        .build();
+  }
+
+  private InternalTable getMostRecentTable(List<LocatedFileStatus> 
parquetFiles) {
+    LocatedFileStatus latestFile = getMostRecentParquetFile(parquetFiles);
+    return createInternalTableFromTable(latestFile);
+  }
+
+  @Override
+  public InternalTable getTable(Long modificationTime) {
+    // get parquetFile at specific time modificationTime
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    LocatedFileStatus file = getParquetFileAt(parquetFiles, modificationTime);
+    return createInternalTableFromTable(file);
+  }
+
+  private List<InternalDataFile> getInternalDataFiles(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .map(
+            file ->
+                InternalDataFile.builder()
+                    .physicalPath(file.getPath().toString())
+                    .fileFormat(FileFormat.APACHE_PARQUET)
+                    .fileSizeBytes(file.getLen())
+                    .partitionValues(
+                        partitionValueExtractor.extractPartitionValues(
+                            partitionValueExtractor.extractParquetPartitions(
+                                parquetMetadataExtractor.readParquetMetadata(
+                                    hadoopConf, file.getPath()),
+                                file.getPath().toString()),
+                            basePath))
+                    .lastModified(file.getModificationTime())
+                    .columnStats(
+                        parquetStatsExtractor.getColumnStatsForaFile(
+                            parquetMetadataExtractor.readParquetMetadata(
+                                hadoopConf, file.getPath())))
+                    .build())
+        .collect(Collectors.toList());
+  }
+
+  private InternalDataFile createInternalDataFileFromParquetFile(FileStatus 
parquetFile) {
+    return InternalDataFile.builder()
+        .physicalPath(parquetFile.getPath().toString())
+        .partitionValues(
+            partitionValueExtractor.extractPartitionValues(
+                partitionValueExtractor.extractParquetPartitions(
+                    parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath()),
+                    parquetFile.getPath().toString()),
+                basePath))
+        .lastModified(parquetFile.getModificationTime())
+        .fileSizeBytes(parquetFile.getLen())
+        .columnStats(
+            parquetStatsExtractor.getColumnStatsForaFile(
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath())))
+        .build();
+  }
+
+  @Override
+  public CommitsBacklog<Long> getCommitsBacklog(InstantsForIncrementalSync 
syncInstants) {
+    // based on either table formats?
+    List<Long> commitsToProcess =
+        
Collections.singletonList(syncInstants.getLastSyncInstant().toEpochMilli());
+    return 
CommitsBacklog.<Long>builder().commitsToProcess(commitsToProcess).build();
+  }
+
+  @Override
+  public TableChange getTableChangeForCommit(Long modificationTime) {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    Set<InternalDataFile> addedInternalDataFiles = new HashSet<>();
+
+    List<FileStatus> tableChangesAfter =
+        parquetFiles.stream()
+            .filter(fileStatus -> fileStatus.getModificationTime() > 
modificationTime)
+            .collect(Collectors.toList());
+    InternalTable internalTable = getMostRecentTable(parquetFiles);
+    for (FileStatus tableStatus : tableChangesAfter) {
+      InternalDataFile currentDataFile = 
createInternalDataFileFromParquetFile(tableStatus);
+      addedInternalDataFiles.add(currentDataFile);
+    }
+
+    return TableChange.builder()
+        .tableAsOfChange(internalTable)
+        
.filesDiff(InternalFilesDiff.builder().filesAdded(addedInternalDataFiles).build())
+        .build();
+  }
+
+  @Override
+  public InternalTable getCurrentTable() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    return getMostRecentTable(parquetFiles);
+  }
+
+  /**
+   * Here to get current snapshot listing all files hence the -1 is being 
passed
+   *
+   * @return
+   */
+  @Override
+  public InternalSnapshot getCurrentSnapshot() {
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    List<InternalDataFile> internalDataFiles = 
getInternalDataFiles(parquetFiles);
+    InternalTable table = getMostRecentTable(parquetFiles);
+    return InternalSnapshot.builder()
+        .table(table)
+        .partitionedDataFiles(PartitionFileGroup.fromFiles(internalDataFiles))
+        .build();
+  }
+
+  private LocatedFileStatus getMostRecentParquetFile(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .max(Comparator.comparing(FileStatus::getModificationTime))
+        .orElseThrow(() -> new IllegalStateException("No files found"));
+  }
+
+  private LocatedFileStatus getParquetFileAt(
+      List<LocatedFileStatus> parquetFiles, long modificationTime) {
+    return parquetFiles.stream()
+        .filter(fileStatus -> fileStatus.getModificationTime() == 
modificationTime)
+        .findFirst()
+        .orElseThrow(
+            () -> new IllegalStateException("No file found at " + 
Long.valueOf(modificationTime)));
+  }
+
+  private List<LocatedFileStatus> getParquetFilesInRange(
+      List<LocatedFileStatus> parquetFiles, Long minModificationDate, Long 
maxModicationDate) {
+    return parquetFiles.stream()
+        .filter(
+            fileStatus ->
+                fileStatus.getModificationTime() > minModificationDate
+                    && fileStatus.getModificationTime() < maxModicationDate)
+        .collect(Collectors.toList());
+  }
+
+  private List<LocatedFileStatus> getParquetFiles(Configuration hadoopConf, 
String basePath) {

Review Comment:
   I see a lot of the methods that use the output of this method later create a 
Stream from the list. We can avoid collecting all the elements in memory at 
once by returning a Stream directly here and then using that.



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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 ParquetStatsExtractor parquetStatsExtractor =
+      ParquetStatsExtractor.getInstance();
+
+  private final ParquetPartitionValueExtractor partitionValueExtractor;
+  private final ConfigurationBasedPartitionSpecExtractor 
partitionSpecExtractor;
+  private final String tableName;
+  private final String basePath;
+  @NonNull private final Configuration hadoopConf;
+
+  private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {
+    ParquetMetadata parquetMetadata =
+        parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+    MessageType parquetSchema = 
parquetMetadataExtractor.getSchema(parquetMetadata);
+    InternalSchema schema =
+        schemaExtractor.toInternalSchema(parquetSchema, 
latestFile.getPath().toString());
+    List<InternalPartitionField> partitionFields = 
partitionSpecExtractor.spec(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.getModificationTime()))
+        .build();
+  }
+
+  private InternalTable getMostRecentTable(List<LocatedFileStatus> 
parquetFiles) {
+    LocatedFileStatus latestFile = getMostRecentParquetFile(parquetFiles);
+    return createInternalTableFromTable(latestFile);
+  }
+
+  @Override
+  public InternalTable getTable(Long modificationTime) {
+    // get parquetFile at specific time modificationTime
+    List<LocatedFileStatus> parquetFiles = getParquetFiles(hadoopConf, 
basePath);
+    LocatedFileStatus file = getParquetFileAt(parquetFiles, modificationTime);
+    return createInternalTableFromTable(file);
+  }
+
+  private List<InternalDataFile> getInternalDataFiles(List<LocatedFileStatus> 
parquetFiles) {
+    return parquetFiles.stream()
+        .map(
+            file ->
+                InternalDataFile.builder()
+                    .physicalPath(file.getPath().toString())
+                    .fileFormat(FileFormat.APACHE_PARQUET)
+                    .fileSizeBytes(file.getLen())
+                    .partitionValues(
+                        partitionValueExtractor.extractPartitionValues(
+                            partitionValueExtractor.extractParquetPartitions(
+                                parquetMetadataExtractor.readParquetMetadata(
+                                    hadoopConf, file.getPath()),
+                                file.getPath().toString()),
+                            basePath))
+                    .lastModified(file.getModificationTime())
+                    .columnStats(
+                        parquetStatsExtractor.getColumnStatsForaFile(
+                            parquetMetadataExtractor.readParquetMetadata(
+                                hadoopConf, file.getPath())))
+                    .build())
+        .collect(Collectors.toList());
+  }
+
+  private InternalDataFile createInternalDataFileFromParquetFile(FileStatus 
parquetFile) {
+    return InternalDataFile.builder()
+        .physicalPath(parquetFile.getPath().toString())
+        .partitionValues(
+            partitionValueExtractor.extractPartitionValues(
+                partitionValueExtractor.extractParquetPartitions(
+                    parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath()),
+                    parquetFile.getPath().toString()),
+                basePath))
+        .lastModified(parquetFile.getModificationTime())
+        .fileSizeBytes(parquetFile.getLen())
+        .columnStats(
+            parquetStatsExtractor.getColumnStatsForaFile(
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
parquetFile.getPath())))
+        .build();
+  }
+
+  @Override
+  public CommitsBacklog<Long> getCommitsBacklog(InstantsForIncrementalSync 
syncInstants) {
+    // based on either table formats?
+    List<Long> commitsToProcess =
+        
Collections.singletonList(syncInstants.getLastSyncInstant().toEpochMilli());
+    return 
CommitsBacklog.<Long>builder().commitsToProcess(commitsToProcess).build();
+  }

Review Comment:
   For this method, we want to return the identifiers for the commits that need 
to be processed given the last sync instant. So we'll want some way to describe 
the time range files that need to be synced



##########
xtable-core/src/main/java/org/apache/xtable/hudi/ConfigurationBasedPartitionSpecExtractor.java:
##########
@@ -25,24 +25,26 @@
 
 import lombok.AllArgsConstructor;
 
+import lombok.NoArgsConstructor;
 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.PartitionFieldSpec;
 import org.apache.xtable.schema.SchemaFieldFinder;
 
 /**
  * Parses the InternalPartitionFields from a configured list of specs with the 
format
  * path:type:format for date types or path:type for value types.
  */
 @AllArgsConstructor
-public class ConfigurationBasedPartitionSpecExtractor implements 
HudiSourcePartitionSpecExtractor {
-  private final HudiSourceConfig config;
+public class ConfigurationBasedPartitionSpecExtractor implements 
PathBasedPartitionSpecExtractor {
+  // private final HudiSourceConfig config;

Review Comment:
   This line can be removed now



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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)

Review Comment:
   Remove this commented out 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> {

Review Comment:
   @unical1988 this is related to the incremental sync paths. We'll want this 
source to have some time range to find the files that need to be synced. If we 
don't do this, we will need to limit this source to snapshot syncs.



##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hudi.common.table.HoodieTableMetaClient;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.schema.MessageType;
+
+import org.apache.xtable.hudi.*;
+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.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 ParquetStatsExtractor parquetStatsExtractor =
+      ParquetStatsExtractor.getInstance();
+
+  private final ParquetPartitionValueExtractor partitionValueExtractor;
+  private final ConfigurationBasedPartitionSpecExtractor 
partitionSpecExtractor;
+  private final String tableName;
+  private final String basePath;
+  @NonNull private final Configuration hadoopConf;
+
+  private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {

Review Comment:
   ```suggestion
     private InternalTable createInternalTableFromFile(LocatedFileStatus 
latestFile) {
   ```



-- 
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