You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Ted Yu (JIRA)" <ji...@apache.org> on 2013/03/11 23:51:13 UTC

[jira] [Updated] (HBASE-8071) TestRestoreFlushSnapshotFromClient fails intermittently in trunk builds

     [ https://issues.apache.org/jira/browse/HBASE-8071?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Ted Yu updated HBASE-8071:
--------------------------

    Description: 
>From https://builds.apache.org/job/HBase-TRUNK/3945/testReport/org.apache.hadoop.hbase.snapshot/TestRestoreFlushSnapshotFromClient/testRestoreSnapshot/:
{code}
2013-03-11 19:35:54,162 DEBUG [IPC Server handler 1 on 51923] master.HMaster(2410): Submitting snapshot request for:{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }
2013-03-11 19:35:54,162 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotDescriptionUtils(235): Creation time not specified, setting to:1363030554162 (current time:1363030554162).
2013-03-11 19:35:54,163 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotManager(465): No existing snapshot, attempting snapshot...
...
2013-03-11 19:36:52,139 DEBUG [pool-1-thread-1] client.HBaseAdmin(2234): Getting current status of snapshot from master...
2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] master.HMaster(2481): Checking to see if snapshot from request:{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } is done
2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotManager(344): Snapshoting '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' is still in progress!
2013-03-11 19:36:52,140 DEBUG [pool-1-thread-1] client.HBaseAdmin(2226): (#16) Sleeping: 8000ms while waiting for snapshot completion.
2013-03-11 19:36:55,740 DEBUG [Timer-28] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	... 2 more
2013-03-11 19:36:55,759 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(96): Creating scanner over .META. starting at key 'testtb-1363030509407,,'
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(209): Advancing internal scanner to startKey at 'testtb-1363030509407,,'
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,764 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(196): Finished with scanning at {NAME => '.META.,,1', STARTKEY => '', ENDKEY => '', ENCODED => 1028785192,}
2013-03-11 19:36:55,790 ERROR [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(152): Failed taking snapshot { ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } due to exception:No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
org.apache.hadoop.hbase.exceptions.CorruptedSnapshotException: No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegion(MasterSnapshotVerifier.java:166)
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegions(MasterSnapshotVerifier.java:151)
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifySnapshot(MasterSnapshotVerifier.java:114)
	at org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler.process(TakeSnapshotHandler.java:145)
	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:130)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
	at java.lang.Thread.run(Thread.java:722)
2013-03-11 19:36:55,791 INFO  [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(203): Stop taking snapshot={ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } because: Failed to take snapshot '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' due to exception
2013-03-11 19:36:55,791 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(159): Launching cleanup of working dir:hdfs://localhost:50807/user/jenkins/hbase/.snapshot/.tmp/snaptb1-1363030509407
2013-03-11 19:36:55,838 DEBUG [Timer-31] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@4baf3db1:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	... 2 more
2013-03-11 19:36:55,839 DEBUG [Timer-31] procedure.ZKProcedureMemberRpcs(285): Aborting procedure (snaptb1-1363030509407) in zk
2013-03-11 19:36:56,061 ERROR [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(225): Procedure 'snaptb1-1363030509407' execution failed!
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:85)
	at org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:371)
	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:215)
	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
	at java.util.concurrent.FutureTask.run(FutureTask.java:166)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
	at java.lang.Thread.run(Thread.java:722)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(228): Running finish phase.
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(279): Finished coordinator procedure - removing self from list of running procedures
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureCoordinatorRpcs(142): Attempting to clean out zk node for op:snaptb1-1363030509407
2013-03-11 19:36:56,061 INFO  [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureUtil(279): Clearing all znodes for procedure snaptb1-1363030509407including nodes /hbase/online-snapshot/acquired /hbase/online-snapshot/reached /hbase/online-snapshot/abort
2013-03-11 19:36:56,080 DEBUG [RegionServer:1;janus.apache.org,59357,1363030252847-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:59357-0x13d5aef12b70002 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
2013-03-11 19:36:56,080 DEBUG [RegionServer:2;janus.apache.org,37320,1363030252872-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:37320-0x13d5aef12b70003 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
{code}

  was:
>From https://builds.apache.org/job/HBase-TRUNK/3945/testReport/org.apache.hadoop.hbase.snapshot/TestRestoreFlushSnapshotFromClient/testRestoreSnapshot/:
{code}
2013-03-11 19:36:52,139 DEBUG [pool-1-thread-1] client.HBaseAdmin(2234): Getting current status of snapshot from master...
2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] master.HMaster(2481): Checking to see if snapshot from request:{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } is done
2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotManager(344): Snapshoting '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' is still in progress!
2013-03-11 19:36:52,140 DEBUG [pool-1-thread-1] client.HBaseAdmin(2226): (#16) Sleeping: 8000ms while waiting for snapshot completion.
2013-03-11 19:36:55,740 DEBUG [Timer-28] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	... 2 more
2013-03-11 19:36:55,759 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(96): Creating scanner over .META. starting at key 'testtb-1363030509407,,'
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(209): Advancing internal scanner to startKey at 'testtb-1363030509407,,'
2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
2013-03-11 19:36:55,764 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(196): Finished with scanning at {NAME => '.META.,,1', STARTKEY => '', ENDKEY => '', ENCODED => 1028785192,}
2013-03-11 19:36:55,790 ERROR [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(152): Failed taking snapshot { ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } due to exception:No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
org.apache.hadoop.hbase.exceptions.CorruptedSnapshotException: No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegion(MasterSnapshotVerifier.java:166)
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegions(MasterSnapshotVerifier.java:151)
	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifySnapshot(MasterSnapshotVerifier.java:114)
	at org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler.process(TakeSnapshotHandler.java:145)
	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:130)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
	at java.lang.Thread.run(Thread.java:722)
2013-03-11 19:36:55,791 INFO  [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(203): Stop taking snapshot={ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } because: Failed to take snapshot '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' due to exception
2013-03-11 19:36:55,791 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(159): Launching cleanup of working dir:hdfs://localhost:50807/user/jenkins/hbase/.snapshot/.tmp/snaptb1-1363030509407
2013-03-11 19:36:55,838 DEBUG [Timer-31] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@4baf3db1:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	... 2 more
2013-03-11 19:36:55,839 DEBUG [Timer-31] procedure.ZKProcedureMemberRpcs(285): Aborting procedure (snaptb1-1363030509407) in zk
2013-03-11 19:36:56,061 ERROR [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(225): Procedure 'snaptb1-1363030509407' execution failed!
org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:85)
	at org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:371)
	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:215)
	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
	at java.util.concurrent.FutureTask.run(FutureTask.java:166)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
	at java.lang.Thread.run(Thread.java:722)
Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
	at java.util.TimerThread.mainLoop(Timer.java:555)
	at java.util.TimerThread.run(Timer.java:505)
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(228): Running finish phase.
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(279): Finished coordinator procedure - removing self from list of running procedures
2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureCoordinatorRpcs(142): Attempting to clean out zk node for op:snaptb1-1363030509407
2013-03-11 19:36:56,061 INFO  [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureUtil(279): Clearing all znodes for procedure snaptb1-1363030509407including nodes /hbase/online-snapshot/acquired /hbase/online-snapshot/reached /hbase/online-snapshot/abort
2013-03-11 19:36:56,080 DEBUG [RegionServer:1;janus.apache.org,59357,1363030252847-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:59357-0x13d5aef12b70002 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
2013-03-11 19:36:56,080 DEBUG [RegionServer:2;janus.apache.org,37320,1363030252872-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:37320-0x13d5aef12b70003 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
{code}

    
> TestRestoreFlushSnapshotFromClient fails intermittently in trunk builds
> -----------------------------------------------------------------------
>
>                 Key: HBASE-8071
>                 URL: https://issues.apache.org/jira/browse/HBASE-8071
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Ted Yu
>
> From https://builds.apache.org/job/HBase-TRUNK/3945/testReport/org.apache.hadoop.hbase.snapshot/TestRestoreFlushSnapshotFromClient/testRestoreSnapshot/:
> {code}
> 2013-03-11 19:35:54,162 DEBUG [IPC Server handler 1 on 51923] master.HMaster(2410): Submitting snapshot request for:{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }
> 2013-03-11 19:35:54,162 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotDescriptionUtils(235): Creation time not specified, setting to:1363030554162 (current time:1363030554162).
> 2013-03-11 19:35:54,163 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotManager(465): No existing snapshot, attempting snapshot...
> ...
> 2013-03-11 19:36:52,139 DEBUG [pool-1-thread-1] client.HBaseAdmin(2234): Getting current status of snapshot from master...
> 2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] master.HMaster(2481): Checking to see if snapshot from request:{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } is done
> 2013-03-11 19:36:52,140 DEBUG [IPC Server handler 1 on 51923] snapshot.SnapshotManager(344): Snapshoting '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' is still in progress!
> 2013-03-11 19:36:52,140 DEBUG [pool-1-thread-1] client.HBaseAdmin(2226): (#16) Sleeping: 8000ms while waiting for snapshot completion.
> 2013-03-11 19:36:55,740 DEBUG [Timer-28] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
> org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
> 	at java.util.TimerThread.mainLoop(Timer.java:555)
> 	at java.util.TimerThread.run(Timer.java:505)
> Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
> 	... 2 more
> 2013-03-11 19:36:55,759 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
> 2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
> 2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
> 2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(96): Creating scanner over .META. starting at key 'testtb-1363030509407,,'
> 2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(209): Advancing internal scanner to startKey at 'testtb-1363030509407,,'
> 2013-03-11 19:36:55,760 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(944): Looking up meta region location in ZK, connection=hconnection 0x25c6f10
> 2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] zookeeper.ZKUtil(1682): hconnection 0x25c6f10-0x13d5aef12b70004 Retrieved 35 byte(s) of data from znode /hbase/meta-region-server; data=janus.apache.org,42570,1363030252791
> 2013-03-11 19:36:55,761 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.HConnectionManager$HConnectionImplementation(949): Looked up meta region location, connection=hconnection 0x25c6f10; serverName=janus.apache.org,42570,1363030252791
> 2013-03-11 19:36:55,764 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] client.ClientScanner(196): Finished with scanning at {NAME => '.META.,,1', STARTKEY => '', ENDKEY => '', ENCODED => 1028785192,}
> 2013-03-11 19:36:55,790 ERROR [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(152): Failed taking snapshot { ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } due to exception:No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
> org.apache.hadoop.hbase.exceptions.CorruptedSnapshotException: No region directory found for region:{NAME => 'testtb-1363030509407,4,1363030509409.8a41cf1a6517ac9f9d4e6aaf2c906588.', STARTKEY => '4', ENDKEY => '5', ENCODED => 8a41cf1a6517ac9f9d4e6aaf2c906588,}
> 	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegion(MasterSnapshotVerifier.java:166)
> 	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifyRegions(MasterSnapshotVerifier.java:151)
> 	at org.apache.hadoop.hbase.master.snapshot.MasterSnapshotVerifier.verifySnapshot(MasterSnapshotVerifier.java:114)
> 	at org.apache.hadoop.hbase.master.snapshot.TakeSnapshotHandler.process(TakeSnapshotHandler.java:145)
> 	at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:130)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> 	at java.lang.Thread.run(Thread.java:722)
> 2013-03-11 19:36:55,791 INFO  [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(203): Stop taking snapshot={ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH } because: Failed to take snapshot '{ ss=snaptb1-1363030509407 table=testtb-1363030509407 type=FLUSH }' due to exception
> 2013-03-11 19:36:55,791 DEBUG [MASTER_TABLE_OPERATIONS-janus.apache.org,51923,1363030251194-0] snapshot.TakeSnapshotHandler(159): Launching cleanup of working dir:hdfs://localhost:50807/user/jenkins/hbase/.snapshot/.tmp/snaptb1-1363030509407
> 2013-03-11 19:36:55,838 DEBUG [Timer-31] errorhandling.ForeignExceptionDispatcher(68):  accepting received exception
> org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@4baf3db1:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:71)
> 	at java.util.TimerThread.mainLoop(Timer.java:555)
> 	at java.util.TimerThread.run(Timer.java:505)
> Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555838, End:1363030615838, diff:60000, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
> 	... 2 more
> 2013-03-11 19:36:55,839 DEBUG [Timer-31] procedure.ZKProcedureMemberRpcs(285): Aborting procedure (snaptb1-1363030509407) in zk
> 2013-03-11 19:36:56,061 ERROR [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(225): Procedure 'snaptb1-1363030509407' execution failed!
> org.apache.hadoop.hbase.errorhandling.TimeoutException via timer-java.util.Timer@1ed8384b:org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher.rethrowException(ForeignExceptionDispatcher.java:85)
> 	at org.apache.hadoop.hbase.procedure.Procedure.waitForLatch(Procedure.java:371)
> 	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:215)
> 	at org.apache.hadoop.hbase.procedure.Procedure.call(Procedure.java:68)
> 	at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> 	at java.lang.Thread.run(Thread.java:722)
> Caused by: org.apache.hadoop.hbase.errorhandling.TimeoutException: Timeout elapsed! Source:Timeout caused Foreign Exception Start:1363030555739, End:1363030615740, diff:60001, max:60000 ms
> 	at org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector$1.run(TimeoutExceptionInjector.java:68)
> 	at java.util.TimerThread.mainLoop(Timer.java:555)
> 	at java.util.TimerThread.run(Timer.java:505)
> 2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(228): Running finish phase.
> 2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.Procedure(279): Finished coordinator procedure - removing self from list of running procedures
> 2013-03-11 19:36:56,061 DEBUG [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureCoordinatorRpcs(142): Attempting to clean out zk node for op:snaptb1-1363030509407
> 2013-03-11 19:36:56,061 INFO  [(janus.apache.org,51923,1363030251194)-proc-coordinator-pool1-thread-1] procedure.ZKProcedureUtil(279): Clearing all znodes for procedure snaptb1-1363030509407including nodes /hbase/online-snapshot/acquired /hbase/online-snapshot/reached /hbase/online-snapshot/abort
> 2013-03-11 19:36:56,080 DEBUG [RegionServer:1;janus.apache.org,59357,1363030252847-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:59357-0x13d5aef12b70002 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
> 2013-03-11 19:36:56,080 DEBUG [RegionServer:2;janus.apache.org,37320,1363030252872-EventThread] zookeeper.ZooKeeperWatcher(274): regionserver:37320-0x13d5aef12b70003 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/hbase/online-snapshot/abort/snaptb1-1363030509407
> {code}

--
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