mirror of https://github.com/apache/druid.git
Nicer handling for cancelled groupBy v2 queries. (#3330)
1. Wrap temporaryStorage in a resource holder, to avoid spurious "Closed" errors from already-running processing tasks. 2. Exit early from the merging accumulator if the query is cancelled.
This commit is contained in:
parent
decefb7477
commit
1aae5bd67d
|
@ -42,6 +42,11 @@ public class ReferenceCountingResourceHolder<T> implements ResourceHolder<T>
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static <T extends Closeable> ReferenceCountingResourceHolder<T> fromCloseable(final T object)
|
||||||
|
{
|
||||||
|
return new ReferenceCountingResourceHolder<>(object, object);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public T get()
|
public T get()
|
||||||
{
|
{
|
||||||
|
|
|
@ -28,7 +28,6 @@ import java.util.Iterator;
|
||||||
|
|
||||||
public class CloseableGrouperIterator<KeyType extends Comparable<KeyType>, T> implements Iterator<T>, Closeable
|
public class CloseableGrouperIterator<KeyType extends Comparable<KeyType>, T> implements Iterator<T>, Closeable
|
||||||
{
|
{
|
||||||
private final Grouper<KeyType> grouper;
|
|
||||||
private final Function<Grouper.Entry<KeyType>, T> transformer;
|
private final Function<Grouper.Entry<KeyType>, T> transformer;
|
||||||
private final Closeable closer;
|
private final Closeable closer;
|
||||||
private final Iterator<Grouper.Entry<KeyType>> iterator;
|
private final Iterator<Grouper.Entry<KeyType>> iterator;
|
||||||
|
@ -40,7 +39,6 @@ public class CloseableGrouperIterator<KeyType extends Comparable<KeyType>, T> im
|
||||||
final Closeable closer
|
final Closeable closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.grouper = grouper;
|
|
||||||
this.transformer = transformer;
|
this.transformer = transformer;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.iterator = grouper.iterator(sorted);
|
this.iterator = grouper.iterator(sorted);
|
||||||
|
|
|
@ -148,15 +148,16 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner
|
||||||
@Override
|
@Override
|
||||||
public CloseableGrouperIterator<RowBasedKey, Row> make()
|
public CloseableGrouperIterator<RowBasedKey, Row> make()
|
||||||
{
|
{
|
||||||
final List<Closeable> closeOnFailure = Lists.newArrayList();
|
final List<ReferenceCountingResourceHolder> resources = Lists.newArrayList();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final LimitedTemporaryStorage temporaryStorage = new LimitedTemporaryStorage(
|
final LimitedTemporaryStorage temporaryStorage = new LimitedTemporaryStorage(
|
||||||
temporaryStorageDirectory,
|
temporaryStorageDirectory,
|
||||||
querySpecificConfig.getMaxOnDiskStorage()
|
querySpecificConfig.getMaxOnDiskStorage()
|
||||||
);
|
);
|
||||||
|
final ReferenceCountingResourceHolder<LimitedTemporaryStorage> temporaryStorageHolder =
|
||||||
closeOnFailure.add(temporaryStorage);
|
ReferenceCountingResourceHolder.fromCloseable(temporaryStorage);
|
||||||
|
resources.add(temporaryStorageHolder);
|
||||||
|
|
||||||
final ReferenceCountingResourceHolder<ByteBuffer> mergeBufferHolder;
|
final ReferenceCountingResourceHolder<ByteBuffer> mergeBufferHolder;
|
||||||
try {
|
try {
|
||||||
|
@ -165,7 +166,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner
|
||||||
if (timeout <= 0 || (mergeBufferHolder = mergeBufferPool.take(timeout)) == null) {
|
if (timeout <= 0 || (mergeBufferHolder = mergeBufferPool.take(timeout)) == null) {
|
||||||
throw new QueryInterruptedException(new TimeoutException());
|
throw new QueryInterruptedException(new TimeoutException());
|
||||||
}
|
}
|
||||||
closeOnFailure.add(mergeBufferHolder);
|
resources.add(mergeBufferHolder);
|
||||||
}
|
}
|
||||||
catch (InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
throw new QueryInterruptedException(e);
|
throw new QueryInterruptedException(e);
|
||||||
|
@ -184,18 +185,9 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner
|
||||||
final Grouper<RowBasedKey> grouper = pair.lhs;
|
final Grouper<RowBasedKey> grouper = pair.lhs;
|
||||||
final Accumulator<Grouper<RowBasedKey>, Row> accumulator = pair.rhs;
|
final Accumulator<Grouper<RowBasedKey>, Row> accumulator = pair.rhs;
|
||||||
|
|
||||||
final ReferenceCountingResourceHolder<Grouper<RowBasedKey>> grouperHolder = new ReferenceCountingResourceHolder<>(
|
final ReferenceCountingResourceHolder<Grouper<RowBasedKey>> grouperHolder =
|
||||||
grouper,
|
ReferenceCountingResourceHolder.fromCloseable(grouper);
|
||||||
new Closeable()
|
resources.add(grouperHolder);
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException
|
|
||||||
{
|
|
||||||
grouper.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
closeOnFailure.add(grouperHolder);
|
|
||||||
|
|
||||||
ListenableFuture<List<Boolean>> futures = Futures.allAsList(
|
ListenableFuture<List<Boolean>> futures = Futures.allAsList(
|
||||||
Lists.newArrayList(
|
Lists.newArrayList(
|
||||||
|
@ -271,16 +263,16 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
grouperHolder.close();
|
for (Closeable closeable : Lists.reverse(resources)) {
|
||||||
mergeBufferHolder.close();
|
CloseQuietly.close(closeable);
|
||||||
CloseQuietly.close(temporaryStorage);
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Throwable e) {
|
catch (Throwable e) {
|
||||||
// Exception caught while setting up the iterator; release resources.
|
// Exception caught while setting up the iterator; release resources.
|
||||||
for (Closeable closeable : Lists.reverse(closeOnFailure)) {
|
for (Closeable closeable : Lists.reverse(resources)) {
|
||||||
CloseQuietly.close(closeable);
|
CloseQuietly.close(closeable);
|
||||||
}
|
}
|
||||||
throw e;
|
throw e;
|
||||||
|
|
|
@ -35,6 +35,7 @@ import com.metamx.common.guava.Accumulator;
|
||||||
import io.druid.data.input.MapBasedRow;
|
import io.druid.data.input.MapBasedRow;
|
||||||
import io.druid.data.input.Row;
|
import io.druid.data.input.Row;
|
||||||
import io.druid.granularity.AllGranularity;
|
import io.druid.granularity.AllGranularity;
|
||||||
|
import io.druid.query.QueryInterruptedException;
|
||||||
import io.druid.query.aggregation.AggregatorFactory;
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
import io.druid.query.dimension.DimensionSpec;
|
import io.druid.query.dimension.DimensionSpec;
|
||||||
import io.druid.query.extraction.ExtractionFn;
|
import io.druid.query.extraction.ExtractionFn;
|
||||||
|
@ -126,6 +127,10 @@ public class RowBasedGrouperHelper
|
||||||
final Row row
|
final Row row
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
if (Thread.interrupted()) {
|
||||||
|
throw new QueryInterruptedException(new InterruptedException());
|
||||||
|
}
|
||||||
|
|
||||||
if (theGrouper == null) {
|
if (theGrouper == null) {
|
||||||
// Pass-through null returns without doing more work.
|
// Pass-through null returns without doing more work.
|
||||||
return null;
|
return null;
|
||||||
|
|
Loading…
Reference in New Issue