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

ReplicationSyncUp doesn't clean up its ZK, needed for tests

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 0.96.1, 0.94.16
    • 0.96.2, 0.98.1, 0.99.0, 0.94.17
    • Replication
    • None
    • Reviewed

    Description

      TestReplicationSyncUpTool failed again:
      https://builds.apache.org/job/HBase-TRUNK/4895/testReport/junit/org.apache.hadoop.hbase.replication/TestReplicationSyncUpTool/testSyncUpTool/

      It's not super obvious why only one of the two tables is replicated, the test could use some more logging, but I understand it this way:

      The first ReplicationSyncUp gets started and for some reason it cannot replicate the data:

      2014-02-06 21:32:19,811 INFO  [Thread-1372] regionserver.ReplicationSourceManager(203): Current list of replicators: [1391722339091.SyncUpTool.replication.org,1234,1, quirinus.apache.org,37045,1391722237951, quirinus.apache.org,33502,1391722238125] other RSs: []
      2014-02-06 21:32:19,811 INFO  [Thread-1372.replicationSource,1] regionserver.ReplicationSource(231): Replicating db42e7fc-7f29-4038-9292-d85ea8b9994b -> 783c0ab2-4ff9-4dc0-bb38-86bf31d1d817
      2014-02-06 21:32:19,892 TRACE [Thread-1372.replicationSource,2] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 1
      2014-02-06 21:32:19,911 TRACE [Thread-1372.replicationSource,1] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 1
      2014-02-06 21:32:20,094 TRACE [Thread-1372.replicationSource,2] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 2
      ...
      2014-02-06 21:32:23,414 TRACE [Thread-1372.replicationSource,1] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 8
      2014-02-06 21:32:23,673 INFO  [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl(169): Moving quirinus.apache.org,37045,1391722237951's hlogs to my queue
      2014-02-06 21:32:23,768 DEBUG [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl(396): Creating quirinus.apache.org%2C37045%2C1391722237951.1391722243779 with data 10803
      2014-02-06 21:32:23,842 DEBUG [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl(396): Creating quirinus.apache.org%2C37045%2C1391722237951.1391722243779 with data 10803
      2014-02-06 21:32:24,297 TRACE [Thread-1372.replicationSource,2] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 9
      2014-02-06 21:32:24,314 TRACE [Thread-1372.replicationSource,1] regionserver.ReplicationSource(596): No log to process, sleeping 100 times 9
      

      Finally it gives up:

      2014-02-06 21:32:30,873 DEBUG [Thread-1372] replication.TestReplicationSyncUpTool(323): SyncUpAfterDelete failed at retry = 0, with rowCount_ht1TargetPeer1 =100 and rowCount_ht2TargetAtPeer1 =200
      

      The syncUp tool has an ID you can follow, grep for syncupReplication1391722338885 or just the timestamp, and you can see it doing things after that. The reason is that the tool closes the ReplicationSourceManager but not the ZK connection, so events still come in and NodeFailoverWorker still tries to recover queues but then there's nothing to process them.

      Later in the logs you can see:

      2014-02-06 21:32:37,381 INFO  [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl(169): Moving quirinus.apache.org,33502,1391722238125's hlogs to my queue
      2014-02-06 21:32:37,567 INFO  [ReplicationExecutor-0] replication.ReplicationQueuesZKImpl(239): Won't transfer the queue, another RS took care of it because of: KeeperErrorCode = NoNode for /1/replication/rs/quirinus.apache.org,33502,1391722238125/lock
      

      There shouldn't' be any racing, but now someone already moved "quirinus.apache.org,33502,1391722238125" away.

      FWIW I can't even make the test fail on my machine so I'm not 100% sure closing the ZK connection fixes the issue, but at least it's the right thing to do.

      Attachments

        1. HBASE-10249.patch
          1 kB
          Jean-Daniel Cryans

        Activity

          People

            jdcryans Jean-Daniel Cryans
            jdcryans Jean-Daniel Cryans
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: