nsivabalan commented on a change in pull request #4278:
URL: https://github.com/apache/hudi/pull/4278#discussion_r767032764



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
##########
@@ -316,9 +316,20 @@ public synchronized HoodieArchivedTimeline 
getArchivedTimeline() {
     return archivedTimeline;
   }
 
+  /**
+   * Get the archived instants as a timeline with all instant details.
+   * This is costly operation, as all data from the archived files are read.
+   *
+   * @return archived instant timeline
+   */
+  public synchronized HoodieArchivedTimeline 
getArchivedTimelineWithInstantDetails() {

Review comment:
       why this needs to be synchronized ?

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieRepairTool.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.utilities;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
+import org.apache.hudi.table.repair.RepairUtils;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+public class HoodieRepairTool {
+
+  private static final Logger LOG = 
LogManager.getLogger(HoodieRepairTool.class);
+  private static final String BACKUP_DIR_PREFIX = "hoodie_repair_backup_";
+  // Repair config
+  private final Config cfg;
+  // Properties with source, hoodie client, key generator etc.
+  private TypedProperties props;
+  // Spark context
+  private final JavaSparkContext jsc;
+  private final HoodieTableMetaClient metaClient;
+  private final FileSystemBackedTableMetadata tableMetadata;
+
+  public HoodieRepairTool(JavaSparkContext jsc, Config cfg) {
+    if (cfg.propsFilePath != null) {
+      cfg.propsFilePath = 
FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString();
+    }
+    this.jsc = jsc;
+    this.cfg = cfg;
+    this.props = cfg.propsFilePath == null
+        ? UtilHelpers.buildProperties(cfg.configs)
+        : readConfigFromFileSystem(jsc, cfg);
+    this.metaClient = HoodieTableMetaClient.builder()
+        .setConf(jsc.hadoopConfiguration()).setBasePath(cfg.basePath)
+        .setLoadActiveTimelineOnLoad(true)
+        .build();
+    this.tableMetadata = new FileSystemBackedTableMetadata(
+        new HoodieSparkEngineContext(jsc),
+        new SerializableConfiguration(jsc.hadoopConfiguration()),
+        cfg.basePath, cfg.assumeDatePartitioning);
+  }
+
+  public void run() {
+    Option<String> startingInstantOption = 
Option.ofNullable(cfg.startingInstantTime);
+    Option<String> endingInstantOption = 
Option.ofNullable(cfg.endingInstantTime);
+
+    if (startingInstantOption.isPresent() && endingInstantOption.isPresent()) {
+      LOG.info(String.format("Start repairing completed instants between %s 
and %s (inclusive)",
+          startingInstantOption.get(), endingInstantOption.get()));
+    } else if (startingInstantOption.isPresent()) {
+      LOG.info(String.format("Start repairing completed instants from %s 
(inclusive)",
+          startingInstantOption.get()));
+    } else if (endingInstantOption.isPresent()) {
+      LOG.info(String.format("Start repairing completed instants till %s 
(inclusive)",
+          endingInstantOption.get()));
+    } else {
+      LOG.info("Start repairing all completed instants");
+    }
+
+    try {
+      Mode mode = Mode.valueOf(cfg.runningMode.toUpperCase());
+      switch (mode) {
+        case REPAIR:
+          LOG.info(" ****** The repair tool is in REPAIR mode, dangling data 
and logs files "

Review comment:
       is "****" intentional or did you plan to remove it 

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieRepairTool.java
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.utilities;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
+import org.apache.hudi.table.repair.RepairUtils;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import scala.Tuple2;
+
+public class HoodieRepairTool {

Review comment:
       java docs.

##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/repair/RepairUtils.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.table.repair;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+
+import java.io.IOException;
+import java.io.Serializable;
+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;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+
+public final class RepairUtils {
+  public static Map<String, List<String>> tagInstantsOfBaseAndLogFiles(
+      String basePath, String baseFileExtension, List<Path> allPaths) {
+    // Instant time -> Set of base and log file paths
+    Map<String, List<String>> instantToFilesMap = new HashMap<>();
+    allPaths.forEach(path -> {
+      String instantTime = path.toString().endsWith(baseFileExtension)
+          ? FSUtils.getCommitTime(path.getName()) : 
FSUtils.getBaseCommitTimeFromLogPath(path);
+      instantToFilesMap.computeIfAbsent(instantTime, k -> new ArrayList<>());
+      instantToFilesMap.get(instantTime).add(
+          FSUtils.getRelativePartitionPath(new Path(basePath), path));
+    });
+    return instantToFilesMap;
+  }
+
+  public static Option<Set<String>> getBaseAndLogFilePathsFromTimeline(
+      HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    if (!instant.isCompleted()) {
+      throw new HoodieException("Cannot get base and log file paths from "
+          + "instant not completed: " + instant.getTimestamp());
+    }
+
+    switch (instant.getAction()) {
+      case COMMIT_ACTION:
+      case DELTA_COMMIT_ACTION:
+        final HoodieCommitMetadata commitMetadata =
+            HoodieCommitMetadata.fromBytes(
+                timeline.getInstantDetails(instant).get(), 
HoodieCommitMetadata.class);
+        return 
Option.of(commitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream)
+            .map(HoodieWriteStat::getPath).collect(Collectors.toSet()));
+      case REPLACE_COMMIT_ACTION:
+        final HoodieReplaceCommitMetadata replaceCommitMetadata =
+            HoodieReplaceCommitMetadata.fromBytes(
+                timeline.getInstantDetails(instant).get(), 
HoodieReplaceCommitMetadata.class);
+        return 
Option.of(replaceCommitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream)
+            .map(HoodieWriteStat::getPath).collect(Collectors.toSet()));
+      default:
+        return Option.empty();

Review comment:
       should we consider other actions like rollback and savepoint etc.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
##########
@@ -143,15 +149,19 @@ public HoodieArchivedTimeline reload() {
   }
 
   private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
-    final String instantTime  = 
record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
+    final String instantTime = 
record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
     final String action = record.get(ACTION_TYPE_KEY).toString();
+    String actionState = record.get(ACTION_STATE).toString();
     if (loadDetails) {
       getMetadataKey(action).map(key -> {
         Object actionData = record.get(key);
-        if (action.equals(HoodieTimeline.COMPACTION_ACTION)) {
-          this.readCommits.put(instantTime, 
HoodieAvroUtils.indexedRecordToBytes((IndexedRecord)actionData));
-        } else {
-          this.readCommits.put(instantTime, 
actionData.toString().getBytes(StandardCharsets.UTF_8));
+        // Only puts instant details if the instant is completed and there is 
action data
+        if (HoodieInstant.State.COMPLETED.toString().equals(actionState) && 
actionData != null) {

Review comment:
       looked at the code to ensure that archival will not move past the 
earlist inflight instant in timline. But just inacse someone wants to double 
check, leaving a comment here. if this assumption is wrong, this fix has to be 
reverted.




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