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:
Alexander Saydakov 2018-05-13 05:48:00 -07:00 committed by Roman Leventov
parent 7a1faa332f
commit 15864434be
46 changed files with 6193 additions and 8 deletions

View File

@ -40,6 +40,10 @@
<artifactId>sketches-core</artifactId>
<version>0.10.3</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-math3</artifactId>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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()
{
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,3 +1,4 @@
io.druid.query.aggregation.datasketches.theta.SketchModule
io.druid.query.aggregation.datasketches.theta.oldapi.OldApiSketchModule
io.druid.query.aggregation.datasketches.quantiles.DoublesSketchModule
io.druid.query.aggregation.datasketches.tuple.ArrayOfDoublesSketchModule

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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
1 2015010101 product_2 key1 1.0
2 2015010101 product_3 key2 1.0
3 2015010101 product_8 key3 1.0
4 2015010101 product_1 key4 1.0
5 2015010101 product_1 key5 1.0
6 2015010101 product_7 key6 1.0
7 2015010101 product_5 key7 1.0
8 2015010101 product_4 key8 1.0
9 2015010101 product_3 key9 1.0
10 2015010101 product_6 key10 1.0
11 2015010101 product_5 key11 1.0
12 2015010101 product_6 key12 1.0
13 2015010101 product_6 key13 1.0
14 2015010101 product_6 key14 1.0
15 2015010101 product_6 key15 1.0
16 2015010101 product_6 key16 1.0
17 2015010101 product_3 key17 1.0
18 2015010101 product_1 key18 1.0
19 2015010101 product_2 key19 1.0
20 2015010101 product_10 key20 1.0
21 2015010101 product_2 key21 1.0
22 2015010101 product_3 key22 1.0
23 2015010101 product_8 key23 1.0
24 2015010101 product_1 key24 1.0
25 2015010101 product_1 key25 1.0
26 2015010101 product_7 key26 1.0
27 2015010101 product_5 key27 1.0
28 2015010101 product_4 key28 1.0
29 2015010101 product_3 key29 1.0
30 2015010101 product_6 key30 1.0
31 2015010101 product_5 key31 1.0
32 2015010101 product_6 key32 1.0
33 2015010101 product_6 key33 1.0
34 2015010101 product_6 key34 1.0
35 2015010101 product_6 key35 1.0
36 2015010101 product_6 key36 1.0
37 2015010101 product_3 key37 1.0
38 2015010101 product_1 key38 1.0
39 2015010101 product_2 key39 1.0
40 2015010101 product_10 key40 1.0

View File

@ -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
1 2015010101 product_2 key1 1.0 2.0
2 2015010101 product_3 key2 1.0 2.0
3 2015010101 product_8 key3 1.0 2.0
4 2015010101 product_1 key4 1.0 2.0
5 2015010101 product_1 key5 1.0 2.0
6 2015010101 product_7 key6 1.0 2.0
7 2015010101 product_5 key7 1.0 2.0
8 2015010101 product_4 key8 1.0 2.0
9 2015010101 product_3 key9 1.0 2.0
10 2015010101 product_6 key10 1.0 2.0
11 2015010101 product_5 key11 1.0 2.0
12 2015010101 product_6 key12 1.0 2.0
13 2015010101 product_6 key13 1.0 2.0
14 2015010101 product_6 key14 1.0 2.0
15 2015010101 product_6 key15 1.0 2.0
16 2015010101 product_6 key16 1.0 2.0
17 2015010101 product_3 key17 1.0 2.0
18 2015010101 product_1 key18 1.0 2.0
19 2015010101 product_2 key19 1.0 2.0
20 2015010101 product_10 key20 1.0 2.0
21 2015010101 product_2 key21 1.0 2.0
22 2015010101 product_3 key22 1.0 2.0
23 2015010101 product_8 key23 1.0 2.0
24 2015010101 product_1 key24 1.0 2.0
25 2015010101 product_1 key25 1.0 2.0
26 2015010101 product_7 key26 1.0 2.0
27 2015010101 product_5 key27 1.0 2.0
28 2015010101 product_4 key28 1.0 2.0
29 2015010101 product_3 key29 1.0 2.0
30 2015010101 product_6 key30 1.0 2.0
31 2015010101 product_5 key31 1.0 2.0
32 2015010101 product_6 key32 1.0 2.0
33 2015010101 product_6 key33 1.0 2.0
34 2015010101 product_6 key34 1.0 2.0
35 2015010101 product_6 key35 1.0 2.0
36 2015010101 product_6 key36 1.0 2.0
37 2015010101 product_3 key37 1.0 2.0
38 2015010101 product_1 key38 1.0 2.0
39 2015010101 product_2 key39 1.0 2.0
40 2015010101 product_10 key40 1.0 2.0

View File

@ -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=
1 2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAFfl9y72GoQXla2HuyIBEIAAAAAAAAPA/AAAAAAAA8D8=
2 2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAukCzwdoGaV3Dl/wSgXCdHgAAAAAAAPA/AAAAAAAA8D8=
3 2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAvTJzckaRzBRA3i7hyds9CAAAAAAAAPA/AAAAAAAA8D8=
4 2015010101 product_8 AQEJAwgBzJP/////////fwIAAAAAAAAA4PSL6pmDw3z+FiET+5i8EAAAAAAAAPA/AAAAAAAA8D8=
5 2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAGtEwC5mMLyJpi7mRuGhXCAAAAAAAAPA/AAAAAAAA8D8=
6 2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAA+zh5iRMkjwHYLSN3S7k1fgAAAAAAAPA/AAAAAAAA8D8=
7 2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAI6VbOBr9dB9XHsScr0GXOgAAAAAAAPA/AAAAAAAA8D8=
8 2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAAbakWvEpmYR4+utyjb2+2IAAAAAAAAPA/AAAAAAAA8D8=
9 2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAZ/it9XjZiz+3YhopjQlLSAAAAAAAAPA/AAAAAAAA8D8=
10 2015010101 product_10 AQEJAwgBzJP/////////fwIAAAAAAAAA4e4o4grYzUTcn/yx1EcobwAAAAAAAPA/AAAAAAAA8D8=
11 2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAA4trPn83qvgRkPRJui9c7SQAAAAAAAPA/AAAAAAAA8D8=
12 2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAT/JN4LwMrhybo4QRBt3JGwAAAAAAAPA/AAAAAAAA8D8=
13 2015010101 product_6 AQEJAwgBzJP/////////fwIAAAAAAAAAXKtUQBM9eT33lvvVCUmqJQAAAAAAAPA/AAAAAAAA8D8=
14 2015010101 product_3 AQEJAwgBzJP/////////fwIAAAAAAAAA4ww1FskK7k6RnfVc9m+6EQAAAAAAAPA/AAAAAAAA8D8=
15 2015010101 product_7 AQEJAwgBzJP/////////fwIAAAAAAAAAmtQYKDOmoS2sP+rYMRoKewAAAAAAAPA/AAAAAAAA8D8=
16 2015010101 product_1 AQEJAwgBzJP/////////fwIAAAAAAAAARuLEMqskVTXLXTAt9sQ8KAAAAAAAAPA/AAAAAAAA8D8=
17 2015010101 product_4 AQEJAwgBzJP/////////fwIAAAAAAAAAZ9D3YBMITC7PFrUBaFIOcgAAAAAAAPA/AAAAAAAA8D8=
18 2015010101 product_2 AQEJAwgBzJP/////////fwIAAAAAAAAA1blD14YpNinCNObiWYQdOwAAAAAAAPA/AAAAAAAA8D8=
19 2015010101 product_5 AQEJAwgBzJP/////////fwIAAAAAAAAAtvKOiU9fciHblwTxetsiPgAAAAAAAPA/AAAAAAAA8D8=
20 2015010101 product_1 AQEJAwgBzJP/////////fwIAAAAAAAAAmRKxCIUTtzDKA2Gq5zOfFQAAAAAAAPA/AAAAAAAA8D8=

View File

@ -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
1 2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAA5Wth7siARCAV+X3LvYahBQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
2 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAw5f8EoFwnR66QLPB2gZpXQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
3 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAQN4u4cnbPQi9MnNyRpHMFAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
4 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4PSL6pmDw3z+FiET+5i8EAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
5 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAaYu5kbhoVwga0TALmYwvIgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
6 2015010101 product_6 AQEJAwgCzJP/////////fwIAAAAAAAAA2C0jd0u5NX77OHmJEySPAQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
7 2015010101 product_1 AQEJAwgCzJP/////////fwIAAAAAAAAAI6VbOBr9dB9XHsScr0GXOgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
8 2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAbakWvEpmYR4+utyjb2+2IAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
9 2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAAZ/it9XjZiz+3YhopjQlLSAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
10 2015010101 product_9 AQEJAwgCzJP/////////fwIAAAAAAAAA4e4o4grYzUTcn/yx1EcobwAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
11 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4trPn83qvgRkPRJui9c7SQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
12 2015010101 product_9 AQEJAwgCzJP/////////fwIAAAAAAAAAT/JN4LwMrhybo4QRBt3JGwAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
13 2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAA95b71QlJqiVcq1RAEz15PQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
14 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAA4ww1FskK7k6RnfVc9m+6EQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
15 2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAArD/q2DEaCnua1BgoM6ahLQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
16 2015010101 product_8 AQEJAwgCzJP/////////fwIAAAAAAAAARuLEMqskVTXLXTAt9sQ8KAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
17 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAZ9D3YBMITC7PFrUBaFIOcgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
18 2015010101 product_2 AQEJAwgCzJP/////////fwIAAAAAAAAAwjTm4lmEHTvVuUPXhik2KQAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
19 2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAtvKOiU9fciHblwTxetsiPgAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA
20 2015010101 product_5 AQEJAwgCzJP/////////fwIAAAAAAAAAygNhqucznxWZErEIhRO3MAAAAAAAAPA/AAAAAAAAAEAAAAAAAADwPwAAAAAAAABA

File diff suppressed because it is too large Load Diff

View File

@ -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_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
*

View File

@ -40,6 +40,7 @@ import io.druid.segment.BaseFloatColumnValueSelector;
import io.druid.segment.BaseLongColumnValueSelector;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.DimensionSelectorUtils;
import io.druid.segment.Segment;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
@ -126,7 +127,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
final Object2IntRBTreeMap<SearchHit> set
)
{
if (selector != null && !isNilSelector(selector)) {
if (selector != null && !DimensionSelectorUtils.isNilSelector(selector)) {
final IndexedInts row = selector.getRow();
for (int i = 0, rowSize = row.size(); i < rowSize; ++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
implements SearchColumnSelectorStrategy<BaseLongColumnValueSelector>
{

View File

@ -270,4 +270,12 @@ public final class DimensionSelectorUtils
return constantSelector(extractionFn.apply(value));
}
}
public static boolean isNilSelector(final DimensionSelector selector)
{
return selector.nameLookupPossibleInAdvance()
&& selector.getValueCardinality() == 1
&& selector.lookupName(0) == null;
}
}