aaronbee commented on code in PR #6266:
URL: https://github.com/apache/hbase/pull/6266#discussion_r2011059633


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RecoveredEditsOutputSink.java:
##########
@@ -89,24 +89,39 @@ private RecoveredEditsWriter 
getRecoveredEditsWriter(TableName tableName, byte[]
 
   @Override
   public List<Path> close() throws IOException {
-    boolean isSuccessful = true;
+    boolean isSuccessful;
     try {
       isSuccessful = finishWriterThreads();
-    } finally {
-      isSuccessful &= closeWriters();
+    } catch (IOException e) {
+      closeWriters(false);
+      throw e;
+    }
+    if (!isSuccessful) {
+      // Even if an exception is not thrown, finishWriterThreads() not being 
successful is an
+      // error case where the WAL files should not be finalized.
+      closeWriters(false);
+      return null;
     }
+    isSuccessful = closeWriters(true);
     return isSuccessful ? splits : null;
   }
 
   /**
-   * Close all of the output streams.
+   * Close all the output streams.
+   * @param finalizeEdits true in the successful close case, false when we 
don't want to rename and
+   *                      finalize the temporary, possibly corrupted WAL 
files, such as when there
+   *                      was a previous failure or exception. Please see 
HBASE-28569.
    * @return true when there is no error.
    */
-  private boolean closeWriters() throws IOException {
+  boolean closeWriters(boolean finalizeEdits) throws IOException {
     List<IOException> thrown = Lists.newArrayList();
     for (RecoveredEditsWriter writer : writers.values()) {
       closeCompletionService.submit(() -> {
-        Path dst = closeRecoveredEditsWriter(writer, thrown);
+        if (!finalizeEdits) {
+          abortRecoveredEditsWriter(writer, thrown);
+          return null;
+        }
+        Path dst = closeRecoveredEditsWriterAndFinalizeEdits(writer, thrown);
         LOG.trace("Closed {}", dst);

Review Comment:
   Add a similar log to the `!finalizeEdits` case?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractRecoveredEditsOutputSink.java:
##########
@@ -73,33 +73,27 @@ protected RecoveredEditsWriter 
createRecoveredEditsWriter(TableName tableName, b
     return new RecoveredEditsWriter(region, regionEditsPath, w, seqId);
   }
 
-  protected Path closeRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
+  /**
+   * abortRecoveredEditsWriter closes the editsWriter, but does not rename and 
finalize the
+   * recovered edits WAL files. Please see HBASE-28569.
+   */
+  protected void abortRecoveredEditsWriter(RecoveredEditsWriter editsWriter,
     List<IOException> thrown) throws IOException {
-    try {
-      editsWriter.writer.close();
-    } catch (IOException ioe) {
-      final String errorMsg = "Could not close recovered edits at " + 
editsWriter.path;
-      LOG.error(errorMsg, ioe);
-      updateStatusWithMsg(errorMsg);
-      thrown.add(ioe);
+    closeRecoveredEditsWriter(editsWriter, thrown);
+    if (editsWriter.editsWritten == 0) {
+      // just remove the empty recovered.edits file
+      removeRecoveredEditsFile(editsWriter);
+    }

Review Comment:
   The recovered edits file is incomplete at this point, correct? Should we 
just always delete it?



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