Exercise v2 engine in the groupBy aggregator and multi-value dimension tests. (#3698)

This also involved some other test changes:

- Added a factory.mergeRunners step to AggregationTestHelper's groupBy chain, since the v2
  engine does merging there.
- Changed test byteBuffer pools from on-heap to off-heap to work around
  https://github.com/DataSketches/sketches-core/pull/116 for datasketches tests.
This commit is contained in:
Gian Merlino 2016-11-16 20:02:25 -08:00 committed by Fangjin Yang
parent 7d36f540e8
commit 7e80d1045a
9 changed files with 288 additions and 146 deletions

View File

@ -19,15 +19,15 @@
package io.druid.query.aggregation;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedRow;
import io.druid.data.input.Row;
import io.druid.granularity.QueryGranularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.segment.ColumnSelectorFactory;
import org.easymock.EasyMock;
import org.joda.time.DateTime;
@ -39,7 +39,6 @@ import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.File;
import java.sql.Timestamp;
import java.util.List;
@ -58,39 +57,49 @@ public class TimestampGroupByAggregationTest
private Timestamp[] values = new Timestamp[10];
@Parameterized.Parameters(name="{index}: Test for {0}")
@Parameterized.Parameters(name="{index}: Test for {0}, config = {1}")
public static Iterable<Object[]> constructorFeeder()
{
return Iterables.transform(
ImmutableList.of(
ImmutableList.of("timeMin", "tmin", "time_min", TimestampMinAggregatorFactory.class, DateTime.parse("2011-01-12T01:00:00.000Z")),
ImmutableList.of("timeMax", "tmax", "time_max", TimestampMaxAggregatorFactory.class, DateTime.parse("2011-01-31T01:00:00.000Z"))
),
new Function<List<?>, Object[]>()
{
@Nullable
@Override
public Object[] apply(List<?> input)
{
return input.toArray();
}
}
final List<Object[]> constructors = Lists.newArrayList();
final List<List<Object>> partialConstructors = ImmutableList.<List<Object>>of(
ImmutableList.<Object>of("timeMin", "tmin", "time_min", TimestampMinAggregatorFactory.class, DateTime.parse("2011-01-12T01:00:00.000Z")),
ImmutableList.<Object>of("timeMax", "tmax", "time_max", TimestampMaxAggregatorFactory.class, DateTime.parse("2011-01-31T01:00:00.000Z"))
);
for (final List<Object> partialConstructor : partialConstructors) {
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
final List<Object> constructor = Lists.newArrayList(partialConstructor);
constructor.add(config);
constructors.add(constructor.toArray());
}
}
return constructors;
}
private String aggType;
private String aggField;
private String groupByField;
private Class<? extends TimestampAggregatorFactory> aggClass;
private DateTime expected;
private final String aggType;
private final String aggField;
private final String groupByField;
private final Class<? extends TimestampAggregatorFactory> aggClass;
private final DateTime expected;
private final GroupByQueryConfig config;
public TimestampGroupByAggregationTest(String aggType, String aggField, String groupByField, Class<? extends TimestampAggregatorFactory> aggClass, DateTime expected)
public TimestampGroupByAggregationTest(
String aggType,
String aggField,
String groupByField,
Class<? extends TimestampAggregatorFactory> aggClass,
DateTime expected,
GroupByQueryConfig config
)
{
this.aggType = aggType;
this.aggField = aggField;
this.groupByField = groupByField;
this.aggClass = aggClass;
this.expected = expected;
this.config = config;
}
@Before
@ -98,6 +107,7 @@ public class TimestampGroupByAggregationTest
{
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
new TimestampMinMaxModule().getJacksonModules(),
config,
temporaryFolder
);

View File

@ -37,21 +37,27 @@ import io.druid.query.aggregation.AggregationTestHelper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
/**
*/
@RunWith(Parameterized.class)
public class SketchAggregationTest
{
private final AggregationTestHelper helper;
@ -59,11 +65,25 @@ public class SketchAggregationTest
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
public SketchAggregationTest()
public SketchAggregationTest(final GroupByQueryConfig config)
{
SketchModule sm = new SketchModule();
sm.configure(null);
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(sm.getJacksonModules(), tempFolder);
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
config,
tempFolder
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@Test
@ -87,10 +107,10 @@ public class SketchAggregationTest
ImmutableMap
.<String, Object>builder()
.put("sids_sketch_count", 50.0)
.put("sids_sketch_count_with_err",
.put("sids_sketch_count_with_err",
new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2))
.put("sketchEstimatePostAgg", 50.0)
.put("sketchEstimatePostAggWithErrorBounds",
.put("sketchEstimatePostAggWithErrorBounds",
new SketchEstimateWithErrorBounds(50.0, 50.0, 50.0, 2))
.put("sketchUnionPostAggEstimate", 50.0)
.put("sketchIntersectionPostAggEstimate", 50.0)
@ -216,7 +236,7 @@ public class SketchAggregationTest
agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, false, null, null);
Assert.assertEquals(sketch, agg.finalizeComputation(sketch));
agg = new SketchMergeAggregatorFactory("name", "fieldName", 16, true, null, 2);
SketchEstimateWithErrorBounds est = (SketchEstimateWithErrorBounds) agg.finalizeComputation(sketch);
Assert.assertEquals(0.0, est.getEstimate(), 0.0001);
@ -247,7 +267,7 @@ public class SketchAggregationTest
null
)
);
assertPostAggregatorSerde(
new SketchEstimatePostAggregator(
"name",

View File

@ -31,6 +31,8 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregationTestHelper;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.select.SelectResultValue;
import io.druid.query.timeseries.TimeseriesResultValue;
import io.druid.query.topn.DimensionAndMetricValueExtractor;
@ -41,23 +43,44 @@ import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Collection;
import java.util.List;
/**
*/
@RunWith(Parameterized.class)
public class SketchAggregationTestWithSimpleData
{
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
private final GroupByQueryConfig config;
private SketchModule sm;
private File s1;
private File s2;
public SketchAggregationTestWithSimpleData(GroupByQueryConfig config)
{
this.config = config;
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@Before
public void setup() throws Exception
{
@ -65,6 +88,7 @@ public class SketchAggregationTestWithSimpleData
sm.configure(null);
AggregationTestHelper toolchest = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
config,
tempFolder
);
@ -97,6 +121,7 @@ public class SketchAggregationTestWithSimpleData
{
AggregationTestHelper gpByQueryAggregationTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
config,
tempFolder
);
@ -178,7 +203,7 @@ public class SketchAggregationTestWithSimpleData
results
);
}
@Test
public void testSimpleDataIngestAndTimeseriesQuery() throws Exception
{

View File

@ -22,7 +22,6 @@ package io.druid.query.aggregation.datasketches.theta.oldapi;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.io.Files;
import io.druid.data.input.MapBasedRow;
import io.druid.granularity.QueryGranularities;
import io.druid.java.util.common.guava.Sequence;
@ -31,19 +30,25 @@ import io.druid.query.aggregation.AggregationTestHelper;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.aggregation.post.FieldAccessPostAggregator;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import org.joda.time.DateTime;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Collection;
import java.util.List;
/**
*/
@RunWith(Parameterized.class)
public class OldApiSketchAggregationTest
{
private final AggregationTestHelper helper;
@ -51,17 +56,28 @@ public class OldApiSketchAggregationTest
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
public OldApiSketchAggregationTest()
public OldApiSketchAggregationTest(final GroupByQueryConfig config)
{
OldApiSketchModule sm = new OldApiSketchModule();
sm.configure(null);
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
sm.getJacksonModules(),
config,
tempFolder
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@Test
public void testSimpleDataIngestAndQuery() throws Exception
{

View File

@ -20,19 +20,27 @@
package io.druid.query.aggregation.histogram;
import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedRow;
import io.druid.granularity.QueryGranularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.aggregation.AggregationTestHelper;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
/**
*/
@RunWith(Parameterized.class)
public class ApproximateHistogramAggregationTest
{
private AggregationTestHelper helper;
@ -40,16 +48,27 @@ public class ApproximateHistogramAggregationTest
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
public ApproximateHistogramAggregationTest()
public ApproximateHistogramAggregationTest(final GroupByQueryConfig config)
{
ApproximateHistogramDruidModule module = new ApproximateHistogramDruidModule();
module.configure(null);
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
Lists.newArrayList(module.getJacksonModules()),
config,
tempFolder
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@Test
public void testIngestWithNullsIgnoredAndQuery() throws Exception
{

View File

@ -43,6 +43,8 @@ import io.druid.query.dimension.ListFilteredDimensionSpec;
import io.druid.query.dimension.RegexFilteredDimensionSpec;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import io.druid.query.groupby.GroupByQueryRunnerTestHelper;
import io.druid.query.spec.LegacySegmentSpec;
import io.druid.query.topn.TopNQuery;
@ -64,15 +66,20 @@ import org.joda.time.DateTime;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
/**
*/
@RunWith(Parameterized.class)
public class MultiValuedDimensionTest
{
private AggregationTestHelper helper;
@ -82,13 +89,27 @@ public class MultiValuedDimensionTest
private static File persistedSegmentDir;
public MultiValuedDimensionTest() throws Exception
public MultiValuedDimensionTest(
final GroupByQueryConfig config
) throws Exception
{
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
ImmutableList.<Module>of(), null
ImmutableList.<Module>of(),
config,
null
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@BeforeClass
public static void setupClass() throws Exception
{
@ -169,14 +190,6 @@ public class MultiValuedDimensionTest
);
TestHelper.assertExpectedObjects(expectedResults, Sequences.toList(result, new ArrayList<Row>()), "");
result = helper.runQueryOnSegmentsObjs(
ImmutableList.<Segment>of(
new QueryableIndexSegment("sid1", queryableIndex),
new IncrementalIndexSegment(incrementalIndex, "sid2")
),
query
);
}
@Test

View File

@ -33,6 +33,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.collections.StupidPool;
import io.druid.data.input.Row;
import io.druid.data.input.impl.InputRowParser;
@ -45,7 +46,6 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.guava.Yielder;
import io.druid.java.util.common.guava.YieldingAccumulator;
import io.druid.query.ConcatQueryRunner;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.IntervalChunkingQueryRunnerDecorator;
import io.druid.query.Query;
@ -128,11 +128,12 @@ public class AggregationTestHelper
public static final AggregationTestHelper createGroupByQueryAggregationTestHelper(
List<? extends Module> jsonModulesToRegister,
GroupByQueryConfig config,
TemporaryFolder tempFolder
)
{
ObjectMapper mapper = new DefaultObjectMapper();
GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig());
GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(mapper, config);
IndexIO indexIO = new IndexIO(
mapper,
@ -479,29 +480,28 @@ public class AggregationTestHelper
toolChest.postMergeQueryDecoration(
toolChest.mergeResults(
toolChest.preMergeQueryDecoration(
new ConcatQueryRunner(
Sequences.simple(
Lists.transform(
segments,
new Function<Segment, QueryRunner>()
{
@Override
public QueryRunner apply(final Segment segment)
{
try {
return makeStringSerdeQueryRunner(
mapper,
toolChest,
query,
factory.createRunner(segment)
);
}
catch (Exception ex) {
throw Throwables.propagate(ex);
}
}
factory.mergeRunners(
MoreExecutors.sameThreadExecutor(),
Lists.transform(
segments,
new Function<Segment, QueryRunner>()
{
@Override
public QueryRunner apply(final Segment segment)
{
try {
return makeStringSerdeQueryRunner(
mapper,
toolChest,
query,
factory.createRunner(segment)
);
}
)
catch (Exception ex) {
throw Throwables.propagate(ex);
}
}
}
)
)
)

View File

@ -26,23 +26,49 @@ import io.druid.jackson.AggregatorsModule;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.aggregation.AggregationTestHelper;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryRunnerTest;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
@RunWith(Parameterized.class)
public class HyperUniquesAggregationTest
{
@Rule
public final TemporaryFolder tempFolder = new TemporaryFolder();
private final GroupByQueryConfig config;
public HyperUniquesAggregationTest(GroupByQueryConfig config)
{
this.config = config;
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
constructors.add(new Object[]{config});
}
return constructors;
}
@Test
public void testIngestAndQuery() throws Exception
{
AggregationTestHelper helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
Lists.newArrayList(new AggregatorsModule()),
config,
tempFolder
);

View File

@ -19,6 +19,7 @@
package io.druid.query.groupby;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
@ -140,78 +141,10 @@ public class GroupByQueryRunnerTest
@Rule
public ExpectedException expectedException = ExpectedException.none();
public static GroupByQueryRunnerFactory makeQueryRunnerFactory(
final GroupByQueryConfig config
)
public static List<GroupByQueryConfig> testConfigs()
{
final Supplier<GroupByQueryConfig> configSupplier = Suppliers.ofInstance(config);
final StupidPool<ByteBuffer> bufferPool = new StupidPool<>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocate(10 * 1024 * 1024);
}
}
);
final BlockingPool<ByteBuffer> mergeBufferPool = new BlockingPool<>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocate(10 * 1024 * 1024);
}
},
2 // There are some tests that need to allocate two buffers (simulating two levels of merging)
);
final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
configSupplier,
new GroupByStrategyV1(
configSupplier,
new GroupByQueryEngine(configSupplier, bufferPool),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
bufferPool
),
new GroupByStrategyV2(
new DruidProcessingConfig()
{
@Override
public String getFormatString()
{
return null;
}
@Override
public int getNumThreads()
{
return 2;
}
},
configSupplier,
bufferPool,
mergeBufferPool,
new DefaultObjectMapper(new SmileFactory()),
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(
configSupplier,
strategySelector,
bufferPool,
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
);
return new GroupByQueryRunnerFactory(
strategySelector,
toolChest
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final GroupByQueryConfig defaultConfig = new GroupByQueryConfig() {
final GroupByQueryConfig defaultConfig = new GroupByQueryConfig()
{
@Override
public String toString()
{
@ -272,7 +205,7 @@ public class GroupByQueryRunnerTest
return "v2SmallBuffer";
}
};
final GroupByQueryConfig epinephelinaeSmallDictionaryConfig = new GroupByQueryConfig()
final GroupByQueryConfig v2SmallDictionaryConfig = new GroupByQueryConfig()
{
@Override
public String getDefaultStrategy()
@ -295,23 +228,103 @@ public class GroupByQueryRunnerTest
@Override
public String toString()
{
return "epinephelinaeSmallDictionary";
return "v2SmallDictionary";
}
};
defaultConfig.setMaxIntermediateRows(10000);
singleThreadedConfig.setMaxIntermediateRows(10000);
final List<Object[]> constructors = Lists.newArrayList();
final List<GroupByQueryConfig> configs = ImmutableList.of(
return ImmutableList.of(
defaultConfig,
singleThreadedConfig,
v2Config,
v2SmallBufferConfig,
epinephelinaeSmallDictionaryConfig
v2SmallDictionaryConfig
);
}
for (GroupByQueryConfig config : configs) {
public static GroupByQueryRunnerFactory makeQueryRunnerFactory(
final GroupByQueryConfig config
)
{
return makeQueryRunnerFactory(new DefaultObjectMapper(new SmileFactory()), config);
}
public static GroupByQueryRunnerFactory makeQueryRunnerFactory(
final ObjectMapper mapper,
final GroupByQueryConfig config
)
{
final Supplier<GroupByQueryConfig> configSupplier = Suppliers.ofInstance(config);
final StupidPool<ByteBuffer> bufferPool = new StupidPool<>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocateDirect(10 * 1024 * 1024);
}
}
);
final BlockingPool<ByteBuffer> mergeBufferPool = new BlockingPool<>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocateDirect(10 * 1024 * 1024);
}
},
2 // There are some tests that need to allocate two buffers (simulating two levels of merging)
);
final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
configSupplier,
new GroupByStrategyV1(
configSupplier,
new GroupByQueryEngine(configSupplier, bufferPool),
QueryRunnerTestHelper.NOOP_QUERYWATCHER,
bufferPool
),
new GroupByStrategyV2(
new DruidProcessingConfig()
{
@Override
public String getFormatString()
{
return null;
}
@Override
public int getNumThreads()
{
return 2;
}
},
configSupplier,
bufferPool,
mergeBufferPool,
mapper,
QueryRunnerTestHelper.NOOP_QUERYWATCHER
)
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(
configSupplier,
strategySelector,
bufferPool,
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
);
return new GroupByQueryRunnerFactory(
strategySelector,
toolChest
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder() throws IOException
{
final List<Object[]> constructors = Lists.newArrayList();
for (GroupByQueryConfig config : testConfigs()) {
final GroupByQueryRunnerFactory factory = makeQueryRunnerFactory(config);
for (QueryRunner<Row> runner : QueryRunnerTestHelper.makeQueryRunners(factory)) {
final String testName = String.format(