Merge pull request #2711 from gianm/filtered-aggregator-impls

All Filters should work with FilteredAggregators.
This commit is contained in:
Fangjin Yang 2016-03-23 13:37:21 -07:00
commit a5d5529749
15 changed files with 285 additions and 226 deletions

View File

@ -204,8 +204,6 @@ A filtered aggregator wraps any given aggregator, but only aggregates the values
This makes it possible to compute the results of a filtered and an unfiltered aggregation simultaneously, without having to issue multiple queries, and use both results as part of post-aggregations.
*Limitations:* The filtered aggregator currently only supports 'or', 'and', 'selector', 'not' and 'Extraction' filters, i.e. matching one or multiple dimensions against a single value.
*Note:* If only the filtered results are required, consider putting the filter on the query itself, which will be much faster since it does not require scanning all the data.
```json

View File

@ -21,12 +21,19 @@ package io.druid.query.aggregation;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.BooleanValueMatcher;
import io.druid.segment.filter.Filters;
import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.Comparator;
import java.util.List;
@ -52,18 +59,19 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@Override
public Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
{
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(columnSelectorFactory);
return new FilteredAggregator(
valueMatcher,
delegate.factorize(columnSelectorFactory)
);
final ValueMatcherFactory valueMatcherFactory = new FilteredAggregatorValueMatcherFactory(columnSelectorFactory);
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(valueMatcherFactory);
return new FilteredAggregator(
valueMatcher,
delegate.factorize(columnSelectorFactory)
);
}
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
{
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(columnSelectorFactory);
final ValueMatcherFactory valueMatcherFactory = new FilteredAggregatorValueMatcherFactory(columnSelectorFactory);
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(valueMatcherFactory);
return new FilteredBufferAggregator(
valueMatcher,
delegate.factorizeBuffered(columnSelectorFactory)
@ -199,4 +207,85 @@ public class FilteredAggregatorFactory extends AggregatorFactory
result = 31 * result + (filter != null ? filter.hashCode() : 0);
return result;
}
private static class FilteredAggregatorValueMatcherFactory implements ValueMatcherFactory
{
private final ColumnSelectorFactory columnSelectorFactory;
public FilteredAggregatorValueMatcherFactory(ColumnSelectorFactory columnSelectorFactory)
{
this.columnSelectorFactory = columnSelectorFactory;
}
@Override
public ValueMatcher makeValueMatcher(final String dimension, final Comparable value)
{
final DimensionSelector selector = columnSelectorFactory.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
// Compare "value" as a String.
final String valueString = value == null ? null : value.toString();
final boolean isNullOrEmpty = valueString == null || valueString.isEmpty();
// Missing columns match a null or empty string value, and don't match anything else.
if (selector == null) {
return new BooleanValueMatcher(isNullOrEmpty);
}
final int valueId = selector.lookupId(valueString);
return new ValueMatcher()
{
@Override
public boolean matches()
{
final IndexedInts row = selector.getRow();
final int size = row.size();
for (int i = 0; i < size; ++i) {
if (row.get(i) == valueId) {
return true;
}
}
return false;
}
};
}
@Override
public ValueMatcher makeValueMatcher(final String dimension, final Predicate predicate)
{
final DimensionSelector selector = columnSelectorFactory.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
if (selector == null) {
return new BooleanValueMatcher(predicate.apply(null));
}
// Check every value in the dimension, as a String.
final int cardinality = selector.getValueCardinality();
final BitSet valueIds = new BitSet(cardinality);
for (int i = 0; i < cardinality; i++) {
if (predicate.apply(selector.lookupName(i))) {
valueIds.set(i);
}
}
return new ValueMatcher()
{
@Override
public boolean matches()
{
final IndexedInts row = selector.getRow();
final int size = row.size();
for (int i = 0; i < size; ++i) {
if (valueIds.get(row.get(i))) {
return true;
}
}
return false;
}
};
}
}
}

View File

@ -20,7 +20,6 @@
package io.druid.query.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.segment.ColumnSelectorFactory;
/**
*/
@ -28,5 +27,4 @@ public interface Filter
{
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector);
public ValueMatcher makeMatcher(ValueMatcherFactory factory);
public ValueMatcher makeMatcher(ColumnSelectorFactory columnSelectorFactory);
}

View File

@ -20,13 +20,11 @@
package io.druid.query.filter;
import com.google.common.base.Predicate;
import com.metamx.collections.spatial.search.Bound;
/**
*/
public interface ValueMatcherFactory
{
public ValueMatcher makeValueMatcher(String dimension, Comparable value);
public ValueMatcher makeValueMatcher(String dimension, Predicate value);
public ValueMatcher makeValueMatcher(String dimension, Bound bound);
public ValueMatcher makeValueMatcher(String dimension, Predicate predicate);
}

View File

@ -25,7 +25,6 @@ import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import java.util.List;
@ -72,16 +71,6 @@ public class AndFilter implements Filter
return makeMatcher(matchers);
}
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
final ValueMatcher[] matchers = new ValueMatcher[filters.size()];
for (int i = 0; i < filters.size(); i++) {
matchers[i] = filters.get(i).makeMatcher(factory);
}
return makeMatcher(matchers);
}
private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers)
{
if (baseMatchers.length == 1) {

View File

@ -27,7 +27,6 @@ import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.data.Indexed;
import javax.annotation.Nullable;
@ -77,10 +76,4 @@ class DimensionPredicateFilter implements Filter
{
return factory.makeValueMatcher(dimension, predicate);
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
throw new UnsupportedOperationException();
}
}

View File

@ -23,18 +23,13 @@ import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import java.util.BitSet;
import java.util.Iterator;
import java.util.List;
@ -121,38 +116,4 @@ public class ExtractionFilter implements Filter
}
);
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory columnSelectorFactory)
{
final DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
if (dimensionSelector == null) {
return new BooleanValueMatcher(value.equals(Strings.nullToEmpty(fn.apply(null))));
} else {
final BitSet bitSetOfIds = new BitSet(dimensionSelector.getValueCardinality());
for (int i = 0; i < dimensionSelector.getValueCardinality(); i++) {
if (value.equals(Strings.nullToEmpty(fn.apply(dimensionSelector.lookupName(i))))) {
bitSetOfIds.set(i);
}
}
return new ValueMatcher()
{
@Override
public boolean matches()
{
final IndexedInts row = dimensionSelector.getRow();
final int size = row.size();
for (int i = 0; i < size; ++i) {
if (bitSetOfIds.get(row.get(i))) {
return true;
}
}
return false;
}
};
}
}
}

View File

@ -27,7 +27,6 @@ import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.data.Indexed;
import org.mozilla.javascript.Context;
import org.mozilla.javascript.Function;
@ -161,11 +160,4 @@ public class JavaScriptFilter implements Filter
return script.hashCode();
}
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
throw new UnsupportedOperationException();
}
}

View File

@ -24,7 +24,6 @@ import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
/**
*/
@ -62,19 +61,4 @@ public class NotFilter implements Filter
}
};
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
final ValueMatcher baseMatcher = baseFilter.makeMatcher(factory);
return new ValueMatcher()
{
@Override
public boolean matches()
{
return !baseMatcher.matches();
}
};
}
}

View File

@ -25,7 +25,6 @@ import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import java.util.List;
@ -72,16 +71,6 @@ public class OrFilter implements Filter
return makeMatcher(matchers);
}
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
final ValueMatcher[] matchers = new ValueMatcher[filters.size()];
for (int i = 0; i < filters.size(); i++) {
matchers[i] = filters.get(i).makeMatcher(factory);
}
return makeMatcher(matchers);
}
private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers){
if (baseMatchers.length == 1) {
return baseMatchers[0];

View File

@ -19,16 +19,11 @@
package io.druid.segment.filter;
import com.google.common.base.Strings;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.IndexedInts;
/**
*/
@ -57,36 +52,4 @@ public class SelectorFilter implements Filter
{
return factory.makeValueMatcher(dimension, value);
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory columnSelectorFactory)
{
final DimensionSelector dimensionSelector = columnSelectorFactory.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
// Missing columns match a null or empty string value and don't match anything else
if (dimensionSelector == null) {
return new BooleanValueMatcher(Strings.isNullOrEmpty(value));
} else {
final int valueId = dimensionSelector.lookupId(value);
return new ValueMatcher()
{
@Override
public boolean matches()
{
final IndexedInts row = dimensionSelector.getRow();
final int size = row.size();
for (int i = 0; i < size; ++i) {
if (row.get(i) == valueId) {
return true;
}
}
return false;
}
};
}
}
}

View File

@ -18,13 +18,17 @@
*/
package io.druid.segment.filter;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.search.Bound;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.incremental.SpatialDimensionRowTransformer;
import java.util.Arrays;
/**
*/
@ -38,8 +42,8 @@ public class SpatialFilter implements Filter
Bound bound
)
{
this.dimension = dimension;
this.bound = bound;
this.dimension = Preconditions.checkNotNull(dimension, "dimension");
this.bound = Preconditions.checkNotNull(bound, "bound");
}
@Override
@ -54,14 +58,19 @@ public class SpatialFilter implements Filter
{
return factory.makeValueMatcher(
dimension,
bound
new Predicate()
{
@Override
public boolean apply(Object input)
{
if (input instanceof String) {
final float[] coordinate = SpatialDimensionRowTransformer.decode((String) input);
return bound.contains(coordinate);
} else {
return false;
}
}
}
);
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
throw new UnsupportedOperationException();
}
}

View File

@ -731,46 +731,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
}
};
}
@Override
public ValueMatcher makeValueMatcher(final String dimension, final Bound bound)
{
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension);
if (dimensionDesc == null) {
return new BooleanValueMatcher(false);
}
final int dimIndex = dimensionDesc.getIndex();
final IncrementalIndex.DimDim dimDim = dimensionDesc.getValues();
return new ValueMatcher()
{
@Override
public boolean matches()
{
int[][] dims = holder.getKey().getDims();
if (dimIndex >= dims.length || dims[dimIndex] == null) {
return false;
}
for (int dimVal : dims[dimIndex]) {
Comparable fullDimVal = dimDim.getValue(dimVal);
// TODO: decide what to do for non-String spatial dims, skip for now
if (!(fullDimVal instanceof String)) {
return false;
}
List<String> stringCoords = Lists.newArrayList(SPLITTER.split((String) fullDimVal));
float[] coords = new float[stringCoords.size()];
for (int j = 0; j < coords.length; j++) {
coords[j] = Float.valueOf(stringCoords.get(j));
}
if (bound.contains(coords)) {
return true;
}
}
return false;
}
};
}
}
@Override

View File

@ -24,6 +24,7 @@ import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
@ -224,11 +225,41 @@ public class SpatialDimensionRowTransformer implements Function<InputRow, InputR
return true;
}
private static Float tryParseFloat(String val) {
private static Float tryParseFloat(String val)
{
try {
return Float.parseFloat(val);
} catch (NullPointerException | NumberFormatException e) {
}
catch (NullPointerException | NumberFormatException e) {
return null;
}
}
/**
* Decodes encodedCoordinate.
*
* @param encodedCoordinate encoded coordinate
*
* @return decoded coordinate, or null if it could not be decoded
*/
public static float[] decode(final String encodedCoordinate)
{
if (encodedCoordinate == null) {
return null;
}
final ImmutableList<String> parts = ImmutableList.copyOf(SPLITTER.split(encodedCoordinate));
final float[] coordinate = new float[parts.size()];
for (int i = 0; i < coordinate.length; i++) {
final Float floatPart = tryParseFloat(parts.get(i));
if (floatPart == null) {
return null;
} else {
coordinate[i] = floatPart;
}
}
return coordinate;
}
}

View File

@ -22,6 +22,7 @@ package io.druid.segment.filter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.collections.spatial.search.RadiusBound;
import com.metamx.collections.spatial.search.RectangularBound;
import io.druid.data.input.MapBasedInputRow;
@ -35,6 +36,7 @@ import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.FilteredAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.filter.SpatialDimFilter;
import io.druid.query.timeseries.TimeseriesQuery;
@ -66,6 +68,7 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Random;
import java.util.Set;
/**
*/
@ -205,9 +208,21 @@ public class SpatialFilterBonusTest
)
);
// Add a bunch of random points
// Add a bunch of random points, without replacement
Set<String> alreadyChosen = Sets.newHashSet();
Random rand = new Random();
for (int i = 6; i < NUM_POINTS; i++) {
String coord = null;
while (coord == null) {
coord = String.format(
"%s,%s",
(float) (rand.nextFloat() * 10 + 10.0),
(float) (rand.nextFloat() * 10 + 10.0)
);
if (!alreadyChosen.add(coord)) {
coord = null;
}
}
theIndex.add(
new MapBasedInputRow(
new DateTime("2013-01-01").getMillis(),
@ -215,11 +230,7 @@ public class SpatialFilterBonusTest
ImmutableMap.<String, Object>of(
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "boo",
"dim.geo", String.format(
"%s,%s",
(float) (rand.nextFloat() * 10 + 10.0),
(float) (rand.nextFloat() * 10 + 10.0)
),
"dim.geo", coord,
"val", i
)
)
@ -389,10 +400,10 @@ public class SpatialFilterBonusTest
"timestamp", new DateTime("2013-01-01").toString(),
"dim", "boo",
"dim.geo", String.format(
"%s,%s",
(float) (rand.nextFloat() * 10 + 10.0),
(float) (rand.nextFloat() * 10 + 10.0)
),
"%s,%s",
(float) (rand.nextFloat() * 10 + 10.0),
(float) (rand.nextFloat() * 10 + 10.0)
),
"val", i
)
)
@ -467,9 +478,9 @@ public class SpatialFilterBonusTest
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 3L)
.put("val", 59L)
.build()
.put("rows", 3L)
.put("val", 59L)
.build()
)
)
);
@ -485,7 +496,7 @@ public class SpatialFilterBonusTest
factory.createRunner(segment),
factory.getToolchest()
);
HashMap<String,Object> context = new HashMap<String, Object>();
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
}
catch (Exception e) {
@ -519,45 +530,45 @@ public class SpatialFilterBonusTest
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 17L)
.build()
.put("rows", 1L)
.put("val", 17L)
.build()
)
),
new Result<TimeseriesResultValue>(
new DateTime("2013-01-02T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 29L)
.build()
.put("rows", 1L)
.put("val", 29L)
.build()
)
),
new Result<TimeseriesResultValue>(
new DateTime("2013-01-03T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 13L)
.build()
.put("rows", 1L)
.put("val", 13L)
.build()
)
),
new Result<TimeseriesResultValue>(
new DateTime("2013-01-04T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 91L)
.build()
.put("rows", 1L)
.put("val", 91L)
.build()
)
),
new Result<TimeseriesResultValue>(
new DateTime("2013-01-05T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 47L)
.build()
.put("rows", 1L)
.put("val", 47L)
.build()
)
)
);
@ -573,7 +584,101 @@ public class SpatialFilterBonusTest
factory.createRunner(segment),
factory.getToolchest()
);
HashMap<String,Object> context = new HashMap<String, Object>();
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
@Test
public void testSpatialQueryFilteredAggregator()
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource("test")
.granularity(QueryGranularity.DAY)
.intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07")))
.aggregators(
Arrays.asList(
new CountAggregatorFactory("rows"),
new FilteredAggregatorFactory(
new LongSumAggregatorFactory("valFiltered", "val"),
new SpatialDimFilter(
"dim.geo",
new RectangularBound(new float[]{0.0f, 0.0f}, new float[]{9.0f, 9.0f})
)
),
new LongSumAggregatorFactory("val", "val")
)
)
.build();
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList(
new Result<>(
new DateTime("2013-01-01T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 4995L)
.put("val", 12497502L)
.put("valFiltered", 17L)
.build()
)
),
new Result<>(
new DateTime("2013-01-02T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 29L)
.put("valFiltered", 29L)
.build()
)
),
new Result<>(
new DateTime("2013-01-03T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 13L)
.put("valFiltered", 13L)
.build()
)
),
new Result<>(
new DateTime("2013-01-04T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 1L)
.put("val", 91L)
.put("valFiltered", 91L)
.build()
)
),
new Result<>(
new DateTime("2013-01-05T00:00:00.000Z"),
new TimeseriesResultValue(
ImmutableMap.<String, Object>builder()
.put("rows", 2L)
.put("val", 548L)
.put("valFiltered", 47L)
.build()
)
)
);
try {
TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
new TimeseriesQueryQueryToolChest(
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
new TimeseriesQueryEngine(),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
QueryRunner runner = new FinalizeResultsQueryRunner(
factory.createRunner(segment),
factory.getToolchest()
);
HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
}
catch (Exception e) {