mirror of https://github.com/apache/druid.git
ArrayOfDoublesSketch module (#5148)
* ArrayOfDoublesSketch module * UTF-8 fix * javadoc, style fixes * more style fixes * null key selector fix * more style fixes * removed @Override, strict compiler doesn't like it * removed @Override, strict compiler doesn't like it * IndexedInts is not autoclosable? removed one more @0verride * synchronized with upstream master * removed unused imports * addressed review points * null fix * addressed review points * IAE from druid package * synchronized aggregate() and get() * use locks per buffer position * corrected javadoc * style fixes * added lock and narrowed the scope * addressed review comments * conflict resolution went wrong * addressed review comments * javadoc * javadoc links * fully qualified name since there is no import for this class * addressed review points * style fix * StandardCharsets.UTF_8 * addressed review points * added @Override * added equals and hashCode tests for post aggs * formatting * suppress warnings * optimal IndexedInts iteration * suppress SelfEquals * added comments about getClass() in equals()
This commit is contained in:
parent
7a1faa332f
commit
15864434be
|
@ -40,6 +40,10 @@
|
||||||
<artifactId>sketches-core</artifactId>
|
<artifactId>sketches-core</artifactId>
|
||||||
<version>0.10.3</version>
|
<version>0.10.3</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>org.apache.commons</groupId>
|
||||||
|
<artifactId>commons-math3</artifactId>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>io.druid</groupId>
|
<groupId>io.druid</groupId>
|
||||||
<artifactId>druid-api</artifactId>
|
<artifactId>druid-api</artifactId>
|
||||||
|
|
|
@ -0,0 +1,311 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.Aggregator;
|
||||||
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.BufferAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||||
|
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||||
|
import io.druid.segment.ColumnSelectorFactory;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
|
import io.druid.segment.NilColumnValueSelector;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.Util;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUnion;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSetOperationBuilder;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchAggregatorFactory extends AggregatorFactory
|
||||||
|
{
|
||||||
|
|
||||||
|
public static final Comparator<ArrayOfDoublesSketch> COMPARATOR =
|
||||||
|
Comparator.nullsFirst(Comparator.comparingDouble(ArrayOfDoublesSketch::getEstimate));
|
||||||
|
|
||||||
|
private final String name;
|
||||||
|
private final String fieldName;
|
||||||
|
private final int nominalEntries;
|
||||||
|
private final int numberOfValues;
|
||||||
|
// if specified indicates building sketched from raw data, and also implies the number of values
|
||||||
|
@Nullable private final List<String> metricColumns;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchAggregatorFactory(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("fieldName") final String fieldName,
|
||||||
|
@JsonProperty("nominalEntries") @Nullable final Integer nominalEntries,
|
||||||
|
@JsonProperty("metricColumns") @Nullable final List<String> metricColumns,
|
||||||
|
@JsonProperty("numberOfValues") @Nullable final Integer numberOfValues
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.name = Preconditions.checkNotNull(name, "Must have a valid, non-null aggregator name");
|
||||||
|
this.fieldName = Preconditions.checkNotNull(fieldName, "Must have a valid, non-null fieldName");
|
||||||
|
this.nominalEntries = nominalEntries == null ? Util.DEFAULT_NOMINAL_ENTRIES : nominalEntries;
|
||||||
|
Util.checkIfPowerOf2(this.nominalEntries, "nominalEntries");
|
||||||
|
this.metricColumns = metricColumns;
|
||||||
|
this.numberOfValues = numberOfValues == null ? (metricColumns == null ? 1 : metricColumns.size()) : numberOfValues;
|
||||||
|
if (metricColumns != null && metricColumns.size() != this.numberOfValues) {
|
||||||
|
throw new IAE(
|
||||||
|
"Number of metricColumns [%d] must agree with numValues [%d]",
|
||||||
|
metricColumns.size(),
|
||||||
|
this.numberOfValues
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator factorize(final ColumnSelectorFactory metricFactory)
|
||||||
|
{
|
||||||
|
if (metricColumns == null) { // input is sketches, use merge aggregator
|
||||||
|
final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector = metricFactory
|
||||||
|
.makeColumnValueSelector(fieldName);
|
||||||
|
if (selector instanceof NilColumnValueSelector) {
|
||||||
|
return new ArrayOfDoublesSketchNoOpAggregator(numberOfValues);
|
||||||
|
}
|
||||||
|
return new ArrayOfDoublesSketchMergeAggregator(selector, nominalEntries, numberOfValues);
|
||||||
|
}
|
||||||
|
// input is raw data (key and array of values), use build aggregator
|
||||||
|
final DimensionSelector keySelector = metricFactory
|
||||||
|
.makeDimensionSelector(new DefaultDimensionSpec(fieldName, fieldName));
|
||||||
|
if (DimensionSelectorUtils.isNilSelector(keySelector)) {
|
||||||
|
return new ArrayOfDoublesSketchNoOpAggregator(numberOfValues);
|
||||||
|
}
|
||||||
|
final List<BaseDoubleColumnValueSelector> valueSelectors = new ArrayList<>();
|
||||||
|
for (final String column : metricColumns) {
|
||||||
|
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(column);
|
||||||
|
valueSelectors.add(valueSelector);
|
||||||
|
}
|
||||||
|
return new ArrayOfDoublesSketchBuildAggregator(keySelector, valueSelectors, nominalEntries);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BufferAggregator factorizeBuffered(final ColumnSelectorFactory metricFactory)
|
||||||
|
{
|
||||||
|
if (metricColumns == null) { // input is sketches, use merge aggregator
|
||||||
|
final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector = metricFactory
|
||||||
|
.makeColumnValueSelector(fieldName);
|
||||||
|
if (selector instanceof NilColumnValueSelector) {
|
||||||
|
return new ArrayOfDoublesSketchNoOpBufferAggregator(numberOfValues);
|
||||||
|
}
|
||||||
|
return new ArrayOfDoublesSketchMergeBufferAggregator(
|
||||||
|
selector,
|
||||||
|
nominalEntries,
|
||||||
|
numberOfValues,
|
||||||
|
getMaxIntermediateSize()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
// input is raw data (key and array of values), use build aggregator
|
||||||
|
final DimensionSelector keySelector = metricFactory
|
||||||
|
.makeDimensionSelector(new DefaultDimensionSpec(fieldName, fieldName));
|
||||||
|
if (DimensionSelectorUtils.isNilSelector(keySelector)) {
|
||||||
|
return new ArrayOfDoublesSketchNoOpBufferAggregator(numberOfValues);
|
||||||
|
}
|
||||||
|
final List<BaseDoubleColumnValueSelector> valueSelectors = new ArrayList<>();
|
||||||
|
for (final String column : metricColumns) {
|
||||||
|
final BaseDoubleColumnValueSelector valueSelector = metricFactory.makeColumnValueSelector(column);
|
||||||
|
valueSelectors.add(valueSelector);
|
||||||
|
}
|
||||||
|
return new ArrayOfDoublesSketchBuildBufferAggregator(
|
||||||
|
keySelector,
|
||||||
|
valueSelectors,
|
||||||
|
nominalEntries,
|
||||||
|
getMaxIntermediateSize()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object deserialize(final Object object)
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketchOperations.deserialize(object);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<ArrayOfDoublesSketch> getComparator()
|
||||||
|
{
|
||||||
|
return COMPARATOR;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object combine(@Nullable final Object lhs, @Nullable final Object rhs)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesUnion union = new ArrayOfDoublesSetOperationBuilder().setNominalEntries(nominalEntries)
|
||||||
|
.setNumberOfValues(numberOfValues).buildUnion();
|
||||||
|
if (lhs != null) {
|
||||||
|
union.update((ArrayOfDoublesSketch) lhs);
|
||||||
|
}
|
||||||
|
if (rhs != null) {
|
||||||
|
union.update((ArrayOfDoublesSketch) rhs);
|
||||||
|
}
|
||||||
|
return union.getResult();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@JsonProperty
|
||||||
|
public String getName()
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getFieldName()
|
||||||
|
{
|
||||||
|
return fieldName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNominalEntries()
|
||||||
|
{
|
||||||
|
return nominalEntries;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public List<String> getMetricColumns()
|
||||||
|
{
|
||||||
|
return metricColumns;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNumberOfValues()
|
||||||
|
{
|
||||||
|
return numberOfValues;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> requiredFields()
|
||||||
|
{
|
||||||
|
return Collections.singletonList(fieldName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
final CacheKeyBuilder builder = new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_CACHE_TYPE_ID)
|
||||||
|
.appendString(name)
|
||||||
|
.appendString(fieldName)
|
||||||
|
.appendInt(nominalEntries)
|
||||||
|
.appendInt(numberOfValues);
|
||||||
|
if (metricColumns != null) {
|
||||||
|
builder.appendStrings(metricColumns);
|
||||||
|
}
|
||||||
|
return builder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxIntermediateSize()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesUnion.getMaxBytes(nominalEntries, numberOfValues);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<AggregatorFactory> getRequiredColumns()
|
||||||
|
{
|
||||||
|
return Collections.<AggregatorFactory>singletonList(
|
||||||
|
new ArrayOfDoublesSketchAggregatorFactory(
|
||||||
|
fieldName,
|
||||||
|
fieldName,
|
||||||
|
nominalEntries,
|
||||||
|
metricColumns,
|
||||||
|
numberOfValues
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AggregatorFactory getCombiningFactory()
|
||||||
|
{
|
||||||
|
return new ArrayOfDoublesSketchAggregatorFactory(name, name, nominalEntries, null, numberOfValues);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object finalizeComputation(final Object object)
|
||||||
|
{
|
||||||
|
return ((ArrayOfDoublesSketch) object).getEstimate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTypeName()
|
||||||
|
{
|
||||||
|
if (metricColumns == null) {
|
||||||
|
return ArrayOfDoublesSketchModule.ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG;
|
||||||
|
}
|
||||||
|
return ArrayOfDoublesSketchModule.ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (!(o instanceof ArrayOfDoublesSketchAggregatorFactory)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchAggregatorFactory that = (ArrayOfDoublesSketchAggregatorFactory) o;
|
||||||
|
if (!name.equals(that.name)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!fieldName.equals(that.fieldName)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (nominalEntries != that.nominalEntries) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!Objects.equals(metricColumns, that.metricColumns)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return numberOfValues != that.numberOfValues;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(name, fieldName, nominalEntries, metricColumns, numberOfValues);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{"
|
||||||
|
+ "fieldName=" + fieldName
|
||||||
|
+ ", name=" + name
|
||||||
|
+ ", nominalEntries=" + nominalEntries
|
||||||
|
+ ", metricColumns=" + metricColumns
|
||||||
|
+ ", numberOfValues=" + numberOfValues
|
||||||
|
+ "}";
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,110 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
|
||||||
|
import io.druid.query.aggregation.Aggregator;
|
||||||
|
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This aggregator builds sketches from raw data.
|
||||||
|
* The input is in the form of a key and array of double values.
|
||||||
|
* The output is {@link com.yahoo.sketches.tuple.ArrayOfDoublesSketch}.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchBuildAggregator implements Aggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final DimensionSelector keySelector;
|
||||||
|
private final BaseDoubleColumnValueSelector[] valueSelectors;
|
||||||
|
private double[] values; // not part of the state, but to reuse in aggregate() method
|
||||||
|
private ArrayOfDoublesUpdatableSketch sketch;
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchBuildAggregator(
|
||||||
|
final DimensionSelector keySelector,
|
||||||
|
final List<BaseDoubleColumnValueSelector> valueSelectors,
|
||||||
|
final int nominalEntries
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.keySelector = keySelector;
|
||||||
|
this.valueSelectors = valueSelectors.toArray(new BaseDoubleColumnValueSelector[0]);
|
||||||
|
values = new double[valueSelectors.size()];
|
||||||
|
sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(nominalEntries)
|
||||||
|
.setNumberOfValues(valueSelectors.size()).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses synchronization because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void aggregate()
|
||||||
|
{
|
||||||
|
final IndexedInts keys = keySelector.getRow();
|
||||||
|
for (int i = 0; i < valueSelectors.length; i++) {
|
||||||
|
values[i] = valueSelectors[i].getDouble();
|
||||||
|
}
|
||||||
|
synchronized (this) {
|
||||||
|
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
|
||||||
|
final String key = keySelector.lookupName(keys.get(i));
|
||||||
|
sketch.update(key, values);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses synchronization because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
* The returned sketch is a separate instance of ArrayOfDoublesCompactSketch
|
||||||
|
* representing the current state of the aggregation, and is not affected by consequent
|
||||||
|
* aggregate() calls
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public synchronized Object get()
|
||||||
|
{
|
||||||
|
return sketch.compact();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
sketch = null;
|
||||||
|
values = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,173 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.google.common.util.concurrent.Striped;
|
||||||
|
import com.yahoo.memory.WritableMemory;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketches;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
|
||||||
|
import io.druid.query.aggregation.BufferAggregator;
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.BaseDoubleColumnValueSelector;
|
||||||
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.data.IndexedInts;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.locks.Lock;
|
||||||
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This aggregator builds sketches from raw data.
|
||||||
|
* The input is in the form of a key and array of double values.
|
||||||
|
* The output is {@link com.yahoo.sketches.tuple.ArrayOfDoublesSketch}.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchBuildBufferAggregator implements BufferAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private static final int NUM_STRIPES = 64; // for locking per buffer position (power of 2 to make index computation faster)
|
||||||
|
|
||||||
|
private final DimensionSelector keySelector;
|
||||||
|
private final BaseDoubleColumnValueSelector[] valueSelectors;
|
||||||
|
private final int nominalEntries;
|
||||||
|
private final int maxIntermediateSize;
|
||||||
|
private double[] values; // not part of the state, but to reuse in aggregate() method
|
||||||
|
private final Striped<ReadWriteLock> stripedLock = Striped.readWriteLock(NUM_STRIPES);
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchBuildBufferAggregator(
|
||||||
|
final DimensionSelector keySelector,
|
||||||
|
final List<BaseDoubleColumnValueSelector> valueSelectors,
|
||||||
|
int nominalEntries,
|
||||||
|
int maxIntermediateSize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.keySelector = keySelector;
|
||||||
|
this.valueSelectors = valueSelectors.toArray(new BaseDoubleColumnValueSelector[0]);
|
||||||
|
this.nominalEntries = nominalEntries;
|
||||||
|
this.maxIntermediateSize = maxIntermediateSize;
|
||||||
|
values = new double[valueSelectors.size()];
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(nominalEntries)
|
||||||
|
.setNumberOfValues(valueSelectors.length)
|
||||||
|
.setNumberOfValues(valueSelectors.length).build(region);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses locks because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void aggregate(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
for (int i = 0; i < valueSelectors.length; i++) {
|
||||||
|
values[i] = valueSelectors[i].getDouble();
|
||||||
|
}
|
||||||
|
final IndexedInts keys = keySelector.getRow();
|
||||||
|
// Wrapping memory and ArrayOfDoublesSketch is inexpensive compared to sketch operations.
|
||||||
|
// Maintaining a cache of wrapped objects per buffer position like in Theta sketch aggregator
|
||||||
|
// might might be considered, but it would increase complexity including relocate() support.
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
final Lock lock = stripedLock.getAt(lockIndex(position)).writeLock();
|
||||||
|
lock.lock();
|
||||||
|
try {
|
||||||
|
final ArrayOfDoublesUpdatableSketch sketch = ArrayOfDoublesSketches.wrapUpdatableSketch(region);
|
||||||
|
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
|
||||||
|
final String key = keySelector.lookupName(keys.get(i));
|
||||||
|
sketch.update(key, values);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses locks because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
* The returned sketch is a separate instance of ArrayOfDoublesCompactSketch
|
||||||
|
* representing the current state of the aggregation, and is not affected by consequent
|
||||||
|
* aggregate() calls
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Object get(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
final Lock lock = stripedLock.getAt(lockIndex(position)).readLock();
|
||||||
|
lock.lock();
|
||||||
|
try {
|
||||||
|
final ArrayOfDoublesUpdatableSketch sketch = (ArrayOfDoublesUpdatableSketch) ArrayOfDoublesSketches
|
||||||
|
.wrapSketch(region);
|
||||||
|
return sketch.compact();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
values = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("keySelector", keySelector);
|
||||||
|
inspector.visit("valueSelectors", valueSelectors);
|
||||||
|
}
|
||||||
|
|
||||||
|
// compute lock index to avoid boxing in Striped.get() call
|
||||||
|
static int lockIndex(final int position)
|
||||||
|
{
|
||||||
|
return smear(position) % NUM_STRIPES;
|
||||||
|
}
|
||||||
|
|
||||||
|
// from https://github.com/google/guava/blob/master/guava/src/com/google/common/util/concurrent/Striped.java#L536-L548
|
||||||
|
private static int smear(int hashCode)
|
||||||
|
{
|
||||||
|
hashCode ^= (hashCode >>> 20) ^ (hashCode >>> 12);
|
||||||
|
return hashCode ^ (hashCode >>> 7) ^ (hashCode >>> 4);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,47 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchBuildComplexMetricSerde extends ArrayOfDoublesSketchMergeComplexMetricSerde
|
||||||
|
{
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ComplexMetricExtractor getExtractor()
|
||||||
|
{
|
||||||
|
return new ComplexMetricExtractor()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<?> extractedClass()
|
||||||
|
{
|
||||||
|
return Object.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object extractValue(final InputRow inputRow, final String metricName)
|
||||||
|
{
|
||||||
|
return inputRow.getRaw(metricName);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,43 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.core.JsonGenerator;
|
||||||
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
|
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||||
|
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchJsonSerializer extends JsonSerializer<ArrayOfDoublesSketch>
|
||||||
|
{
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void serialize(
|
||||||
|
final ArrayOfDoublesSketch sketch,
|
||||||
|
final JsonGenerator generator,
|
||||||
|
final SerializerProvider provider
|
||||||
|
) throws IOException, JsonProcessingException
|
||||||
|
{
|
||||||
|
generator.writeBinary(sketch.toByteArray());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,99 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSetOperationBuilder;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUnion;
|
||||||
|
import io.druid.query.aggregation.Aggregator;
|
||||||
|
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This aggregator merges existing sketches.
|
||||||
|
* The input column contains ArrayOfDoublesSketch.
|
||||||
|
* The output is {@link ArrayOfDoublesSketch} that is a union of the input sketches.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchMergeAggregator implements Aggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector;
|
||||||
|
private ArrayOfDoublesUnion union;
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchMergeAggregator(
|
||||||
|
final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector,
|
||||||
|
final int nominalEntries,
|
||||||
|
final int numberOfValues
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.selector = selector;
|
||||||
|
union = new ArrayOfDoublesSetOperationBuilder().setNominalEntries(nominalEntries).setNumberOfValues(numberOfValues)
|
||||||
|
.buildUnion();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses synchronization because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void aggregate()
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch update = selector.getObject();
|
||||||
|
if (update == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
synchronized (this) {
|
||||||
|
union.update(update);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses synchronization because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
* The returned sketch is a separate instance of ArrayOfDoublesCompactSketch
|
||||||
|
* representing the current state of the aggregation, and is not affected by consequent
|
||||||
|
* aggregate() calls
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public synchronized Object get()
|
||||||
|
{
|
||||||
|
return union.getResult();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
union = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,149 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.google.common.util.concurrent.Striped;
|
||||||
|
import com.yahoo.memory.WritableMemory;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSetOperationBuilder;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketches;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUnion;
|
||||||
|
import io.druid.query.aggregation.BufferAggregator;
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import io.druid.segment.BaseObjectColumnValueSelector;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.concurrent.locks.Lock;
|
||||||
|
import java.util.concurrent.locks.ReadWriteLock;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This aggregator merges existing sketches.
|
||||||
|
* The input column contains ArrayOfDoublesSketch.
|
||||||
|
* The output is {@link ArrayOfDoublesSketch} that is a union of the input sketches.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchMergeBufferAggregator implements BufferAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private static final int NUM_STRIPES = 64; // for locking per buffer position
|
||||||
|
|
||||||
|
private final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector;
|
||||||
|
private final int nominalEntries;
|
||||||
|
private final int numberOfValues;
|
||||||
|
private final int maxIntermediateSize;
|
||||||
|
private final Striped<ReadWriteLock> stripedLock = Striped.readWriteLock(NUM_STRIPES);
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchMergeBufferAggregator(
|
||||||
|
final BaseObjectColumnValueSelector<ArrayOfDoublesSketch> selector,
|
||||||
|
final int nominalEntries,
|
||||||
|
final int numberOfValues,
|
||||||
|
final int maxIntermediateSize
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.selector = selector;
|
||||||
|
this.nominalEntries = nominalEntries;
|
||||||
|
this.numberOfValues = numberOfValues;
|
||||||
|
this.maxIntermediateSize = maxIntermediateSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
new ArrayOfDoublesSetOperationBuilder().setNominalEntries(nominalEntries)
|
||||||
|
.setNumberOfValues(numberOfValues).buildUnion(region);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses locks because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void aggregate(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch update = selector.getObject();
|
||||||
|
if (update == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
// Wrapping memory and ArrayOfDoublesUnion is inexpensive compared to union operations.
|
||||||
|
// Maintaining a cache of wrapped objects per buffer position like in Theta sketch aggregator
|
||||||
|
// might might be considered, but it would increase complexity including relocate() support.
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
final Lock lock = stripedLock.getAt(ArrayOfDoublesSketchBuildBufferAggregator.lockIndex(position)).writeLock();
|
||||||
|
lock.lock();
|
||||||
|
try {
|
||||||
|
final ArrayOfDoublesUnion union = ArrayOfDoublesSketches.wrapUnion(region);
|
||||||
|
union.update(update);
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method uses locks because it can be used during indexing,
|
||||||
|
* and Druid can call aggregate() and get() concurrently
|
||||||
|
* https://github.com/druid-io/druid/pull/3956
|
||||||
|
* The returned sketch is a separate instance of ArrayOfDoublesCompactSketch
|
||||||
|
* representing the current state of the aggregation, and is not affected by consequent
|
||||||
|
* aggregate() calls
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Object get(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
final WritableMemory mem = WritableMemory.wrap(buf);
|
||||||
|
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
|
||||||
|
final Lock lock = stripedLock.getAt(ArrayOfDoublesSketchBuildBufferAggregator.lockIndex(position)).readLock();
|
||||||
|
lock.lock();
|
||||||
|
try {
|
||||||
|
final ArrayOfDoublesUnion union = ArrayOfDoublesSketches.wrapUnion(region);
|
||||||
|
return union.getResult();
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
lock.unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("selector", selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,89 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import io.druid.data.input.InputRow;
|
||||||
|
import io.druid.segment.GenericColumnSerializer;
|
||||||
|
import io.druid.segment.column.ColumnBuilder;
|
||||||
|
import io.druid.segment.data.GenericIndexed;
|
||||||
|
import io.druid.segment.data.ObjectStrategy;
|
||||||
|
import io.druid.segment.serde.ComplexColumnPartSupplier;
|
||||||
|
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||||
|
import io.druid.segment.serde.ComplexMetricSerde;
|
||||||
|
import io.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
|
||||||
|
import io.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchMergeComplexMetricSerde extends ComplexMetricSerde
|
||||||
|
{
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTypeName()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketchModule.ARRAY_OF_DOUBLES_SKETCH;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ComplexMetricExtractor getExtractor()
|
||||||
|
{
|
||||||
|
return new ComplexMetricExtractor()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public Class<?> extractedClass()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketch.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object extractValue(final InputRow inputRow, final String metricName)
|
||||||
|
{
|
||||||
|
final Object object = inputRow.getRaw(metricName);
|
||||||
|
if (object == null || object instanceof ArrayOfDoublesSketch) {
|
||||||
|
return object;
|
||||||
|
}
|
||||||
|
return ArrayOfDoublesSketchOperations.deserialize(object);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void deserializeColumn(final ByteBuffer buffer, final ColumnBuilder builder)
|
||||||
|
{
|
||||||
|
final GenericIndexed<ArrayOfDoublesSketch> ge = GenericIndexed.read(buffer, ArrayOfDoublesSketchObjectStrategy.STRATEGY);
|
||||||
|
builder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), ge));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ObjectStrategy<ArrayOfDoublesSketch> getObjectStrategy()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketchObjectStrategy.STRATEGY;
|
||||||
|
}
|
||||||
|
|
||||||
|
// support large columns
|
||||||
|
@Override
|
||||||
|
public GenericColumnSerializer getSerializer(final SegmentWriteOutMedium segmentWriteOutMedium, final String column)
|
||||||
|
{
|
||||||
|
return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,120 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.Module;
|
||||||
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
|
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||||
|
import com.google.inject.Binder;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.initialization.DruidModule;
|
||||||
|
import io.druid.segment.serde.ComplexMetrics;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This module is to support numeric Tuple sketches, which extend the functionality of the count-distinct
|
||||||
|
* Theta sketches by adding arrays of double values associated with unique keys.
|
||||||
|
*
|
||||||
|
* See <a href=https://datasketches.github.io/docs/Tuple/TupleOverview.html>Tuple Sketch Overview</a>
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchModule implements DruidModule
|
||||||
|
{
|
||||||
|
|
||||||
|
public static final String ARRAY_OF_DOUBLES_SKETCH = "arrayOfDoublesSketch";
|
||||||
|
|
||||||
|
public static final String ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG = "arrayOfDoublesSketchMerge";
|
||||||
|
public static final String ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG = "arrayOfDoublesSketchBuild";
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void configure(final Binder binder)
|
||||||
|
{
|
||||||
|
if (ComplexMetrics.getSerdeForType(ARRAY_OF_DOUBLES_SKETCH) == null) {
|
||||||
|
ComplexMetrics.registerSerde(ARRAY_OF_DOUBLES_SKETCH, new ArrayOfDoublesSketchMergeComplexMetricSerde());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ComplexMetrics.getSerdeForType(ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG) == null) {
|
||||||
|
ComplexMetrics.registerSerde(
|
||||||
|
ARRAY_OF_DOUBLES_SKETCH_MERGE_AGG,
|
||||||
|
new ArrayOfDoublesSketchMergeComplexMetricSerde()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ComplexMetrics.getSerdeForType(ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG) == null) {
|
||||||
|
ComplexMetrics.registerSerde(
|
||||||
|
ARRAY_OF_DOUBLES_SKETCH_BUILD_AGG,
|
||||||
|
new ArrayOfDoublesSketchBuildComplexMetricSerde()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<? extends Module> getJacksonModules()
|
||||||
|
{
|
||||||
|
return Arrays.<Module>asList(
|
||||||
|
new SimpleModule("ArrayOfDoublesSketchModule").registerSubtypes(
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchAggregatorFactory.class,
|
||||||
|
ARRAY_OF_DOUBLES_SKETCH
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToEstimatePostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToEstimate"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToEstimateAndBounds"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToNumEntriesPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToNumEntries"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToMeansPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToMeans"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToVariancesPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToVariances"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToQuantilesSketchPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToQuantilesSketch"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchSetOpPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchSetOp"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchTTestPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchTTest"
|
||||||
|
),
|
||||||
|
new NamedType(
|
||||||
|
ArrayOfDoublesSketchToStringPostAggregator.class,
|
||||||
|
"arrayOfDoublesSketchToString"
|
||||||
|
)
|
||||||
|
).addSerializer(ArrayOfDoublesSketch.class, new ArrayOfDoublesSketchJsonSerializer())
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,96 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for post aggs taking multiple sketches as input
|
||||||
|
*/
|
||||||
|
public abstract class ArrayOfDoublesSketchMultiPostAggregator extends ArrayOfDoublesSketchPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final List<PostAggregator> fields;
|
||||||
|
private Set<String> dependentFields;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchMultiPostAggregator(final String name, final List<PostAggregator> fields)
|
||||||
|
{
|
||||||
|
super(name);
|
||||||
|
this.fields = fields;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getDependentFields()
|
||||||
|
{
|
||||||
|
if (dependentFields == null) {
|
||||||
|
dependentFields = Sets.newHashSet(super.getDependentFields());
|
||||||
|
for (final PostAggregator field : fields) {
|
||||||
|
dependentFields.addAll(field.getDependentFields());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return dependentFields;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public List<PostAggregator> getFields()
|
||||||
|
{
|
||||||
|
return fields;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{"
|
||||||
|
+ "name='" + getName() + '\''
|
||||||
|
+ ", fields=" + fields
|
||||||
|
+ "}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (!super.equals(o)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
// this check is used here instead of instanceof because this is an abstract class
|
||||||
|
// and subclasses not overriding equals should not be equal to each other
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchMultiPostAggregator that = (ArrayOfDoublesSketchMultiPostAggregator) o;
|
||||||
|
return fields.equals(that.fields);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), fields);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.Aggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchNoOpAggregator implements Aggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final ArrayOfDoublesSketch emptySketch;
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchNoOpAggregator(final int numberOfValues)
|
||||||
|
{
|
||||||
|
emptySketch = new ArrayOfDoublesUpdatableSketchBuilder().setNumberOfValues(numberOfValues).build().compact();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void aggregate()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get()
|
||||||
|
{
|
||||||
|
return emptySketch;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong()
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,78 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.BufferAggregator;
|
||||||
|
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchNoOpBufferAggregator implements BufferAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final ArrayOfDoublesSketch emptySketch;
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchNoOpBufferAggregator(final int numberOfValues)
|
||||||
|
{
|
||||||
|
emptySketch = new ArrayOfDoublesUpdatableSketchBuilder().setNumberOfValues(numberOfValues).build().compact();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void init(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void aggregate(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object get(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
return emptySketch;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float getFloat(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLong(final ByteBuffer buf, final int position)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("Not implemented");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,65 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import io.druid.segment.data.ObjectStrategy;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import com.yahoo.memory.Memory;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketches;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchObjectStrategy implements ObjectStrategy<ArrayOfDoublesSketch>
|
||||||
|
{
|
||||||
|
|
||||||
|
static final ArrayOfDoublesSketchObjectStrategy STRATEGY = new ArrayOfDoublesSketchObjectStrategy();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int compare(final ArrayOfDoublesSketch s1, final ArrayOfDoublesSketch s2)
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketchAggregatorFactory.COMPARATOR.compare(s1, s2);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<? extends ArrayOfDoublesSketch> getClazz()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketch.class;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ArrayOfDoublesSketch fromByteBuffer(final ByteBuffer buffer, final int numBytes)
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketches.wrapSketch(Memory.wrap(buffer).region(buffer.position(), numBytes));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Nullable
|
||||||
|
public byte[] toBytes(@Nullable final ArrayOfDoublesSketch sketch)
|
||||||
|
{
|
||||||
|
if (sketch == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return sketch.toByteArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,133 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.yahoo.memory.Memory;
|
||||||
|
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
|
||||||
|
import org.apache.commons.codec.binary.Base64;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesAnotB;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesCombiner;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesIntersection;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketches;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUnion;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.java.util.common.ISE;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSetOperationBuilder;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchOperations
|
||||||
|
{
|
||||||
|
|
||||||
|
public enum Operation
|
||||||
|
{
|
||||||
|
UNION {
|
||||||
|
@Override
|
||||||
|
public ArrayOfDoublesSketch apply(final int nominalEntries, final int numberOfValues, final ArrayOfDoublesSketch[] sketches)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesUnion union = new ArrayOfDoublesSetOperationBuilder().setNominalEntries(nominalEntries)
|
||||||
|
.setNumberOfValues(numberOfValues).buildUnion();
|
||||||
|
for (final ArrayOfDoublesSketch sketch : sketches) {
|
||||||
|
union.update(sketch);
|
||||||
|
}
|
||||||
|
return union.getResult();
|
||||||
|
}
|
||||||
|
},
|
||||||
|
INTERSECT {
|
||||||
|
@Override
|
||||||
|
public ArrayOfDoublesSketch apply(final int nominalEntries, final int numberOfValues, final ArrayOfDoublesSketch[] sketches)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesIntersection intersection = new ArrayOfDoublesSetOperationBuilder()
|
||||||
|
.setNominalEntries(nominalEntries).setNumberOfValues(numberOfValues).buildIntersection();
|
||||||
|
for (final ArrayOfDoublesSketch sketch : sketches) {
|
||||||
|
intersection.update(sketch, COMBINER);
|
||||||
|
}
|
||||||
|
return intersection.getResult();
|
||||||
|
}
|
||||||
|
},
|
||||||
|
NOT {
|
||||||
|
@Override
|
||||||
|
public ArrayOfDoublesSketch apply(final int nominalEntries, final int numberOfValues, final ArrayOfDoublesSketch[] sketches)
|
||||||
|
{
|
||||||
|
if (sketches.length < 1) {
|
||||||
|
throw new IAE("A-Not-B requires at least 1 sketch");
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sketches.length == 1) {
|
||||||
|
return sketches[0];
|
||||||
|
}
|
||||||
|
|
||||||
|
ArrayOfDoublesSketch result = sketches[0];
|
||||||
|
for (int i = 1; i < sketches.length; i++) {
|
||||||
|
final ArrayOfDoublesAnotB aNotB = new ArrayOfDoublesSetOperationBuilder().setNumberOfValues(numberOfValues)
|
||||||
|
.buildAnotB();
|
||||||
|
aNotB.update(result, sketches[i]);
|
||||||
|
result = aNotB.getResult();
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public abstract ArrayOfDoublesSketch apply(int nominalEntries, int numberOfValues, ArrayOfDoublesSketch[] sketches);
|
||||||
|
}
|
||||||
|
|
||||||
|
// This is how to combine values for sketch intersection.
|
||||||
|
// Might not fit all use cases.
|
||||||
|
private static ArrayOfDoublesCombiner COMBINER = new ArrayOfDoublesCombiner()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public double[] combine(final double[] a, final double[] b)
|
||||||
|
{
|
||||||
|
final double[] result = new double[a.length];
|
||||||
|
for (int i = 0; i < a.length; i++) {
|
||||||
|
result[i] = a[i] + b[i];
|
||||||
|
}
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
public static ArrayOfDoublesSketch deserialize(final Object serializedSketch)
|
||||||
|
{
|
||||||
|
if (serializedSketch instanceof String) {
|
||||||
|
return deserializeFromBase64EncodedString((String) serializedSketch);
|
||||||
|
} else if (serializedSketch instanceof byte[]) {
|
||||||
|
return deserializeFromByteArray((byte[]) serializedSketch);
|
||||||
|
} else if (serializedSketch instanceof ArrayOfDoublesSketch) {
|
||||||
|
return (ArrayOfDoublesSketch) serializedSketch;
|
||||||
|
}
|
||||||
|
throw new ISE("Object is not of a type that can deserialize to sketch: %s", serializedSketch.getClass());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ArrayOfDoublesSketch deserializeFromBase64EncodedString(final String str)
|
||||||
|
{
|
||||||
|
return deserializeFromByteArray(Base64.decodeBase64(str.getBytes(StandardCharsets.UTF_8)));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static ArrayOfDoublesSketch deserializeFromByteArray(final byte[] data)
|
||||||
|
{
|
||||||
|
final Memory mem = Memory.wrap(data);
|
||||||
|
return ArrayOfDoublesSketches.wrapSketch(mem);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,83 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.AggregatorFactory;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for all post aggs
|
||||||
|
*/
|
||||||
|
public abstract class ArrayOfDoublesSketchPostAggregator implements PostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final String name;
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchPostAggregator(final String name)
|
||||||
|
{
|
||||||
|
this.name = Preconditions.checkNotNull(name, "name is null");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@JsonProperty
|
||||||
|
public String getName()
|
||||||
|
{
|
||||||
|
return name;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getDependentFields()
|
||||||
|
{
|
||||||
|
return Collections.emptySet();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (!(o instanceof ArrayOfDoublesSketchPostAggregator)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchPostAggregator that = (ArrayOfDoublesSketchPostAggregator) o;
|
||||||
|
return name.equals(that.getName());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return name.hashCode();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PostAggregator decorate(final Map<String, AggregatorFactory> map)
|
||||||
|
{
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,152 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.Util;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a result of a specified set operation on the given array of sketches. Supported operations are:
|
||||||
|
* union, intersection and set difference (UNION, INTERSECT, NOT).
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchSetOpPostAggregator extends ArrayOfDoublesSketchMultiPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final ArrayOfDoublesSketchOperations.Operation operation;
|
||||||
|
private final int nominalEntries;
|
||||||
|
private final int numberOfValues;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("operation") final String operation,
|
||||||
|
@JsonProperty("nominalEntries") @Nullable final Integer nominalEntries,
|
||||||
|
@JsonProperty("numberOfValues") @Nullable final Integer numberOfValues,
|
||||||
|
@JsonProperty("fields") List<PostAggregator> fields
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, fields);
|
||||||
|
this.operation = ArrayOfDoublesSketchOperations.Operation.valueOf(operation);
|
||||||
|
this.nominalEntries = nominalEntries == null ? Util.DEFAULT_NOMINAL_ENTRIES : nominalEntries;
|
||||||
|
this.numberOfValues = numberOfValues == null ? 1 : numberOfValues;
|
||||||
|
Util.checkIfPowerOf2(this.nominalEntries, "size");
|
||||||
|
|
||||||
|
if (fields.size() <= 1) {
|
||||||
|
throw new IAE("Illegal number of fields[%d], must be > 1", fields.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<ArrayOfDoublesSketch> getComparator()
|
||||||
|
{
|
||||||
|
return ArrayOfDoublesSketchAggregatorFactory.COMPARATOR;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ArrayOfDoublesSketch compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch[] sketches = new ArrayOfDoublesSketch[getFields().size()];
|
||||||
|
for (int i = 0; i < sketches.length; i++) {
|
||||||
|
sketches[i] = (ArrayOfDoublesSketch) getFields().get(i).compute(combinedAggregators);
|
||||||
|
}
|
||||||
|
return operation.apply(nominalEntries, numberOfValues, sketches);
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public String getOperation()
|
||||||
|
{
|
||||||
|
return operation.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNominalEntries()
|
||||||
|
{
|
||||||
|
return nominalEntries;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNumberOfValues()
|
||||||
|
{
|
||||||
|
return numberOfValues;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{"
|
||||||
|
+ "name='" + getName() + '\''
|
||||||
|
+ ", fields=" + getFields()
|
||||||
|
+ ", operation=" + operation
|
||||||
|
+ ", nominalEntries=" + nominalEntries
|
||||||
|
+ ", numberOfValues=" + numberOfValues
|
||||||
|
+ "}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (!super.equals(o)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!(o instanceof ArrayOfDoublesSketchSetOpPostAggregator)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchSetOpPostAggregator that = (ArrayOfDoublesSketchSetOpPostAggregator) o;
|
||||||
|
if (nominalEntries != that.nominalEntries) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (numberOfValues != that.numberOfValues) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return operation.equals(that.operation);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), operation, nominalEntries, numberOfValues);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_SET_OP_CACHE_TYPE_ID)
|
||||||
|
.appendCacheables(getFields())
|
||||||
|
.appendInt(nominalEntries)
|
||||||
|
.appendInt(numberOfValues)
|
||||||
|
.appendString(operation.toString())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,113 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
|
||||||
|
import org.apache.commons.math3.stat.inference.TTest;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketchIterator;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Performs Student's t-test and returns a list of p-values given two instances of {@link ArrayOfDoublesSketch}.
|
||||||
|
* The result will be N double values, where N is the number of double values kept in the sketch per key.
|
||||||
|
* See <a href=http://commons.apache.org/proper/commons-math/javadocs/api-3.4/org/apache/commons/math3/stat/inference/TTest.html>Student's t-test</a>
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchTTestPostAggregator extends ArrayOfDoublesSketchMultiPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("fields") List<PostAggregator> fields
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, fields);
|
||||||
|
if (fields.size() != 2) {
|
||||||
|
throw new IAE("Illegal number of fields[%d], must be 2", fields.size());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<double[]> getComparator()
|
||||||
|
{
|
||||||
|
throw new IAE("Comparing arrays of p values is not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double[] compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch1 = (ArrayOfDoublesSketch) getFields().get(0).compute(combinedAggregators);
|
||||||
|
final ArrayOfDoublesSketch sketch2 = (ArrayOfDoublesSketch) getFields().get(1).compute(combinedAggregators);
|
||||||
|
if (sketch1.getNumValues() != sketch2.getNumValues()) {
|
||||||
|
throw new IAE(
|
||||||
|
"Sketches have different number of values: %d and %d",
|
||||||
|
sketch1.getNumValues(),
|
||||||
|
sketch2.getNumValues()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
final SummaryStatistics[] stats1 = getStats(sketch1);
|
||||||
|
final SummaryStatistics[] stats2 = getStats(sketch2);
|
||||||
|
|
||||||
|
final int numberOfValues = sketch1.getNumValues();
|
||||||
|
final double[] pValues = new double[numberOfValues];
|
||||||
|
final TTest test = new TTest();
|
||||||
|
for (int i = 0; i < pValues.length; i++) {
|
||||||
|
pValues[i] = test.tTest(stats1[i], stats2[i]);
|
||||||
|
}
|
||||||
|
return pValues;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static SummaryStatistics[] getStats(final ArrayOfDoublesSketch sketch)
|
||||||
|
{
|
||||||
|
final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
|
||||||
|
Arrays.setAll(stats, i -> new SummaryStatistics());
|
||||||
|
final ArrayOfDoublesSketchIterator it = sketch.iterator();
|
||||||
|
while (it.next()) {
|
||||||
|
final double[] values = it.getValues();
|
||||||
|
for (int i = 0; i < values.length; i++) {
|
||||||
|
stats[i].addValue(values[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return stats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_T_TEST_CACHE_TYPE_ID)
|
||||||
|
.appendCacheables(getFields())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,120 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a distinct count estimate and error bounds from a given {@link ArrayOfDoublesSketch}."
|
||||||
|
* The result will be three double values: estimate of the number of distinct keys, lower bound and upper bound.
|
||||||
|
* The bounds are provided at the given number of standard deviations (optional, defaults to 1).
|
||||||
|
* This must be an integer value of 1, 2 or 3 corresponding to approximately 68.3%, 95.4% and 99.7%
|
||||||
|
* confidence intervals.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final int numStdDevs;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field,
|
||||||
|
@JsonProperty("numStdDevs") @Nullable final Integer numStdDevs
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
this.numStdDevs = numStdDevs == null ? 1 : numStdDevs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getNumStdDevs()
|
||||||
|
{
|
||||||
|
return numStdDevs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double[] compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
return new double[] {sketch.getEstimate(), sketch.getLowerBound(numStdDevs), sketch.getUpperBound(numStdDevs)};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<double[]> getComparator()
|
||||||
|
{
|
||||||
|
throw new IAE("Comparing arrays of estimates and error bounds is not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{" +
|
||||||
|
"name='" + getName() + '\'' +
|
||||||
|
", field=" + getField() +
|
||||||
|
", numStdDevs=" + numStdDevs +
|
||||||
|
"}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (!super.equals(o)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!(o instanceof ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator that = (ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator) o;
|
||||||
|
if (numStdDevs != that.numStdDevs) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), numStdDevs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_ESTIMATE_AND_BOUNDS_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.appendInt(numStdDevs)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a distinct count estimate from a given {@link ArrayOfDoublesSketch}.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToEstimatePostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Double compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
return sketch.getEstimate();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<Double> getComparator()
|
||||||
|
{
|
||||||
|
return Comparator.naturalOrder();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_ESTIMATE_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,86 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketchIterator;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a list of mean values from a given {@link ArrayOfDoublesSketch}.
|
||||||
|
* The result will be N double values, where N is the number of double values kept in the sketch per key.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToMeansPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToMeansPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double[] compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
|
||||||
|
Arrays.setAll(stats, i -> new SummaryStatistics());
|
||||||
|
final ArrayOfDoublesSketchIterator it = sketch.iterator();
|
||||||
|
while (it.next()) {
|
||||||
|
final double[] values = it.getValues();
|
||||||
|
for (int i = 0; i < values.length; i++) {
|
||||||
|
stats[i].addValue(values[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final double[] means = new double[sketch.getNumValues()];
|
||||||
|
Arrays.setAll(means, i -> stats[i].getMean());
|
||||||
|
return means;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<double[]> getComparator()
|
||||||
|
{
|
||||||
|
throw new IAE("Comparing arrays of mean values is not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_MEANS_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,69 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the number of retained entries from a given {@link ArrayOfDoublesSketch}.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToNumEntriesPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Integer compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
return sketch.getRetainedEntries();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<Integer> getComparator()
|
||||||
|
{
|
||||||
|
return Comparator.naturalOrder();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_NUM_ENTRIES_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,146 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.datasketches.quantiles.DoublesSketchAggregatorFactory;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketchIterator;
|
||||||
|
import com.yahoo.sketches.quantiles.DoublesSketch;
|
||||||
|
import com.yahoo.sketches.quantiles.UpdateDoublesSketch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a quanitles {@link DoublesSketch} constructed from a given column of double values from a given
|
||||||
|
* {@link ArrayOfDoublesSketch} using parameter k that determines the accuracy and size of the quantiles sketch.
|
||||||
|
* The column number is optional (the default is 1).
|
||||||
|
* The parameter k is optional (the default is defined in the sketch library).
|
||||||
|
* The result is a quantiles sketch.
|
||||||
|
* See <a href=https://datasketches.github.io/docs/Quantiles/QuantilesOverview.html>Quantiles Sketch Overview</a>
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToQuantilesSketchPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private static final int DEFAULT_QUANTILES_SKETCH_SIZE = 128;
|
||||||
|
|
||||||
|
private final int column;
|
||||||
|
private final int k;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToQuantilesSketchPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field,
|
||||||
|
@JsonProperty("column") @Nullable final Integer column,
|
||||||
|
@JsonProperty("k") @Nullable final Integer k
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
this.column = column == null ? 1 : column;
|
||||||
|
this.k = k == null ? DEFAULT_QUANTILES_SKETCH_SIZE : k;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<DoublesSketch> getComparator()
|
||||||
|
{
|
||||||
|
return DoublesSketchAggregatorFactory.COMPARATOR;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DoublesSketch compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
final UpdateDoublesSketch qs = UpdateDoublesSketch.builder().setK(k).build();
|
||||||
|
final ArrayOfDoublesSketchIterator it = sketch.iterator();
|
||||||
|
while (it.next()) {
|
||||||
|
qs.update(it.getValues()[column - 1]); // convert 1-based column number to zero-based index
|
||||||
|
}
|
||||||
|
return qs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getColumn()
|
||||||
|
{
|
||||||
|
return column;
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public int getK()
|
||||||
|
{
|
||||||
|
return k;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{" +
|
||||||
|
"name='" + getName() + '\'' +
|
||||||
|
", field=" + getField() +
|
||||||
|
", column=" + column +
|
||||||
|
", k=" + k +
|
||||||
|
"}";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (!super.equals(o)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (!(o instanceof ArrayOfDoublesSketchToQuantilesSketchPostAggregator)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchToQuantilesSketchPostAggregator that = (ArrayOfDoublesSketchToQuantilesSketchPostAggregator) o;
|
||||||
|
if (column != that.column) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (k != that.k) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), column, k);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_QUANTILES_SKETCH_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.appendInt(column)
|
||||||
|
.appendInt(k)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a human-readable summary of a given {@link ArrayOfDoublesSketch}.
|
||||||
|
* This is a string returned by toString() method of the sketch.
|
||||||
|
* This can be useful for debugging.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToStringPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
return sketch.toString();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<String> getComparator()
|
||||||
|
{
|
||||||
|
throw new IAE("Comparing sketch summaries is not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,86 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
import org.apache.commons.math3.stat.descriptive.SummaryStatistics;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesSketchIterator;
|
||||||
|
|
||||||
|
import io.druid.java.util.common.IAE;
|
||||||
|
import io.druid.query.aggregation.AggregatorUtil;
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.cache.CacheKeyBuilder;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a list of variance values from a given {@link ArrayOfDoublesSketch}.
|
||||||
|
* The result will be N double values, where N is the number of double values kept in the sketch per key.
|
||||||
|
*/
|
||||||
|
public class ArrayOfDoublesSketchToVariancesPostAggregator extends ArrayOfDoublesSketchUnaryPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchToVariancesPostAggregator(
|
||||||
|
@JsonProperty("name") final String name,
|
||||||
|
@JsonProperty("field") final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name, field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double[] compute(final Map<String, Object> combinedAggregators)
|
||||||
|
{
|
||||||
|
final ArrayOfDoublesSketch sketch = (ArrayOfDoublesSketch) getField().compute(combinedAggregators);
|
||||||
|
final SummaryStatistics[] stats = new SummaryStatistics[sketch.getNumValues()];
|
||||||
|
Arrays.setAll(stats, i -> new SummaryStatistics());
|
||||||
|
final ArrayOfDoublesSketchIterator it = sketch.iterator();
|
||||||
|
while (it.next()) {
|
||||||
|
final double[] values = it.getValues();
|
||||||
|
for (int i = 0; i < values.length; i++) {
|
||||||
|
stats[i].addValue(values[i]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final double[] variances = new double[sketch.getNumValues()];
|
||||||
|
Arrays.setAll(variances, i -> stats[i].getVariance());
|
||||||
|
return variances;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Comparator<double[]> getComparator()
|
||||||
|
{
|
||||||
|
throw new IAE("Comparing arrays of variance values is not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new CacheKeyBuilder(AggregatorUtil.ARRAY_OF_DOUBLES_SKETCH_TO_VARIANCES_CACHE_TYPE_ID)
|
||||||
|
.appendCacheable(getField())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,97 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.Sets;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Base class for post aggs taking one sketch as input
|
||||||
|
*/
|
||||||
|
public abstract class ArrayOfDoublesSketchUnaryPostAggregator extends ArrayOfDoublesSketchPostAggregator
|
||||||
|
{
|
||||||
|
|
||||||
|
private final PostAggregator field;
|
||||||
|
private Set<String> dependentFields;
|
||||||
|
|
||||||
|
@JsonCreator
|
||||||
|
public ArrayOfDoublesSketchUnaryPostAggregator(
|
||||||
|
final String name,
|
||||||
|
final PostAggregator field
|
||||||
|
)
|
||||||
|
{
|
||||||
|
super(name);
|
||||||
|
this.field = Preconditions.checkNotNull(field, "field is null");
|
||||||
|
}
|
||||||
|
|
||||||
|
@JsonProperty
|
||||||
|
public PostAggregator getField()
|
||||||
|
{
|
||||||
|
return field;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<String> getDependentFields()
|
||||||
|
{
|
||||||
|
if (dependentFields == null) {
|
||||||
|
dependentFields = Sets.newHashSet(super.getDependentFields());
|
||||||
|
dependentFields.addAll(field.getDependentFields());
|
||||||
|
}
|
||||||
|
return dependentFields;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(final Object o)
|
||||||
|
{
|
||||||
|
if (!super.equals(o)) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
// this check is used here instead of instanceof because this is an abstract class
|
||||||
|
// and subclasses not overriding equals should not be equal to each other
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
final ArrayOfDoublesSketchUnaryPostAggregator that = (ArrayOfDoublesSketchUnaryPostAggregator) o;
|
||||||
|
return field.equals(that.getField());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return Objects.hash(super.hashCode(), field);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return this.getClass().getSimpleName() + "{" +
|
||||||
|
"name='" + getName() + '\'' +
|
||||||
|
", field=" + getField() +
|
||||||
|
"}";
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -1,3 +1,4 @@
|
||||||
io.druid.query.aggregation.datasketches.theta.SketchModule
|
io.druid.query.aggregation.datasketches.theta.SketchModule
|
||||||
io.druid.query.aggregation.datasketches.theta.oldapi.OldApiSketchModule
|
io.druid.query.aggregation.datasketches.theta.oldapi.OldApiSketchModule
|
||||||
io.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule
|
io.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule
|
||||||
|
io.druid.query.aggregation.datasketches.tuple.ArrayOfDoublesSketchModule
|
||||||
|
|
|
@ -0,0 +1,565 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import com.yahoo.sketches.quantiles.DoublesSketch;
|
||||||
|
import io.druid.data.input.Row;
|
||||||
|
import io.druid.initialization.DruidModule;
|
||||||
|
import io.druid.java.util.common.granularity.Granularities;
|
||||||
|
import io.druid.java.util.common.guava.Sequence;
|
||||||
|
import io.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
import io.druid.query.groupby.GroupByQueryConfig;
|
||||||
|
import io.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.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
@RunWith(Parameterized.class)
|
||||||
|
public class ArrayOfDoublesSketchAggregationTest
|
||||||
|
{
|
||||||
|
|
||||||
|
private final AggregationTestHelper helper;
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder tempFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
public ArrayOfDoublesSketchAggregationTest(final GroupByQueryConfig config)
|
||||||
|
{
|
||||||
|
DruidModule module = new ArrayOfDoublesSketchModule();
|
||||||
|
module.configure(null);
|
||||||
|
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||||
|
module.getJacksonModules(), config, tempFolder);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Parameterized.Parameters(name = "{0}")
|
||||||
|
public static Collection<?> constructorFeeder()
|
||||||
|
{
|
||||||
|
final List<Object[]> constructors = Lists.newArrayList();
|
||||||
|
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||||
|
constructors.add(new Object[] {config});
|
||||||
|
}
|
||||||
|
return constructors;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void ingestingSketches() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(this.getClass().getClassLoader().getResource("tuple/array_of_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\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"non_existing_sketch\", \"fieldName\": \"non_existing_sketch\"}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
10, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"non_existing_sketch\", \"fieldName\": \"non_existing_sketch\"}",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"estimate\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToQuantilesSketch\", \"name\": \"quantiles-sketch\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"union\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"union\",",
|
||||||
|
" \"operation\": \"UNION\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"intersection\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"intersection\",",
|
||||||
|
" \"operation\": \"INTERSECT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"anotb\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"anotb\",",
|
||||||
|
" \"operation\": \"NOT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }}",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Assert.assertEquals(0, (double) row.getMetric("non_existing_sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
|
||||||
|
Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
|
||||||
|
|
||||||
|
Object obj = row.getRaw("quantiles-sketch");
|
||||||
|
Assert.assertTrue(obj instanceof DoublesSketch);
|
||||||
|
DoublesSketch ds = (DoublesSketch) obj;
|
||||||
|
Assert.assertEquals(40, ds.getN());
|
||||||
|
Assert.assertEquals(1.0, ds.getMinValue(), 0);
|
||||||
|
Assert.assertEquals(1.0, ds.getMaxValue(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void ingestingSketchesTwoValues() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_sketch_data_two_values.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\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
10, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"estimate\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToQuantilesSketch\", \"name\": \"quantiles-sketch\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"union\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"union\",",
|
||||||
|
" \"operation\": \"UNION\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"intersection\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"intersection\",",
|
||||||
|
" \"operation\": \"INTERSECT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"anotb\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"anotb\",",
|
||||||
|
" \"operation\": \"NOT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchToMeans\",",
|
||||||
|
" \"name\": \"means\",",
|
||||||
|
" \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}",
|
||||||
|
" }",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
|
||||||
|
Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
|
||||||
|
|
||||||
|
Object meansObj = row.getRaw("means");
|
||||||
|
Assert.assertTrue(meansObj instanceof double[]);
|
||||||
|
double[] means = (double[]) meansObj;
|
||||||
|
Assert.assertEquals(2, means.length);
|
||||||
|
Assert.assertEquals(1.0, means[0], 0);
|
||||||
|
Assert.assertEquals(2.0, means[1], 0);
|
||||||
|
|
||||||
|
Object quantilesObj = row.getRaw("quantiles-sketch");
|
||||||
|
Assert.assertTrue(quantilesObj instanceof DoublesSketch);
|
||||||
|
DoublesSketch ds = (DoublesSketch) quantilesObj;
|
||||||
|
Assert.assertEquals(40, ds.getN());
|
||||||
|
Assert.assertEquals(1.0, ds.getMinValue(), 0);
|
||||||
|
Assert.assertEquals(1.0, ds.getMaxValue(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void buildingSketchesAtIngestionTime() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data.tsv").getFile()),
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"type\": \"string\",",
|
||||||
|
" \"parseSpec\": {",
|
||||||
|
" \"format\": \"tsv\",",
|
||||||
|
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||||
|
" \"dimensionsSpec\": {",
|
||||||
|
" \"dimensions\": [\"product\"],",
|
||||||
|
" \"dimensionExclusions\": [],",
|
||||||
|
" \"spatialDimensions\": []",
|
||||||
|
" },",
|
||||||
|
" \"columns\": [\"timestamp\", \"product\", \"key\", \"value\"]",
|
||||||
|
" }",
|
||||||
|
"}"),
|
||||||
|
String.join("\n",
|
||||||
|
"[",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"key\", \"metricColumns\": [\"value\"], \"nominalEntries\": 1024}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
10, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"size\": 1024}",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"estimate\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToQuantilesSketch\", \"name\": \"quantiles-sketch\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"union\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"union\",",
|
||||||
|
" \"operation\": \"UNION\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"intersection\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"intersection\",",
|
||||||
|
" \"operation\": \"INTERSECT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"anotb\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"anotb\",",
|
||||||
|
" \"operation\": \"NOT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }}",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
|
||||||
|
Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
|
||||||
|
|
||||||
|
Object obj = row.getRaw("quantiles-sketch");
|
||||||
|
Assert.assertTrue(obj instanceof DoublesSketch);
|
||||||
|
DoublesSketch ds = (DoublesSketch) obj;
|
||||||
|
Assert.assertEquals(40, ds.getN());
|
||||||
|
Assert.assertEquals(1.0, ds.getMinValue(), 0);
|
||||||
|
Assert.assertEquals(1.0, ds.getMaxValue(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void buildingSketchesAtIngestionTimeTwoValues() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(
|
||||||
|
this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data_two_values.tsv").getFile()),
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"type\": \"string\",",
|
||||||
|
" \"parseSpec\": {",
|
||||||
|
" \"format\": \"tsv\",",
|
||||||
|
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||||
|
" \"dimensionsSpec\": {",
|
||||||
|
" \"dimensions\": [\"product\"],",
|
||||||
|
" \"dimensionExclusions\": [],",
|
||||||
|
" \"spatialDimensions\": []",
|
||||||
|
" },",
|
||||||
|
" \"columns\": [\"timestamp\", \"product\", \"key\", \"value1\", \"value2\"]",
|
||||||
|
" }",
|
||||||
|
"}"),
|
||||||
|
String.join("\n",
|
||||||
|
"[",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"key\", \"metricColumns\": [ \"value1\", \"value2\" ], \"nominalEntries\": 1024}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
10, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"nominalEntries\": 1024, \"numberOfValues\": 2}",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"estimate\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToQuantilesSketch\", \"name\": \"quantiles-sketch\", \"column\": 2, \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"union\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"union\",",
|
||||||
|
" \"operation\": \"UNION\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"numberOfValues\": 2,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"intersection\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"intersection\",",
|
||||||
|
" \"operation\": \"INTERSECT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"numberOfValues\": 2,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"anotb\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"anotb\",",
|
||||||
|
" \"operation\": \"NOT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"numberOfValues\": 2,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchToMeans\",",
|
||||||
|
" \"name\": \"means\",",
|
||||||
|
" \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}",
|
||||||
|
" }",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("estimate"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("union"), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("intersection"), 0);
|
||||||
|
Assert.assertEquals(0, (double) row.getRaw("anotb"), 0);
|
||||||
|
|
||||||
|
Object meansObj = row.getRaw("means");
|
||||||
|
Assert.assertTrue(meansObj instanceof double[]);
|
||||||
|
double[] means = (double[]) meansObj;
|
||||||
|
Assert.assertEquals(2, means.length);
|
||||||
|
Assert.assertEquals(1.0, means[0], 0);
|
||||||
|
Assert.assertEquals(2.0, means[1], 0);
|
||||||
|
|
||||||
|
Object obj = row.getRaw("quantiles-sketch");
|
||||||
|
Assert.assertTrue(obj instanceof DoublesSketch);
|
||||||
|
DoublesSketch ds = (DoublesSketch) obj;
|
||||||
|
Assert.assertEquals(40, ds.getN());
|
||||||
|
Assert.assertEquals(2.0, ds.getMinValue(), 0);
|
||||||
|
Assert.assertEquals(2.0, ds.getMaxValue(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void buildingSketchesAtQueryTime() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(this.getClass().getClassLoader().getResource("tuple/array_of_doubles_build_data.tsv").getFile()),
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"type\": \"string\",",
|
||||||
|
" \"parseSpec\": {",
|
||||||
|
" \"format\": \"tsv\",",
|
||||||
|
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||||
|
" \"dimensionsSpec\": {",
|
||||||
|
" \"dimensions\": [\"product\", \"key\"],",
|
||||||
|
" \"dimensionExclusions\": [],",
|
||||||
|
" \"spatialDimensions\": []",
|
||||||
|
" },",
|
||||||
|
" \"columns\": [\"timestamp\", \"product\", \"key\", \"value\"]",
|
||||||
|
" }",
|
||||||
|
"}"),
|
||||||
|
String.join("\n",
|
||||||
|
"[",
|
||||||
|
" {\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
40, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"key\", \"metricColumns\": [\"value\"], \"nominalEntries\": 1024},",
|
||||||
|
" {\"type\": \"count\", \"name\":\"cnt\"}",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"estimate\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToQuantilesSketch\", \"name\": \"quantiles-sketch\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"union\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"union\",",
|
||||||
|
" \"operation\": \"UNION\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"intersection\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"intersection\",",
|
||||||
|
" \"operation\": \"INTERSECT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }},",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchToEstimate\", \"name\": \"anotb\", \"field\": {",
|
||||||
|
" \"type\": \"arrayOfDoublesSketchSetOp\",",
|
||||||
|
" \"name\": \"anotb\",",
|
||||||
|
" \"operation\": \"NOT\",",
|
||||||
|
" \"nominalEntries\": 1024,",
|
||||||
|
" \"fields\": [{\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}, {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}]",
|
||||||
|
" }}",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2015-01-01T00:00:00.000Z/2015-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Assert.assertEquals(40.0, new Double(row.getRaw("cnt").toString()), 0);
|
||||||
|
Assert.assertEquals(40.0, (double) row.getRaw("sketch"), 0);
|
||||||
|
Assert.assertEquals(40.0, new Double(row.getRaw("estimate").toString()), 0);
|
||||||
|
Assert.assertEquals(40.0, new Double(row.getRaw("union").toString()), 0);
|
||||||
|
Assert.assertEquals(40.0, new Double(row.getRaw("intersection").toString()), 0);
|
||||||
|
Assert.assertEquals(0, new Double(row.getRaw("anotb").toString()), 0);
|
||||||
|
|
||||||
|
Object obj = row.getRaw("quantiles-sketch");
|
||||||
|
Assert.assertTrue(obj instanceof DoublesSketch);
|
||||||
|
DoublesSketch ds = (DoublesSketch) obj;
|
||||||
|
Assert.assertEquals(40, ds.getN());
|
||||||
|
Assert.assertEquals(1.0, ds.getMinValue(), 0);
|
||||||
|
Assert.assertEquals(1.0, ds.getMaxValue(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Two buckets with statistically significant difference.
|
||||||
|
// See GenerateTestData class for details.
|
||||||
|
@Test
|
||||||
|
public void buildingSketchesAtQueryTimeAndTTest() throws Exception
|
||||||
|
{
|
||||||
|
Sequence<Row> seq = helper.createIndexAndRunQueryOnSegment(
|
||||||
|
new File(this.getClass().getClassLoader().getResource("tuple/bucket_test_data.tsv").getFile()),
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"type\": \"string\",",
|
||||||
|
" \"parseSpec\": {",
|
||||||
|
" \"format\": \"tsv\",",
|
||||||
|
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMdd\"},",
|
||||||
|
" \"dimensionsSpec\": {",
|
||||||
|
" \"dimensions\": [\"label\", \"userid\"]",
|
||||||
|
" },",
|
||||||
|
" \"columns\": [\"timestamp\", \"label\", \"userid\", \"parameter\"]",
|
||||||
|
" }",
|
||||||
|
"}"),
|
||||||
|
String.join("\n",
|
||||||
|
"[",
|
||||||
|
" {\"type\": \"doubleSum\", \"name\": \"parameter\", \"fieldName\": \"parameter\"}",
|
||||||
|
"]"),
|
||||||
|
0, // minTimestamp
|
||||||
|
Granularities.NONE,
|
||||||
|
2000, // maxRowCount
|
||||||
|
String.join("\n",
|
||||||
|
"{",
|
||||||
|
" \"queryType\": \"groupBy\",",
|
||||||
|
" \"dataSource\": \"test_datasource\",",
|
||||||
|
" \"granularity\": \"ALL\",",
|
||||||
|
" \"dimensions\": [],",
|
||||||
|
" \"aggregations\": [",
|
||||||
|
" {",
|
||||||
|
" \"type\": \"filtered\",",
|
||||||
|
" \"filter\": {\"type\": \"selector\", \"dimension\": \"label\", \"value\": \"test\"},",
|
||||||
|
" \"aggregator\": {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch-test\", \"fieldName\": \"userid\", \"metricColumns\": [\"parameter\"]}",
|
||||||
|
" },",
|
||||||
|
" {",
|
||||||
|
" \"type\": \"filtered\",",
|
||||||
|
" \"filter\": {\"type\": \"selector\", \"dimension\": \"label\", \"value\": \"control\"},",
|
||||||
|
" \"aggregator\": {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch-control\", \"fieldName\": \"userid\", \"metricColumns\": [\"parameter\"]}",
|
||||||
|
" }",
|
||||||
|
" ],",
|
||||||
|
" \"postAggregations\": [",
|
||||||
|
" {\"type\": \"arrayOfDoublesSketchTTest\",",
|
||||||
|
" \"name\": \"p-value\", \"fields\": [",
|
||||||
|
" {\"type\": \"fieldAccess\", \"fieldName\": \"sketch-test\"},",
|
||||||
|
" {\"type\": \"fieldAccess\", \"fieldName\": \"sketch-control\"}",
|
||||||
|
" ]",
|
||||||
|
" }",
|
||||||
|
" ],",
|
||||||
|
" \"intervals\": [\"2017-01-01T00:00:00.000Z/2017-01-31T00:00:00.000Z\"]",
|
||||||
|
"}"));
|
||||||
|
List<Row> results = seq.toList();
|
||||||
|
Assert.assertEquals(1, results.size());
|
||||||
|
Row row = results.get(0);
|
||||||
|
Object obj = row.getRaw("p-value");
|
||||||
|
Assert.assertTrue(obj instanceof double[]);
|
||||||
|
double[] array = (double[]) obj;
|
||||||
|
Assert.assertEquals(1, array.length);
|
||||||
|
double pValue = array[0];
|
||||||
|
// Test and control buckets were constructed to have different means, so we
|
||||||
|
// expect very low p value
|
||||||
|
Assert.assertEquals(0, pValue, 0.001);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,90 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchSetOpPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
"a",
|
||||||
|
"UNION",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
"a",
|
||||||
|
"UNION",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different operation
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
"a",
|
||||||
|
"INTERSECT",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
"a",
|
||||||
|
"UNION",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 1))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
"a",
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,81 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchTTestPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
"a",
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
"a",
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
"a",
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 1))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchTTestPostAggregator(
|
||||||
|
"b",
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchSetOpPostAggregator(
|
||||||
|
"a",
|
||||||
|
"UNION",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
Arrays.asList(new ConstantPostAggregator("", 0), new ConstantPostAggregator("", 0))
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,80 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToEstimateAndBoundsPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different numStdDevs
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToEstimateAndBoundsPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
2
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToEstimatePostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToEstimatePostAggregator(
|
||||||
|
"b",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, also not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToMeansPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToMeansPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToMeansPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToMeansPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToMeansPostAggregator(
|
||||||
|
"b",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, also not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToNumEntriesPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
"b",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, also not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,84 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToQuantilesSketchPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToQuantilesSketchPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToQuantilesSketchPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToQuantilesSketchPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1),
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different column
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToQuantilesSketchPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0),
|
||||||
|
2,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToStringPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"b",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, also not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToNumEntriesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import io.druid.query.aggregation.PostAggregator;
|
||||||
|
import io.druid.query.aggregation.post.ConstantPostAggregator;
|
||||||
|
|
||||||
|
public class ArrayOfDoublesSketchToVariancesPostAggregatorTest
|
||||||
|
{
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void equalsAndHashCode()
|
||||||
|
{
|
||||||
|
final PostAggregator postAgg1 = new ArrayOfDoublesSketchToVariancesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
@SuppressWarnings("ObjectEqualsNull")
|
||||||
|
final boolean equalsNull = postAgg1.equals(null);
|
||||||
|
Assert.assertFalse(equalsNull);
|
||||||
|
@SuppressWarnings({"EqualsWithItself", "SelfEquals"})
|
||||||
|
final boolean equalsSelf = postAgg1.equals(postAgg1);
|
||||||
|
Assert.assertTrue(equalsSelf);
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg1.hashCode());
|
||||||
|
|
||||||
|
// equals
|
||||||
|
final PostAggregator postAgg2 = new ArrayOfDoublesSketchToVariancesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertTrue(postAgg1.equals(postAgg2));
|
||||||
|
Assert.assertEquals(postAgg1.hashCode(), postAgg2.hashCode());
|
||||||
|
|
||||||
|
// same class, different field
|
||||||
|
final PostAggregator postAgg3 = new ArrayOfDoublesSketchToVariancesPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 1)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg3));
|
||||||
|
|
||||||
|
// same class, different name
|
||||||
|
final PostAggregator postAgg4 = new ArrayOfDoublesSketchToVariancesPostAggregator(
|
||||||
|
"b",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg4));
|
||||||
|
|
||||||
|
// different class, same parent, also not overriding equals and hashCode
|
||||||
|
final PostAggregator postAgg5 = new ArrayOfDoublesSketchToStringPostAggregator(
|
||||||
|
"a",
|
||||||
|
new ConstantPostAggregator("", 0)
|
||||||
|
);
|
||||||
|
Assert.assertFalse(postAgg1.equals(postAgg5));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,95 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. Metamarkets licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package io.druid.query.aggregation.datasketches.tuple;
|
||||||
|
|
||||||
|
import java.io.BufferedWriter;
|
||||||
|
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 org.apache.commons.codec.binary.Base64;
|
||||||
|
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketch;
|
||||||
|
import com.yahoo.sketches.tuple.ArrayOfDoublesUpdatableSketchBuilder;
|
||||||
|
|
||||||
|
//This is used for generating test data for ArrayOfDoublesSketchAggregationTest
|
||||||
|
class GenerateTestData
|
||||||
|
{
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception
|
||||||
|
{
|
||||||
|
generateSketches();
|
||||||
|
generateBucketTestData();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void generateSketches() throws Exception
|
||||||
|
{
|
||||||
|
Path path = FileSystems.getDefault().getPath("array_of_doubles_sketch_data.tsv");
|
||||||
|
try (BufferedWriter out = Files.newBufferedWriter(path, StandardCharsets.UTF_8)) {
|
||||||
|
Random rand = new Random();
|
||||||
|
int key = 0;
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
ArrayOfDoublesUpdatableSketch sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(1024)
|
||||||
|
.build();
|
||||||
|
sketch.update(key++, new double[] {1});
|
||||||
|
sketch.update(key++, new double[] {1});
|
||||||
|
out.write("2015010101");
|
||||||
|
out.write('\t');
|
||||||
|
out.write("product_" + (rand.nextInt(10) + 1));
|
||||||
|
out.write('\t');
|
||||||
|
out.write(Base64.encodeBase64String(sketch.compact().toByteArray()));
|
||||||
|
out.newLine();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Data for two buckets: test and control.
|
||||||
|
// Each user ID is associated with a numeric parameter
|
||||||
|
// randomly drawn from normal distribution.
|
||||||
|
// Buckets have different means.
|
||||||
|
private static void generateBucketTestData() throws Exception
|
||||||
|
{
|
||||||
|
double meanTest = 10;
|
||||||
|
double meanControl = 10.2;
|
||||||
|
Path path = FileSystems.getDefault().getPath("bucket_test_data.tsv");
|
||||||
|
try (BufferedWriter out = Files.newBufferedWriter(path, StandardCharsets.UTF_8)) {
|
||||||
|
Random rand = new Random();
|
||||||
|
for (int i = 0; i < 1000; i++) {
|
||||||
|
writeBucketTestRecord(out, "test", i, rand.nextGaussian() + meanTest);
|
||||||
|
writeBucketTestRecord(out, "control", i, rand.nextGaussian() + meanControl);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void writeBucketTestRecord(BufferedWriter out, String label, int id, double parameter) throws Exception
|
||||||
|
{
|
||||||
|
out.write("20170101");
|
||||||
|
out.write("\t");
|
||||||
|
out.write(label);
|
||||||
|
out.write("\t");
|
||||||
|
out.write(Integer.toString(id));
|
||||||
|
out.write("\t");
|
||||||
|
out.write(Double.toString(parameter));
|
||||||
|
out.newLine();
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,40 @@
|
||||||
|
2015010101 product_2 key1 1.0
|
||||||
|
2015010101 product_3 key2 1.0
|
||||||
|
2015010101 product_8 key3 1.0
|
||||||
|
2015010101 product_1 key4 1.0
|
||||||
|
2015010101 product_1 key5 1.0
|
||||||
|
2015010101 product_7 key6 1.0
|
||||||
|
2015010101 product_5 key7 1.0
|
||||||
|
2015010101 product_4 key8 1.0
|
||||||
|
2015010101 product_3 key9 1.0
|
||||||
|
2015010101 product_6 key10 1.0
|
||||||
|
2015010101 product_5 key11 1.0
|
||||||
|
2015010101 product_6 key12 1.0
|
||||||
|
2015010101 product_6 key13 1.0
|
||||||
|
2015010101 product_6 key14 1.0
|
||||||
|
2015010101 product_6 key15 1.0
|
||||||
|
2015010101 product_6 key16 1.0
|
||||||
|
2015010101 product_3 key17 1.0
|
||||||
|
2015010101 product_1 key18 1.0
|
||||||
|
2015010101 product_2 key19 1.0
|
||||||
|
2015010101 product_10 key20 1.0
|
||||||
|
2015010101 product_2 key21 1.0
|
||||||
|
2015010101 product_3 key22 1.0
|
||||||
|
2015010101 product_8 key23 1.0
|
||||||
|
2015010101 product_1 key24 1.0
|
||||||
|
2015010101 product_1 key25 1.0
|
||||||
|
2015010101 product_7 key26 1.0
|
||||||
|
2015010101 product_5 key27 1.0
|
||||||
|
2015010101 product_4 key28 1.0
|
||||||
|
2015010101 product_3 key29 1.0
|
||||||
|
2015010101 product_6 key30 1.0
|
||||||
|
2015010101 product_5 key31 1.0
|
||||||
|
2015010101 product_6 key32 1.0
|
||||||
|
2015010101 product_6 key33 1.0
|
||||||
|
2015010101 product_6 key34 1.0
|
||||||
|
2015010101 product_6 key35 1.0
|
||||||
|
2015010101 product_6 key36 1.0
|
||||||
|
2015010101 product_3 key37 1.0
|
||||||
|
2015010101 product_1 key38 1.0
|
||||||
|
2015010101 product_2 key39 1.0
|
||||||
|
2015010101 product_10 key40 1.0
|
|
|
@ -0,0 +1,40 @@
|
||||||
|
2015010101 product_2 key1 1.0 2.0
|
||||||
|
2015010101 product_3 key2 1.0 2.0
|
||||||
|
2015010101 product_8 key3 1.0 2.0
|
||||||
|
2015010101 product_1 key4 1.0 2.0
|
||||||
|
2015010101 product_1 key5 1.0 2.0
|
||||||
|
2015010101 product_7 key6 1.0 2.0
|
||||||
|
2015010101 product_5 key7 1.0 2.0
|
||||||
|
2015010101 product_4 key8 1.0 2.0
|
||||||
|
2015010101 product_3 key9 1.0 2.0
|
||||||
|
2015010101 product_6 key10 1.0 2.0
|
||||||
|
2015010101 product_5 key11 1.0 2.0
|
||||||
|
2015010101 product_6 key12 1.0 2.0
|
||||||
|
2015010101 product_6 key13 1.0 2.0
|
||||||
|
2015010101 product_6 key14 1.0 2.0
|
||||||
|
2015010101 product_6 key15 1.0 2.0
|
||||||
|
2015010101 product_6 key16 1.0 2.0
|
||||||
|
2015010101 product_3 key17 1.0 2.0
|
||||||
|
2015010101 product_1 key18 1.0 2.0
|
||||||
|
2015010101 product_2 key19 1.0 2.0
|
||||||
|
2015010101 product_10 key20 1.0 2.0
|
||||||
|
2015010101 product_2 key21 1.0 2.0
|
||||||
|
2015010101 product_3 key22 1.0 2.0
|
||||||
|
2015010101 product_8 key23 1.0 2.0
|
||||||
|
2015010101 product_1 key24 1.0 2.0
|
||||||
|
2015010101 product_1 key25 1.0 2.0
|
||||||
|
2015010101 product_7 key26 1.0 2.0
|
||||||
|
2015010101 product_5 key27 1.0 2.0
|
||||||
|
2015010101 product_4 key28 1.0 2.0
|
||||||
|
2015010101 product_3 key29 1.0 2.0
|
||||||
|
2015010101 product_6 key30 1.0 2.0
|
||||||
|
2015010101 product_5 key31 1.0 2.0
|
||||||
|
2015010101 product_6 key32 1.0 2.0
|
||||||
|
2015010101 product_6 key33 1.0 2.0
|
||||||
|
2015010101 product_6 key34 1.0 2.0
|
||||||
|
2015010101 product_6 key35 1.0 2.0
|
||||||
|
2015010101 product_6 key36 1.0 2.0
|
||||||
|
2015010101 product_3 key37 1.0 2.0
|
||||||
|
2015010101 product_1 key38 1.0 2.0
|
||||||
|
2015010101 product_2 key39 1.0 2.0
|
||||||
|
2015010101 product_10 key40 1.0 2.0
|
|
|
@ -0,0 +1,20 @@
|
||||||
|
2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAFfl9y72GoQXla2HuyIBEIAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAukCzwdoGaV3Dl/wSgXCdHgAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAvTJzckaRzBRA3i7hyds9CAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_8 AQEJAwgBzJP/////////fwIAAAAAAAAA4PSL6pmDw3z+FiET+5i8EAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAGtEwC5mMLyJpi7mRuGhXCAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAA+zh5iRMkjwHYLSN3S7k1fgAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAI6VbOBr9dB9XHsScr0GXOgAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAAbakWvEpmYR4+utyjb2+2IAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAZ/it9XjZiz+3YhopjQlLSAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_10 AQEJAwgBzJP/////////fwIAAAAAAAAA4e4o4grYzUTcn/yx1EcobwAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAA4trPn83qvgRkPRJui9c7SQAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAT/JN4LwMrhybo4QRBt3JGwAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAXKtUQBM9eT33lvvVCUmqJQAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_3 AQEJAwgBzJP/////////fwIAAAAAAAAA4ww1FskK7k6RnfVc9m+6EQAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_7 AQEJAwgBzJP/////////fwIAAAAAAAAAmtQYKDOmoS2sP+rYMRoKewAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_1 AQEJAwgBzJP/////////fwIAAAAAAAAARuLEMqskVTXLXTAt9sQ8KAAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAAZ9D3YBMITC7PFrUBaFIOcgAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_2 AQEJAwgBzJP/////////fwIAAAAAAAAA1blD14YpNinCNObiWYQdOwAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAtvKOiU9fciHblwTxetsiPgAAAAAAAPA/AAAAAAAA8D8=
|
||||||
|
2015010101 product_1 AQEJAwgBzJP/////////fwIAAAAAAAAAmRKxCIUTtzDKA2Gq5zOfFQAAAAAAAPA/AAAAAAAA8D8=
|
|
|
@ -0,0 +1,20 @@
|
||||||
|
2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAA5Wth7siARCAV+X3LvYahBQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAw5f8EoFwnR66QLPB2gZpXQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAQN4u4cnbPQi9MnNyRpHMFAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4PSL6pmDw3z+FiET+5i8EAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAaYu5kbhoVwga0TALmYwvIgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_6 AQEJAwgCzJP/////////fwIAAAAAAAAA2C0jd0u5NX77OHmJEySPAQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_1 AQEJAwgCzJP/////////fwIAAAAAAAAAI6VbOBr9dB9XHsScr0GXOgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAbakWvEpmYR4+utyjb2+2IAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAAZ/it9XjZiz+3YhopjQlLSAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_9 AQEJAwgCzJP/////////fwIAAAAAAAAA4e4o4grYzUTcn/yx1EcobwAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4trPn83qvgRkPRJui9c7SQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_9 AQEJAwgCzJP/////////fwIAAAAAAAAAT/JN4LwMrhybo4QRBt3JGwAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAA95b71QlJqiVcq1RAEz15PQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4ww1FskK7k6RnfVc9m+6EQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAArD/q2DEaCnua1BgoM6ahLQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAARuLEMqskVTXLXTAt9sQ8KAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAZ9D3YBMITC7PFrUBaFIOcgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAwjTm4lmEHTvVuUPXhik2KQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAtvKOiU9fciHblwTxetsiPgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
||||||
|
2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAygNhqucznxWZErEIhRO3MAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
|
|
File diff suppressed because it is too large
Load Diff
|
@ -82,6 +82,18 @@ public class AggregatorUtil
|
||||||
public static final byte QUANTILES_DOUBLES_SKETCH_TO_QUANTILES_CACHE_TYPE_ID = 0x1F;
|
public static final byte QUANTILES_DOUBLES_SKETCH_TO_QUANTILES_CACHE_TYPE_ID = 0x1F;
|
||||||
public static final byte QUANTILES_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x20;
|
public static final byte QUANTILES_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x20;
|
||||||
|
|
||||||
|
// ArrayOfDoublesSketch aggregator
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_CACHE_TYPE_ID = 0x21;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_SET_OP_CACHE_TYPE_ID = 0x22;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_ESTIMATE_CACHE_TYPE_ID = 0x23;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_ESTIMATE_AND_BOUNDS_CACHE_TYPE_ID = 0x24;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_MEANS_CACHE_TYPE_ID = 0x25;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_VARIANCES_CACHE_TYPE_ID = 0x26;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_NUM_ENTRIES_CACHE_TYPE_ID = 0x27;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_QUANTILES_SKETCH_CACHE_TYPE_ID = 0x28;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_T_TEST_CACHE_TYPE_ID = 0x29;
|
||||||
|
public static final byte ARRAY_OF_DOUBLES_SKETCH_TO_STRING_CACHE_TYPE_ID = 0x2A;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg
|
* returns the list of dependent postAggregators that should be calculated in order to calculate given postAgg
|
||||||
*
|
*
|
||||||
|
|
|
@ -40,6 +40,7 @@ import io.druid.segment.BaseFloatColumnValueSelector;
|
||||||
import io.druid.segment.BaseLongColumnValueSelector;
|
import io.druid.segment.BaseLongColumnValueSelector;
|
||||||
import io.druid.segment.ColumnValueSelector;
|
import io.druid.segment.ColumnValueSelector;
|
||||||
import io.druid.segment.DimensionSelector;
|
import io.druid.segment.DimensionSelector;
|
||||||
|
import io.druid.segment.DimensionSelectorUtils;
|
||||||
import io.druid.segment.Segment;
|
import io.druid.segment.Segment;
|
||||||
import io.druid.segment.column.ColumnCapabilities;
|
import io.druid.segment.column.ColumnCapabilities;
|
||||||
import io.druid.segment.column.ValueType;
|
import io.druid.segment.column.ValueType;
|
||||||
|
@ -126,7 +127,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
||||||
final Object2IntRBTreeMap<SearchHit> set
|
final Object2IntRBTreeMap<SearchHit> set
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (selector != null && !isNilSelector(selector)) {
|
if (selector != null && !DimensionSelectorUtils.isNilSelector(selector)) {
|
||||||
final IndexedInts row = selector.getRow();
|
final IndexedInts row = selector.getRow();
|
||||||
for (int i = 0, rowSize = row.size(); i < rowSize; ++i) {
|
for (int i = 0, rowSize = row.size(); i < rowSize; ++i) {
|
||||||
final String dimVal = selector.lookupName(row.get(i));
|
final String dimVal = selector.lookupName(row.get(i));
|
||||||
|
@ -141,13 +142,6 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static boolean isNilSelector(final DimensionSelector selector)
|
|
||||||
{
|
|
||||||
return selector.nameLookupPossibleInAdvance()
|
|
||||||
&& selector.getValueCardinality() == 1
|
|
||||||
&& selector.lookupName(0) == null;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static class LongSearchColumnSelectorStrategy
|
public static class LongSearchColumnSelectorStrategy
|
||||||
implements SearchColumnSelectorStrategy<BaseLongColumnValueSelector>
|
implements SearchColumnSelectorStrategy<BaseLongColumnValueSelector>
|
||||||
{
|
{
|
||||||
|
|
|
@ -270,4 +270,12 @@ public final class DimensionSelectorUtils
|
||||||
return constantSelector(extractionFn.apply(value));
|
return constantSelector(extractionFn.apply(value));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static boolean isNilSelector(final DimensionSelector selector)
|
||||||
|
{
|
||||||
|
return selector.nameLookupPossibleInAdvance()
|
||||||
|
&& selector.getValueCardinality() == 1
|
||||||
|
&& selector.lookupName(0) == null;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue