Jelajahi Sumber

HDFS-4130. BKJM: The reading for editlog at NN starting using bkjm is not efficient. Contributed by Han Xiao.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1419649 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 12 tahun lalu
induk
melakukan
103eff1fad

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

@@ -414,6 +414,9 @@ Release 2.0.3-alpha - Unreleased
     HDFS-3680. Allow customized audit logging in HDFS FSNamesystem. (Marcelo
     Vanzin via atm)
 
+    HDFS-4130. BKJM: The reading for editlog at NN starting using bkjm is not efficient.
+    (Han Xiao via umamahesh)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 26 - 44
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -500,16 +500,18 @@ public class BookKeeperJournalManager implements JournalManager {
     } 
   }
 
-  EditLogInputStream getInputStream(long fromTxId, boolean inProgressOk)
-      throws IOException {
-    for (EditLogLedgerMetadata l : getLedgerList(inProgressOk)) {
-      long lastTxId = l.getLastTxId();
-      if (l.isInProgress()) {
-        lastTxId = recoverLastTxId(l, false);
-      }
-
-      if (fromTxId >= l.getFirstTxId() && fromTxId <= lastTxId) {
-        try {
+  @Override
+  public void selectInputStreams(Collection<EditLogInputStream> streams,
+      long fromTxId, boolean inProgressOk) throws IOException {
+    List<EditLogLedgerMetadata> currentLedgerList = getLedgerList(inProgressOk);
+    try {
+      BookKeeperEditLogInputStream elis = null;
+      for (EditLogLedgerMetadata l : currentLedgerList) {
+        long lastTxId = l.getLastTxId();
+        if (l.isInProgress()) {
+          lastTxId = recoverLastTxId(l, false);
+        }
+        if (fromTxId >= l.getFirstTxId() && fromTxId <= lastTxId) {
           LedgerHandle h;
           if (l.isInProgress()) { // we don't want to fence the current journal
             h = bkc.openLedgerNoRecovery(l.getLedgerId(),
@@ -518,42 +520,22 @@ public class BookKeeperJournalManager implements JournalManager {
             h = bkc.openLedger(l.getLedgerId(), BookKeeper.DigestType.MAC,
                 digestpw.getBytes());
           }
-          BookKeeperEditLogInputStream s = new BookKeeperEditLogInputStream(h,
-              l);
-          s.skipTo(fromTxId);
-          return s;
-        } catch (BKException e) {
-          throw new IOException("Could not open ledger for " + fromTxId, e);
-        } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted opening ledger for "
-                                         + fromTxId, ie);
+          elis = new BookKeeperEditLogInputStream(h, l);
+          elis.skipTo(fromTxId);
+        } else {
+          return;
         }
+        streams.add(elis);
+        if (elis.getLastTxId() == HdfsConstants.INVALID_TXID) {
+          return;
+        }
+        fromTxId = elis.getLastTxId() + 1;
       }
-    }
-    return null;
-  }
-
-  @Override
-  public void selectInputStreams(Collection<EditLogInputStream> streams,
-      long fromTxId, boolean inProgressOk) {
-    // NOTE: could probably be rewritten more efficiently
-    while (true) {
-      EditLogInputStream elis;
-      try {
-        elis = getInputStream(fromTxId, inProgressOk);
-      } catch (IOException e) {
-        LOG.error(e);
-        return;
-      }
-      if (elis == null) {
-        return;
-      }
-      streams.add(elis);
-      if (elis.getLastTxId() == HdfsConstants.INVALID_TXID) {
-        return;
-      }
-      fromTxId = elis.getLastTxId() + 1;
+    } catch (BKException e) {
+      throw new IOException("Could not open ledger for " + fromTxId, e);
+    } catch (InterruptedException ie) {
+      Thread.currentThread().interrupt();
+      throw new IOException("Interrupted opening ledger for " + fromTxId, ie);
     }
   }
 

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperJournalManager.java

@@ -28,6 +28,7 @@ import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Random;
@@ -315,13 +316,13 @@ public class TestBookKeeperJournalManager {
     out.close();
     bkjm.finalizeLogSegment(1, numTransactions);
 
-     
-    EditLogInputStream in = bkjm.getInputStream(1, true);
+    List<EditLogInputStream> in = new ArrayList<EditLogInputStream>();
+    bkjm.selectInputStreams(in, 1, true);
     try {
       assertEquals(numTransactions, 
-                   FSEditLogTestUtil.countTransactionsInStream(in));
+                   FSEditLogTestUtil.countTransactionsInStream(in.get(0)));
     } finally {
-      in.close();
+      in.get(0).close();
     }
   }