1) Fix PostAggregations with GroupBy (Fixes #24)

2) Add GroupByTimeseriesQueryRunnerTest which wraps a GroupByQueryRunner to look like a TimeseriesQueryRunner and leverages the tests there to help verify behavior (#8)
This commit is contained in:
Eric Tschetter 2012-11-21 12:51:43 -06:00
parent 7497a31943
commit 06b8e4230c
7 changed files with 335 additions and 66 deletions

View File

@ -0,0 +1,85 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.query.filter;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.List;
/**
*/
public class DimFilters
{
public static SelectorDimFilter dimEquals(String dimension, String value)
{
return new SelectorDimFilter(dimension, value);
}
public static AndDimFilter and(DimFilter... filters)
{
return and(Arrays.asList(filters));
}
public static AndDimFilter and(List<DimFilter> filters)
{
return new AndDimFilter(filters);
}
public static OrDimFilter or(DimFilter... filters)
{
return or(Arrays.asList(filters));
}
public static OrDimFilter or(List<DimFilter> filters)
{
return new OrDimFilter(filters);
}
public static NotDimFilter not(DimFilter filter)
{
return new NotDimFilter(filter);
}
public static RegexDimFilter regex(String dimension, String pattern)
{
return new RegexDimFilter(dimension, pattern);
}
public static DimFilter dimEquals(final String dimension, String... values)
{
return or(
Lists.transform(
Arrays.asList(values),
new Function<String, DimFilter>()
{
@Override
public DimFilter apply(String input)
{
return dimEquals(dimension, input);
}
}
)
);
}
}

View File

@ -73,7 +73,7 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
@Override
public Sequence<Row> run(Query<Row> input)
{
GroupByQuery query = (GroupByQuery) input;
final GroupByQuery query = (GroupByQuery) input;
List<Interval> condensed = query.getIntervals();
final List<AggregatorFactory> aggs = Lists.transform(
@ -119,7 +119,7 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
}
);
return Sequences.simple(index);
return Sequences.simple(index.iterableWithPostAggregations(query.getPostAggregatorSpecs()));
}
};
}

View File

