Review Board 1.7.22


HBASE-5099 ZK event thread waiting for root region while server shutdown handler waiting for event thread to finish distributed log splitting to recover the region sever the root region is on

Review Request #3323 - Created Dec. 28, 2011 and submitted

Jimmy Xiang
trunk
HBASE-5099
Reviewers
hbase
stack, tedyu
hbase-git
Per discussion with Ted (on issues), I put up a patch to run tryRecoveringExpiredZKSession() in a separate thread and time it out and fail the recovery if it is stuck somewhere.

I added a test to test the abort method.  However, for the mini cluster, becomeActiveMaster() doesn't succeed so the abort method ends up always aborted.  So the actually success recovery is not tested.
mvn -PlocalTests -Dtest=TestMaster* clean test

Diff revision 2

This is not the most recent revision of the diff. The latest diff is revision 6. See what's changed.

1 2 3 4 5 6
1 2 3 4 5 6

  1. src/main/java/org/apache/hadoop/hbase/master/HMaster.java: Loading...
  2. src/test/java/org/apache/hadoop/hbase/master/TestMaster.java: Loading...
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
Revision a5935a6 New Change
[20] 26 lines
[+20]
27
import java.util.ArrayList;
27
import java.util.ArrayList;
28
import java.util.List;
28
import java.util.List;
29
import java.util.Map;
29
import java.util.Map;
30
import java.util.Set;
30
import java.util.Set;
31
import java.util.concurrent.atomic.AtomicReference;
31
import java.util.concurrent.atomic.AtomicReference;

    
   
32
import java.util.concurrent.Callable;

    
   
33
import java.util.concurrent.ExecutionException;

    
   
34
import java.util.concurrent.Executors;

    
   
35
import java.util.concurrent.Future;

    
   
36
import java.util.concurrent.TimeUnit;
32

    
   
37

   
33
import org.apache.commons.logging.Log;
38
import org.apache.commons.logging.Log;
34
import org.apache.commons.logging.LogFactory;
39
import org.apache.commons.logging.LogFactory;
35
import org.apache.hadoop.conf.Configuration;
40
import org.apache.hadoop.conf.Configuration;
36
import org.apache.hadoop.hbase.Chore;
41
import org.apache.hadoop.hbase.Chore;
[+20] [20] 1360 lines
[+20] [+] public void abort(final String msg, final Throwable t) {
1397
      stop("Aborting");
1402
      stop("Aborting");
1398
    }
1403
    }
1399
  }
1404
  }
1400

    
   
1405

   
1401
  /**
1406
  /**
1402
   * We do the following.
1407
   * We do the following in a different thread.  If it is not completed

    
   
1408
   * in time, we will time it out and assume it is not easy to recover.

    
   
1409
   *
1403
   * 1. Create a new ZK session. (since our current one is expired)
1410
   * 1. Create a new ZK session. (since our current one is expired)
1404
   * 2. Try to become a primary master again
1411
   * 2. Try to become a primary master again
1405
   * 3. Initialize all ZK based system trackers.
1412
   * 3. Initialize all ZK based system trackers.
1406
   * 4. Assign root and meta. (they are already assigned, but we need to update our
1413
   * 4. Assign root and meta. (they are already assigned, but we need to update our
1407
   * internal memory state to reflect it)
1414
   * internal memory state to reflect it)
1408
   * 5. Process any RIT if any during the process of our recovery.
1415
   * 5. Process any RIT if any during the process of our recovery.
1409
   *
1416
   *
1410
   * @return True if we could successfully recover from ZK session expiry.
1417
   * @return True if we could successfully recover from ZK session expiry.
1411
   * @throws InterruptedException
1418
   * @throws InterruptedException
1412
   * @throws IOException
1419
   * @throws IOException

    
   
1420
   * @throws Throwable
1413
   */
1421
   */
1414
  private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
1422
  private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
1415
      IOException, KeeperException {
1423
      IOException, KeeperException, Throwable {

    
   
1424

   
1416
    this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":"
1425
    this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":"
1417
        + this.serverName.getPort(), this, true);
1426
      + this.serverName.getPort(), this, true);
1418

    
   
1427

   
1419
    MonitoredTask status = 
1428
    Callable<Boolean> callable = new Callable<Boolean> () {

    
   
1429
      public Boolean call() throws InterruptedException,

    
   
1430
          IOException, KeeperException {

    
   
1431
        MonitoredTask status =
1420
      TaskMonitor.get().createStatus("Recovering expired ZK session");
1432
          TaskMonitor.get().createStatus("Recovering expired ZK session");
1421
    try {
1433
        try {
1422
      if (!becomeActiveMaster(status)) {
1434
          if (!becomeActiveMaster(status)) {
1423
        return false;
1435
            return Boolean.FALSE;
1424
      }
1436
          }
1425
      initializeZKBasedSystemTrackers();
1437
          initializeZKBasedSystemTrackers();
1426
      // Update in-memory structures to reflect our earlier Root/Meta assignment.
1438
          // Update in-memory structures to reflect our earlier Root/Meta assignment.
1427
      assignRootAndMeta(status);
1439
          assignRootAndMeta(status);
1428
      // process RIT if any
1440
          // process RIT if any
1429
      // TODO: Why does this not call AssignmentManager.joinCluster?  Otherwise
1441
          // TODO: Why does this not call AssignmentManager.joinCluster?  Otherwise
1430
      // we are not processing dead servers if any.
1442
          // we are not processing dead servers if any.
1431
      this.assignmentManager.processDeadServersAndRegionsInTransition();
1443
          assignmentManager.processDeadServersAndRegionsInTransition();
1432
      return true;
1444
          return Boolean.TRUE;
1433
    } finally {
1445
        } finally {
1434
      status.cleanup();
1446
          status.cleanup();
1435
    }
1447
        }
1436
  }
1448
      }

    
   
1449
    };

    
   
1450

   

    
   
1451
    long timeout = conf.getLong("hbase.master.zksession.recover.timeout", 300000);

    
   
1452
    java.util.concurrent.ExecutorService executor = Executors.newSingleThreadExecutor();

    
   
1453
    Future<Boolean> result = executor.submit(callable);

    
   
1454
    executor.shutdown();

    
   
1455
    if (executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)

    
   
1456
        && result.isDone()) {

    
   
1457
      try {

    
   
1458
        Boolean recovered = result.get();

    
   
1459
        if (recovered != null) {

    
   
1460
          return recovered.booleanValue();

    
   
1461
        }

    
   
1462
      } catch (ExecutionException ee) {

    
   
1463
        throw ee.getCause() ;

    
   
1464
      }

    
   
1465
    }

    
   
1466
    return false;

    
   
1467
  }
1437

    
   
1468

   
1438
  /**
1469
  /**
1439
   * Check to see if the current trigger for abort is due to ZooKeeper session
1470
   * Check to see if the current trigger for abort is due to ZooKeeper session
1440
   * expiry, and If yes, whether we can recover from ZK session expiry.
1471
   * expiry, and If yes, whether we can recover from ZK session expiry.
1441
   *
1472
   *
[+20] [20] 262 lines
src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
Revision b7a8270 New Change
 
  1. src/main/java/org/apache/hadoop/hbase/master/HMaster.java: Loading...
  2. src/test/java/org/apache/hadoop/hbase/master/TestMaster.java: Loading...