mirror of https://github.com/apache/druid.git
TDigest backed sketch aggregators (#7331)
* First set of changes for tDigest histogram * Add license * Address code review comments * Add a doc page for new T-Digest sketch aggregators. Minor code cleanup and comments. * Remove synchronization from BufferAggregators. Address code review comments * Fix typo
This commit is contained in:
parent
b32c9abdb9
commit
b542bb9f34
|
@ -334,7 +334,9 @@
|
|||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-virtual-columns</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-moving-average-query</argument> </arguments>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-moving-average-query</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:druid-tdigestsketch</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
|
|
|
@ -0,0 +1,159 @@
|
|||
---
|
||||
layout: doc_page
|
||||
title: "T-Digest Quantiles Sketch module"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ Licensed to the Apache Software Foundation (ASF) under one
|
||||
~ or more contributor license agreements. See the NOTICE file
|
||||
~ distributed with this work for additional information
|
||||
~ regarding copyright ownership. The ASF 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.
|
||||
-->
|
||||
|
||||
# T-Digest Quantiles Sketch module
|
||||
|
||||
This module provides Apache Druid (incubating) approximate sketch aggregators based on T-Digest.
|
||||
T-Digest (https://github.com/tdunning/t-digest) is a popular datastructure for accurate on-line accumulation of
|
||||
rank-based statistics such as quantiles and trimmed means.
|
||||
The datastructure is also designed for parallel programming use cases like distributed aggregations or map reduce jobs by making combining two intermediate t-digests easy and efficient.
|
||||
|
||||
There are three flavors of T-Digest sketch aggregator available in Apache Druid (incubating):
|
||||
|
||||
1. buildTDigestSketch - used for building T-Digest sketches from raw numeric values. It generally makes sense to
|
||||
use this aggregator when ingesting raw data into Druid. One can also use this aggregator during query time too to
|
||||
generate sketches, just that one would be building these sketches on every query execution instead of building them
|
||||
once during ingestion.
|
||||
2. mergeTDigestSketch - used for merging pre-built T-Digest sketches. This aggregator is generally used during
|
||||
query time to combine sketches generated by buildTDigestSketch aggregator.
|
||||
3. quantilesFromTDigestSketch - used for generating quantiles from T-Digest sketches. This aggregator is generally used
|
||||
during query time to generate quantiles from sketches built using the above two sketch generating aggregators.
|
||||
|
||||
To use this aggregator, make sure you [include](../../operations/including-extensions.html) the extension in your config file:
|
||||
|
||||
```
|
||||
druid.extensions.loadList=["druid-tdigestsketch"]
|
||||
```
|
||||
|
||||
### Aggregator
|
||||
|
||||
The result of the aggregation is a T-Digest sketch that is built ingesting numeric values from the raw data.
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "buildTDigestSketch",
|
||||
"name" : <output_name>,
|
||||
"fieldName" : <metric_name>,
|
||||
"compression": <parameter that controls size and accuracy>
|
||||
}
|
||||
```
|
||||
Example:
|
||||
```json
|
||||
{
|
||||
"type": "buildTDigestSketch",
|
||||
"name": "sketch",
|
||||
"fieldName": "session_duration",
|
||||
"compression": 200
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This String should always be "buildTDigestSketch"|yes|
|
||||
|name|A String for the output (result) name of the calculation.|yes|
|
||||
|fieldName|A String for the name of the input field containing raw numeric values.|yes|
|
||||
|compression|Parameter that determines the accuracy and size of the sketch. Higher compression means higher accuracy but more space to store sketches.|no, defaults to 100|
|
||||
|
||||
|
||||
The result of the aggregation is a T-Digest sketch that is built by merging pre-built T-Digest sketches.
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "mergeTDigestSketch",
|
||||
"name" : <output_name>,
|
||||
"fieldName" : <metric_name>,
|
||||
"compression": <parameter that controls size and accuracy>
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This String should always be "mergeTDigestSketch"|yes|
|
||||
|name|A String for the output (result) name of the calculation.|yes|
|
||||
|fieldName|A String for the name of the input field containing raw numeric values.|yes|
|
||||
|compression|Parameter that determines the accuracy and size of the sketch. Higher compression means higher accuracy but more space to store sketches.|no, defaults to 100|
|
||||
|
||||
Example:
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "test_datasource",
|
||||
"granularity": "ALL",
|
||||
"dimensions": [],
|
||||
"aggregations": [{
|
||||
"type": "mergeTDigestSketch",
|
||||
"name": "merged_sketch",
|
||||
"fieldName": "ingested_sketch",
|
||||
"compression": 200
|
||||
}],
|
||||
"intervals": ["2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z"]
|
||||
}
|
||||
```
|
||||
### Post Aggregators
|
||||
|
||||
#### Quantiles
|
||||
|
||||
This returns an array of quantiles corresponding to a given array of fractions.
|
||||
|
||||
```json
|
||||
{
|
||||
"type" : "quantilesFromTDigestSketch",
|
||||
"name": <output name>,
|
||||
"field" : <post aggregator that refers to a TDigestSketch (fieldAccess or another post aggregator)>,
|
||||
"fractions" : <array of fractions>
|
||||
}
|
||||
```
|
||||
|
||||
|property|description|required?|
|
||||
|--------|-----------|---------|
|
||||
|type|This String should always be "quantilesFromTDigestSketch"|yes|
|
||||
|name|A String for the output (result) name of the calculation.|yes|
|
||||
|fieldName|A String for the name of the input field containing raw numeric values.|yes|
|
||||
|fractions|Non-empty array of fractions between 0 and 1|yes|
|
||||
|
||||
Example:
|
||||
```json
|
||||
{
|
||||
"queryType": "groupBy",
|
||||
"dataSource": "test_datasource",
|
||||
"granularity": "ALL",
|
||||
"dimensions": [],
|
||||
"aggregations": [{
|
||||
"type": "mergeTDigestSketch",
|
||||
"name": "merged_sketch",
|
||||
"fieldName": "ingested_sketch",
|
||||
"compression": 200
|
||||
}],
|
||||
"postAggregations": [{
|
||||
"type": "quantilesFromTDigestSketch",
|
||||
"name": "quantiles",
|
||||
"fractions": [0, 0.5, 1],
|
||||
"field": {
|
||||
"type": "fieldAccess",
|
||||
"fieldName": "merged_sketch"
|
||||
}
|
||||
}],
|
||||
"intervals": ["2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z"]
|
||||
}
|
||||
```
|
|
@ -0,0 +1,94 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ Licensed to the Apache Software Foundation (ASF) under one
|
||||
~ or more contributor license agreements. See the NOTICE file
|
||||
~ distributed with this work for additional information
|
||||
~ regarding copyright ownership. The ASF 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.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<parent>
|
||||
<artifactId>druid</artifactId>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<version>0.15.0-incubating-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
|
||||
<groupId>io.druid.extensions.contrib</groupId>
|
||||
<artifactId>druid-tdigestsketch</artifactId>
|
||||
<name>tdigestsketch</name>
|
||||
<description>Druid extension for generating tdigest backed sketches</description>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>com.tdunning</groupId>
|
||||
<artifactId>t-digest</artifactId>
|
||||
<version>3.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<version>${guava.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
</project>
|
|
@ -0,0 +1,93 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import javax.annotation.concurrent.GuardedBy;
|
||||
|
||||
/**
|
||||
* Aggregator to build T-Digest sketches on numeric values.
|
||||
* It generally makes sense to use this aggregator during the ingestion time.
|
||||
* Then during query time one can use {@link TDigestMergeSketchAggregator} to merge
|
||||
* these pre-aggregated sketches.
|
||||
* <p>
|
||||
* One can use this aggregator to build these sketches during query time too, just
|
||||
* that it will be slower and more resource intensive.
|
||||
*/
|
||||
public class TDigestBuildSketchAggregator implements Aggregator
|
||||
{
|
||||
|
||||
private final ColumnValueSelector selector;
|
||||
|
||||
@GuardedBy("this")
|
||||
private MergingDigest histogram;
|
||||
|
||||
|
||||
public TDigestBuildSketchAggregator(ColumnValueSelector selector, @Nullable Integer compression)
|
||||
{
|
||||
this.selector = selector;
|
||||
if (compression != null) {
|
||||
this.histogram = new MergingDigest(compression);
|
||||
} else {
|
||||
this.histogram = new MergingDigest(TDigestBuildSketchAggregatorFactory.DEFAULT_COMPRESSION);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
if (selector.getObject() instanceof Number) {
|
||||
synchronized (this) {
|
||||
histogram.add(((Number) selector.getObject()).doubleValue());
|
||||
}
|
||||
} else {
|
||||
TDigestSketchUtils.throwExceptionForWrongType(selector);
|
||||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
@Override
|
||||
public synchronized Object get()
|
||||
{
|
||||
return histogram;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
throw new UnsupportedOperationException("Casting to float type is not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
throw new UnsupportedOperationException("Casting to long type is not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close()
|
||||
{
|
||||
histogram = null;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,268 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import com.tdunning.math.stats.TDigest;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactoryNotMergeableException;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Aggregation operations over the tdigest-based quantile sketch
|
||||
* available on <a href="https://github.com/tdunning/t-digest">github</a> and described
|
||||
* in the paper
|
||||
* <a href="https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf">
|
||||
* Computing extremely accurate quantiles using t-digests</a>.
|
||||
* <p>
|
||||
* <p>
|
||||
* At the time of writing this implementation, there are two flavors of {@link TDigest}
|
||||
* available - {@link MergingDigest} and {@link com.tdunning.math.stats.AVLTreeDigest}.
|
||||
* This implementation uses {@link MergingDigest} since it is more suited for the cases
|
||||
* when we have to merge intermediate aggregations which Druid needs to do as
|
||||
* part of query processing.
|
||||
*/
|
||||
public class TDigestBuildSketchAggregatorFactory extends AggregatorFactory
|
||||
{
|
||||
|
||||
// Default compression
|
||||
public static final int DEFAULT_COMPRESSION = 100;
|
||||
|
||||
@Nonnull
|
||||
private final String name;
|
||||
@Nonnull
|
||||
private final String fieldName;
|
||||
|
||||
private final int compression;
|
||||
|
||||
@Nonnull
|
||||
private final byte cacheTypeId;
|
||||
|
||||
public static final String TYPE_NAME = "buildTDigestSketch";
|
||||
|
||||
@JsonCreator
|
||||
public TDigestBuildSketchAggregatorFactory(
|
||||
@JsonProperty("name") final String name,
|
||||
@JsonProperty("fieldName") final String fieldName,
|
||||
@Nullable @JsonProperty("compression") final Integer compression
|
||||
)
|
||||
{
|
||||
this(name, fieldName, compression, AggregatorUtil.TDIGEST_BUILD_SKETCH_CACHE_TYPE_ID);
|
||||
}
|
||||
|
||||
TDigestBuildSketchAggregatorFactory(
|
||||
final String name,
|
||||
final String fieldName,
|
||||
@Nullable final Integer compression,
|
||||
final byte cacheTypeId
|
||||
)
|
||||
{
|
||||
this.name = Objects.requireNonNull(name, "Must have a valid, non-null aggregator name");
|
||||
this.fieldName = Objects.requireNonNull(fieldName, "Parameter fieldName must be specified");
|
||||
this.compression = compression == null ? DEFAULT_COMPRESSION : compression;
|
||||
this.cacheTypeId = cacheTypeId;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new CacheKeyBuilder(
|
||||
cacheTypeId
|
||||
).appendString(fieldName).appendInt(compression).build();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
||||
final ColumnValueSelector<Double> selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new TDigestBuildSketchAggregator(selector, compression);
|
||||
} else {
|
||||
final ColumnValueSelector<MergingDigest> selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new TDigestMergeSketchAggregator(selector, compression);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
||||
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
||||
final ColumnValueSelector<Double> selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new TDigestBuildSketchBufferAggregator(selector, compression);
|
||||
} else {
|
||||
final ColumnValueSelector<MergingDigest> selector = metricFactory.makeColumnValueSelector(fieldName);
|
||||
return new TDigestMergeSketchBufferAggregator(selector, compression);
|
||||
}
|
||||
}
|
||||
|
||||
public static final Comparator<TDigest> COMPARATOR = Comparator.nullsFirst(
|
||||
Comparator.comparingLong(a -> a.size())
|
||||
);
|
||||
|
||||
@Override
|
||||
public Comparator getComparator()
|
||||
{
|
||||
return COMPARATOR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object combine(@Nullable Object lhs, @Nullable Object rhs)
|
||||
{
|
||||
if (lhs == null) {
|
||||
return rhs;
|
||||
}
|
||||
if (rhs == null) {
|
||||
return lhs;
|
||||
}
|
||||
TDigest union = (TDigest) lhs;
|
||||
union.add((TDigest) rhs);
|
||||
return union;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getCombiningFactory()
|
||||
{
|
||||
return new TDigestMergeSketchAggregatorFactory(name, name, compression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public AggregatorFactory getMergingFactory(AggregatorFactory other) throws AggregatorFactoryNotMergeableException
|
||||
{
|
||||
if (other.getName().equals(this.getName()) && this.getClass() == other.getClass()) {
|
||||
return getCombiningFactory();
|
||||
} else {
|
||||
throw new AggregatorFactoryNotMergeableException(this, other);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<AggregatorFactory> getRequiredColumns()
|
||||
{
|
||||
return Collections.singletonList(
|
||||
new TDigestBuildSketchAggregatorFactory(
|
||||
fieldName,
|
||||
fieldName,
|
||||
compression
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object deserialize(Object serializedSketch)
|
||||
{
|
||||
return TDigestSketchUtils.deserialize(serializedSketch);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object finalizeComputation(Object object)
|
||||
{
|
||||
return object;
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getFieldName()
|
||||
{
|
||||
return fieldName;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getCompression()
|
||||
{
|
||||
return compression;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> requiredFields()
|
||||
{
|
||||
return Collections.singletonList(fieldName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTypeName()
|
||||
{
|
||||
return TYPE_NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getMaxIntermediateSize()
|
||||
{
|
||||
return TDigestSketchUtils.getMaxIntermdiateTDigestSize(compression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || !getClass().equals(o.getClass())) {
|
||||
return false;
|
||||
}
|
||||
final TDigestBuildSketchAggregatorFactory that = (TDigestBuildSketchAggregatorFactory) o;
|
||||
|
||||
return Objects.equals(name, that.name) &&
|
||||
Objects.equals(fieldName, that.fieldName) &&
|
||||
compression == that.compression;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(name, fieldName, compression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getClass().getSimpleName() + "{"
|
||||
+ "name=" + name
|
||||
+ ", fieldName=" + fieldName
|
||||
+ ", compression=" + compression
|
||||
+ "}";
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,121 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
|
||||
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.IdentityHashMap;
|
||||
|
||||
/**
|
||||
* Aggregator that builds T-Digest backed sketch using numeric values read from {@link ByteBuffer}
|
||||
*/
|
||||
public class TDigestBuildSketchBufferAggregator implements BufferAggregator
|
||||
{
|
||||
|
||||
@Nonnull
|
||||
private final ColumnValueSelector selector;
|
||||
private final int compression;
|
||||
private final IdentityHashMap<ByteBuffer, Int2ObjectMap<MergingDigest>> sketchCache = new IdentityHashMap();
|
||||
|
||||
public TDigestBuildSketchBufferAggregator(
|
||||
final ColumnValueSelector valueSelector,
|
||||
@Nullable final Integer compression
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(valueSelector);
|
||||
this.selector = valueSelector;
|
||||
if (compression != null) {
|
||||
this.compression = compression;
|
||||
} else {
|
||||
this.compression = TDigestBuildSketchAggregatorFactory.DEFAULT_COMPRESSION;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(ByteBuffer buffer, int position)
|
||||
{
|
||||
MergingDigest emptyDigest = new MergingDigest(compression);
|
||||
addToCache(buffer, position, emptyDigest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buffer, int position)
|
||||
{
|
||||
MergingDigest sketch = sketchCache.get(buffer).get(position);
|
||||
Object x = selector.getObject();
|
||||
if (x instanceof Number) {
|
||||
sketch.add(((Number) x).doubleValue());
|
||||
} else {
|
||||
TDigestSketchUtils.throwExceptionForWrongType(selector);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(final ByteBuffer buffer, final int position)
|
||||
{
|
||||
// sketchCache is an IdentityHashMap where the reference of buffer is used for equality checks.
|
||||
// So the returned object isn't impacted by the changes in the buffer object made by concurrent threads.
|
||||
return sketchCache.get(buffer).get(position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(final ByteBuffer buffer, final int position)
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(final ByteBuffer buffer, final int position)
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
sketchCache.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
|
||||
{
|
||||
MergingDigest sketch = sketchCache.get(oldBuffer).get(oldPosition);
|
||||
addToCache(newBuffer, newPosition, sketch);
|
||||
final Int2ObjectMap<MergingDigest> map = sketchCache.get(oldBuffer);
|
||||
map.remove(oldPosition);
|
||||
if (map.isEmpty()) {
|
||||
sketchCache.remove(oldBuffer);
|
||||
}
|
||||
}
|
||||
|
||||
private void addToCache(final ByteBuffer buffer, final int position, final MergingDigest sketch)
|
||||
{
|
||||
Int2ObjectMap<MergingDigest> map = sketchCache.computeIfAbsent(buffer, b -> new Int2ObjectOpenHashMap<>());
|
||||
map.put(position, sketch);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,83 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import javax.annotation.concurrent.GuardedBy;
|
||||
|
||||
/**
|
||||
* Aggregator that merges T-Digest based sketches generated from {@link TDigestBuildSketchAggregator}
|
||||
*/
|
||||
public class TDigestMergeSketchAggregator implements Aggregator
|
||||
{
|
||||
private final ColumnValueSelector<MergingDigest> selector;
|
||||
|
||||
@GuardedBy("this")
|
||||
private MergingDigest tdigestSketch;
|
||||
|
||||
public TDigestMergeSketchAggregator(
|
||||
ColumnValueSelector<MergingDigest> selector,
|
||||
final Integer compression
|
||||
)
|
||||
{
|
||||
this.selector = selector;
|
||||
this.tdigestSketch = new MergingDigest(compression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
final MergingDigest sketch = selector.getObject();
|
||||
if (sketch == null) {
|
||||
return;
|
||||
}
|
||||
synchronized (this) {
|
||||
this.tdigestSketch.add(sketch);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Object get()
|
||||
{
|
||||
return tdigestSketch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat()
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong()
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close()
|
||||
{
|
||||
tdigestSketch = null;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,64 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.query.aggregation.Aggregator;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
/**
|
||||
* Factory to create {@link TDigestMergeSketchAggregator}
|
||||
*/
|
||||
public class TDigestMergeSketchAggregatorFactory extends TDigestBuildSketchAggregatorFactory
|
||||
{
|
||||
public static final String TYPE_NAME = "mergeTDigestSketch";
|
||||
|
||||
@JsonCreator
|
||||
public TDigestMergeSketchAggregatorFactory(
|
||||
@JsonProperty("name") final String name,
|
||||
@JsonProperty("fieldName") final String fieldName,
|
||||
@JsonProperty("compression") final Integer compression
|
||||
)
|
||||
{
|
||||
super(name, fieldName, compression, AggregatorUtil.TDIGEST_MERGE_SKETCH_CACHE_TYPE_ID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Aggregator factorize(final ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
final ColumnValueSelector<MergingDigest> selector = metricFactory.makeColumnValueSelector(
|
||||
getFieldName());
|
||||
return new TDigestMergeSketchAggregator(selector, getCompression());
|
||||
}
|
||||
|
||||
@Override
|
||||
public BufferAggregator factorizeBuffered(final ColumnSelectorFactory metricFactory)
|
||||
{
|
||||
final ColumnValueSelector<MergingDigest> selector = metricFactory.makeColumnValueSelector(
|
||||
getFieldName()
|
||||
);
|
||||
return new TDigestMergeSketchBufferAggregator(selector, getCompression());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,110 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
|
||||
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
|
||||
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.IdentityHashMap;
|
||||
|
||||
/**
|
||||
* Aggregator that is capable of combining T-Digest sketch serialized as {@link ByteBuffer}
|
||||
*/
|
||||
public class TDigestMergeSketchBufferAggregator implements BufferAggregator
|
||||
{
|
||||
private final ColumnValueSelector<MergingDigest> selector;
|
||||
private final int compression;
|
||||
private final IdentityHashMap<ByteBuffer, Int2ObjectMap<MergingDigest>> sketchCache = new IdentityHashMap();
|
||||
|
||||
public TDigestMergeSketchBufferAggregator(
|
||||
ColumnValueSelector<MergingDigest> selector,
|
||||
int compression
|
||||
)
|
||||
{
|
||||
this.selector = selector;
|
||||
this.compression = compression;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(ByteBuffer buffer, int position)
|
||||
{
|
||||
MergingDigest emptyDigest = new MergingDigest(compression);
|
||||
addToCache(buffer, position, emptyDigest);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buffer, int position)
|
||||
{
|
||||
final MergingDigest sketch = selector.getObject();
|
||||
if (sketch == null) {
|
||||
return;
|
||||
}
|
||||
final MergingDigest union = sketchCache.get(buffer).get(position);
|
||||
union.add(sketch);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object get(ByteBuffer buffer, int position)
|
||||
{
|
||||
// sketchCache is an IdentityHashMap where the reference of buffer is used for equality checks.
|
||||
// So the returned object isn't impacted by the changes in the buffer object made by concurrent threads.
|
||||
return sketchCache.get(buffer).get(position);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float getFloat(final ByteBuffer buffer, final int position)
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLong(final ByteBuffer buffer, final int position)
|
||||
{
|
||||
throw new UnsupportedOperationException("Not implemented");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close()
|
||||
{
|
||||
sketchCache.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
|
||||
{
|
||||
MergingDigest sketch = sketchCache.get(oldBuffer).get(oldPosition);
|
||||
addToCache(newBuffer, newPosition, sketch);
|
||||
final Int2ObjectMap<MergingDigest> map = sketchCache.get(oldBuffer);
|
||||
map.remove(oldPosition);
|
||||
if (map.isEmpty()) {
|
||||
sketchCache.remove(oldBuffer);
|
||||
}
|
||||
}
|
||||
|
||||
private void addToCache(final ByteBuffer buffer, final int position, final MergingDigest union)
|
||||
{
|
||||
Int2ObjectMap<MergingDigest> map = sketchCache.computeIfAbsent(buffer, b -> new Int2ObjectOpenHashMap<>());
|
||||
map.put(position, union);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,111 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.segment.GenericColumnSerializer;
|
||||
import org.apache.druid.segment.column.ColumnBuilder;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
import org.apache.druid.segment.data.ObjectStrategy;
|
||||
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
|
||||
import org.apache.druid.segment.serde.ComplexMetricExtractor;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class TDigestSketchComplexMetricSerde extends ComplexMetricSerde
|
||||
{
|
||||
private static final TDigestSketchObjectStrategy STRATEGY = new TDigestSketchObjectStrategy();
|
||||
|
||||
@Override
|
||||
public String getTypeName()
|
||||
{
|
||||
return TDigestBuildSketchAggregatorFactory.TYPE_NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ComplexMetricExtractor getExtractor()
|
||||
{
|
||||
return new ComplexMetricExtractor()
|
||||
{
|
||||
@Override
|
||||
public Class<?> extractedClass()
|
||||
{
|
||||
return MergingDigest.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object extractValue(final InputRow inputRow, final String metricName)
|
||||
{
|
||||
final Object object = inputRow.getRaw(metricName);
|
||||
if (object == null || object instanceof Number || object instanceof MergingDigest) {
|
||||
return object;
|
||||
}
|
||||
if (object instanceof String) {
|
||||
String objectString = (String) object;
|
||||
if (Character.isDigit((objectString).charAt(0))) {
|
||||
// Base64 representation of MergingDigest starts with A. So if it's a
|
||||
// string that starts with a digit, we assume it is a number.
|
||||
try {
|
||||
Double doubleValue = Double.parseDouble(objectString);
|
||||
return doubleValue;
|
||||
}
|
||||
catch (NumberFormatException e) {
|
||||
throw new IAE("Expected a string with a number, received value " + objectString);
|
||||
}
|
||||
}
|
||||
}
|
||||
return TDigestSketchUtils.deserialize(object);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deserializeColumn(ByteBuffer buffer, ColumnBuilder builder)
|
||||
{
|
||||
final GenericIndexed<MergingDigest> column = GenericIndexed.read(
|
||||
buffer,
|
||||
STRATEGY,
|
||||
builder.getFileMapper()
|
||||
);
|
||||
builder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ObjectStrategy<MergingDigest> getObjectStrategy()
|
||||
{
|
||||
return STRATEGY;
|
||||
}
|
||||
|
||||
@Override
|
||||
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
|
||||
{
|
||||
return LargeColumnSupportedComplexColumnSerializer.create(
|
||||
segmentWriteOutMedium,
|
||||
column,
|
||||
this.getObjectStrategy()
|
||||
);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class TDigestSketchJsonSerializer extends JsonSerializer<MergingDigest>
|
||||
{
|
||||
@Override
|
||||
public void serialize(
|
||||
MergingDigest tDigest,
|
||||
JsonGenerator jsonGenerator,
|
||||
SerializerProvider serializerProvider
|
||||
) throws IOException
|
||||
{
|
||||
jsonGenerator.writeBinary(TDigestSketchUtils.toBytes(tDigest));
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Module defining aggregators for the T-Digest based sketches
|
||||
*/
|
||||
public class TDigestSketchModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule(
|
||||
getClass().getSimpleName()
|
||||
).registerSubtypes(
|
||||
new NamedType(
|
||||
TDigestBuildSketchAggregatorFactory.class,
|
||||
TDigestBuildSketchAggregatorFactory.TYPE_NAME
|
||||
),
|
||||
new NamedType(
|
||||
TDigestMergeSketchAggregatorFactory.class,
|
||||
TDigestMergeSketchAggregatorFactory.TYPE_NAME
|
||||
),
|
||||
new NamedType(
|
||||
TDigestSketchToQuantilesPostAggregator.class,
|
||||
TDigestSketchToQuantilesPostAggregator.TYPE_NAME
|
||||
)
|
||||
).addSerializer(MergingDigest.class, new TDigestSketchJsonSerializer())
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
registerSerde();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static void registerSerde()
|
||||
{
|
||||
if (ComplexMetrics.getSerdeForType(TDigestBuildSketchAggregatorFactory.TYPE_NAME) == null) {
|
||||
ComplexMetrics.registerSerde(TDigestBuildSketchAggregatorFactory.TYPE_NAME, new TDigestSketchComplexMetricSerde());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,63 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.segment.data.ObjectStrategy;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class TDigestSketchObjectStrategy implements ObjectStrategy<MergingDigest>
|
||||
{
|
||||
private static final byte[] EMPTY_BYTES = new byte[0];
|
||||
|
||||
@Override
|
||||
public Class<MergingDigest> getClazz()
|
||||
{
|
||||
return MergingDigest.class;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergingDigest fromByteBuffer(ByteBuffer buffer, int numBytes)
|
||||
{
|
||||
if (numBytes == 0) {
|
||||
return null;
|
||||
}
|
||||
ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
|
||||
readOnlyBuffer.limit(buffer.position() + numBytes);
|
||||
return MergingDigest.fromBytes(readOnlyBuffer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] toBytes(@Nullable MergingDigest val)
|
||||
{
|
||||
if (val == null) {
|
||||
return EMPTY_BYTES;
|
||||
}
|
||||
return TDigestSketchUtils.toBytes(val);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(MergingDigest o1, MergingDigest o2)
|
||||
{
|
||||
return TDigestBuildSketchAggregatorFactory.COMPARATOR.compare(o1, o2);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,159 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Post aggregation operator that can take in aggregated T-Digest sketches and
|
||||
* generate quantiles from it.
|
||||
*/
|
||||
public class TDigestSketchToQuantilesPostAggregator implements PostAggregator
|
||||
{
|
||||
|
||||
private final String name;
|
||||
private final PostAggregator field;
|
||||
private final double[] fractions;
|
||||
|
||||
public static final String TYPE_NAME = "quantilesFromTDigestSketch";
|
||||
|
||||
@JsonCreator
|
||||
public TDigestSketchToQuantilesPostAggregator(
|
||||
@JsonProperty("name") final String name,
|
||||
@JsonProperty("field") final PostAggregator field,
|
||||
@JsonProperty("fractions") final double[] fractions
|
||||
)
|
||||
{
|
||||
this.name = Preconditions.checkNotNull(name, "name is null");
|
||||
this.field = Preconditions.checkNotNull(field, "field is null");
|
||||
this.fractions = Preconditions.checkNotNull(fractions, "array of fractions is null");
|
||||
Preconditions.checkArgument(this.fractions.length > 1, "Array of fractions cannot be empty");
|
||||
}
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
public String getName()
|
||||
{
|
||||
return name;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public PostAggregator getField()
|
||||
{
|
||||
return field;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public double[] getFractions()
|
||||
{
|
||||
return fractions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object compute(final Map<String, Object> combinedAggregators)
|
||||
{
|
||||
final MergingDigest sketch = (MergingDigest) field.compute(combinedAggregators);
|
||||
double[] quantiles = new double[fractions.length];
|
||||
int i = 0;
|
||||
|
||||
for (double f : fractions) {
|
||||
quantiles[i++] = sketch.quantile(f);
|
||||
}
|
||||
return quantiles;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<double[]> getComparator()
|
||||
{
|
||||
throw new IAE("Comparing arrays of quantiles is not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
return field.getDependentFields();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return getClass().getSimpleName() + "{" +
|
||||
"name='" + name + '\'' +
|
||||
", field=" + field +
|
||||
", fractions=" + Arrays.toString(fractions) +
|
||||
"}";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
final TDigestSketchToQuantilesPostAggregator that = (TDigestSketchToQuantilesPostAggregator) o;
|
||||
if (!name.equals(that.name)) {
|
||||
return false;
|
||||
}
|
||||
if (!Arrays.equals(fractions, that.fractions)) {
|
||||
return false;
|
||||
}
|
||||
return field.equals(that.field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return (name.hashCode() * 31 + field.hashCode()) * 31 + Arrays.hashCode(fractions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
final CacheKeyBuilder builder = new CacheKeyBuilder(
|
||||
AggregatorUtil.TDIGEST_SKETCH_TO_QUANTILES_CACHE_TYPE_ID).appendCacheable(field);
|
||||
for (final double value : fractions) {
|
||||
builder.appendDouble(value);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostAggregator decorate(final Map<String, AggregatorFactory> map)
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.segment.ColumnValueSelector;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class TDigestSketchUtils
|
||||
{
|
||||
// Class is not meant to be instantiated
|
||||
private TDigestSketchUtils()
|
||||
{
|
||||
}
|
||||
|
||||
public static MergingDigest deserialize(Object serializedSketch)
|
||||
{
|
||||
if (serializedSketch instanceof String) {
|
||||
String str = (String) serializedSketch;
|
||||
return MergingDigest.fromBytes(ByteBuffer.wrap(StringUtils.decodeBase64(StringUtils.toUtf8(str))));
|
||||
} else if (serializedSketch instanceof byte[]) {
|
||||
return MergingDigest.fromBytes(ByteBuffer.wrap((byte[]) serializedSketch));
|
||||
} else if (serializedSketch instanceof MergingDigest) {
|
||||
return (MergingDigest) serializedSketch;
|
||||
}
|
||||
throw new IAE(
|
||||
"Object cannot be deserialized to a TDigest Sketch: "
|
||||
+ serializedSketch.getClass()
|
||||
);
|
||||
}
|
||||
|
||||
static byte[] toBytes(MergingDigest tDigest)
|
||||
{
|
||||
byte[] arr = new byte[tDigest.byteSize()];
|
||||
ByteBuffer result = ByteBuffer.wrap(arr);
|
||||
tDigest.asBytes(result);
|
||||
return result.array();
|
||||
}
|
||||
|
||||
/**
|
||||
* This method computes an estimate of the max intermediate size of a {@link MergingDigest}.
|
||||
* Since there is no utility available in the T-Digest library to compute this size,
|
||||
* the below code is inspired by looking at
|
||||
* {@link MergingDigest#MergingDigest(double, int, int)}
|
||||
* This method is current as of 3.2 version and many need to change in future.
|
||||
*/
|
||||
static int getMaxIntermdiateTDigestSize(int compression)
|
||||
{
|
||||
int intermediateSize = 0;
|
||||
int centroidArraySize = (int) (2 * Math.ceil(compression)) + 10;
|
||||
intermediateSize += 2 * centroidArraySize * Double.BYTES;
|
||||
int tempBufferSize = (int) (5 * Math.ceil(compression));
|
||||
intermediateSize += 2 * tempBufferSize * Double.BYTES;
|
||||
intermediateSize += tempBufferSize * Integer.BYTES;
|
||||
// Adding an extra buffer of 1K for overhead
|
||||
return intermediateSize + 1000;
|
||||
}
|
||||
|
||||
static void throwExceptionForWrongType(ColumnValueSelector selector)
|
||||
{
|
||||
final String msg = selector.getObject() == null
|
||||
? StringUtils.format("Expected a number, but received null")
|
||||
: StringUtils.format(
|
||||
"Expected a number, but received [%s] of type [%s]",
|
||||
selector.getObject(),
|
||||
selector.getObject().getClass()
|
||||
);
|
||||
throw new IAE(msg);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,16 @@
|
|||
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
# contributor license agreements. See the NOTICE file distributed with
|
||||
# this work for additional information regarding copyright ownership.
|
||||
# The ASF 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.
|
||||
|
||||
org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchModule
|
|
@ -0,0 +1,79 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.tdunning.math.stats.MergingDigest;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
/**
|
||||
* This class is largely a copy of GenerateTestData class for Datasketches project.
|
||||
* It is used for generating test data for {@link TDigestSketchAggregatorTest}.
|
||||
*/
|
||||
public class GenerateTestData
|
||||
{
|
||||
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
Path buildPath = FileSystems.getDefault().getPath("doubles_build_data.tsv");
|
||||
Path sketchPath = FileSystems.getDefault().getPath("doubles_sketch_data.tsv");
|
||||
BufferedWriter buildData = Files.newBufferedWriter(buildPath, StandardCharsets.UTF_8);
|
||||
BufferedWriter sketchData = Files.newBufferedWriter(sketchPath, StandardCharsets.UTF_8);
|
||||
Random rand = ThreadLocalRandom.current();
|
||||
int sequenceNumber = 0;
|
||||
for (int i = 0; i < 20; i++) {
|
||||
int product = rand.nextInt(10);
|
||||
MergingDigest sketch = new MergingDigest(100);
|
||||
for (int j = 0; j < 20; j++) {
|
||||
double value = rand.nextDouble();
|
||||
buildData.write("2016010101");
|
||||
buildData.write('\t');
|
||||
buildData.write(Integer.toString(sequenceNumber)); // dimension with unique numbers for ingesting raw data
|
||||
buildData.write('\t');
|
||||
buildData.write(Integer.toString(product)); // product dimension
|
||||
buildData.write('\t');
|
||||
buildData.write(Double.toString(value));
|
||||
buildData.newLine();
|
||||
sketch.add(value);
|
||||
sequenceNumber++;
|
||||
}
|
||||
sketchData.write("2016010101");
|
||||
sketchData.write('\t');
|
||||
sketchData.write(Integer.toString(product)); // product dimension
|
||||
sketchData.write('\t');
|
||||
byte[] bytes = new byte[sketch.byteSize()];
|
||||
ByteBuffer buffer = ByteBuffer.wrap(bytes);
|
||||
sketch.asBytes(buffer);
|
||||
sketchData.write(StringUtils.encodeBase64String(buffer.array()));
|
||||
sketchData.newLine();
|
||||
}
|
||||
buildData.close();
|
||||
sketchData.close();
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,279 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.tdigestsketch;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class TDigestSketchAggregatorTest
|
||||
{
|
||||
|
||||
private final AggregationTestHelper helper;
|
||||
|
||||
@Rule
|
||||
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||
|
||||
public TDigestSketchAggregatorTest(final GroupByQueryConfig config)
|
||||
{
|
||||
TDigestSketchModule.registerSerde();
|
||||
TDigestSketchModule module = new TDigestSketchModule();
|
||||
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||
module.getJacksonModules(), config, tempFolder);
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
return constructors;
|
||||
}
|
||||
|
||||
// this is to test Json properties and equals
|
||||
@Test
|
||||
public void serializeDeserializeFactoryWithFieldName() throws Exception
|
||||
{
|
||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||
new TDigestSketchModule().getJacksonModules().forEach(objectMapper::registerModule);
|
||||
TDigestBuildSketchAggregatorFactory factory = new TDigestBuildSketchAggregatorFactory("name", "filedName", 128);
|
||||
|
||||
AggregatorFactory other = objectMapper.readValue(
|
||||
objectMapper.writeValueAsString(factory),
|
||||
AggregatorFactory.class
|
||||
);
|
||||
|
||||
Assert.assertEquals(factory, other);
|
||||
}
|
||||
|
||||
// this is to test Json properties and equals for the combining factory
|
||||
@Test
|
||||
public void serializeDeserializeCombiningFactoryWithFieldName() throws Exception
|
||||
{
|
||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||
new TDigestSketchModule().getJacksonModules().forEach(objectMapper::registerModule);
|
||||
TDigestMergeSketchAggregatorFactory factory = new TDigestMergeSketchAggregatorFactory("name", "fieldName", 128);
|
||||
|
||||
AggregatorFactory other = objectMapper.readValue(
|
||||
objectMapper.writeValueAsString(factory),
|
||||
AggregatorFactory.class
|
||||
);
|
||||
|
||||
Assert.assertEquals(factory, other);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildingSketchesAtIngestionTime() throws Exception
|
||||
{
|
||||
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"type\": \"string\",",
|
||||
" \"parseSpec\": {",
|
||||
" \"format\": \"tsv\",",
|
||||
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||
" \"dimensionsSpec\": {",
|
||||
" \"dimensions\": [\"product\"],",
|
||||
" \"dimensionExclusions\": [ \"sequenceNumber\"],",
|
||||
" \"spatialDimensions\": []",
|
||||
" },",
|
||||
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
|
||||
" }",
|
||||
"}"
|
||||
),
|
||||
"[{\"type\": \"buildTDigestSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"compression\": 200}]",
|
||||
0, // minTimestamp
|
||||
Granularities.NONE,
|
||||
10, // maxRowCount
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"queryType\": \"groupBy\",",
|
||||
" \"dataSource\": \"test_datasource\",",
|
||||
" \"granularity\": \"ALL\",",
|
||||
" \"dimensions\": [],",
|
||||
" \"aggregations\": [",
|
||||
" {\"type\": \"mergeTDigestSketch\", \"name\": \"merged_sketch\", \"fieldName\": \"sketch\", "
|
||||
+ "\"compression\": "
|
||||
+ "200}",
|
||||
" ],",
|
||||
" \"postAggregations\": [",
|
||||
" {\"type\": \"quantilesFromTDigestSketch\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], "
|
||||
+ "\"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"merged_sketch\"}}",
|
||||
" ],",
|
||||
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
||||
"}"
|
||||
)
|
||||
);
|
||||
List<Row> results = seq.toList();
|
||||
Assert.assertEquals(1, results.size());
|
||||
Row row = results.get(0);
|
||||
|
||||
// post agg
|
||||
Object quantilesObject = row.getRaw("quantiles");
|
||||
Assert.assertTrue(quantilesObject instanceof double[]);
|
||||
double[] quantiles = (double[]) quantilesObject;
|
||||
Assert.assertEquals(0, quantiles[0], 0.05); // min value
|
||||
Assert.assertEquals(0.5, quantiles[1], 0.05); // median value
|
||||
Assert.assertEquals(1, quantiles[2], 0.05); // max value
|
||||
}
|
||||
|
||||
@Test
|
||||
public void buildingSketchesAtQueryTime() throws Exception
|
||||
{
|
||||
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("doubles_build_data.tsv").getFile()),
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"type\": \"string\",",
|
||||
" \"parseSpec\": {",
|
||||
" \"format\": \"tsv\",",
|
||||
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||
" \"dimensionsSpec\": {",
|
||||
" \"dimensions\": [\"sequenceNumber\", \"product\"],",
|
||||
" \"dimensionExclusions\": [],",
|
||||
" \"spatialDimensions\": []",
|
||||
" },",
|
||||
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
|
||||
" }",
|
||||
"}"
|
||||
),
|
||||
"[{\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}]",
|
||||
0, // minTimestamp
|
||||
Granularities.NONE,
|
||||
10, // maxRowCount
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"queryType\": \"groupBy\",",
|
||||
" \"dataSource\": \"test_datasource\",",
|
||||
" \"granularity\": \"ALL\",",
|
||||
" \"dimensions\": [],",
|
||||
" \"aggregations\": [",
|
||||
" {\"type\": \"buildTDigestSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"compression\": 200}",
|
||||
" ],",
|
||||
" \"postAggregations\": [",
|
||||
" {\"type\": \"quantilesFromTDigestSketch\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
|
||||
" ],",
|
||||
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
||||
"}"
|
||||
)
|
||||
);
|
||||
List<Row> results = seq.toList();
|
||||
Assert.assertEquals(1, results.size());
|
||||
Row row = results.get(0);
|
||||
|
||||
|
||||
// post agg
|
||||
Object quantilesObject = row.getRaw("quantiles");
|
||||
Assert.assertTrue(quantilesObject instanceof double[]);
|
||||
double[] quantiles = (double[]) quantilesObject;
|
||||
Assert.assertEquals(0, quantiles[0], 0.05); // min value
|
||||
Assert.assertEquals(0.5, quantiles[1], 0.05); // median value
|
||||
Assert.assertEquals(1, quantiles[2], 0.05); // max value
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngestingSketches() throws Exception
|
||||
{
|
||||
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(this.getClass().getClassLoader().getResource("doubles_sketch_data.tsv").getFile()),
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"type\": \"string\",",
|
||||
" \"parseSpec\": {",
|
||||
" \"format\": \"tsv\",",
|
||||
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||
" \"dimensionsSpec\": {",
|
||||
" \"dimensions\": [\"product\"],",
|
||||
" \"dimensionExclusions\": [],",
|
||||
" \"spatialDimensions\": []",
|
||||
" },",
|
||||
" \"columns\": [\"timestamp\", \"product\", \"sketch\"]",
|
||||
" }",
|
||||
"}"
|
||||
),
|
||||
String.join(
|
||||
"\n",
|
||||
"[",
|
||||
" {\"type\": \"mergeTDigestSketch\", \"name\": \"first_level_merge_sketch\", \"fieldName\": \"sketch\", "
|
||||
+ "\"compression\": "
|
||||
+ "200}",
|
||||
"]"
|
||||
),
|
||||
0, // minTimestamp
|
||||
Granularities.NONE,
|
||||
10, // maxRowCount
|
||||
String.join(
|
||||
"\n",
|
||||
"{",
|
||||
" \"queryType\": \"groupBy\",",
|
||||
" \"dataSource\": \"test_datasource\",",
|
||||
" \"granularity\": \"ALL\",",
|
||||
" \"dimensions\": [],",
|
||||
" \"aggregations\": [",
|
||||
" {\"type\": \"mergeTDigestSketch\", \"name\": \"second_level_merge_sketch\", \"fieldName\": "
|
||||
+ "\"first_level_merge_sketch\", \"compression\": "
|
||||
+ "200}",
|
||||
" ],",
|
||||
" \"postAggregations\": [",
|
||||
" {\"type\": \"quantilesFromTDigestSketch\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"second_level_merge_sketch\"}}",
|
||||
" ],",
|
||||
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
||||
"}"
|
||||
)
|
||||
);
|
||||
List<Row> results = seq.toList();
|
||||
Assert.assertEquals(1, results.size());
|
||||
Row row = results.get(0);
|
||||
|
||||
// post agg
|
||||
Object quantilesObject = row.getRaw("quantiles");
|
||||
Assert.assertTrue(quantilesObject instanceof double[]);
|
||||
double[] quantiles = (double[]) quantilesObject;
|
||||
Assert.assertEquals(0, quantiles[0], 0.05); // min value
|
||||
Assert.assertEquals(0.5, quantiles[1], 0.05); // median value
|
||||
Assert.assertEquals(1, quantiles[2], 0.05); // max value
|
||||
}
|
||||
}
|
|
@ -0,0 +1,400 @@
|
|||
2016010101 0 4 0.18400342240873802
|
||||
2016010101 1 4 0.6865245424441776
|
||||
2016010101 2 4 0.7739519201819897
|
||||
2016010101 3 4 0.5856367722263963
|
||||
2016010101 4 4 0.7975517383656839
|
||||
2016010101 5 4 0.43358568598862635
|
||||
2016010101 6 4 0.12275185396963084
|
||||
2016010101 7 4 0.9993974605827314
|
||||
2016010101 8 4 0.4874039882191328
|
||||
2016010101 9 4 0.5692609022340854
|
||||
2016010101 10 4 0.26678754447065534
|
||||
2016010101 11 4 0.1989527055444228
|
||||
2016010101 12 4 0.9110129077294297
|
||||
2016010101 13 4 0.5538363335848467
|
||||
2016010101 14 4 0.10562455403723603
|
||||
2016010101 15 4 0.6935674808894914
|
||||
2016010101 16 4 0.4726911873373467
|
||||
2016010101 17 4 0.11900464634546115
|
||||
2016010101 18 4 0.4559981547074664
|
||||
2016010101 19 4 0.904574218307998
|
||||
2016010101 20 4 0.9776521615693896
|
||||
2016010101 21 4 0.2539035002418517
|
||||
2016010101 22 4 0.20805734530287534
|
||||
2016010101 23 4 0.41417993613962933
|
||||
2016010101 24 4 0.4952370716312732
|
||||
2016010101 25 4 0.07011744863814762
|
||||
2016010101 26 4 0.4531974535282244
|
||||
2016010101 27 4 0.6424395861589562
|
||||
2016010101 28 4 0.7228621538271517
|
||||
2016010101 29 4 0.5843296221504063
|
||||
2016010101 30 4 0.331742294401003
|
||||
2016010101 31 4 0.8294570315308673
|
||||
2016010101 32 4 0.9120986646735282
|
||||
2016010101 33 4 0.2286300818426008
|
||||
2016010101 34 4 0.8624446565970323
|
||||
2016010101 35 4 0.5343087459095744
|
||||
2016010101 36 4 0.4130981692773146
|
||||
2016010101 37 4 0.6713822529812259
|
||||
2016010101 38 4 0.6359244178847552
|
||||
2016010101 39 4 0.8614097204164315
|
||||
2016010101 40 6 0.855433193186756
|
||||
2016010101 41 6 0.5494808135474287
|
||||
2016010101 42 6 0.2494867642163876
|
||||
2016010101 43 6 0.44676257684891874
|
||||
2016010101 44 6 0.26941623048140995
|
||||
2016010101 45 6 0.24707447705353103
|
||||
2016010101 46 6 0.7763084722362548
|
||||
2016010101 47 6 0.7197776524998396
|
||||
2016010101 48 6 0.8547881573358272
|
||||
2016010101 49 6 0.04962070578119382
|
||||
2016010101 50 6 0.8577013635322285
|
||||
2016010101 51 6 0.3248850193586539
|
||||
2016010101 52 6 0.7099486817951516
|
||||
2016010101 53 6 0.9086831898801163
|
||||
2016010101 54 6 0.25911405409920296
|
||||
2016010101 55 6 0.48594375975487214
|
||||
2016010101 56 6 0.6328081792059475
|
||||
2016010101 57 6 0.2541410518420717
|
||||
2016010101 58 6 0.8855225528099031
|
||||
2016010101 59 6 0.8947324865996268
|
||||
2016010101 60 3 0.4115770024239612
|
||||
2016010101 61 3 0.407209253168066
|
||||
2016010101 62 3 0.20712349395555596
|
||||
2016010101 63 3 0.9019620910065339
|
||||
2016010101 64 3 0.9024166734838822
|
||||
2016010101 65 3 0.9229600784866859
|
||||
2016010101 66 3 0.985147872082527
|
||||
2016010101 67 3 0.10673099772511685
|
||||
2016010101 68 3 0.5996259775565335
|
||||
2016010101 69 3 0.5326920547476084
|
||||
2016010101 70 3 0.2908715666501662
|
||||
2016010101 71 3 0.9554390881150112
|
||||
2016010101 72 3 0.36984132919562984
|
||||
2016010101 73 3 0.2817272979933143
|
||||
2016010101 74 3 0.739014257182305
|
||||
2016010101 75 3 0.4547691759148418
|
||||
2016010101 76 3 0.5517497118489398
|
||||
2016010101 77 3 0.2277857035162798
|
||||
2016010101 78 3 0.8173939363090669
|
||||
2016010101 79 3 0.3930423279427335
|
||||
2016010101 80 6 0.9919726250736455
|
||||
2016010101 81 6 0.7670059532445269
|
||||
2016010101 82 6 0.042563956599481534
|
||||
2016010101 83 6 0.9637944309418425
|
||||
2016010101 84 6 0.7332037660613114
|
||||
2016010101 85 6 0.4426495381649056
|
||||
2016010101 86 6 0.7166993196134611
|
||||
2016010101 87 6 0.24743532603299845
|
||||
2016010101 88 6 0.4799158294749326
|
||||
2016010101 89 6 0.6653475668391361
|
||||
2016010101 90 6 0.775227487806507
|
||||
2016010101 91 6 0.6351773012367866
|
||||
2016010101 92 6 0.11859244615305142
|
||||
2016010101 93 6 0.47937789482404425
|
||||
2016010101 94 6 0.9176591572395486
|
||||
2016010101 95 6 0.31531929207681375
|
||||
2016010101 96 6 0.2620957460429699
|
||||
2016010101 97 6 0.48434732612327625
|
||||
2016010101 98 6 0.5404432154037893
|
||||
2016010101 99 6 0.0756673384583817
|
||||
2016010101 100 2 0.03467763042527283
|
||||
2016010101 101 2 0.6599322068206671
|
||||
2016010101 102 2 0.5851566930666781
|
||||
2016010101 103 2 0.9289365437820967
|
||||
2016010101 104 2 0.18676909308684186
|
||||
2016010101 105 2 0.9879999829181275
|
||||
2016010101 106 2 0.5989887945104968
|
||||
2016010101 107 2 0.5116753904494058
|
||||
2016010101 108 2 0.47180963140539234
|
||||
2016010101 109 2 0.3817179664370707
|
||||
2016010101 110 2 0.43867514596244317
|
||||
2016010101 111 2 0.8682947779291971
|
||||
2016010101 112 2 0.4531002982651964
|
||||
2016010101 113 2 0.4137446783476031
|
||||
2016010101 114 2 0.7828722097979959
|
||||
2016010101 115 2 0.7544029243316186
|
||||
2016010101 116 2 0.8073610464180303
|
||||
2016010101 117 2 0.3988232509895424
|
||||
2016010101 118 2 0.7420954813030962
|
||||
2016010101 119 2 0.9682767534483183
|
||||
2016010101 120 5 0.932266170435191
|
||||
2016010101 121 5 0.9442118007962537
|
||||
2016010101 122 5 0.3459369305182035
|
||||
2016010101 123 5 0.7353091264347587
|
||||
2016010101 124 5 0.09173481217567525
|
||||
2016010101 125 5 0.4841125699937996
|
||||
2016010101 126 5 0.09980357210224322
|
||||
2016010101 127 5 0.6310323689510959
|
||||
2016010101 128 5 0.5150126421118538
|
||||
2016010101 129 5 0.25952271088776446
|
||||
2016010101 130 5 0.5831931427843442
|
||||
2016010101 131 5 0.4857630294618446
|
||||
2016010101 132 5 0.1236953097751542
|
||||
2016010101 133 5 0.26865146249505867
|
||||
2016010101 134 5 0.758753349838143
|
||||
2016010101 135 5 0.4470022609108373
|
||||
2016010101 136 5 0.17341364635583423
|
||||
2016010101 137 5 0.7912507932298367
|
||||
2016010101 138 5 0.5888308222727439
|
||||
2016010101 139 5 0.3588616442423166
|
||||
2016010101 140 4 0.17634206104241368
|
||||
2016010101 141 4 0.5410328570948041
|
||||
2016010101 142 4 0.37692978933091825
|
||||
2016010101 143 4 0.7141908938615491
|
||||
2016010101 144 4 0.07267656582545412
|
||||
2016010101 145 4 0.8558523550312013
|
||||
2016010101 146 4 0.9813834781771359
|
||||
2016010101 147 4 0.8901977712477311
|
||||
2016010101 148 4 0.9560822202831745
|
||||
2016010101 149 4 0.3835659668522904
|
||||
2016010101 150 4 0.24970317775825412
|
||||
2016010101 151 4 0.23101157937207617
|
||||
2016010101 152 4 0.9962568046377394
|
||||
2016010101 153 4 0.8361247330090413
|
||||
2016010101 154 4 0.840684054948499
|
||||
2016010101 155 4 0.9052044460975798
|
||||
2016010101 156 4 0.6463926661125562
|
||||
2016010101 157 4 0.04411048858353084
|
||||
2016010101 158 4 0.38084927682182457
|
||||
2016010101 159 4 0.9341567696712807
|
||||
2016010101 160 2 0.6444600998039801
|
||||
2016010101 161 2 0.9959038761023891
|
||||
2016010101 162 2 0.6277409184324876
|
||||
2016010101 163 2 0.15567670882328122
|
||||
2016010101 164 2 0.9399539050143508
|
||||
2016010101 165 2 0.9427950658336692
|
||||
2016010101 166 2 0.32695366924165714
|
||||
2016010101 167 2 0.06494423391083837
|
||||
2016010101 168 2 0.7350656166921584
|
||||
2016010101 169 2 0.6597571880397293
|
||||
2016010101 170 2 0.6232896262457435
|
||||
2016010101 171 2 0.3991049576034902
|
||||
2016010101 172 2 0.030384991176152565
|
||||
2016010101 173 2 0.31798195313124733
|
||||
2016010101 174 2 0.35924006958852484
|
||||
2016010101 175 2 0.6967463287336736
|
||||
2016010101 176 2 0.48130500580323377
|
||||
2016010101 177 2 0.6307327262923675
|
||||
2016010101 178 2 0.8278978150044358
|
||||
2016010101 179 2 0.18235069592612707
|
||||
2016010101 180 6 0.33506690463108024
|
||||
2016010101 181 6 0.7467491419926762
|
||||
2016010101 182 6 0.935716427990453
|
||||
2016010101 183 6 0.27312535493840295
|
||||
2016010101 184 6 0.08411352680155793
|
||||
2016010101 185 6 0.7286593203874426
|
||||
2016010101 186 6 0.4944768219354745
|
||||
2016010101 187 6 0.3657278465928795
|
||||
2016010101 188 6 0.9032089768136577
|
||||
2016010101 189 6 0.07319490990372934
|
||||
2016010101 190 6 0.2711714964325377
|
||||
2016010101 191 6 0.43076666049459356
|
||||
2016010101 192 6 0.3929942938831871
|
||||
2016010101 193 6 0.06702713588446585
|
||||
2016010101 194 6 0.2880711799296227
|
||||
2016010101 195 6 0.2753747150639939
|
||||
2016010101 196 6 0.4504200108910671
|
||||
2016010101 197 6 0.785606159065031
|
||||
2016010101 198 6 0.6424553371008628
|
||||
2016010101 199 6 0.03389914063403954
|
||||
2016010101 200 7 0.7231577934981064
|
||||
2016010101 201 7 0.8020703973388846
|
||||
2016010101 202 7 0.34618571636736384
|
||||
2016010101 203 7 0.8870323946331024
|
||||
2016010101 204 7 0.22829122357756182
|
||||
2016010101 205 7 0.7391887247919631
|
||||
2016010101 206 7 0.32667298905806685
|
||||
2016010101 207 7 0.28966062987780283
|
||||
2016010101 208 7 0.35051206876737706
|
||||
2016010101 209 7 0.8028045292717977
|
||||
2016010101 210 7 0.8416165660849151
|
||||
2016010101 211 7 0.7392329281431951
|
||||
2016010101 212 7 0.338124145223625
|
||||
2016010101 213 7 0.42239056684731546
|
||||
2016010101 214 7 0.8257949422958452
|
||||
2016010101 215 7 0.4420414268433974
|
||||
2016010101 216 7 0.29256916602260286
|
||||
2016010101 217 7 0.6850345466358918
|
||||
2016010101 218 7 0.7464113033810255
|
||||
2016010101 219 7 0.35843523655425125
|
||||
2016010101 220 0 0.5941513813898424
|
||||
2016010101 221 0 0.027368543668205403
|
||||
2016010101 222 0 0.0646593769174727
|
||||
2016010101 223 0 0.9569978913839153
|
||||
2016010101 224 0 0.6811984053832584
|
||||
2016010101 225 0 0.5606358380083493
|
||||
2016010101 226 0 0.26670488547550186
|
||||
2016010101 227 0 0.9014400833785653
|
||||
2016010101 228 0 0.637566015918935
|
||||
2016010101 229 0 0.019791340881956998
|
||||
2016010101 230 0 0.4245542806308241
|
||||
2016010101 231 0 0.444662141732598
|
||||
2016010101 232 0 0.1758335119170732
|
||||
2016010101 233 0 0.0159214929875503
|
||||
2016010101 234 0 0.1771063482497166
|
||||
2016010101 235 0 0.9604671407661743
|
||||
2016010101 236 0 0.47384714026832164
|
||||
2016010101 237 0 0.18977081242542304
|
||||
2016010101 238 0 0.4766476987574778
|
||||
2016010101 239 0 0.9294775624405623
|
||||
2016010101 240 3 0.12594080257677187
|
||||
2016010101 241 3 0.7559152350398667
|
||||
2016010101 242 3 0.05745443874940148
|
||||
2016010101 243 3 0.6717027081764814
|
||||
2016010101 244 3 0.5700849510416076
|
||||
2016010101 245 3 0.8261816952276474
|
||||
2016010101 246 3 0.7060976979132971
|
||||
2016010101 247 3 0.4176561761388906
|
||||
2016010101 248 3 0.5256334635141646
|
||||
2016010101 249 3 0.49816409290688646
|
||||
2016010101 250 3 0.9830449049508984
|
||||
2016010101 251 3 0.5777317502988757
|
||||
2016010101 252 3 0.69224309816127
|
||||
2016010101 253 3 0.640944588547068
|
||||
2016010101 254 3 0.17549906190868492
|
||||
2016010101 255 3 0.5626366786777154
|
||||
2016010101 256 3 0.5489319715869551
|
||||
2016010101 257 3 0.5001626533570611
|
||||
2016010101 258 3 0.5248247072184595
|
||||
2016010101 259 3 0.5157966135463232
|
||||
2016010101 260 0 0.043824985443986386
|
||||
2016010101 261 0 0.4355867507058666
|
||||
2016010101 262 0 0.8517353983128922
|
||||
2016010101 263 0 0.050511310614060556
|
||||
2016010101 264 0 0.018975576091520008
|
||||
2016010101 265 0 0.24288753566173815
|
||||
2016010101 266 0 0.42592232986751477
|
||||
2016010101 267 0 0.8219313240908653
|
||||
2016010101 268 0 0.9200298974447165
|
||||
2016010101 269 0 0.9038295212557719
|
||||
2016010101 270 0 0.5268384730163258
|
||||
2016010101 271 0 0.8370249923326465
|
||||
2016010101 272 0 0.20764198291870795
|
||||
2016010101 273 0 0.28224686510065466
|
||||
2016010101 274 0 0.05114706164317673
|
||||
2016010101 275 0 0.6822109116561871
|
||||
2016010101 276 0 0.1920487580753658
|
||||
2016010101 277 0 0.5374714224193021
|
||||
2016010101 278 0 0.1407286063444365
|
||||
2016010101 279 0 0.6300903200679441
|
||||
2016010101 280 3 0.5761961995211345
|
||||
2016010101 281 3 0.48860594440363003
|
||||
2016010101 282 3 0.9621962588672488
|
||||
2016010101 283 3 0.3795960509843891
|
||||
2016010101 284 3 0.157214495528595
|
||||
2016010101 285 3 0.8775922187262636
|
||||
2016010101 286 3 0.04594168614675043
|
||||
2016010101 287 3 0.856217645027161
|
||||
2016010101 288 3 0.4401812473830602
|
||||
2016010101 289 3 0.45997954611887704
|
||||
2016010101 290 3 0.7575967536722598
|
||||
2016010101 291 3 0.30081626796169747
|
||||
2016010101 292 3 0.5556330504516347
|
||||
2016010101 293 3 0.2610662583323312
|
||||
2016010101 294 3 0.9456901737901005
|
||||
2016010101 295 3 0.7732531252738661
|
||||
2016010101 296 3 0.9312784670391524
|
||||
2016010101 297 3 0.1700551978587308
|
||||
2016010101 298 3 0.09416802177766626
|
||||
2016010101 299 3 0.047486020483062275
|
||||
2016010101 300 0 0.2368720948888663
|
||||
2016010101 301 0 0.1363540247675844
|
||||
2016010101 302 0 0.6796632164100268
|
||||
2016010101 303 0 0.4847790313081668
|
||||
2016010101 304 0 0.8923911191374748
|
||||
2016010101 305 0 0.4629209972203546
|
||||
2016010101 306 0 0.8770504803374757
|
||||
2016010101 307 0 0.16132164307745944
|
||||
2016010101 308 0 0.07724195103053844
|
||||
2016010101 309 0 0.7818522541007664
|
||||
2016010101 310 0 0.12646019668431752
|
||||
2016010101 311 0 0.7833272631599443
|
||||
2016010101 312 0 0.6456747363593918
|
||||
2016010101 313 0 0.007262692215570188
|
||||
2016010101 314 0 0.7252559157521992
|
||||
2016010101 315 0 0.3561896202462319
|
||||
2016010101 316 0 0.4916774723249263
|
||||
2016010101 317 0 0.41665575732393756
|
||||
2016010101 318 0 0.8873537893988267
|
||||
2016010101 319 0 2.671170814837476E-4
|
||||
2016010101 320 3 0.38601998905690915
|
||||
2016010101 321 3 0.3386538450586696
|
||||
2016010101 322 3 0.9029330123077748
|
||||
2016010101 323 3 0.7119643269179263
|
||||
2016010101 324 3 0.999586987542997
|
||||
2016010101 325 3 0.2606007123119771
|
||||
2016010101 326 3 0.758173586634067
|
||||
2016010101 327 3 0.8745954276793834
|
||||
2016010101 328 3 0.6271248733821071
|
||||
2016010101 329 3 0.9022512107069566
|
||||
2016010101 330 3 0.7581268831412983
|
||||
2016010101 331 3 0.5960487672639095
|
||||
2016010101 332 3 0.9976334716314451
|
||||
2016010101 333 3 0.16634963588078155
|
||||
2016010101 334 3 0.845497495310386
|
||||
2016010101 335 3 0.29561984090341287
|
||||
2016010101 336 3 0.3852437651241116
|
||||
2016010101 337 3 0.6298152822162192
|
||||
2016010101 338 3 0.22342878011982603
|
||||
2016010101 339 3 0.7315688774325575
|
||||
2016010101 340 1 0.5184463792301018
|
||||
2016010101 341 1 0.8248209481750183
|
||||
2016010101 342 1 0.19301501207164873
|
||||
2016010101 343 1 0.9789732956413038
|
||||
2016010101 344 1 0.0320870306724641
|
||||
2016010101 345 1 0.7513122728857949
|
||||
2016010101 346 1 0.4256312747004757
|
||||
2016010101 347 1 0.8885035744639196
|
||||
2016010101 348 1 0.9385374330083497
|
||||
2016010101 349 1 0.5124115905623042
|
||||
2016010101 350 1 0.1723444960263597
|
||||
2016010101 351 1 0.6268182721826895
|
||||
2016010101 352 1 0.8325112646795693
|
||||
2016010101 353 1 0.8753479069746972
|
||||
2016010101 354 1 0.17435564546281812
|
||||
2016010101 355 1 0.7733161692529222
|
||||
2016010101 356 1 0.09477460168514973
|
||||
2016010101 357 1 0.5522517284701014
|
||||
2016010101 358 1 0.2309322491262773
|
||||
2016010101 359 1 0.4553785462375177
|
||||
2016010101 360 3 0.4207538591898
|
||||
2016010101 361 3 0.5777884912044923
|
||||
2016010101 362 3 0.09598914695635141
|
||||
2016010101 363 3 0.8298751445992327
|
||||
2016010101 364 3 0.8455400406757709
|
||||
2016010101 365 3 0.7082357095169295
|
||||
2016010101 366 3 0.3428513232921617
|
||||
2016010101 367 3 0.002034459046941328
|
||||
2016010101 368 3 0.8041741490355258
|
||||
2016010101 369 3 0.5046992061463758
|
||||
2016010101 370 3 0.5968379402186867
|
||||
2016010101 371 3 0.49251728555072416
|
||||
2016010101 372 3 0.499514272967044
|
||||
2016010101 373 3 0.3163953106310815
|
||||
2016010101 374 3 0.9141264759479766
|
||||
2016010101 375 3 0.5654017237196727
|
||||
2016010101 376 3 0.870688146748015
|
||||
2016010101 377 3 0.8346158948878163
|
||||
2016010101 378 3 0.2534223361266069
|
||||
2016010101 379 3 0.6615127870881999
|
||||
2016010101 380 0 0.9404409994982728
|
||||
2016010101 381 0 0.7181624188084241
|
||||
2016010101 382 0 0.8688492337400265
|
||||
2016010101 383 0 0.7423992107648576
|
||||
2016010101 384 0 0.962538954563567
|
||||
2016010101 385 0 0.3065544881403711
|
||||
2016010101 386 0 0.5581815487224352
|
||||
2016010101 387 0 0.057834629796017833
|
||||
2016010101 388 0 0.7639391385884952
|
||||
2016010101 389 0 0.7851115146196147
|
||||
2016010101 390 0 0.5742903447712463
|
||||
2016010101 391 0 0.9505151407243474
|
||||
2016010101 392 0 0.7991385460697642
|
||||
2016010101 393 0 0.42243212914925077
|
||||
2016010101 394 0 0.5001511188980361
|
||||
2016010101 395 0 0.4162146563124456
|
||||
2016010101 396 0 0.053188648211970335
|
||||
2016010101 397 0 0.5917396934991566
|
||||
2016010101 398 0 0.1840362970791214
|
||||
2016010101 399 0 0.24913435524980954
|
|
|
@ -0,0 +1,20 @@
|
|||
2016010101 4 AAAAAT+7CjX1PphwP+/7EGIdEKJAWQAAAAAAAAAAABQ/8AAAAAAAAD+7CjX1PphwP/AAAAAAAAA/vncWqCA0iD/wAAAAAAAAP79sql5Sq7A/8AAAAAAAAD/HjWyUzH5EP/AAAAAAAAA/yXdIQeHLND/wAAAAAAAAP9ETDBCe0rQ/8AAAAAAAAD/bv94tVW6aP/AAAAAAAAA/3S8S4mBQjj/wAAAAAAAAP95AkomuKgA/8AAAAAAAAD/fMaB/VdVQP/AAAAAAAAA/4bkG+YKruz/wAAAAAAAAP+I3YqO/lEc/8AAAAAAAAD/ivYlUAYQVP/AAAAAAAAA/5fgCUTZfFT/wAAAAAAAAP+YxtG3/qb4/8AAAAAAAAD/oxDbRO3UhP/AAAAAAAAA/6YWLOSTBAz/wAAAAAAAAP+zyRaGOAPM/8AAAAAAAAD/tJwSKndEHP/AAAAAAAAA/7/sQYh0Qog==
|
||||
2016010101 4 AAAAAT+x8zeUx6E4P+9I7S+ZvE5AWQAAAAAAAAAAABQ/8AAAAAAAAD+x8zeUx6E4P/AAAAAAAAA/yqGfguJb4D/wAAAAAAAAP81DwCIyqLg/8AAAAAAAAD/QP/R3eDxQP/AAAAAAAAA/1TtECEm/ej/wAAAAAAAAP9pwM03FWLA/8AAAAAAAAD/ageyQGELKP/AAAAAAAAA/3QEv5GIwZD/wAAAAAAAAP9+x9tSbFIY/8AAAAAAAAD/hGQ6ntL+pP/AAAAAAAAA/4rLUCScKgT/wAAAAAAAAP+RZfioxXx4/8AAAAAAAAD/kjt12hqv2P/AAAAAAAAA/5Xv2onVpyj/wAAAAAAAAP+chr8/GhQU/8AAAAAAAAD/qiul4+5gXP/AAAAAAAAA/65CrHjSkeD/wAAAAAAAAP+uZJYlWNuw/8AAAAAAAAD/tL+mJ7/A5P/AAAAAAAAA/70jtL5m8Tg==
|
||||
2016010101 6 AAAAAT+pZ+KZEb+wP+0T7sTeuMtAWQAAAAAAAAAAABQ/8AAAAAAAAD+pZ+KZEb+wP/AAAAAAAAA/z6Ai70+EFD/wAAAAAAAAP8/vLqqMEFA/8AAAAAAAAD/QQ9jUjuUwP/AAAAAAAAA/0JVTHRKQND/wAAAAAAAAP9E+HZK7eOY/8AAAAAAAAD/UyuqJRsSIP/AAAAAAAAA/3JfCECkjhD/wAAAAAAAAP98Zs9r16tA/8AAAAAAAAD/hlVjJfubaP/AAAAAAAAA/5D/28EqWVj/wAAAAAAAAP+a35kxEwGc/8AAAAAAAAD/nCGskvBxgP/AAAAAAAAA/6NeE3XuZGD/wAAAAAAAAP+tabLGYftE/8AAAAAAAAD/rX7VulMyIP/AAAAAAAAA/63JKIUNgfT/wAAAAAAAAP+xWM2SGDAQ/8AAAAAAAAD/soaYGE6cpP/AAAAAAAAA/7RPuxN64yw==
|
||||
2016010101 3 AAAAAT+7UrkAsuZYP++GVNSKMLNAWQAAAAAAAAAAABQ/8AAAAAAAAD+7UrkAsuZYP/AAAAAAAAA/yoMFzGLc6D/wAAAAAAAAP80oFPmMpKA/8AAAAAAAAD/SB9Hu0WQMP/AAAAAAAAA/0p2jxoZR5D/wAAAAAAAAP9erevdmrn4/8AAAAAAAAD/ZJ5sCHUu8P/AAAAAAAAA/2g+3Zj8RQD/wAAAAAAAAP9pXRxFMlYY/8AAAAAAAAD/dGvAsch6UP/AAAAAAAAA/4QvQNT9cUj/wAAAAAAAAP+Gn7wL+/vA/8AAAAAAAAD/jMCLRbf6zP/AAAAAAAAA/56YBOjwGAT/wAAAAAAAAP+ooF1QMrUE/8AAAAAAAAD/s3N+aY1rXP/AAAAAAAAA/7OCY7n9O7T/wAAAAAAAAP+2I45MTpSU/8AAAAAAAAD/ukvT+mMTmP/AAAAAAAAA/74ZU1Iowsw==
|
||||
2016010101 6 AAAAAT+lyvFjKpDgP+++PV/m/pVAWQAAAAAAAAAAABQ/8AAAAAAAAD+lyvFjKpDgP/AAAAAAAAA/s17vSA3aUD/wAAAAAAAAP75cExXHreA/8AAAAAAAAD/Pq/X0l+noP/AAAAAAAAA/0MYtPGs3vj/wAAAAAAAAP9QuMPfRI2w/8AAAAAAAAD/cVF66gH9cP/AAAAAAAAA/3q4gnyx0Sj/wAAAAAAAAP9628OIbXPQ/8AAAAAAAAD/e/4vtZrBsP/AAAAAAAAA/4UtPkfAjtD/wAAAAAAAAP+RTX1j/Jzs/8AAAAAAAAD/lSob7AYMvP/AAAAAAAAA/5u8zaVnEvz/wAAAAAAAAP+d2Z76RMh0/8AAAAAAAAD/oi1ARoLi3P/AAAAAAAAA/6M6p4GLanT/wAAAAAAAAP+1ddrynAEU/8AAAAAAAAD/u12drHsniP/AAAAAAAAA/7749X+b+lQ==
|
||||
2016010101 2 AAAAAT+hwUQxKJzQP++dsiPinw9AWQAAAAAAAAAAABQ/8AAAAAAAAD+hwUQxKJzQP/AAAAAAAAA/x+gMtVsUlD/wAAAAAAAAP9huETGJJFw/8AAAAAAAAD/ZhlH0+JtkP/AAAAAAAAA/2nrK9ZloXD/wAAAAAAAAP9wTQOtegpQ/8AAAAAAAAD/c/5hkttbcP/AAAAAAAAA/3jIhBjS8AD/wAAAAAAAAP+BfpRGEvko/8AAAAAAAAD/iuZqHeDh2P/AAAAAAAAA/4yrqjGL6QD/wAAAAAAAAP+UeKiW75mM/8AAAAAAAAD/nvz8F1pkCP/AAAAAAAAA/6CQRmgBapD/wAAAAAAAAP+kNSgVA8tY/8AAAAAAAAD/p1ebVTcC+P/AAAAAAAAA/68kSIU/Knz/wAAAAAAAAP+252SFzTt4/8AAAAAAAAD/u/B+HsTPgP/AAAAAAAAA/752yI+KfDw==
|
||||
2016010101 5 AAAAAT+3e+7CMwCYP+42+6qdWYJAWQAAAAAAAAAAABQ/8AAAAAAAAD+3e+7CMwCYP/AAAAAAAAA/uYy6FjP/MD/wAAAAAAAAP7+qfu4nIug/8AAAAAAAAD/GMmsZ46MwP/AAAAAAAAA/0JwFJPVC9D/wAAAAAAAAP9ExledcFhQ/8AAAAAAAAD/WI9Smw3nKP/AAAAAAAAA/1veW1HPPcj/wAAAAAAAAP9ybr172aCA/8AAAAAAAAD/e+7NJ7S0kP/AAAAAAAAA/3xa90Uk/mj/wAAAAAAAAP+B6+8rcs+8/8AAAAAAAAD/iqYSqcFOyP/AAAAAAAAA/4tezvJE5ID/wAAAAAAAAP+QxastrmHo/8AAAAAAAAD/nh6cBT5qQP/AAAAAAAAA/6Ee1Gukg8T/wAAAAAAAAP+lR7S77Zgw/8AAAAAAAAD/t1R/dJfY7P/AAAAAAAAA/7jb7qp1Zgg==
|
||||
2016010101 4 AAAAAT+mlaZjwPIAP+/hVfNKxQhAWQAAAAAAAAAAABQ/8AAAAAAAAD+mlaZjwPIAP/AAAAAAAAA/sprucWfqKD/wAAAAAAAAP8aSYGyLDzw/8AAAAAAAAD/NkcmVMz+AP/AAAAAAAAA/z/ZGExbosD/wAAAAAAAAP9gfnh+EICw/8AAAAAAAAD/YX9WlKetQP/AAAAAAAAA/2IxYRN9PLD/wAAAAAAAAP+FQJCNpE3U/8AAAAAAAAD/krz+sKnxTP/AAAAAAAAA/5tqm3IeQRD/wAAAAAAAAP+rBiKf00Ww/8AAAAAAAAD/q5uI/SLgTP/AAAAAAAAA/62MkemII7z/wAAAAAAAAP+x8gAlPZSA/8AAAAAAAAD/s929QhdpRP/AAAAAAAAA/7eScvOJnrD/wAAAAAAAAP+6YOb2M59o/8AAAAAAAAD/vZ35S82B9P/AAAAAAAAA/7+FV80rFCA==
|
||||
2016010101 2 AAAAAT+fHT49kDQgP+/ecc46OIBAWQAAAAAAAAAAABQ/8AAAAAAAAD+fHT49kDQgP/AAAAAAAAA/sKAvcLX4oD/wAAAAAAAAP8PtNuKSjFQ/8AAAAAAAAD/HV0SBs+lEP/AAAAAAAAA/1FnQ+lqusD/wAAAAAAAAP9TszxUszr4/8AAAAAAAAD/W/coPkukeP/AAAAAAAAA/2YrvhSUFcD/wAAAAAAAAP97Ns4LU3Ng/8AAAAAAAAD/j8f0WFStEP/AAAAAAAAA/5BZ0H2DpyD/wAAAAAAAAP+Qu9mX+Khc/8AAAAAAAAD/kn2rJh4ULP/AAAAAAAAA/5Ry7Gz3PVj/wAAAAAAAAP+ZLvvTwnTs/8AAAAAAAAD/nhahUA2upP/AAAAAAAAA/6n4jjvv3Ej/wAAAAAAAAP+4UGjY5F5I/8AAAAAAAAD/uK2CO0r6uP/AAAAAAAAA/795xzjo4gA==
|
||||
2016010101 6 AAAAAT+hWzpoxXsQP+3xY5QRkN9AWQAAAAAAAAAAABQ/8AAAAAAAAD+hWzpoxXsQP/AAAAAAAAA/sSiwvJF+AD/wAAAAAAAAP7K85tBFKsg/8AAAAAAAAD+1iHbOw4+oP/AAAAAAAAA/0VrfsTI/kj/wAAAAAAAAP9F64sTKzig/8AAAAAAAAD/Rn71E1hdOP/AAAAAAAAA/0m/CGi3xcD/wAAAAAAAAP9VxvHVXMjo/8AAAAAAAAD/XaBXFFpZIP/AAAAAAAAA/2SbRie+Lsj/wAAAAAAAAP9uRrlPCArI/8AAAAAAAAD/c0650D24iP/AAAAAAAAA/36WCHLXxuD/wAAAAAAAAP+SO/n6/pMA/8AAAAAAAAD/nUS1Z36s7P/AAAAAAAAA/5+VedOWWXD/wAAAAAAAAP+kjr4cXDjA/8AAAAAAAAD/s5xaDG8j4P/AAAAAAAAA/7fFjlBGQ3w==
|
||||
2016010101 7 AAAAAT/NOKWVnV6wP+xikcKuIr1AWQAAAAAAAAAAABQ/8AAAAAAAAD/NOKWVnV6wP/AAAAAAAAA/0onMvRDkID/wAAAAAAAAP9K5dAX4pQw/8AAAAAAAAD/U6DXTH2vwP/AAAAAAAAA/1aPTdG5MlD/wAAAAAAAAP9Yn6CKI+GA/8AAAAAAAAD/WbsosDWNUP/AAAAAAAAA/1vCaWK8Clj/wAAAAAAAAP9sIcnGv4To/8AAAAAAAAD/cSmgf8UCcP/AAAAAAAAA/5evNkc3Njj/wAAAAAAAAP+ckG9AdgBw/8AAAAAAAAD/np28c0bWqP/AAAAAAAAA/56fL0FAK/D/wAAAAAAAAP+fimfUsXBE/8AAAAAAAAD/pqo+JtSClP/AAAAAAAAA/6bCTH8myjT/wAAAAAAAAP+ps6YPLnJE/8AAAAAAAAD/q7oXdY2i4P/AAAAAAAAA/7GKRwq4ivQ==
|
||||
2016010101 0 AAAAAT+QTblOvXZgP+68JZXPJalAWQAAAAAAAAAAABQ/8AAAAAAAAD+QTblOvXZgP/AAAAAAAAA/lEQuZ1P2AD/wAAAAAAAAP5wGf9/58sA/8AAAAAAAAD+wjYRVPYMoP/AAAAAAAAA/xoG2Z5zBhD/wAAAAAAAAP8ara7rSwSg/8AAAAAAAAD/ISmj0jSAcP/AAAAAAAAA/0RGxXjM+xD/wAAAAAAAAP9sr5ber6qI/8AAAAAAAAD/cdVgzILFcP/AAAAAAAAA/3lOC9LBbMj/wAAAAAAAAP96BZVl4ITY/8AAAAAAAAD/h8LqRpsPnP/AAAAAAAAA/4wNJwf4pvz/wAAAAAAAAP+Rm8NhtNf8/8AAAAAAAAD/lzGCZJquVP/AAAAAAAAA/7NiY361DLz/wAAAAAAAAP+2+R7qhiVM/8AAAAAAAAD/un7oKurdPP/AAAAAAAAA/7rwllc8lqQ==
|
||||
2016010101 3 AAAAAT+taqsO4J8wP+91GpaobuVAWQAAAAAAAAAAABQ/8AAAAAAAAD+taqsO4J8wP/AAAAAAAAA/wB7UBiZNGD/wAAAAAAAAP8Z2wNWwMIg/8AAAAAAAAD/auuD4X0r4P/AAAAAAAAA/3+HrpcTmvD/wAAAAAAAAP+ABVRvbLFY/8AAAAAAAAD/ggWfmUjVSP/AAAAAAAAA/4MtdLzRamD/wAAAAAAAAP+DR/UTvP94/8AAAAAAAAD/hkNnINjuVP/AAAAAAAAA/4gEeos5tWT/wAAAAAAAAP+I+IsuVS38/8AAAAAAAAD/ifMdLrJ/SP/AAAAAAAAA/5IKeOctzXz/wAAAAAAAAP+V+lq2IFXU/8AAAAAAAAD/mJtr/b3/7P/AAAAAAAAA/5phaMwoxJD/wAAAAAAAAP+gwdSWha5E/8AAAAAAAAD/qcBSYMc+LP/AAAAAAAAA/73Ualqhu5Q==
|
||||
2016010101 0 AAAAAT+TblVa8X+AP+1w4oobwNpAWQAAAAAAAAAAABQ/8AAAAAAAAD+TblVa8X+AP/AAAAAAAAA/pnA6fnWN0D/wAAAAAAAAP6ncnlZUutA/8AAAAAAAAD+qL/KaFn9QP/AAAAAAAAA/wgNlHO41rD/wAAAAAAAAP8iVDb+V5wA/8AAAAAAAAD/KlAMy9MrgP/AAAAAAAAA/zxbwUyL4MD/wAAAAAAAAP9IQVSfAX5w/8AAAAAAAAD/bQk+7WrHEP/AAAAAAAAA/2+CnQDaRCD/wAAAAAAAAP+Db3Ft8Jnw/8AAAAAAAAD/hMvdEumqHP/AAAAAAAAA/5CmzLMb5yz/wAAAAAAAAP+XUq/pRNJU/8AAAAAAAAD/qTULrkOdfP/AAAAAAAAA/6sjoowAblj/wAAAAAAAAP+tBapgTLlU/8AAAAAAAAD/s7CvjXn34P/AAAAAAAAA/7XDiihvA2g==
|
||||
2016010101 3 AAAAAT+nhasvC/mwP+7KT88FY2ZAWQAAAAAAAAAAABQ/8AAAAAAAAD+nhasvC/mwP/AAAAAAAAA/qFAWcfvfYD/wAAAAAAAAP7gbZT3dNSg/8AAAAAAAAD/EH5rGYFBoP/AAAAAAAAA/xcReZKi2ID/wAAAAAAAAP9C1T0BoF8I/8AAAAAAAAD/TQJLgQARQP/AAAAAAAAA/2EtNPCrL+j/wAAAAAAAAP9wr7fd0p5g/8AAAAAAAAD/dcE4M2jVCP/AAAAAAAAA/30VR3fYQ9j/wAAAAAAAAP+HHvvaIhxQ/8AAAAAAAAD/icDMDILp7P/AAAAAAAAA/6D47jBJ//T/wAAAAAAAAP+i+fVaSl+4/8AAAAAAAAD/rZiKL9MscP/AAAAAAAAA/7BU8RtTofz/wAAAAAAAAP+3NCH/s3ps/8AAAAAAAAD/uQxgKEncYP/AAAAAAAAA/7spPzwVjZg==
|
||||
2016010101 0 AAAAAT8xgXshEagAP+yOd9H99h9AWQAAAAAAAAAAABQ/8AAAAAAAAD8xgXshEagAP/AAAAAAAAA/fb98GL+kgD/wAAAAAAAAP7PGIOWOLnA/8AAAAAAAAD/AL9kEgJ+gP/AAAAAAAAA/wXQMdocCBD/wAAAAAAAAP8SmMAaTzFQ/8AAAAAAAAD/OUdMmcPv4P/AAAAAAAAA/1svPjIh2Yj/wAAAAAAAAP9qqfOjZXwQ/8AAAAAAAAD/doH9j7Fw+P/AAAAAAAAA/3waeoVBUKD/wAAAAAAAAP993pMn0MTY/8AAAAAAAAD/kqV4QkItYP/AAAAAAAAA/5b/NEtjOuz/wAAAAAAAAP+c1S+TsW/g/8AAAAAAAAD/pBO8EtVVkP/AAAAAAAAA/6REEVirJ5z/wAAAAAAAAP+wQzCs/spQ/8AAAAAAAAD/sZTPGLmNjP/AAAAAAAAA/7I530f32Hw==
|
||||
2016010101 3 AAAAAT/FSvHi55yMP+/8ndmgI3xAWQAAAAAAAAAAABQ/8AAAAAAAAD/FSvHi55yMP/AAAAAAAAA/zJlQc8zHKD/wAAAAAAAAP9Ctrpwsbog/8AAAAAAAAD/S6297LqX+P/AAAAAAAAA/1ayBLUxDyD/wAAAAAAAAP9in1XcMirA/8AAAAAAAAD/YtI0vJoJMP/AAAAAAAAA/4xLU3UcHBT/wAAAAAAAAP+QRaC615bY/8AAAAAAAAD/kJ3Jg9HgxP/AAAAAAAAA/5shpaYD/Jj/wAAAAAAAAP+dpAyJrBDw/8AAAAAAAAD/oQpNPKe3lP/AAAAAAAAA/6EL1QOkZUT/wAAAAAAAAP+sOUMNmqBw/8AAAAAAAAD/r/K+M46aVP/AAAAAAAAA/7N897lhouT/wAAAAAAAAP+zk08XK5qw/8AAAAAAAAD/v7J0Hwan9P/AAAAAAAAA/7/yd2aAjfA==
|
||||
2016010101 1 AAAAAT+gbbYWulvgP+9Tv84N+cRAWQAAAAAAAAAAABQ/8AAAAAAAAD+gbbYWulvgP/AAAAAAAAA/uEMl9rqpOD/wAAAAAAAAP8YPYmsKFqw/8AAAAAAAAD/GUUkpkWEQP/AAAAAAAAA/yLS3Rj4NKD/wAAAAAAAAP82PMBzLZyg/8AAAAAAAAD/bPYr1P5LOP/AAAAAAAAA/3STsDtj4bD/wAAAAAAAAP+BlrP3xzlk/8AAAAAAAAD/glxzccLubP/AAAAAAAAA/4awL0R4Iqj/wAAAAAAAAP+QO5TFx6oU/8AAAAAAAAD/oCsAJJBflP/AAAAAAAAA/6L8BjQ0ZAD/wAAAAAAAAP+pk7uau9Q8/8AAAAAAAAD/qo+6p6zYRP/AAAAAAAAA/7ALZnSKCDD/wAAAAAAAAP+xunwxWc8w/8AAAAAAAAD/uCH+nmvcDP/AAAAAAAAA/71O/zg35xA==
|
||||
2016010101 3 AAAAAT9gqpHiSxMAP+1AhirTVlhAWQAAAAAAAAAAABQ/8AAAAAAAAD9gqpHiSxMAP/AAAAAAAAA/uJK+pvLOwD/wAAAAAAAAP9A4ElFvVy4/8AAAAAAAAD/UP9Id8SscP/AAAAAAAAA/1fFGrTuHtD/wAAAAAAAAP9rtoZg4s+Y/8AAAAAAAAD/fhWc4iexmP/AAAAAAAAA/3/gKtpHVvj/wAAAAAAAAP+AmfvMW6DY/8AAAAAAAAD/iF8VbD0v0P/AAAAAAAAA/4n0+SjdPsT/wAAAAAAAAP+MZS+FIClg/8AAAAAAAAD/lKxzdTb+HP/AAAAAAAAA/5qnd70eBlT/wAAAAAAAAP+m7y2zMrcc/8AAAAAAAAD/qjlZRuiSqP/AAAAAAAAA/6rUsZKh6bT/wAAAAAAAAP+sOqfzFJng/8AAAAAAAAD/r3K1jaYW4P/AAAAAAAAA/7UCGKtNWWA==
|
||||
2016010101 0 AAAAAT+rO4rhLX9AP+7NHn5fP+ZAWQAAAAAAAAAAABQ/8AAAAAAAAD+rO4rhLX9AP/AAAAAAAAA/rZyAJxmvED/wAAAAAAAAP8eOgFqdqog/8AAAAAAAAD/P46JyDdI4P/AAAAAAAAA/056Wt0BQgj/wAAAAAAAAP9qjQsw+lNQ/8AAAAAAAAD/bCSDE3nGMP/AAAAAAAAA/4AE861cyKD/wAAAAAAAAP+Hcn40fyqs/8AAAAAAAAD/iYJYlJm8lP/AAAAAAAAA/4u+IFOpduj/wAAAAAAAAP+b7L8C/8zI/8AAAAAAAAD/nwbv9WfX2P/AAAAAAAAA/6HIwfgvnaT/wAAAAAAAAP+kfoi7gIv0/8AAAAAAAAD/pkosACvbMP/AAAAAAAAA/682c6IItqT/wAAAAAAAAP+4YF7kVNEw/8AAAAAAAAD/uap66eHHzP/AAAAAAAAA/7s0efl8/5g==
|
|
1
pom.xml
1
pom.xml
|
@ -174,6 +174,7 @@
|
|||
<module>extensions-contrib/materialized-view-selection</module>
|
||||
<module>extensions-contrib/momentsketch</module>
|
||||
<module>extensions-contrib/moving-average-query</module>
|
||||
<module>extensions-contrib/tdigestsketch</module>
|
||||
<!-- distribution packaging -->
|
||||
<module>distribution</module>
|
||||
</modules>
|
||||
|
|
|
@ -121,6 +121,11 @@ public class AggregatorUtil
|
|||
public static final byte MOMENTS_SKETCH_BUILD_CACHE_TYPE_ID = 0x36;
|
||||
public static final byte MOMENTS_SKETCH_MERGE_CACHE_TYPE_ID = 0x37;
|
||||
|
||||
// TDigest sketch aggregators
|
||||
public static final byte TDIGEST_BUILD_SKETCH_CACHE_TYPE_ID = 0x38;
|
||||
public static final byte TDIGEST_MERGE_SKETCH_CACHE_TYPE_ID = 0x39;
|
||||
public static final byte TDIGEST_SKETCH_TO_QUANTILES_CACHE_TYPE_ID = 0x40;
|
||||
|
||||
/**
|
||||
* returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg
|
||||
*
|
||||
|
|
Loading…
Reference in New Issue