mirror of https://github.com/apache/druid.git
UTs for hyperUnique aggregation
This commit is contained in:
parent
160d5fe6b7
commit
215c1ab01e
|
@ -82,7 +82,7 @@ Example JavaScript aggregator:
|
|||
The hyperUniqueCardinality post aggregator is used to wrap a hyperUnique object such that it can be used in post aggregations.
|
||||
|
||||
```json
|
||||
{ "type" : "hyperUniqueCardinality", "fieldName" : <the name field value of the hyperUnique aggregator>}
|
||||
{ "type" : "hyperUniqueCardinality", "name": <output name>, "fieldName" : <the name field value of the hyperUnique aggregator>}
|
||||
```
|
||||
|
||||
It can be used in a sample calculation as so:
|
||||
|
|
|
@ -19,6 +19,7 @@ package io.druid.query.aggregation.hyperloglog;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
||||
|
@ -30,14 +31,20 @@ import java.util.Set;
|
|||
*/
|
||||
public class HyperUniqueFinalizingPostAggregator implements PostAggregator
|
||||
{
|
||||
private final String name;
|
||||
private final String fieldName;
|
||||
|
||||
@JsonCreator
|
||||
public HyperUniqueFinalizingPostAggregator(
|
||||
@JsonProperty("name") String name,
|
||||
@JsonProperty("fieldName") String fieldName
|
||||
)
|
||||
{
|
||||
this.fieldName = fieldName;
|
||||
this.fieldName = Preconditions.checkNotNull(fieldName, "fieldName is null");
|
||||
//Note that, in general, name shouldn't be null, we are defaulting
|
||||
//to fieldName here just to be backward compatible with 0.7.x
|
||||
this.name = name == null ? fieldName : name;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -59,8 +66,14 @@ public class HyperUniqueFinalizingPostAggregator implements PostAggregator
|
|||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty("fieldName")
|
||||
@JsonProperty("name")
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty("fieldName")
|
||||
public String getFieldName()
|
||||
{
|
||||
return fieldName;
|
||||
}
|
||||
|
|
|
@ -152,7 +152,7 @@ public class QueryRunnerTestHelper
|
|||
public static ArithmeticPostAggregator hyperUniqueFinalizingPostAgg = new ArithmeticPostAggregator(
|
||||
hyperUniqueFinalizingPostAggMetric,
|
||||
"+",
|
||||
Lists.newArrayList(new HyperUniqueFinalizingPostAggregator(uniqueMetric), new ConstantPostAggregator(null, 1))
|
||||
Lists.newArrayList(new HyperUniqueFinalizingPostAggregator(uniqueMetric, uniqueMetric), new ConstantPostAggregator(null, 1))
|
||||
);
|
||||
|
||||
public static final List<AggregatorFactory> commonAggregators = Arrays.asList(
|
||||
|
|
|
@ -36,7 +36,7 @@ public class HyperUniqueFinalizingPostAggregatorTest
|
|||
{
|
||||
Random random = new Random(0L);
|
||||
HyperUniqueFinalizingPostAggregator postAggregator = new HyperUniqueFinalizingPostAggregator(
|
||||
"uniques"
|
||||
"uniques", "uniques"
|
||||
);
|
||||
HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
|
||||
|
||||
|
|
|
@ -0,0 +1,92 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.query.aggregation.hyperloglog;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.jackson.AggregatorsModule;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
|
||||
public class HyperUniquesAggregationTest
|
||||
{
|
||||
@Test
|
||||
public void testIngestAndQuery() throws Exception
|
||||
{
|
||||
AggregationTestHelper helper = new AggregationTestHelper(Lists.newArrayList(new AggregatorsModule()));
|
||||
|
||||
String metricSpec = "[{"
|
||||
+ "\"type\": \"hyperUnique\","
|
||||
+ "\"name\": \"index_hll\","
|
||||
+ "\"fieldName\": \"market\""
|
||||
+ "}]";
|
||||
|
||||
String parseSpec = "{"
|
||||
+ "\"type\" : \"string\","
|
||||
+ "\"parseSpec\" : {"
|
||||
+ " \"format\" : \"tsv\","
|
||||
+ " \"timestampSpec\" : {"
|
||||
+ " \"column\" : \"timestamp\","
|
||||
+ " \"format\" : \"auto\""
|
||||
+ "},"
|
||||
+ " \"dimensionsSpec\" : {"
|
||||
+ " \"dimensions\": [],"
|
||||
+ " \"dimensionExclusions\" : [],"
|
||||
+ " \"spatialDimensions\" : []"
|
||||
+ " },"
|
||||
+ " \"columns\": [\"timestamp\", \"market\", \"quality\", \"placement\", \"placementish\", \"index\"]"
|
||||
+ " }"
|
||||
+ "}";
|
||||
|
||||
String query = "{"
|
||||
+ "\"queryType\": \"groupBy\","
|
||||
+ "\"dataSource\": \"test_datasource\","
|
||||
+ "\"granularity\": \"ALL\","
|
||||
+ "\"dimensions\": [],"
|
||||
+ "\"aggregations\": ["
|
||||
+ " { \"type\": \"hyperUnique\", \"name\": \"index_hll\", \"fieldName\": \"index_hll\" }"
|
||||
+ "],"
|
||||
+ "\"postAggregations\": ["
|
||||
+ " { \"type\": \"hyperUniqueCardinality\", \"name\": \"index_unique_count\", \"fieldName\": \"index_hll\" }"
|
||||
+ "],"
|
||||
+ "\"intervals\": [ \"1970/2050\" ]"
|
||||
+ "}";
|
||||
|
||||
Sequence seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("druid.sample.tsv").getFile()),
|
||||
parseSpec,
|
||||
metricSpec,
|
||||
0,
|
||||
QueryGranularity.NONE,
|
||||
50000,
|
||||
query
|
||||
);
|
||||
|
||||
MapBasedRow row = (MapBasedRow) Sequences.toList(seq, Lists.newArrayList()).get(0);
|
||||
Assert.assertEquals(3.0, row.getFloatMetric("index_hll"), 0.1);
|
||||
Assert.assertEquals(3.0, row.getFloatMetric("index_unique_count"), 0.1);
|
||||
}
|
||||
}
|
|
@ -289,7 +289,7 @@ public class GroupByQueryRunnerTest
|
|||
)
|
||||
.setPostAggregatorSpecs(
|
||||
Arrays.<PostAggregator>asList(
|
||||
new HyperUniqueFinalizingPostAggregator("quality_uniques")
|
||||
new HyperUniqueFinalizingPostAggregator("quality_uniques", "quality_uniques")
|
||||
)
|
||||
)
|
||||
.setGranularity(QueryRunnerTestHelper.allGran)
|
||||
|
|
Loading…
Reference in New Issue