zhangduo created HBASE-13136:
--------------------------------

             Summary: TestSplitLogManager.testGetPreviousRecoveryMode is flakey
                 Key: HBASE-13136
                 URL: https://issues.apache.org/jira/browse/HBASE-13136
             Project: HBase
          Issue Type: Bug
            Reporter: zhangduo


Add test code to run it 100 times then we can make it fail always.
{code:title=TestSplitLogManager.java}
  @Test
  public void test() throws Exception {
    for (int i = 0; i < 100; i++) {
      setup();
      testGetPreviousRecoveryMode();
      teardown();
    }
  }
{code}

Add then add some ugly debug logs(Yeah I usually debug in this way...)
{code:title=ZKSplitLogManagerCoordination.java}
  @Override
  public void setRecoveryMode(boolean isForInitialization) throws IOException {
    synchronized(this) {
      if (this.isDrainingDone) {
        // when there is no outstanding splitlogtask after master start up, we 
already have up to 
        // date recovery mode
        return;
      }
    }
    if (this.watcher == null) {
      // when watcher is null(testing code) and recovery mode can only be 
LOG_SPLITTING
      synchronized(this) {
        this.isDrainingDone = true;
        this.recoveryMode = RecoveryMode.LOG_SPLITTING;
      }
      return;
    }
    boolean hasSplitLogTask = false;
    boolean hasRecoveringRegions = false;
    RecoveryMode previousRecoveryMode = RecoveryMode.UNKNOWN;
    RecoveryMode recoveryModeInConfig =
        (isDistributedLogReplay(conf)) ? RecoveryMode.LOG_REPLAY : 
RecoveryMode.LOG_SPLITTING;

    // Firstly check if there are outstanding recovering regions
    try {
      List<String> regions = ZKUtil.listChildrenNoWatch(watcher, 
watcher.recoveringRegionsZNode);
      LOG.debug("=======" + regions);
      if (regions != null && !regions.isEmpty()) {
        hasRecoveringRegions = true;
        previousRecoveryMode = RecoveryMode.LOG_REPLAY;
      }
      if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
        // Secondly check if there are outstanding split log task
        List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, 
watcher.splitLogZNode);
        LOG.debug("=======" + tasks);
        if (tasks != null && !tasks.isEmpty()) {
          hasSplitLogTask = true;
          if (isForInitialization) {
            // during initialization, try to get recovery mode from splitlogtask
            int listSize = tasks.size();
            for (int i = 0; i < listSize; i++) {
              String task = tasks.get(i);
              try {
                byte[] data =
                    ZKUtil.getData(this.watcher, 
ZKUtil.joinZNode(watcher.splitLogZNode, task));
                if (data == null) continue;
                SplitLogTask slt = SplitLogTask.parseFrom(data);
                previousRecoveryMode = slt.getMode();
                if (previousRecoveryMode == RecoveryMode.UNKNOWN) {
                  // created by old code base where we don't set recovery mode 
in splitlogtask
                  // we can safely set to LOG_SPLITTING because we're in master 
initialization code
                  // before SSH is enabled & there is no outstanding recovering 
regions
                  previousRecoveryMode = RecoveryMode.LOG_SPLITTING;
                }
                break;
              } catch (DeserializationException e) {
                LOG.warn("Failed parse data for znode " + task, e);
              } catch (InterruptedException e) {
                throw new InterruptedIOException();
              }
            }
          }
        }
      }
    } catch (KeeperException e) {
      throw new IOException(e);
    }

    synchronized (this) {
      if (this.isDrainingDone) {
        return;
      }
      if (!hasSplitLogTask && !hasRecoveringRegions) {
        this.isDrainingDone = true;
        LOG.debug("====set to " + recoveryModeInConfig);
        this.recoveryMode = recoveryModeInConfig;
        return;
      } else if (!isForInitialization) {
        // splitlogtask hasn't drained yet, keep existing recovery mode
        return;
      }

      if (previousRecoveryMode != RecoveryMode.UNKNOWN) {
        LOG.debug("====set to " + previousRecoveryMode);
        this.isDrainingDone = (previousRecoveryMode == recoveryModeInConfig);
        this.recoveryMode = previousRecoveryMode;
      } else {
        LOG.debug("====set to " + recoveryModeInConfig);
        this.recoveryMode = recoveryModeInConfig;
      }
    }
  }
{code}

When failing, I got this
{noformat}
2015-03-02 12:26:12,555 INFO  [main] master.TestSplitLogManager(654): 
testGetPreviousRecoveryMode
2015-03-02 12:26:12,600 DEBUG [main-EventThread] 
zookeeper.ZooKeeperWatcher(388): 
split-log-manager-tests465da653-db4c-475b-bf73-91ac0ca13323-0x14bd8bbf5d90000, 
quorum=localhost:53589, baseZNode=/hbase Received ZooKeeper Event, type=None, 
state=Disconnected, path=null
2015-03-02 12:26:12,600 DEBUG [main-EventThread] 
zookeeper.ZooKeeperWatcher(477): 
split-log-manager-tests465da653-db4c-475b-bf73-91ac0ca13323-0x14bd8bbf5d90000, 
quorum=localhost:53589, baseZNode=/hbase Received Disconnected from ZooKeeper, 
ignoring
2015-03-02 12:26:12,621 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(875): Distributed log replay=true
2015-03-02 12:26:12,626 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(798): =======null
2015-03-02 12:26:12,627 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(806): =======[testRecovery]
2015-03-02 12:26:12,628 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(855): ====set to LOG_SPLITTING
2015-03-02 12:26:12,629 INFO  [main] 
coordination.ZKSplitLogManagerCoordination(594): found orphan task testRecovery
2015-03-02 12:26:12,630 INFO  [main] 
coordination.ZKSplitLogManagerCoordination(598): Found 1 orphan tasks and 0 
rescan nodes
2015-03-02 12:26:12,631 DEBUG [main-EventThread] 
coordination.ZKSplitLogManagerCoordination(464): task not yet acquired 
/hbase/splitWAL/testRecovery ver = 0
2015-03-02 12:26:12,631 INFO  [main-EventThread] 
coordination.ZKSplitLogManagerCoordination(548): creating orphan task 
/hbase/splitWAL/testRecovery
2015-03-02 12:26:12,631 INFO  [main-EventThread] 
coordination.ZKSplitLogManagerCoordination(178): resubmitting unassigned orphan 
task /hbase/splitWAL/testRecovery
2015-03-02 12:26:12,631 INFO  [main-EventThread] 
coordination.ZKSplitLogManagerCoordination(229): resubmitting task 
/hbase/splitWAL/testRecovery
2015-03-02 12:26:12,632 INFO  [main] master.TestSplitLogManager(665): 
Mode1=LOG_SPLITTING
2015-03-02 12:26:12,633 DEBUG [main-EventThread] 
zookeeper.ZooKeeperWatcher(388): 
split-log-manager-tests9317eabf-735c-4a9d-bc5b-8a9269bb6d62-0x14bd8bbf8be0000, 
quorum=localhost:55209, baseZNode=/hbase Received ZooKeeper Event, 
type=NodeDataChanged, state=SyncConnected, path=/hbase/splitWAL/testRecovery
2015-03-02 12:26:12,639 INFO  [main] master.TestSplitLogManager(668): 
Mode2=LOG_SPLITTING
2015-03-02 12:26:12,640 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(875): Distributed log replay=true
2015-03-02 12:26:12,641 WARN  [main-EventThread] 
coordination.ZKSplitLogManagerCoordination$GetDataAsyncCallback(1000): task 
znode /hbase/splitWAL/testRecovery vanished or not created yet.
2015-03-02 12:26:12,641 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(798): =======null
2015-03-02 12:26:12,642 INFO  [main-EventThread] 
coordination.ZKSplitLogManagerCoordination(472): task 
/hbase/splitWAL/RESCAN0000000001 entered state: DONE dummy-master,1,1
2015-03-02 12:26:12,642 DEBUG [main] 
coordination.ZKSplitLogManagerCoordination(806): =======[RESCAN0000000001]
2015-03-02 12:26:12,642 INFO  [main] master.TestSplitLogManager(670): 
Mode3=LOG_SPLITTING
2015-03-02 12:26:12,642 INFO  [main] hbase.ChoreService(303): Chore service 
for: dummy-master,1,1_splitLogManager_ had 
[java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@35cd95e8] 
on shutdown
2015-03-02 12:26:12,643 INFO  [main] zookeeper.MiniZooKeeperCluster(244): 
Shutdown MiniZK cluster with all ZK servers
{noformat}

We got some tasks here and cause hasSplitLogTask to be true.
{code:title=}
        List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, 
watcher.splitLogZNode);
        LOG.debug("=======" + tasks);
        if (tasks != null && !tasks.isEmpty()) {
          hasSplitLogTask = true;
{code}
So we skipped the mode setting stage and returned.

Not sure if it is a race condition in ZKSplitLogManagerCoordination or just a 
unit test issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to