mirror of https://github.com/apache/druid.git
groupBy query cancellation
This commit is contained in:
parent
1fb9b21cf0
commit
6550cb1776
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -113,6 +113,7 @@ public class GroupByQueryRunnerTest
|
|||
|
||||
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
|
||||
engine,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
|
||||
configSupplier,
|
||||
new GroupByQueryQueryToolChest(configSupplier, engine)
|
||||
);
|
||||
|
|
|
@ -72,6 +72,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
|
|||
|
||||
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
|
||||
engine,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
|
||||
configSupplier,
|
||||
new GroupByQueryQueryToolChest(configSupplier, engine)
|
||||
);
|
||||
|
|
Loading…
Reference in New Issue