unical1988 commented on code in PR #728:
URL: https://github.com/apache/incubator-xtable/pull/728#discussion_r2224257262


##########
xtable-core/src/main/java/org/apache/xtable/parquet/ParquetConversionSource.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.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 String tableName;
+    private final String basePath;
+    private final String configPath;
+    @NonNull
+    private final Configuration hadoopConf;
+
+    private InternalTable createInternalTableFromTable(LocatedFileStatus 
latestFile) {
+        ParquetMetadata parquetMetadata =
+                parquetMetadataExtractor.readParquetMetadata(hadoopConf, 
latestFile.getPath());
+
+        List<InternalPartitionField> partitionFields =
+                partitionValueExtractor.extractParquetPartitions(
+                        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();
+
+    }
+
+    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());
+        List<FileStatus> tableChangesBefore =
+                parquetFiles.stream()
+                        .filter(fileStatus -> fileStatus.getModificationTime() 
< modificationTime)
+                        .collect(Collectors.toList());
+        InternalTable internalTable = getMostRecentTable(parquetFiles);
+
+        for (FileStatus tableStatus : tableChangesAfter) {
+            boolean isPresent = tableChangesBefore.contains(tableStatus);

Review Comment:
   I removed the check ` tableChangesBefore.contains(tableStatus);`



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