UTs for hyperUnique aggregation

This commit is contained in:
Himanshu Gupta 2015-05-22 17:02:39 -05:00
parent 160d5fe6b7
commit 215c1ab01e
6 changed files with 111 additions and 6 deletions

View File

@ -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:

View File

@ -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;
}

View File

@ -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(

View File

@ -36,7 +36,7 @@ public class HyperUniqueFinalizingPostAggregatorTest
{
Random random = new Random(0L);
HyperUniqueFinalizingPostAggregator postAggregator = new HyperUniqueFinalizingPostAggregator(
"uniques"
"uniques", "uniques"
);
HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();

View File

@ -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);
}
}

View File

@ -289,7 +289,7 @@ public class GroupByQueryRunnerTest
)
.setPostAggregatorSpecs(
Arrays.<PostAggregator>asList(
new HyperUniqueFinalizingPostAggregator("quality_uniques")
new HyperUniqueFinalizingPostAggregator("quality_uniques", "quality_uniques")
)
)
.setGranularity(QueryRunnerTestHelper.allGran)