Browse Source

HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang via Colin P. McCabe)

(cherry picked from commit 7b424f938c3c306795d574792b086d84e4f06425)
(cherry picked from commit cbd11681ce8a51d187d91748b67a708681e599de)
Colin Patrick Mccabe 10 năm trước cách đây
mục cha
commit
2e86d1f254

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -29,6 +29,10 @@ Release 2.6.1 - UNRELEASED
 
   OPTIMIZATIONS
 
+    HDFS-8480. Fix performance and timeout issues in HDFS-7929 by using
+    hard-links to preserve old edit logs, instead of copying them. (Zhe Zhang
+    via Colin P. McCabe)
+
   BUG FIXES
 
     HDFS-7425. NameNode block deletion logging uses incorrect appender.

+ 2 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNUpgradeUtil.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.File;
 import java.io.FilenameFilter;
 import java.io.IOException;
+import java.nio.file.Files;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
@@ -130,23 +131,8 @@ public abstract class NNUpgradeUtil {
 
     for (String s : fileNameList) {
       File prevFile = new File(tmpDir, s);
-      Preconditions.checkState(prevFile.canRead(),
-          "Edits log file " + s + " is not readable.");
       File newFile = new File(curDir, prevFile.getName());
-      Preconditions.checkState(newFile.createNewFile(),
-          "Cannot create new edits log file in " + curDir);
-      EditLogFileInputStream in = new EditLogFileInputStream(prevFile);
-      EditLogFileOutputStream out =
-          new EditLogFileOutputStream(conf, newFile, 512*1024);
-      FSEditLogOp logOp = in.nextValidOp();
-      while (logOp != null) {
-        out.write(logOp);
-        logOp = in.nextOp();
-      }
-      out.setReadyToFlush();
-      out.flushAndSync(true);
-      out.close();
-      in.close();
+      Files.createLink(newFile.toPath(), prevFile.toPath());
     }
   }
 

+ 43 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -32,13 +32,17 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedList;
+import java.nio.file.Files;
 import java.util.List;
 import java.util.regex.Pattern;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.inotify.Event;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -46,7 +50,11 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.namenode.TestParallelImageWrite;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils;
@@ -54,6 +62,8 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
+
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 
@@ -466,29 +476,49 @@ public class TestDFSUpgrade {
     log("Normal NameNode upgrade", 1);
     File[] created =
         UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
-    List<String> beforeUpgrade = new LinkedList<String>();
     for (final File createdDir : created) {
       String[] fileNameList = createdDir.list(EditLogsFilter.INSTANCE);
-      Collections.addAll(beforeUpgrade, fileNameList);
+      for (String fileName : fileNameList) {
+        String tmpFileName = fileName + ".tmp";
+        File existingFile = new File(createdDir, fileName);
+        File tmpFile = new File(createdDir, tmpFileName);
+        Files.move(existingFile.toPath(), tmpFile.toPath());
+        File newFile = new File(createdDir, fileName);
+        Preconditions.checkState(newFile.createNewFile(),
+            "Cannot create new edits log file in " + createdDir);
+        EditLogFileInputStream in = new EditLogFileInputStream(tmpFile,
+            HdfsConstants.INVALID_TXID, HdfsConstants.INVALID_TXID,
+            false);
+        EditLogFileOutputStream out = new EditLogFileOutputStream(conf, newFile,
+            (int)tmpFile.length());
+        out.create(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION + 1);
+        FSEditLogOp logOp = in.readOp();
+        while (logOp != null) {
+          out.write(logOp);
+          logOp = in.readOp();
+        }
+        out.setReadyToFlush();
+        out.flushAndSync(true);
+        out.close();
+        Files.delete(tmpFile.toPath());
+      }
     }
 
     cluster = createCluster();
 
-    List<String> afterUpgrade = new LinkedList<String>();
-    for (final File createdDir : created) {
-      String[] fileNameList = createdDir.list(EditLogsFilter.INSTANCE);
-      Collections.addAll(afterUpgrade, fileNameList);
-    }
-
-    for (String s : beforeUpgrade) {
-      assertTrue(afterUpgrade.contains(s));
-    }
-
+    DFSInotifyEventInputStream ieis =
+        cluster.getFileSystem().getInotifyEventStream(0);
+    EventBatch batch = ieis.poll();
+    Event[] events = batch.getEvents();
+    assertTrue("Should be able to get transactions before the upgrade.",
+        events.length > 0);
+    assertEquals(events[0].getEventType(), Event.EventType.CREATE);
+    assertEquals(((CreateEvent) events[0]).getPath(), "/TestUpgrade");
     cluster.shutdown();
     UpgradeUtilities.createEmptyDirs(nameNodeDirs);
   }
 
-  private static enum EditLogsFilter implements FilenameFilter {
+  private enum EditLogsFilter implements FilenameFilter {
     INSTANCE;
 
     @Override