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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/CommitMetadataUtils.java:
##########
@@ -0,0 +1,225 @@
+/*
+ * 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 basePathStr = config.getBasePath();
+        // populate partition -> map (fileId -> List <missing log file names>)
+        HoodiePairData<String, Map<String, List<String>>> 
partitionToMissingLogFilesHoodieData =
+            getPartitionToFileIdToMissingLogFileMap(basePathStr, 
logFilesMarkerPath, context, config.getFileListingParallelism());
+
+        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 = 
getWriteStatsForMissingLogFiles(partitionToWriteStatHoodieData,
+            partitionToMissingLogFilesHoodieData, serializableConfiguration, 
basePathStr);
+
+        for (Pair<String, List<HoodieWriteStat>> partitionDeltaStats : 
additionalLogFileWriteStat) {
+          String partitionPath = partitionDeltaStats.getKey();
+          partitionDeltaStats.getValue().forEach(ws -> 
commitMetadata.addWriteStat(partitionPath, ws));

Review Comment:
   So the incremental queries would see these `missing log files` right? If 
there are any valid log blocks in these file, could there be any duplications 
for consumption ?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to