Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-5635

If getTaskList() returns null, splitlogWorker would go down and it won't serve any requests

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.92.1
    • 0.94.0, 0.95.0
    • wal
    • None
    • Reviewed

    Description

      During the hlog split operation if all the zookeepers are down ,then the paths will be returned as null and the splitworker thread wil be exited
      Now this regionserver wil not be able to acquire any other tasks since the splitworker thread is exited
      Please find the attached code for more details

      private List<String> getTaskList() {
          for (int i = 0; i < zkretries; i++) {
            try {
              return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
                  this.watcher.splitLogZNode));
            } catch (KeeperException e) {
              LOG.warn("Could not get children of znode " +
                  this.watcher.splitLogZNode, e);
              try {
                Thread.sleep(1000);
              } catch (InterruptedException e1) {
                LOG.warn("Interrupted while trying to get task list ...", e1);
                Thread.currentThread().interrupt();
                return null;
              }
            }
          }
      

      in the org.apache.hadoop.hbase.regionserver.SplitLogWorker

      Attachments

        1. HBASE-5635_0.94.patch
          2 kB
          ramkrishna.s.vasudevan
        2. HBASE-5635._trunk.patch
          2 kB
          ramkrishna.s.vasudevan
        3. HBASE-5635.2.patch
          2 kB
          Chinna Rao Lalam
        4. HBASE-5635.1.patch
          2 kB
          Chinna Rao Lalam
        5. HBASE-5635.patch
          2 kB
          Chinna Rao Lalam

        Activity

          People

            chinnalalam Chinna Rao Lalam
            kristamswathi Kristam Subba Swathi
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: