mirror of https://github.com/apache/druid.git
Merge pull request #2752 from druid-io/cleanup
cleanup distinct count agg
This commit is contained in:
commit
5120b8ab5c
|
@ -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
|
||||||
|
|
||||||
|
|
|
@ -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 -->
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
*
|
*
|
||||||
|
|
Loading…
Reference in New Issue