mirror of https://github.com/apache/druid.git
Merge pull request #1060 from metamx/fix-test-timeout
fix BatchServerInventoryViewTest test timeout
This commit is contained in:
commit
7082487075
|
@ -42,7 +42,9 @@ import org.apache.curator.framework.CuratorFramework;
|
||||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||||
import org.apache.curator.retry.ExponentialBackoffRetry;
|
import org.apache.curator.retry.ExponentialBackoffRetry;
|
||||||
import org.apache.curator.test.TestingCluster;
|
import org.apache.curator.test.TestingCluster;
|
||||||
|
import org.apache.curator.test.Timing;
|
||||||
import org.easymock.EasyMock;
|
import org.easymock.EasyMock;
|
||||||
|
import org.easymock.IAnswer;
|
||||||
import org.easymock.LogicalOperator;
|
import org.easymock.LogicalOperator;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -56,6 +58,7 @@ import org.junit.rules.ExpectedException;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -65,6 +68,7 @@ public class BatchServerInventoryViewTest
|
||||||
private static final String testBasePath = "/test";
|
private static final String testBasePath = "/test";
|
||||||
public static final DateTime SEGMENT_INTERVAL_START = new DateTime("2013-01-01");
|
public static final DateTime SEGMENT_INTERVAL_START = new DateTime("2013-01-01");
|
||||||
public static final int INITIAL_SEGMENTS = 100;
|
public static final int INITIAL_SEGMENTS = 100;
|
||||||
|
private static final Timing timing = new Timing();
|
||||||
|
|
||||||
private TestingCluster testingCluster;
|
private TestingCluster testingCluster;
|
||||||
private CuratorFramework cf;
|
private CuratorFramework cf;
|
||||||
|
@ -244,6 +248,8 @@ public class BatchServerInventoryViewTest
|
||||||
@Test
|
@Test
|
||||||
public void testRunWithFilterCallback() throws Exception
|
public void testRunWithFilterCallback() throws Exception
|
||||||
{
|
{
|
||||||
|
final CountDownLatch removeCallbackLatch = new CountDownLatch(1);
|
||||||
|
|
||||||
segmentAnnouncer.announceSegments(testSegments);
|
segmentAnnouncer.announceSegments(testSegments);
|
||||||
|
|
||||||
waitForSync(filteredBatchServerInventoryView, testSegments);
|
waitForSync(filteredBatchServerInventoryView, testSegments);
|
||||||
|
@ -280,7 +286,17 @@ public class BatchServerInventoryViewTest
|
||||||
EasyMock.cmp(makeSegment(INITIAL_SEGMENTS + 2), dataSegmentComparator, LogicalOperator.EQUAL)
|
EasyMock.cmp(makeSegment(INITIAL_SEGMENTS + 2), dataSegmentComparator, LogicalOperator.EQUAL)
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
.andReturn(ServerView.CallbackAction.CONTINUE)
|
.andAnswer(
|
||||||
|
new IAnswer<ServerView.CallbackAction>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public ServerView.CallbackAction answer() throws Throwable
|
||||||
|
{
|
||||||
|
removeCallbackLatch.countDown();
|
||||||
|
return ServerView.CallbackAction.CONTINUE;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
)
|
||||||
.times(1);
|
.times(1);
|
||||||
|
|
||||||
|
|
||||||
|
@ -316,6 +332,8 @@ public class BatchServerInventoryViewTest
|
||||||
testSegments.remove(segment2);
|
testSegments.remove(segment2);
|
||||||
|
|
||||||
waitForSync(filteredBatchServerInventoryView, testSegments);
|
waitForSync(filteredBatchServerInventoryView, testSegments);
|
||||||
|
timing.forWaiting().awaitLatch(removeCallbackLatch);
|
||||||
|
|
||||||
EasyMock.verify(callback);
|
EasyMock.verify(callback);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -335,11 +353,12 @@ public class BatchServerInventoryViewTest
|
||||||
|
|
||||||
private static void waitForSync(BatchServerInventoryView batchServerInventoryView, Set<DataSegment> testSegments) throws Exception
|
private static void waitForSync(BatchServerInventoryView batchServerInventoryView, Set<DataSegment> testSegments) throws Exception
|
||||||
{
|
{
|
||||||
|
final Timing forWaitingTiming = timing.forWaiting();
|
||||||
Stopwatch stopwatch = Stopwatch.createStarted();
|
Stopwatch stopwatch = Stopwatch.createStarted();
|
||||||
while (Iterables.isEmpty(batchServerInventoryView.getInventory())
|
while (Iterables.isEmpty(batchServerInventoryView.getInventory())
|
||||||
|| Iterables.get(batchServerInventoryView.getInventory(), 0).getSegments().size() != testSegments.size()) {
|
|| Iterables.get(batchServerInventoryView.getInventory(), 0).getSegments().size() != testSegments.size()) {
|
||||||
Thread.sleep(500);
|
Thread.sleep(500);
|
||||||
if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 2000) {
|
if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > forWaitingTiming.milliseconds()) {
|
||||||
throw new ISE("BatchServerInventoryView is not updating");
|
throw new ISE("BatchServerInventoryView is not updating");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue