mirror of https://github.com/apache/druid.git
Merge pull request #1710 from metamx/incrementalIndexConcurrentTestLatching
Add some basic latching to concurrency testing in IncrementalIndexTest
This commit is contained in:
commit
8b071a7230
|
@ -292,6 +292,8 @@ public class IncrementalIndexTest
|
||||||
final AtomicInteger currentlyRunning = new AtomicInteger(0);
|
final AtomicInteger currentlyRunning = new AtomicInteger(0);
|
||||||
final AtomicBoolean concurrentlyRan = new AtomicBoolean(false);
|
final AtomicBoolean concurrentlyRan = new AtomicBoolean(false);
|
||||||
final AtomicInteger someoneRan = new AtomicInteger(0);
|
final AtomicInteger someoneRan = new AtomicInteger(0);
|
||||||
|
final CountDownLatch startLatch = new CountDownLatch(1);
|
||||||
|
try {
|
||||||
for (int j = 0; j < taskCount; j++) {
|
for (int j = 0; j < taskCount; j++) {
|
||||||
indexFutures.add(
|
indexFutures.add(
|
||||||
indexExecutor.submit(
|
indexExecutor.submit(
|
||||||
|
@ -300,6 +302,13 @@ public class IncrementalIndexTest
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
currentlyRunning.incrementAndGet();
|
currentlyRunning.incrementAndGet();
|
||||||
try {
|
try {
|
||||||
for (int i = 0; i < elementsPerThread; i++) {
|
for (int i = 0; i < elementsPerThread; i++) {
|
||||||
|
@ -315,6 +324,13 @@ public class IncrementalIndexTest
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.dataSource("xxx")
|
||||||
|
.granularity(QueryGranularity.ALL)
|
||||||
|
.intervals(ImmutableList.of(queryInterval))
|
||||||
|
.aggregators(queryAggregatorFactories)
|
||||||
|
.build();
|
||||||
queryFutures.add(
|
queryFutures.add(
|
||||||
queryExecutor.submit(
|
queryExecutor.submit(
|
||||||
new Runnable()
|
new Runnable()
|
||||||
|
@ -322,16 +338,17 @@ public class IncrementalIndexTest
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
|
try {
|
||||||
|
startLatch.await();
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
QueryRunner<Result<TimeseriesResultValue>> runner = new FinalizeResultsQueryRunner<Result<TimeseriesResultValue>>(
|
QueryRunner<Result<TimeseriesResultValue>> runner = new FinalizeResultsQueryRunner<Result<TimeseriesResultValue>>(
|
||||||
factory.createRunner(incrementalIndexSegment),
|
factory.createRunner(incrementalIndexSegment),
|
||||||
factory.getToolchest()
|
factory.getToolchest()
|
||||||
);
|
);
|
||||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
|
||||||
.dataSource("xxx")
|
|
||||||
.granularity(QueryGranularity.ALL)
|
|
||||||
.intervals(ImmutableList.of(queryInterval))
|
|
||||||
.aggregators(queryAggregatorFactories)
|
|
||||||
.build();
|
|
||||||
Map<String, Object> context = new HashMap<String, Object>();
|
Map<String, Object> context = new HashMap<String, Object>();
|
||||||
for (Result<TimeseriesResultValue> result :
|
for (Result<TimeseriesResultValue> result :
|
||||||
Sequences.toList(
|
Sequences.toList(
|
||||||
|
@ -344,17 +361,24 @@ public class IncrementalIndexTest
|
||||||
final Double sumResult = result.getValue().getDoubleMetric("doubleSumResult0");
|
final Double sumResult = result.getValue().getDoubleMetric("doubleSumResult0");
|
||||||
// Eventually consistent, but should be somewhere in that range
|
// Eventually consistent, but should be somewhere in that range
|
||||||
// Actual result is validated after all writes are guaranteed done.
|
// Actual result is validated after all writes are guaranteed done.
|
||||||
Assert.assertTrue(String.format("%d >= %g >= 0 violated", ranCount, sumResult), sumResult >= 0 && sumResult <= ranCount);
|
Assert.assertTrue(
|
||||||
}
|
String.format("%d >= %g >= 0 violated", ranCount, sumResult),
|
||||||
|
sumResult >= 0 && sumResult <= ranCount
|
||||||
|
);
|
||||||
}
|
}
|
||||||
if (currentlyRunning.get() > 0) {
|
if (currentlyRunning.get() > 0) {
|
||||||
concurrentlyRan.set(true);
|
concurrentlyRan.set(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
startLatch.countDown();
|
||||||
|
}
|
||||||
List<ListenableFuture<?>> allFutures = new ArrayList<>(queryFutures.size() + indexFutures.size());
|
List<ListenableFuture<?>> allFutures = new ArrayList<>(queryFutures.size() + indexFutures.size());
|
||||||
allFutures.addAll(queryFutures);
|
allFutures.addAll(queryFutures);
|
||||||
allFutures.addAll(indexFutures);
|
allFutures.addAll(indexFutures);
|
||||||
|
|
Loading…
Reference in New Issue