mirror of https://github.com/apache/druid.git
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:
parent
7d36f540e8
commit
7e80d1045a
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
)
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
Loading…
Reference in New Issue