You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-dev@hadoop.apache.org by "Han Xiao (JIRA)" <ji...@apache.org> on 2012/10/30 10:02:12 UTC

[jira] [Created] (HDFS-4130) The reading for editlog at NN starting using bkjm is not efficient

Han Xiao created HDFS-4130:
------------------------------

             Summary: The reading for editlog at NN starting using bkjm  is not efficient
                 Key: HDFS-4130
                 URL: https://issues.apache.org/jira/browse/HDFS-4130
             Project: Hadoop HDFS
          Issue Type: Improvement
          Components: ha, performance
    Affects Versions: 2.0.2-alpha
            Reporter: Han Xiao


Now, the method of BookKeeperJournalManager.selectInputStreams is written like:

    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;
    }
 
EditLogInputstream is got from getInputStream(), which will read the ledgers from zookeeper in each calling.
This will be a larger cost of times when the the number ledgers becomes large.
The reading of ledgers from zk is not necessary for every calling of getInputStream().

The log of time wasting here is as follows:
2012-10-30 16:44:52,995 INFO org.apache.hadoop.hdfs.server.namenode.NameNode: Caching file names occuring more than 10 times
2012-10-30 16:49:24,643 INFO hidden.bkjournal.org.apache.bookkeeper.proto.PerChannelBookieClient: Successfully connected to bookie: /167.52.1.121:318

The stack of the process when blocking between the two lines of log is like:
"main" prio=10 tid=0x000000004011f000 nid=0x39ba in Object.wait() [0x00007fca020fe000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:485)
        at hidden.bkjournal.org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1253)
        - locked <0x00000006fb8495a8> (a hidden.bkjournal.org.apache.zookeeper.ClientCnxn$Packet)
        at hidden.bkjournal.org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1129)
        at org.apache.hadoop.contrib.bkjournal.utils.RetryableZookeeper.getData(RetryableZookeeper.java:501)
        at hidden.bkjournal.org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:1160)
        at org.apache.hadoop.contrib.bkjournal.EditLogLedgerMetadata.read(EditLogLedgerMetadata.java:113)
        at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.getLedgerList(BookKeeperJournalManager.java:725)
        at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.getInputStream(BookKeeperJournalManager.java:442)
        at org.apache.hadoop.contrib.bkjournal.BookKeeperJournalManager.selectInputStreams(BookKeeperJournalManager.java:480)
        
betweent different time, the diff of stack is:
diff stack stack2
1c1
< 2012-10-30 16:44:53
---
> 2012-10-30 16:46:17
106c106
<       - locked <0x00000006fb8495a8> (a hidden.bkjournal.org.apache.zookeeper.ClientCnxn$Packet)
---
>       - locked <0x00000006fae58468> (a hidden.bkjournal.org.apache.zookeeper.ClientCnxn$Packet)

In our environment, the waiting time could even reach to tens of minutes.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira