nsivabalan commented on code in PR #9553:
URL: https://github.com/apache/hudi/pull/9553#discussion_r1316523537


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/CommitMetadataUtils.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.hudi.client.utils;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.function.SerializableBiFunction;
+import org.apache.hudi.common.function.SerializableFunction;
+import org.apache.hudi.common.function.SerializablePairFunction;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieDeltaWriteStat;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.marker.WriteMarkers;
+import org.apache.hudi.table.marker.WriteMarkersFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CommitMetadataUtils {
+
+  /* In spark mor table, task retries may generate log files which are not 
included in write status.
+   * We need to add these to CommitMetadata so that it will be synced to MDT.
+   */
+  public static HoodieCommitMetadata 
reconcileMetadataForMissingFiles(HoodieTable table, String commitActionType, 
String instantTime,
+                                                                      
HoodieCommitMetadata commitMetadata, HoodieWriteConfig config,
+                                                                      
HoodieEngineContext context, Configuration hadoopConf, String 
classNameForContext) throws IOException {
+    if 
(!table.getMetaClient().getTableConfig().getTableType().equals(HoodieTableType.MERGE_ON_READ)
+        || !commitActionType.equals(HoodieActiveTimeline.DELTA_COMMIT_ACTION)) 
{
+      return commitMetadata;
+    }
+
+    WriteMarkers markers = WriteMarkersFactory.get(config.getMarkersType(), 
table, instantTime);
+    // if there is log files in this delta commit, we search any invalid log 
files generated by failed spark task
+    boolean hasLogFileInDeltaCommit = commitMetadata.getPartitionToWriteStats()
+        .values().stream().flatMap(List::stream)
+        .anyMatch(writeStat -> FSUtils.isLogFile(new 
Path(config.getBasePath(), writeStat.getPath()).getName()));
+    if (hasLogFileInDeltaCommit) { // skip for COW table
+      // get all log files generated by makers
+      Set<String> allLogFilesMarkerPath = new 
HashSet<>(markers.getAppendedLogPaths(context, 
config.getFinalizeWriteParallelism()));
+      Set<String> logFilesMarkerPath = new HashSet<>();
+      allLogFilesMarkerPath.stream().filter(logFilePath -> 
!logFilePath.endsWith("cdc")).forEach(logFilePath -> 
logFilesMarkerPath.add(logFilePath));
+
+      // remove valid log files
+      for (Map.Entry<String, List<HoodieWriteStat>> partitionAndWriteStats : 
commitMetadata.getPartitionToWriteStats().entrySet()) {
+        for (HoodieWriteStat hoodieWriteStat : 
partitionAndWriteStats.getValue()) {
+          logFilesMarkerPath.remove(hoodieWriteStat.getPath());
+        }
+      }
+
+      // remaining are log files generated by retried spark task, let's 
generate write stat for them
+      if (logFilesMarkerPath.size() > 0) {
+        SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(hadoopConf);
+        context.setJobStatus(classNameForContext, "Preparing data for missing 
files to assit with generatin write stats");
+        // populate partition -> map (fileId -> HoodieWriteStat) // we just 
need one write stat per fileID to fetch some info about
+        // the file slice of interest to populate WriteStat.
+        HoodiePairData<String, Map<String, HoodieWriteStat>> 
partitionToWriteStatHoodieData = getPartitionToFileIdToFilesMap(commitMetadata, 
context);
+
+        String localBasePath = config.getBasePath();
+        // populate partition -> map (fileId -> List <missing log file names>)
+        HoodiePairData<String, Map<String, List<String>>> 
partitionToMissingLogFilesHoodieData =
+            getPartitionToFileIdToMissingLogFileMap(localBasePath, 
logFilesMarkerPath, context, config.getFileListingParallelism());
+
+        // TODO: make one call per partition to fetch file sizes.
+        context.setJobStatus(classNameForContext, "Generating writeStat for 
missing log files");
+
+        // lets join both to generate write stats for missing log files
+        List<Pair<String, List<HoodieWriteStat>>> additionalLogFileWriteStat = 
partitionToWriteStatHoodieData
+            .join(partitionToMissingLogFilesHoodieData)
+            .map((SerializableFunction<Pair<String, Pair<Map<String, 
HoodieWriteStat>, Map<String, List<String>>>>, Pair<String, 
List<HoodieWriteStat>>>) v1 -> {
+              final Path basePathLocal = new Path(localBasePath);
+              String partitionPath = v1.getKey();
+              Map<String, HoodieWriteStat> fileIdToOriginalWriteStat = 
v1.getValue().getKey();
+              Map<String, List<String>> missingFileIdToLogFileNames = 
v1.getValue().getValue();
+              List<String> missingLogFileNames = new ArrayList<>();
+              missingFileIdToLogFileNames.values().forEach(logfiles -> 
missingLogFileNames.addAll(logfiles));
+
+              // fetch file sizes from FileSystem
+              Path fullPartitionPath = 
StringUtils.isNullOrEmpty(partitionPath) ? new Path(localBasePath) : new 
Path(localBasePath, partitionPath);
+              FileSystem fileSystem = 
fullPartitionPath.getFileSystem(serializableConfiguration.get());
+              List<Option<FileStatus>> fileStatuesOpt = 
FSUtils.getFileStatusesUnderPartition(fileSystem, fullPartitionPath, 
missingLogFileNames, true);
+              List<FileStatus> fileStatuses = 
fileStatuesOpt.stream().filter(fileStatusOpt -> 
fileStatusOpt.isPresent()).map(fileStatusOption -> 
fileStatusOption.get()).collect(Collectors.toList());
+
+              // populate fileId -> List<FileStatus>
+              Map<String, List<FileStatus>> missingFileIdToLogFilesList = new 
HashMap<>();
+              fileStatuses.forEach(fileStatus -> {
+                String fileId = 
FSUtils.getFileIdFromLogPath(fileStatus.getPath());
+                if (!missingFileIdToLogFilesList.containsKey(fileId)) {
+                  missingFileIdToLogFilesList.put(fileId, new ArrayList<>());
+                }
+                missingFileIdToLogFilesList.get(fileId).add(fileStatus);
+              });
+
+              List<HoodieWriteStat> missingWriteStats = new ArrayList();
+              missingFileIdToLogFilesList.forEach((k, logFileStatuses) -> {
+                String fileId = k;
+                HoodieDeltaWriteStat originalWriteStat =
+                    (HoodieDeltaWriteStat) 
fileIdToOriginalWriteStat.get(fileId); // are there chances that there won't be 
any write stat in original list?
+                logFileStatuses.forEach(fileStatus -> {
+                  // for every missing file, add a new HoodieDeltaWriteStat
+                  HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
+                  HoodieLogFile logFile = new HoodieLogFile(fileStatus);
+                  writeStat.setPath(basePathLocal, logFile.getPath());
+                  writeStat.setPartitionPath(partitionPath);
+                  writeStat.setFileId(fileId);
+                  writeStat.setTotalWriteBytes(logFile.getFileSize());
+                  writeStat.setFileSizeInBytes(logFile.getFileSize());
+                  writeStat.setLogVersion(logFile.getLogVersion());
+                  List<String> logFiles = new 
ArrayList<>(originalWriteStat.getLogFiles());
+                  logFiles.add(logFile.getFileName());
+                  writeStat.setLogFiles(logFiles);
+                  writeStat.setBaseFile(originalWriteStat.getBaseFile());
+                  writeStat.setPrevCommit(logFile.getBaseCommitTime());
+                  missingWriteStats.add(writeStat);
+                });
+              });
+              return Pair.of(partitionPath, missingWriteStats);
+            }).collectAsList();
+
+        // add these write stat to commit meta. deltaWriteStat can be empty 
due to file missing. See code above to address FileNotFoundException
+        for (Pair<String, List<HoodieWriteStat>> partitionDeltaStats : 
additionalLogFileWriteStat) {
+          String partitionPath = partitionDeltaStats.getKey();
+          partitionDeltaStats.getValue().forEach(ws -> 
commitMetadata.addWriteStat(partitionPath, ws));
+        }
+      }
+    }
+    return commitMetadata;
+  }
+
+  private static HoodiePairData<String, Map<String, HoodieWriteStat>> 
getPartitionToFileIdToFilesMap(HoodieCommitMetadata commitMetadata, 
HoodieEngineContext context) {
+
+    List<Map.Entry<String, List<HoodieWriteStat>>> partitionToWriteStats = new 
ArrayList(commitMetadata.getPartitionToWriteStats().entrySet());
+
+    return context.parallelize(partitionToWriteStats)
+        .mapToPair((SerializablePairFunction<Map.Entry<String, 
List<HoodieWriteStat>>, String, Map<String, HoodieWriteStat>>) t -> {
+          Map<String, HoodieWriteStat> fileIdToWriteStat = new HashMap<>();
+          t.getValue().forEach(writeStat -> {
+            if (!fileIdToWriteStat.containsKey(writeStat.getFileId())) {
+              fileIdToWriteStat.put(writeStat.getFileId(), writeStat);
+            }
+          });
+          return Pair.of(t.getKey(), fileIdToWriteStat);
+        });
+  }
+
+  private static HoodiePairData<String, Map<String, List<String>>> 
getPartitionToFileIdToMissingLogFileMap(String basePathStr, Set<String> 
logFilesMarkerPath, HoodieEngineContext context,

Review Comment:
   this looks ok to me. 
   after this, we have to do the join anyways so that we can create write stat 
for each missing file. 
   
   instead, in the cur logic, we are keeping everything in one join and map 
call. 
   lmk if you think we have compelling reason to go w/ your suggestion 
   



-- 
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...@hudi.apache.org

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

Reply via email to