Merge pull request #2752 from druid-io/cleanup

cleanup distinct count agg
This commit is contained in:
binlijin 2016-03-29 09:58:18 +08:00
commit 5120b8ab5c
4 changed files with 8 additions and 3 deletions

View File

@ -43,11 +43,11 @@ If you'd like to take on maintenance for a community extension, please post on [
|druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/extensions-contrib/azure.html)| |druid-azure-extensions|Microsoft Azure deep storage.|[link](../development/extensions-contrib/azure.html)|
|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.html)| |druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.html)|
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)| |druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)|
|druid-distinctcount|DistinctCount aggregator|[link](../development/extensions-contrib/distinctcount.html)|
|druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/extensions-contrib/kafka-simple.html)| |druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer).|[link](../development/extensions-contrib/kafka-simple.html)|
|druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)| |druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)|
|druid-rocketmq|RocketMQ firehose.|[link](../development/extensions-contrib/rocketmq.html)| |druid-rocketmq|RocketMQ firehose.|[link](../development/extensions-contrib/rocketmq.html)|
|graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)| |graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)|
|druid-distinctcount|DistinctCount aggregator|[link](../development/extensions-contrib/distinctcount.html)|
## Promoting Community Extension to Core Extension ## Promoting Community Extension to Core Extension

View File

@ -39,6 +39,7 @@
<groupId>io.druid</groupId> <groupId>io.druid</groupId>
<artifactId>druid-processing</artifactId> <artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version> <version>${project.parent.version}</version>
<scope>provided</scope>
</dependency> </dependency>
<!-- Tests --> <!-- Tests -->

View File

@ -27,6 +27,7 @@ import com.metamx.common.StringUtils;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.AggregatorUtil;
import io.druid.query.aggregation.BufferAggregator; import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DefaultDimensionSpec;
@ -151,7 +152,7 @@ public class DistinctCountAggregatorFactory extends AggregatorFactory
return fieldName; return fieldName;
} }
@JsonProperty("bitmap") @JsonProperty("bitmapFactory")
public BitMapFactory getBitMapFactory() public BitMapFactory getBitMapFactory()
{ {
return bitMapFactory; return bitMapFactory;
@ -175,9 +176,10 @@ public class DistinctCountAggregatorFactory extends AggregatorFactory
{ {
byte[] fieldNameBytes = StringUtils.toUtf8(fieldName); byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
byte[] bitMapFactoryCacheKey = StringUtils.toUtf8(bitMapFactory.toString()); byte[] bitMapFactoryCacheKey = StringUtils.toUtf8(bitMapFactory.toString());
return ByteBuffer.allocate(1 + fieldNameBytes.length + bitMapFactoryCacheKey.length) return ByteBuffer.allocate(2 + fieldNameBytes.length + bitMapFactoryCacheKey.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(fieldNameBytes) .put(fieldNameBytes)
.put(AggregatorUtil.STRING_SEPARATOR)
.put(bitMapFactoryCacheKey) .put(bitMapFactoryCacheKey)
.array(); .array();
} }

View File

@ -29,6 +29,8 @@ import java.util.Set;
public class AggregatorUtil public class AggregatorUtil
{ {
public static final byte STRING_SEPARATOR = (byte) 0xFF;
/** /**
* returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg * returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg
* *