mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
js/cardinality/hyperUnique happy groupBy family
This commit is contained in:
parent
fd47efd59e
commit
bfa3caa589
@ -73,12 +73,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
||||
return Aggregators.noopAggregator();
|
||||
}
|
||||
|
||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||
final Class classOfObject = selector.classOfObject();
|
||||
if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) {
|
||||
return new HyperUniquesAggregator(name, selector);
|
||||
}
|
||||
|
||||
throw new IAE(
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject()
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject
|
||||
);
|
||||
}
|
||||
|
||||
@ -91,12 +92,13 @@ public class HyperUniquesAggregatorFactory implements AggregatorFactory
|
||||
return Aggregators.noopBufferAggregator();
|
||||
}
|
||||
|
||||
if (HyperLogLogCollector.class.isAssignableFrom(selector.classOfObject())) {
|
||||
final Class classOfObject = selector.classOfObject();
|
||||
if (classOfObject.equals(Object.class) || HyperLogLogCollector.class.isAssignableFrom(classOfObject)) {
|
||||
return new HyperUniquesBufferAggregator(selector);
|
||||
}
|
||||
|
||||
throw new IAE(
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, selector.classOfObject()
|
||||
"Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -24,12 +24,14 @@ import com.google.common.collect.Lists;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Accumulator;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.Rows;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
@ -248,7 +248,7 @@ public class IncrementalIndex implements Iterable<Row>
|
||||
final ObjectColumnSelector<Object> rawColumnSelector = new ObjectColumnSelector<Object>()
|
||||
{
|
||||
@Override
|
||||
public Class<Object> classOfObject()
|
||||
public Class classOfObject()
|
||||
{
|
||||
return Object.class;
|
||||
}
|
||||
@ -294,22 +294,17 @@ public class IncrementalIndex implements Iterable<Row>
|
||||
public DimensionSelector makeDimensionSelector(final String dimension)
|
||||
{
|
||||
final String dimensionName = dimension.toLowerCase();
|
||||
final List<String> dimensionValues = in.getDimension(dimensionName);
|
||||
if (dimensionValues == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final IncrementalIndex.DimDim dimValLookup = getDimension(dimensionName);
|
||||
final int maxId = dimValLookup.size();
|
||||
return new DimensionSelector()
|
||||
{
|
||||
@Override
|
||||
public IndexedInts getRow()
|
||||
{
|
||||
final List<String> dimensionValues = in.getDimension(dimensionName);
|
||||
final ArrayList<Integer> vals = Lists.newArrayList();
|
||||
for (String dimVal : dimensionValues) {
|
||||
int id = dimValLookup.getId(dimVal);
|
||||
vals.add(id);
|
||||
if (dimensionValues != null) {
|
||||
for (int i = 0; i < dimensionValues.size(); ++i) {
|
||||
vals.add(i);
|
||||
}
|
||||
}
|
||||
|
||||
return new IndexedInts()
|
||||
@ -337,19 +332,19 @@ public class IncrementalIndex implements Iterable<Row>
|
||||
@Override
|
||||
public int getValueCardinality()
|
||||
{
|
||||
return maxId;
|
||||
throw new UnsupportedOperationException("value cardinality is unknown in incremental index");
|
||||
}
|
||||
|
||||
@Override
|
||||
public String lookupName(int id)
|
||||
{
|
||||
return dimValLookup.getValue(id);
|
||||
return in.getDimension(dimensionName).get(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int lookupId(String name)
|
||||
{
|
||||
return dimValLookup.getId(name);
|
||||
return in.getDimension(dimensionName).indexOf(name);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -28,6 +28,7 @@ import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.JavaScriptAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.cardinality.CardinalityAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniqueFinalizingPostAggregator;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.aggregation.post.ArithmeticPostAggregator;
|
||||
@ -110,6 +111,11 @@ public class QueryRunnerTestHelper
|
||||
"uniques",
|
||||
"quality_uniques"
|
||||
);
|
||||
public static final CardinalityAggregatorFactory qualityCardinality = new CardinalityAggregatorFactory(
|
||||
"cardinality",
|
||||
Arrays.asList("quality"),
|
||||
false
|
||||
);
|
||||
public static final ConstantPostAggregator constant = new ConstantPostAggregator("const", 1L, null);
|
||||
public static final FieldAccessPostAggregator rowsPostAgg = new FieldAccessPostAggregator("rows", "rows");
|
||||
public static final FieldAccessPostAggregator indexPostAgg = new FieldAccessPostAggregator("index", "index");
|
||||
|
@ -217,6 +217,36 @@ public class GroupByQueryRunnerTest
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithCardinality()
|
||||
{
|
||||
GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(
|
||||
Arrays.<AggregatorFactory>asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
QueryRunnerTestHelper.qualityCardinality
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow(
|
||||
"2011-04-01",
|
||||
"rows",
|
||||
26L,
|
||||
"cardinality",
|
||||
QueryRunnerTestHelper.UNIQUES_9
|
||||
)
|
||||
);
|
||||
|
||||
Iterable<Row> results = runQuery(query);
|
||||
TestHelper.assertExpectedObjects(expectedResults, results, "");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGroupByWithDimExtractionFn()
|
||||
{
|
||||
@ -1209,7 +1239,7 @@ public class GroupByQueryRunnerTest
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSubqueryWithEverything()
|
||||
public void testSubqueryWithMultiColumnAggregators()
|
||||
{
|
||||
final GroupByQuery subquery = GroupByQuery
|
||||
.builder()
|
||||
@ -1220,11 +1250,11 @@ public class GroupByQueryRunnerTest
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
QueryRunnerTestHelper.rowsCount,
|
||||
new LongSumAggregatorFactory("idx_subagg", "index"),
|
||||
new DoubleSumAggregatorFactory("idx_subagg", "index"),
|
||||
new JavaScriptAggregatorFactory(
|
||||
"js_agg",
|
||||
Arrays.asList("index", "provider"),
|
||||
"function(index, dim){return index + dim.length;}",
|
||||
"function(current, index, dim){return current + index + dim.length;}",
|
||||
"function(){return 0;}",
|
||||
"function(a,b){return a + b;}"
|
||||
)
|
||||
@ -1295,23 +1325,11 @@ public class GroupByQueryRunnerTest
|
||||
.build();
|
||||
|
||||
List<Row> expectedResults = Arrays.asList(
|
||||
createExpectedRow("2011-04-01", "alias", "automotive", "rows", 1L, "idx", 11135.0),
|
||||
createExpectedRow("2011-04-01", "alias", "business", "rows", 1L, "idx", 11118.0),
|
||||
createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0),
|
||||
createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0),
|
||||
createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0),
|
||||
createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0),
|
||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0),
|
||||
|
||||
createExpectedRow("2011-04-02", "alias", "automotive", "rows", 1L, "idx", 11147.0),
|
||||
createExpectedRow("2011-04-02", "alias", "business", "rows", 1L, "idx", 11112.0),
|
||||
createExpectedRow("2011-04-02", "alias", "entertainment", "rows", 1L, "idx", 11166.0),
|
||||
createExpectedRow("2011-04-02", "alias", "health", "rows", 1L, "idx", 11113.0),
|
||||
createExpectedRow("2011-04-02", "alias", "mezzanine", "rows", 3L, "idx", 13447.0),
|
||||
createExpectedRow("2011-04-02", "alias", "news", "rows", 1L, "idx", 11114.0),
|
||||
createExpectedRow("2011-04-02", "alias", "premium", "rows", 3L, "idx", 13505.0),
|
||||
createExpectedRow("2011-04-02", "alias", "technology", "rows", 1L, "idx", 11097.0),
|
||||
createExpectedRow("2011-04-02", "alias", "travel", "rows", 1L, "idx", 11126.0)
|
||||
createExpectedRow("2011-04-01", "alias", "travel", "rows", 1L, "idx", 11119.0, "js_outer_agg", 123.92274475097656),
|
||||
createExpectedRow("2011-04-01", "alias", "technology", "rows", 1L, "idx", 11078.0, "js_outer_agg", 82.62254333496094),
|
||||
createExpectedRow("2011-04-01", "alias", "news", "rows", 1L, "idx", 11121.0, "js_outer_agg", 125.58358001708984),
|
||||
createExpectedRow("2011-04-01", "alias", "health", "rows", 1L, "idx", 11120.0, "js_outer_agg", 124.13470458984375),
|
||||
createExpectedRow("2011-04-01", "alias", "entertainment", "rows", 1L, "idx", 11158.0, "js_outer_agg", 162.74722290039062)
|
||||
);
|
||||
|
||||
// Subqueries are handled by the ToolChest
|
||||
|
Loading…
x
Reference in New Issue
Block a user