mirror of https://github.com/apache/druid.git
update to sketches-core-0.2.2 .
adds support for "cardinality" aggregator. do not create sketch per event at ingestion time to make realtime ingestion faster
This commit is contained in:
parent
24ba20c52f
commit
fde9df2720
|
@ -5,6 +5,7 @@ layout: doc_page
|
|||
## DataSketches aggregator
|
||||
Druid aggregators based on [datasketches]()http://datasketches.github.io/) library. Note that sketch algorithms are approxiate, see details in the "Accuracy" section of the datasketches doc.
|
||||
At ingestion time, this aggregator creates the theta sketch objects which get stored in Druid segments. Logically speaking, a theta sketch object can be thought of as a Set data structure. At query time, sketches are read and aggregated(set unioned) together. In the end, by default, you receive the estimate of number of unique entries in the sketch object. Also, You can use post aggregators to do union, intersection or difference on sketch columns in the same row.
|
||||
Note that you can use `thetaSketch` aggregator on columns which were not ingested using same, it will return estimated cardinality of the column. It is recommended to use it at ingestion time as well to make querying faster.
|
||||
|
||||
### Aggregators
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@
|
|||
<dependency>
|
||||
<groupId>com.yahoo.datasketches</groupId>
|
||||
<artifactId>sketches-core</artifactId>
|
||||
<version>0.1.1</version>
|
||||
<version>0.2.2</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
|
|
|
@ -19,17 +19,22 @@
|
|||
|
||||
package io.druid.query.aggregation.datasketches.theta;
|
||||
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.yahoo.sketches.Family;
|
||||
import com.yahoo.sketches.memory.Memory;
|
||||
import com.yahoo.sketches.theta.SetOpReturnState;
|
||||
import com.yahoo.sketches.theta.SetOperation;
|
||||
import com.yahoo.sketches.theta.Sketch;
|
||||
import com.yahoo.sketches.theta.Union;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.segment.ObjectColumnSelector;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class SketchAggregator implements Aggregator
|
||||
{
|
||||
private static final Logger logger = new Logger(SketchAggregator.class);
|
||||
|
||||
private final ObjectColumnSelector selector;
|
||||
private final String name;
|
||||
private final int size;
|
||||
|
@ -48,21 +53,11 @@ public class SketchAggregator implements Aggregator
|
|||
public void aggregate()
|
||||
{
|
||||
Object update = selector.get();
|
||||
|
||||
if(update == null) {
|
||||
if (update == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
SetOpReturnState success;
|
||||
if (update instanceof Memory) {
|
||||
success = union.update((Memory) update);
|
||||
} else {
|
||||
success = union.update((Sketch) update);
|
||||
}
|
||||
|
||||
if(success != SetOpReturnState.Success) {
|
||||
throw new IllegalStateException("Sketch Aggregation failed with state " + success);
|
||||
}
|
||||
updateUnion(union, update);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -105,4 +100,31 @@ public class SketchAggregator implements Aggregator
|
|||
{
|
||||
union = null;
|
||||
}
|
||||
|
||||
static void updateUnion(Union union, Object update)
|
||||
{
|
||||
if (update instanceof Memory) {
|
||||
union.update((Memory) update);
|
||||
} else if (update instanceof Sketch) {
|
||||
union.update((Sketch) update);
|
||||
} else if (update instanceof String) {
|
||||
union.update((String) update);
|
||||
} else if (update instanceof byte[]) {
|
||||
union.update((byte[]) update);
|
||||
} else if (update instanceof Double) {
|
||||
union.update(((Double) update));
|
||||
} else if (update instanceof Integer || update instanceof Long) {
|
||||
union.update(((Number) update).longValue());
|
||||
} else if (update instanceof int[]) {
|
||||
union.update((int[]) update);
|
||||
} else if (update instanceof long[]) {
|
||||
union.update((long[]) update);
|
||||
} else if (update instanceof List) {
|
||||
for (Object entry : (List) update) {
|
||||
union.update(entry.toString());
|
||||
}
|
||||
} else {
|
||||
throw new ISE("Illegal type received while theta sketch merging [%s]", update.getClass());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,13 +19,12 @@
|
|||
|
||||
package io.druid.query.aggregation.datasketches.theta;
|
||||
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.yahoo.sketches.Family;
|
||||
import com.yahoo.sketches.memory.Memory;
|
||||
import com.yahoo.sketches.memory.MemoryRegion;
|
||||
import com.yahoo.sketches.memory.NativeMemory;
|
||||
import com.yahoo.sketches.theta.SetOpReturnState;
|
||||
import com.yahoo.sketches.theta.SetOperation;
|
||||
import com.yahoo.sketches.theta.Sketch;
|
||||
import com.yahoo.sketches.theta.Union;
|
||||
import io.druid.query.aggregation.BufferAggregator;
|
||||
import io.druid.segment.ObjectColumnSelector;
|
||||
|
@ -36,6 +35,8 @@ import java.util.Map;
|
|||
|
||||
public class SketchBufferAggregator implements BufferAggregator
|
||||
{
|
||||
private static final Logger logger = new Logger(SketchAggregator.class);
|
||||
|
||||
private final ObjectColumnSelector selector;
|
||||
private final int size;
|
||||
private final int maxIntermediateSize;
|
||||
|
@ -59,28 +60,19 @@ public class SketchBufferAggregator implements BufferAggregator
|
|||
}
|
||||
|
||||
Memory mem = new MemoryRegion(nm, position, maxIntermediateSize);
|
||||
unions.put(position, (Union) SetOperation.builder().setMemory(mem).build(size, Family.UNION));
|
||||
unions.put(position, (Union) SetOperation.builder().initMemory(mem).build(size, Family.UNION));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void aggregate(ByteBuffer buf, int position)
|
||||
{
|
||||
Object update = selector.get();
|
||||
if(update == null) {
|
||||
if (update == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
Union union = getUnion(buf, position);
|
||||
SetOpReturnState success;
|
||||
if (update instanceof Memory) {
|
||||
success = union.update((Memory) update);
|
||||
} else {
|
||||
success = union.update((Sketch) update);
|
||||
}
|
||||
|
||||
if(success != SetOpReturnState.Success) {
|
||||
throw new IllegalStateException("Sketch Buffer Aggregation failed with state " + update);
|
||||
}
|
||||
SketchAggregator.updateUnion(union, update);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,16 +19,9 @@
|
|||
|
||||
package io.druid.query.aggregation.datasketches.theta;
|
||||
|
||||
import com.metamx.common.IAE;
|
||||
import com.yahoo.sketches.memory.Memory;
|
||||
import com.yahoo.sketches.theta.Sketch;
|
||||
import com.yahoo.sketches.theta.Sketches;
|
||||
import com.yahoo.sketches.theta.UpdateSketch;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.segment.serde.ComplexMetricExtractor;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SketchBuildComplexMetricSerde extends SketchMergeComplexMetricSerde
|
||||
|
@ -48,33 +41,7 @@ public class SketchBuildComplexMetricSerde extends SketchMergeComplexMetricSerde
|
|||
@Override
|
||||
public Object extractValue(InputRow inputRow, String metricName)
|
||||
{
|
||||
Object obj = inputRow.getRaw(metricName);
|
||||
if (obj == null || obj instanceof Sketch || obj instanceof Memory) {
|
||||
return obj;
|
||||
}
|
||||
|
||||
UpdateSketch sketch = Sketches.updateSketchBuilder().build(4096);
|
||||
if (obj instanceof String) {
|
||||
sketch.update((String) obj);
|
||||
} else if (obj instanceof byte[]) {
|
||||
sketch.update((byte[]) obj);
|
||||
} else if (obj instanceof Double) {
|
||||
sketch.update(((Double) obj));
|
||||
} else if (obj instanceof Integer || obj instanceof Long) {
|
||||
sketch.update(((Number) obj).longValue());
|
||||
} else if (obj instanceof int[]) {
|
||||
sketch.update((int[]) obj);
|
||||
} else if (obj instanceof long[]) {
|
||||
sketch.update((long[]) obj);
|
||||
} else if (obj instanceof List) {
|
||||
for (Object entry : (List) obj) {
|
||||
sketch.update(entry.toString());
|
||||
}
|
||||
} else {
|
||||
throw new IAE("Unknown object type[%s] received for ingestion.", obj.getClass());
|
||||
}
|
||||
|
||||
return sketch;
|
||||
return inputRow.getRaw(metricName);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.yahoo.sketches.Family;
|
|||
import com.yahoo.sketches.memory.NativeMemory;
|
||||
import com.yahoo.sketches.theta.AnotB;
|
||||
import com.yahoo.sketches.theta.Intersection;
|
||||
import com.yahoo.sketches.theta.SetOpReturnState;
|
||||
import com.yahoo.sketches.theta.SetOperation;
|
||||
import com.yahoo.sketches.theta.Sketch;
|
||||
import com.yahoo.sketches.theta.Sketches;
|
||||
|
@ -92,33 +91,28 @@ public class SketchOperations
|
|||
case UNION:
|
||||
Union union = (Union) SetOperation.builder().build(sketchSize, Family.UNION);
|
||||
for(Sketch sketch : sketches) {
|
||||
SetOpReturnState success = union.update(sketch);
|
||||
if(success != SetOpReturnState.Success) {
|
||||
throw new IllegalStateException("Sketch operation failed " + func);
|
||||
}
|
||||
union.update(sketch);
|
||||
}
|
||||
return union.getResult(false, null);
|
||||
case INTERSECT:
|
||||
Intersection intersection = (Intersection) SetOperation.builder().build(sketchSize, Family.INTERSECTION);
|
||||
for(Sketch sketch : sketches) {
|
||||
SetOpReturnState success = intersection.update(sketch);
|
||||
if(success != SetOpReturnState.Success) {
|
||||
throw new IllegalStateException("Sketch operation failed " + func);
|
||||
}
|
||||
intersection.update(sketch);
|
||||
}
|
||||
return intersection.getResult(false, null);
|
||||
case NOT:
|
||||
if(sketches.length < 2) {
|
||||
throw new IllegalArgumentException("A-Not-B requires atleast 2 sketches");
|
||||
if(sketches.length < 1) {
|
||||
throw new IllegalArgumentException("A-Not-B requires atleast 1 sketch");
|
||||
}
|
||||
|
||||
if(sketches.length == 1) {
|
||||
return sketches[0];
|
||||
}
|
||||
|
||||
Sketch result = sketches[0];
|
||||
for (int i = 1; i < sketches.length; i++) {
|
||||
AnotB anotb = (AnotB) SetOperation.builder().build(sketchSize, Family.A_NOT_B);
|
||||
SetOpReturnState success = anotb.update(result, sketches[i]);
|
||||
if(success != SetOpReturnState.Success) {
|
||||
throw new IllegalStateException("Sketch operation failed " + func);
|
||||
}
|
||||
anotb.update(result, sketches[i]);
|
||||
result = anotb.getResult(false, null);
|
||||
}
|
||||
return result;
|
||||
|
|
|
@ -21,7 +21,6 @@ package io.druid.query.aggregation.datasketches.theta;
|
|||
|
||||
import com.yahoo.sketches.memory.Memory;
|
||||
import com.yahoo.sketches.theta.CompactSketch;
|
||||
import com.yahoo.sketches.theta.SetOpReturnState;
|
||||
import com.yahoo.sketches.theta.Sketch;
|
||||
import com.yahoo.sketches.theta.Union;
|
||||
|
||||
|
@ -37,15 +36,51 @@ public class SynchronizedUnion implements Union
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized SetOpReturnState update(Sketch sketch)
|
||||
public synchronized void update(Sketch sketchIn)
|
||||
{
|
||||
return delegate.update(sketch);
|
||||
delegate.update(sketchIn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized SetOpReturnState update(Memory memory)
|
||||
public synchronized void update(Memory mem)
|
||||
{
|
||||
return delegate.update(memory);
|
||||
delegate.update(mem);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(long datum)
|
||||
{
|
||||
delegate.update(datum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(double datum)
|
||||
{
|
||||
delegate.update(datum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(String datum)
|
||||
{
|
||||
delegate.update(datum);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(byte[] data)
|
||||
{
|
||||
delegate.update(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(int[] data)
|
||||
{
|
||||
delegate.update(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void update(long[] data)
|
||||
{
|
||||
delegate.update(data);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -54,6 +89,12 @@ public class SynchronizedUnion implements Union
|
|||
return delegate.getResult(b, memory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized CompactSketch getResult()
|
||||
{
|
||||
return delegate.getResult();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] toByteArray()
|
||||
{
|
||||
|
|
|
@ -123,6 +123,43 @@ public class SketchAggregationTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testThetaCardinalityOnSimpleColumn() throws Exception
|
||||
{
|
||||
Sequence seq = helper.createIndexAndRunQueryOnSegment(
|
||||
new File(SketchAggregationTest.class.getClassLoader().getResource("simple_test_data.tsv").getFile()),
|
||||
readFileFromClasspathAsString("simple_test_data_record_parser2.json"),
|
||||
"["
|
||||
+ " {"
|
||||
+ " \"type\": \"count\","
|
||||
+ " \"name\": \"count\""
|
||||
+ " }"
|
||||
+ "]",
|
||||
0,
|
||||
QueryGranularity.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("simple_test_data_group_by_query.json")
|
||||
);
|
||||
|
||||
List results = Sequences.toList(seq, Lists.newArrayList());
|
||||
Assert.assertEquals(1, results.size());
|
||||
Assert.assertEquals(
|
||||
new MapBasedRow(
|
||||
DateTime.parse("2014-10-19T00:00:00.000Z"),
|
||||
ImmutableMap
|
||||
.<String, Object>builder()
|
||||
.put("sketch_count", 50.0)
|
||||
.put("sketchEstimatePostAgg", 50.0)
|
||||
.put("sketchUnionPostAggEstimate", 50.0)
|
||||
.put("sketchIntersectionPostAggEstimate", 50.0)
|
||||
.put("sketchAnotBPostAggEstimate", 0.0)
|
||||
.put("non_existing_col_validation", 0.0)
|
||||
.build()
|
||||
),
|
||||
results.get(0)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSketchMergeAggregatorFactorySerde() throws Exception
|
||||
{
|
||||
|
|
|
@ -0,0 +1,16 @@
|
|||
{
|
||||
"type" : "string",
|
||||
"parseSpec" : {
|
||||
"format" : "tsv",
|
||||
"timestampSpec" : {
|
||||
"column" : "timestamp",
|
||||
"format" : "yyyyMMddHH"
|
||||
},
|
||||
"dimensionsSpec" : {
|
||||
"dimensions": ["product", "pty_country"],
|
||||
"dimensionExclusions" : [],
|
||||
"spatialDimensions" : []
|
||||
},
|
||||
"columns": ["timestamp", "product", "pty_country"]
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue