groupBy query cancellation

This commit is contained in:
Xavier Léauté 2014-06-10 10:15:31 -07:00
parent 1fb9b21cf0
commit 6550cb1776
4 changed files with 105 additions and 46 deletions

View File

@ -26,6 +26,10 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.metamx.common.Pair;
import com.metamx.common.guava.Accumulator;
import com.metamx.common.guava.Sequence;
@ -39,37 +43,44 @@ import io.druid.segment.incremental.IncrementalIndex;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
public class GroupByParallelQueryRunner implements QueryRunner<Row>
{
private static final Logger log = new Logger(GroupByParallelQueryRunner.class);
private final Iterable<QueryRunner<Row>> queryables;
private final ExecutorService exec;
private final ListeningExecutorService exec;
private final Ordering<Row> ordering;
private final Supplier<GroupByQueryConfig> configSupplier;
private final QueryWatcher queryWatcher;
public GroupByParallelQueryRunner(
ExecutorService exec,
Ordering<Row> ordering,
Supplier<GroupByQueryConfig> configSupplier,
QueryWatcher queryWatcher,
QueryRunner<Row>... queryables
)
{
this(exec, ordering, configSupplier, Arrays.asList(queryables));
this(exec, ordering, configSupplier, queryWatcher, Arrays.asList(queryables));
}
public GroupByParallelQueryRunner(
ExecutorService exec,
Ordering<Row> ordering, Supplier<GroupByQueryConfig> configSupplier,
QueryWatcher queryWatcher,
Iterable<QueryRunner<Row>> queryables
)
{
this.exec = exec;
this.exec = MoreExecutors.listeningDecorator(exec);
this.ordering = ordering;
this.queryWatcher = queryWatcher;
this.queryables = Iterables.unmodifiableIterable(Iterables.filter(queryables, Predicates.notNull()));
this.configSupplier = configSupplier;
}
@ -88,48 +99,67 @@ public class GroupByParallelQueryRunner implements QueryRunner<Row>
if (Iterables.isEmpty(queryables)) {
log.warn("No queryables found.");
}
List<Future<Boolean>> futures = Lists.newArrayList(
Iterables.transform(
queryables,
new Function<QueryRunner<Row>, Future<Boolean>>()
{
@Override
public Future<Boolean> apply(final QueryRunner<Row> input)
{
return exec.submit(
new AbstractPrioritizedCallable<Boolean>(priority)
{
@Override
public Boolean call() throws Exception
{
try {
input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
return true;
ListenableFuture<List<Boolean>> futures = Futures.allAsList(
Lists.newArrayList(
Iterables.transform(
queryables,
new Function<QueryRunner<Row>, ListenableFuture<Boolean>>()
{
@Override
public ListenableFuture<Boolean> apply(final QueryRunner<Row> input)
{
return exec.submit(
new AbstractPrioritizedCallable<Boolean>(priority)
{
@Override
public Boolean call() throws Exception
{
try {
input.run(queryParam).accumulate(indexAccumulatorPair.lhs, indexAccumulatorPair.rhs);
return true;
}
catch (QueryInterruptedException e) {
throw Throwables.propagate(e);
}
catch (Exception e) {
log.error(e, "Exception with one of the sequences!");
throw Throwables.propagate(e);
}
}
}
catch (Exception e) {
log.error(e, "Exception with one of the sequences!");
throw Throwables.propagate(e);
}
}
}
);
}
}
);
}
}
)
)
);
// Let the runners complete
for (Future<Boolean> future : futures) {
try {
future.get();
}
catch (InterruptedException e) {
throw new RuntimeException(e);
}
catch (ExecutionException e) {
throw new RuntimeException(e);
try {
queryWatcher.registerQuery(query, futures);
final Number timeout = query.getContextValue("timeout", (Number) null);
if(timeout == null) {
futures.get();
} else {
futures.get(timeout.longValue(), TimeUnit.MILLISECONDS);
}
}
catch (InterruptedException e) {
log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId());
futures.cancel(true);
throw new QueryInterruptedException("Query interrupted");
}
catch(CancellationException e) {
throw new QueryInterruptedException("Query cancelled");
}
catch(TimeoutException e) {
log.info("Query timeout, cancelling pending results for query id [%s]", query.getId());
futures.cancel(true);
throw new QueryInterruptedException("Query timeout");
}
catch (ExecutionException e) {
throw Throwables.propagate(e.getCause());
}
return Sequences.simple(indexAccumulatorPair.lhs.iterableWithPostAggregations(null));
}

View File

@ -24,42 +24,55 @@ import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Longs;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.guava.ExecutorExecutingSequence;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
import io.druid.data.input.Row;
import io.druid.query.ConcatQueryRunner;
import io.druid.query.GroupByParallelQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryInterruptedException;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
import io.druid.query.QueryWatcher;
import io.druid.segment.Segment;
import io.druid.segment.StorageAdapter;
import java.util.concurrent.Callable;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
/**
*/
public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupByQuery>
{
private final GroupByQueryEngine engine;
private final QueryWatcher queryWatcher;
private final Supplier<GroupByQueryConfig> config;
private final GroupByQueryQueryToolChest toolChest;
private static final Logger log = new Logger(GroupByQueryRunnerFactory.class);
@Inject
public GroupByQueryRunnerFactory(
GroupByQueryEngine engine,
QueryWatcher queryWatcher,
Supplier<GroupByQueryConfig> config,
GroupByQueryQueryToolChest toolChest
)
{
this.engine = engine;
this.queryWatcher = queryWatcher;
this.config = config;
this.toolChest = toolChest;
}
@ -71,8 +84,10 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
}
@Override
public QueryRunner<Row> mergeRunners(final ExecutorService queryExecutor, Iterable<QueryRunner<Row>> queryRunners)
public QueryRunner<Row> mergeRunners(final ExecutorService exec, Iterable<QueryRunner<Row>> queryRunners)
{
// mergeRunners should take ListeningExecutorService at some point
final ListeningExecutorService queryExecutor = MoreExecutors.listeningDecorator(exec);
if (config.get().isSingleThreaded()) {
return new ConcatQueryRunner<Row>(
Sequences.map(
@ -88,7 +103,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
public Sequence<Row> run(final Query<Row> query)
{
Future<Sequence<Row>> future = queryExecutor.submit(
ListenableFuture<Sequence<Row>> future = queryExecutor.submit(
new Callable<Sequence<Row>>()
{
@Override
@ -102,13 +117,25 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
}
);
try {
return future.get();
queryWatcher.registerQuery(query, future);
final Number timeout = query.getContextValue("timeout", (Number)null);
return timeout == null ? future.get() : future.get(timeout.longValue(), TimeUnit.MILLISECONDS);
}
catch (InterruptedException e) {
throw Throwables.propagate(e);
log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId());
future.cancel(true);
throw new QueryInterruptedException("Query interrupted");
}
catch(CancellationException e) {
throw new QueryInterruptedException("Query cancelled");
}
catch(TimeoutException e) {
log.info("Query timeout, cancelling pending results for query id [%s]", query.getId());
future.cancel(true);
throw new QueryInterruptedException("Query timeout");
}
catch (ExecutionException e) {
throw Throwables.propagate(e);
throw Throwables.propagate(e.getCause());
}
}
};
@ -117,7 +144,7 @@ public class GroupByQueryRunnerFactory implements QueryRunnerFactory<Row, GroupB
)
);
} else {
return new GroupByParallelQueryRunner(queryExecutor, new RowOrdering(), config, queryRunners);
return new GroupByParallelQueryRunner(queryExecutor, new RowOrdering(), config, queryWatcher, queryRunners);
}
}

View File

@ -113,6 +113,7 @@ public class GroupByQueryRunnerTest
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
engine,
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
configSupplier,
new GroupByQueryQueryToolChest(configSupplier, engine)
);

View File

@ -72,6 +72,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
engine,
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
configSupplier,
new GroupByQueryQueryToolChest(configSupplier, engine)
);