HBASE-18905 addendum fix hanging TestFlushLifeCycleTracker
This commit is contained in:
parent
82b1c320f0
commit
090c10f2a4
|
@ -45,7 +45,6 @@ import org.junit.After;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
|
@ -53,13 +52,12 @@ import org.junit.experimental.categories.Category;
|
|||
* Confirm that the function of FlushLifeCycleTracker is OK as we do not use it in our own code.
|
||||
*/
|
||||
@Category({ CoprocessorTests.class, MediumTests.class })
|
||||
@Ignore
|
||||
public class TestFlushLifeCycleTracker {
|
||||
|
||||
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
|
||||
|
||||
private static final TableName NAME =
|
||||
TableName.valueOf(TestCompactionLifeCycleTracker.class.getSimpleName());
|
||||
TableName.valueOf(TestFlushLifeCycleTracker.class.getSimpleName());
|
||||
|
||||
private static final byte[] CF = Bytes.toBytes("CF");
|
||||
|
||||
|
@ -116,11 +114,11 @@ public class TestFlushLifeCycleTracker {
|
|||
CountDownLatch arrive = ARRIVE;
|
||||
if (arrive != null) {
|
||||
arrive.countDown();
|
||||
}
|
||||
try {
|
||||
BLOCK.await();
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
try {
|
||||
BLOCK.await();
|
||||
} catch (InterruptedException e) {
|
||||
throw new InterruptedIOException();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue