[ 
https://issues.apache.org/jira/browse/HBASE-14570?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14956220#comment-14956220
 ] 

Heng Chen commented on HBASE-14570:
-----------------------------------

Hi, [~stack] and [~eclark],  After analysis the log stack posted above,  I 
don't think TestHbaseFsck was hanged.

We can see the when the test failed,  jstack about TestHbaseFsck is
{code}
"Time-limited test" daemon prio=10 tid=0x00007f5830aec800 nid=0x6206 waiting on 
condition [0x00007f55fdada000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000007f294f240> (a 
java.util.concurrent.FutureTask)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:425)
        at java.util.concurrent.FutureTask.get(FutureTask.java:187)
        at 
java.util.concurrent.AbstractExecutorService.invokeAll(AbstractExecutorService.java:243)
        at 
org.apache.hadoop.hbase.util.HBaseFsck.checkRegionConsistencyConcurrently(HBaseFsck.java:1876)
        at 
org.apache.hadoop.hbase.util.HBaseFsck.checkAndFixConsistency(HBaseFsck.java:1834)
        at 
org.apache.hadoop.hbase.util.HBaseFsck.onlineConsistencyRepair(HBaseFsck.java:675)
        at org.apache.hadoop.hbase.util.HBaseFsck.onlineHbck(HBaseFsck.java:697)
        at 
org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck(HbckTestingUtil.java:71)
        at 
org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck(HbckTestingUtil.java:43)
        at 
org.apache.hadoop.hbase.util.hbck.HbckTestingUtil.doFsck(HbckTestingUtil.java:38)
        at 
org.apache.hadoop.hbase.util.TestHBaseFsck.testRegionShouldNotBeDeployed(TestHBaseFsck.java:1632)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at java.lang.Thread.run(Thread.java:744)
{code}

The relates code is 
{code}
private void checkRegionConsistencyConcurrently(
    final List<CheckRegionConsistencyWorkItem> workItems)
    throws IOException, KeeperException, InterruptedException {
    if (workItems.isEmpty()) {
      return;  // nothing to check
    }

    List<Future<Void>> workFutures = executor.invokeAll(workItems);
    for(Future<Void> f: workFutures) {
      try {
        f.get(); //blocking here
      } catch(ExecutionException e1) {
{code}

And jstack about thread CheckRegionConsistencyWorkItem is 
{code}
"pool-104-thread-1" prio=10 tid=0x00007f57e8026800 nid=0x65ea waiting on 
condition [0x00007f560aeae000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at 
org.apache.hadoop.hbase.master.ServerManager.closeRegionSilentlyAndWait(ServerManager.java:863)
        at 
org.apache.hadoop.hbase.util.HBaseFsckRepair.closeRegionSilentlyAndWait(HBaseFsckRepair.java:156)
        at 
org.apache.hadoop.hbase.util.HBaseFsckRepair.fixMultiAssignment(HBaseFsckRepair.java:74)
        at 
org.apache.hadoop.hbase.util.HBaseFsck.checkRegionConsistency(HBaseFsck.java:2387)
        at org.apache.hadoop.hbase.util.HBaseFsck.access$900(HBaseFsck.java:192)
        at 
org.apache.hadoop.hbase.util.HBaseFsck$CheckRegionConsistencyWorkItem.call(HBaseFsck.java:1907)
        - locked <0x00000007f294ef10> (a 
org.apache.hadoop.hbase.util.HBaseFsck$CheckRegionConsistencyWorkItem)
        at 
org.apache.hadoop.hbase.util.HBaseFsck$CheckRegionConsistencyWorkItem.call(HBaseFsck.java:1895)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:178)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:292)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)
{code}

And relates code is in {{ServerManager}}
{code}
  /**
   * Contacts a region server and waits up to timeout ms
   * to close the region.  This bypasses the active hmaster.
   */
  public static void closeRegionSilentlyAndWait(ClusterConnection connection, 
    ServerName server, HRegionInfo region, long timeout) throws IOException, 
InterruptedException {
    AdminService.BlockingInterface rs = connection.getAdmin(server);
    ...
    long expiration = timeout + System.currentTimeMillis();
    while (System.currentTimeMillis() < expiration) {
      try {
        HRegionInfo rsRegion = ProtobufUtil.getRegionInfo(rs, 
region.getRegionName());
        if (rsRegion == null) return;
      } catch (IOException ioe) {
         ...
      }
      Thread.sleep(1000); //block here
    }
    throw new IOException("Region " + region + " failed to close within"
        + " timeout " + timeout);
  }
{code}

It means blocking thread is waiting for region to be closed,  it has timeout 
180s.  

But during this waiting time,  mvn test was broken by some failed testcases.  

So we can't see clean up after TestHbaseFsck running.

IMO this is not TestHbaseFack's fault.  But i argee we do spilt this testcase, 
because it is too big.





> Cleanup hanging TestHBaseFsck
> -----------------------------
>
>                 Key: HBASE-14570
>                 URL: https://issues.apache.org/jira/browse/HBASE-14570
>             Project: HBase
>          Issue Type: Sub-task
>          Components: test
>    Affects Versions: 2.0.0
>            Reporter: stack
>         Attachments: HBASE-14570.patch
>
>
> This one hangs regularly. Let me at least add timeouts. Looking in log, a 
> bunch of tests are potentially hanging tests since they don't see to clean up 
> after themselves. Will start watching and just disable likely candidates 
> unless someone wants to have a go at fixing this.



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

Reply via email to