@ -35,6 +35,7 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.QueryGranularity;
import com.metamx.druid.aggregation.Aggregator;
import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.aggregation.post.PostAggregator;
import com.metamx.druid.index.v1.serde.ComplexMetricExtractor;
import com.metamx.druid.index.v1.serde.ComplexMetricSerde;
import com.metamx.druid.index.v1.serde.ComplexMetrics;
@ -53,7 +54,6 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap;
@ -341,34 +341,52 @@ public class IncrementalIndex implements Iterable<Row>
@Override
public Iterator<Row> iterator()
{
return Iterators.transform(
facts.entrySet().iterator(),
new Function<Map.Entry<TimeAndDims, Aggregator[]>, Row>()
{
@Override
public Row apply(final Map.Entry<TimeAndDims, Aggregator[]> input)
{
final TimeAndDims timeAndDims = input.getKey();
final Aggregator[] aggregators = input.getValue();
return iterableWithPostAggregations(null).iterator();
}
String[][] theDims = timeAndDims.getDims();
public Iterable<Row> iterableWithPostAggregations(final List<PostAggregator> postAggs)
{
return new Iterable<Row>()
{
@Override
public Iterator<Row> iterator()
{
return Iterators.transform(
facts.entrySet().iterator(),
new Function<Map.Entry<TimeAndDims, Aggregator[]>, Row>()
{
@Override
public Row apply(final Map.Entry<TimeAndDims, Aggregator[]> input)
{
final TimeAndDims timeAndDims = input.getKey();
final Aggregator[] aggregators = input.getValue();
Map<String, Object> theVals = Maps.newLinkedHashMap();
for (int i = 0; i < theDims.length; ++i) {
String[] dim = theDims[i];
if (dim != null && dim.length != 0) {
theVals.put(dimensions.get(i), dim.length == 1 ? dim[0] : Arrays.asList(dim));
String[][] theDims = timeAndDims.getDims();
Map<String, Object> theVals = Maps.newLinkedHashMap();
for (int i = 0; i < theDims.length; ++i) {
String[] dim = theDims[i];
if (dim != null && dim.length != 0) {
theVals.put(dimensions.get(i), dim.length == 1 ? dim[0] : Arrays.asList(dim));
}
}
for (int i = 0; i < aggregators.length; ++i) {
theVals.put(metrics[i].getName(), aggregators[i].get());
}
if (postAggs != null) {
for (PostAggregator postAgg : postAggs) {
theVals.put(postAgg.getName(), postAgg.compute(theVals));
}
}
return new MapBasedRow(timeAndDims.getTimestamp(), theVals);
}
}
for (int i = 0; i < aggregators.length; ++i) {
theVals.put(metrics[i].getName(), aggregators[i].get());
}
return new MapBasedRow(timeAndDims.getTimestamp(), theVals);
}
}
);
);
}
};
}
static class DimensionHolder

View File

@ -352,7 +352,7 @@ public class GroupByQueryEngine
}
for (PostAggregator postAggregator : query.getPostAggregatorSpecs()) {
postAggregator.compute(theEvent);
theEvent.put(postAggregator.getName(), postAggregator.compute(theEvent));
}
return new MapBasedRow(timestamp, theEvent);

View File

@ -52,7 +52,7 @@ import java.util.List;
public class QueryRunnerTestHelper
{
public static final String dataSource = "testing";
public static final QueryGranularity gran = QueryGranularity.DAY;
public static final QueryGranularity dayGran = QueryGranularity.DAY;
public static final QueryGranularity allGran = QueryGranularity.ALL;
public static final String providerDimension = "proVider";
public static final String qualityDimension = "quality";

View File

@ -0,0 +1,168 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.query.group;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.druid.GroupByQueryEngine;
import com.metamx.druid.GroupByQueryEngineConfig;
import com.metamx.druid.Query;
import com.metamx.druid.collect.StupidPool;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.input.MapBasedRow;
import com.metamx.druid.input.Row;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.query.QueryRunnerTestHelper;
import com.metamx.druid.query.timeseries.TimeseriesQuery;
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerTest;
import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeseriesResultValue;
import org.joda.time.DateTime;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collection;
/**
*/
@RunWith(Parameterized.class)
public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
{
@Parameterized.Parameters
public static Collection<?> constructorFeeder() throws IOException
{
final GroupByQueryRunnerFactory factory = new GroupByQueryRunnerFactory(
new GroupByQueryEngine(
new GroupByQueryEngineConfig()
{
@Override
public int getMaxIntermediateRows()
{
return 10000;
}
},
new StupidPool<ByteBuffer>(
new Supplier<ByteBuffer>()
{
@Override
public ByteBuffer get()
{
return ByteBuffer.allocate(1024 * 1024);
}
}
)
)
);
final Collection<?> objects = QueryRunnerTestHelper.makeQueryRunners(factory);
Object[][] newObjects = new Object[objects.size()][];
int i = 0;
for (Object object : objects) {
if (object instanceof Object[]) {
Object[] queryRunnerArray = (Object[]) object;
Preconditions.checkState(queryRunnerArray.length == 1);
Preconditions.checkState(queryRunnerArray[0] instanceof QueryRunner);
final QueryRunner groupByRunner = (QueryRunner) queryRunnerArray[0];
QueryRunner timeseriesRunner = new QueryRunner()
{
@Override
public Sequence run(Query query)
{
TimeseriesQuery tsQuery = (TimeseriesQuery) query;
return Sequences.map(
groupByRunner.run(
GroupByQuery.builder()
.setDataSource(tsQuery.getDataSource())
.setQuerySegmentSpec(tsQuery.getQuerySegmentSpec())
.setGranularity(tsQuery.getGranularity())
.setDimFilter(tsQuery.getDimensionsFilter())
.setAggregatorSpecs(tsQuery.getAggregatorSpecs())
.setPostAggregatorSpecs(tsQuery.getPostAggregatorSpecs())
.build()
),
new Function<Row, Result<TimeseriesResultValue>>()
{
@Override
public Result<TimeseriesResultValue> apply(final Row input)
{
MapBasedRow row = (MapBasedRow) input;
return new Result<TimeseriesResultValue>(
new DateTime(input.getTimestampFromEpoch()), new TimeseriesResultValue(row.getEvent())
);
}
}
);
}
};
newObjects[i] = new Object[]{timeseriesRunner};
++i;
}
}
return Arrays.asList(newObjects);
}
public GroupByTimeseriesQueryRunnerTest(QueryRunner runner)
{
super(runner);
}
@Override
public void testFullOnTimeseries()
{
// Skip this test because the timeseries test expects a skipped day to be filled in, but group by doesn't
// fill anything in.
}
@Override
public void testFullOnTimeseriesWithFilter()
{
// Skip this test because the timeseries test expects a skipped day to be filled in, but group by doesn't
// fill anything in.
}
@Override
public void testTimeseriesWithNonExistentFilter()
{
// Skip this test because the timeseries test expects a day that doesn't have a filter match to be filled in,
// but group by just doesn't return a value if the filter doesn't match.
}
@Override
public void testTimeseriesWithNonExistentFilterAndMultiDim()
{
// Skip this test because the timeseries test expects a day that doesn't have a filter match to be filled in,
// but group by just doesn't return a value if the filter doesn't match.
}
}

View File

@ -79,7 +79,7 @@ public class TimeseriesQueryRunnerTest
QueryGranularity gran = QueryGranularity.DAY;
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.aggregators(
Arrays.asList(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexDoubleSum)
@ -105,24 +105,23 @@ public class TimeseriesQueryRunnerTest
result.getTimestamp().isAfter(expectedLast)
);
Map value = result.getValue().getBaseObject();
final TimeseriesResultValue value = result.getValue();
Assert.assertTrue(result.toString(), value.containsKey("rows"));
Assert.assertEquals(
result.toString(),
QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0L : 13L,
value.get("rows")
value.getLongMetric("rows").longValue()
);
Assert.assertEquals(
result.toString(),
QueryRunnerTestHelper.expectedFullOnIndexValues[count],
String.valueOf(value.get("index"))
String.valueOf(value.getDoubleMetric("index"))
);
Assert.assertEquals(
result.toString(),
new Double(QueryRunnerTestHelper.expectedFullOnIndexValues[count]) +
(QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0L : 13L) + 1L,
new Double(String.valueOf(value.get("addRowsIndexConstant"))),
value.getDoubleMetric("addRowsIndexConstant"),
0.0
);
@ -163,12 +162,10 @@ public class TimeseriesQueryRunnerTest
result.getTimestamp().isAfter(expectedLast)
);
Map value = result.getValue().getBaseObject();
final TimeseriesResultValue value = result.getValue();
Assert.assertTrue(result.toString(), value.containsKey("maxIndex"));
Assert.assertTrue(result.toString(), value.containsKey("minIndex"));
Assert.assertEquals(result.toString(), 1870.06103515625, value.get("maxIndex"));
Assert.assertEquals(result.toString(), 59.02102279663086, value.get("minIndex"));
Assert.assertEquals(result.toString(), 1870.06103515625, value.getDoubleMetric("maxIndex"), 0.0);
Assert.assertEquals(result.toString(), 59.02102279663086, value.getDoubleMetric("minIndex"), 0.0);
}
@Test
@ -178,7 +175,7 @@ public class TimeseriesQueryRunnerTest
QueryGranularity gran = QueryGranularity.DAY;
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "upfront")
.intervals(QueryRunnerTestHelper.fullOnInterval)
.aggregators(Arrays.<AggregatorFactory>asList(QueryRunnerTestHelper.rowsCount))
@ -207,11 +204,12 @@ public class TimeseriesQueryRunnerTest
result.getTimestamp().isAfter(expectedLast)
);
Map value = result.getValue().getBaseObject();
final TimeseriesResultValue value = result.getValue();
Assert.assertTrue(result.toString(), value.containsKey("rows"));
Assert.assertEquals(
result.toString(), QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0L : 2L, value.get("rows")
result.toString(),
QueryRunnerTestHelper.skippedDay.equals(result.getTimestamp()) ? 0L : 2L,
value.getLongMetric("rows").longValue()
);
expectedEarliest = gran.toDateTime(gran.next(expectedEarliest.getMillis()));
@ -223,7 +221,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
Arrays.<AggregatorFactory>asList(
@ -419,7 +417,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(
new MultipleIntervalSegmentSpec(
Arrays.asList(
@ -454,7 +452,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "spot", "upfront", "total_market")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
@ -501,7 +499,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(new RegexDimFilter(QueryRunnerTestHelper.providerDimension, "^.p.*$")) // spot and upfront
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(Arrays.<AggregatorFactory>asList(QueryRunnerTestHelper.rowsCount, QueryRunnerTestHelper.indexLongSum))
@ -543,7 +541,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "spot")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
@ -590,7 +588,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "upfront")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
@ -637,7 +635,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "total_market")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(
@ -698,7 +696,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -754,7 +752,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -810,7 +808,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -852,7 +850,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(
QueryRunnerTestHelper.providerDimension,
"spot",
@ -923,7 +921,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -965,7 +963,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.providerDimension, "billy")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1021,7 +1019,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1063,7 +1061,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.placementishDimension, "preferred")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1074,7 +1072,7 @@ public class TimeseriesQueryRunnerTest
runner.run(
Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
.postAggregators(Arrays.<PostAggregator>asList(QueryRunnerTestHelper.addRowsIndexConstant))
@ -1094,7 +1092,7 @@ public class TimeseriesQueryRunnerTest
{
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.placementishDimension, "a")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1105,7 +1103,7 @@ public class TimeseriesQueryRunnerTest
runner.run(
Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(QueryRunnerTestHelper.qualityDimension, "automotive")
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1140,7 +1138,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1166,7 +1164,7 @@ public class TimeseriesQueryRunnerTest
runner.run(
Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter2)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1200,7 +1198,7 @@ public class TimeseriesQueryRunnerTest
.build();
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)
@ -1226,7 +1224,7 @@ public class TimeseriesQueryRunnerTest
runner.run(
Druids.newTimeseriesQueryBuilder()
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.gran)
.granularity(QueryRunnerTestHelper.dayGran)
.filters(andDimFilter2)
.intervals(QueryRunnerTestHelper.firstToThird)
.aggregators(QueryRunnerTestHelper.commonAggregators)