Add a small LRU cache and use utf8 bytes in ArrayOfDoubles (#12130)

* Add a small LRU cache and use utf8 bytes in ArrayOfDoubles

* Add tests for extra branches

* Even more tests for branch coverage

* Fix Style
This commit is contained in:
imply-cheddar 2022-01-12 06:04:11 +09:00 committed by GitHub
parent 08fea7a46a
commit b153cb2342
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 546 additions and 51 deletions

View File

@ -28,7 +28,10 @@ import org.apache.druid.segment.data.IndexedInts;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
* This aggregator builds sketches from raw data. * This aggregator builds sketches from raw data.
@ -45,6 +48,17 @@ public class ArrayOfDoublesSketchBuildAggregator implements Aggregator
@Nullable @Nullable
private ArrayOfDoublesUpdatableSketch sketch; private ArrayOfDoublesUpdatableSketch sketch;
private final boolean canLookupUtf8;
private final boolean canCacheById;
private final LinkedHashMap<Integer, Object> stringCache = new LinkedHashMap<Integer, Object>()
{
@Override
protected boolean removeEldestEntry(Map.Entry eldest)
{
return size() >= 10;
}
};
public ArrayOfDoublesSketchBuildAggregator( public ArrayOfDoublesSketchBuildAggregator(
final DimensionSelector keySelector, final DimensionSelector keySelector,
final List<BaseDoubleColumnValueSelector> valueSelectors, final List<BaseDoubleColumnValueSelector> valueSelectors,
@ -55,7 +69,10 @@ public class ArrayOfDoublesSketchBuildAggregator implements Aggregator
this.valueSelectors = valueSelectors.toArray(new BaseDoubleColumnValueSelector[0]); this.valueSelectors = valueSelectors.toArray(new BaseDoubleColumnValueSelector[0]);
values = new double[valueSelectors.size()]; values = new double[valueSelectors.size()];
sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(nominalEntries) sketch = new ArrayOfDoublesUpdatableSketchBuilder().setNominalEntries(nominalEntries)
.setNumberOfValues(valueSelectors.size()).build(); .setNumberOfValues(valueSelectors.size()).build();
this.canCacheById = this.keySelector.nameLookupPossibleInAdvance();
this.canLookupUtf8 = this.keySelector.supportsLookupNameUtf8();
} }
/** /**
@ -75,9 +92,35 @@ public class ArrayOfDoublesSketchBuildAggregator implements Aggregator
} }
} }
synchronized (this) { synchronized (this) {
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) { if (canLookupUtf8) {
final String key = keySelector.lookupName(keys.get(i)); for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
sketch.update(key, values); final ByteBuffer key;
if (canCacheById) {
key = (ByteBuffer) stringCache.computeIfAbsent(keys.get(i), keySelector::lookupNameUtf8);
} else {
key = keySelector.lookupNameUtf8(keys.get(i));
}
if (key != null) {
byte[] bytes = new byte[key.remaining()];
key.mark();
key.get(bytes);
key.reset();
sketch.update(bytes, values);
}
}
} else {
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
final String key;
if (canCacheById) {
key = (String) stringCache.computeIfAbsent(keys.get(i), keySelector::lookupName);
} else {
key = keySelector.lookupName(keys.get(i));
}
sketch.update(key, values);
}
} }
} }
} }

View File

@ -32,7 +32,9 @@ import org.apache.druid.segment.data.IndexedInts;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map;
/** /**
* This aggregator builds sketches from raw data. * This aggregator builds sketches from raw data.
@ -48,6 +50,18 @@ public class ArrayOfDoublesSketchBuildBufferAggregator implements BufferAggregat
@Nullable @Nullable
private double[] values; // not part of the state, but to reuse in aggregate() method private double[] values; // not part of the state, but to reuse in aggregate() method
private final boolean canLookupUtf8;
private final boolean canCacheById;
private final LinkedHashMap<Integer, Object> stringCache = new LinkedHashMap<Integer, Object>()
{
@Override
protected boolean removeEldestEntry(Map.Entry eldest)
{
return size() >= 10;
}
};
public ArrayOfDoublesSketchBuildBufferAggregator( public ArrayOfDoublesSketchBuildBufferAggregator(
final DimensionSelector keySelector, final DimensionSelector keySelector,
final List<BaseDoubleColumnValueSelector> valueSelectors, final List<BaseDoubleColumnValueSelector> valueSelectors,
@ -60,6 +74,9 @@ public class ArrayOfDoublesSketchBuildBufferAggregator implements BufferAggregat
this.nominalEntries = nominalEntries; this.nominalEntries = nominalEntries;
this.maxIntermediateSize = maxIntermediateSize; this.maxIntermediateSize = maxIntermediateSize;
values = new double[valueSelectors.size()]; values = new double[valueSelectors.size()];
this.canCacheById = this.keySelector.nameLookupPossibleInAdvance();
this.canLookupUtf8 = this.keySelector.supportsLookupNameUtf8();
} }
@Override @Override
@ -82,16 +99,42 @@ public class ArrayOfDoublesSketchBuildBufferAggregator implements BufferAggregat
values[i] = valueSelectors[i].getDouble(); values[i] = valueSelectors[i].getDouble();
} }
} }
final IndexedInts keys = keySelector.getRow();
// Wrapping memory and ArrayOfDoublesSketch is inexpensive compared to sketch operations. // Wrapping memory and ArrayOfDoublesSketch is inexpensive compared to sketch operations.
// Maintaining a cache of wrapped objects per buffer position like in Theta sketch aggregator // 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. // might might be considered, but it would increase complexity including relocate() support.
final WritableMemory mem = WritableMemory.writableWrap(buf, ByteOrder.LITTLE_ENDIAN); final WritableMemory mem = WritableMemory.writableWrap(buf, ByteOrder.LITTLE_ENDIAN);
final WritableMemory region = mem.writableRegion(position, maxIntermediateSize); final WritableMemory region = mem.writableRegion(position, maxIntermediateSize);
final ArrayOfDoublesUpdatableSketch sketch = ArrayOfDoublesSketches.wrapUpdatableSketch(region); final ArrayOfDoublesUpdatableSketch sketch = ArrayOfDoublesSketches.wrapUpdatableSketch(region);
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) { final IndexedInts keys = keySelector.getRow();
final String key = keySelector.lookupName(keys.get(i)); if (canLookupUtf8) {
sketch.update(key, values); for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
final ByteBuffer key;
if (canCacheById) {
key = (ByteBuffer) stringCache.computeIfAbsent(keys.get(i), keySelector::lookupNameUtf8);
} else {
key = keySelector.lookupNameUtf8(keys.get(i));
}
if (key != null) {
byte[] bytes = new byte[key.remaining()];
key.mark();
key.get(bytes);
key.reset();
sketch.update(bytes, values);
}
}
} else {
for (int i = 0, keysSize = keys.size(); i < keysSize; i++) {
final String key;
if (canCacheById) {
key = (String) stringCache.computeIfAbsent(keys.get(i), keySelector::lookupName);
} else {
key = keySelector.lookupName(keys.get(i));
}
sketch.update(key, values);
}
} }
} }

View File

@ -24,10 +24,12 @@ import org.apache.druid.common.config.NullHandling;
import org.apache.druid.initialization.DruidModule; import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.aggregation.AggregationTestHelper;
import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -49,6 +51,7 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
@Rule @Rule
public final TemporaryFolder tempFolder = new TemporaryFolder(); public final TemporaryFolder tempFolder = new TemporaryFolder();
private final AggregationTestHelper helper; private final AggregationTestHelper helper;
private final AggregationTestHelper tsHelper;
public ArrayOfDoublesSketchAggregationTest(final GroupByQueryConfig config) public ArrayOfDoublesSketchAggregationTest(final GroupByQueryConfig config)
{ {
@ -56,6 +59,7 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
module.configure(null); module.configure(null);
helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
module.getJacksonModules(), config, tempFolder); module.getJacksonModules(), config, tempFolder);
tsHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(module.getJacksonModules(), tempFolder);
} }
@Parameterized.Parameters(name = "{0}") @Parameterized.Parameters(name = "{0}")
@ -298,7 +302,7 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
" \"dimensionExclusions\": [],", " \"dimensionExclusions\": [],",
" \"spatialDimensions\": []", " \"spatialDimensions\": []",
" },", " },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"value\"]", " \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"value\"]",
" }", " }",
"}" "}"
), ),
@ -469,6 +473,109 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
Assert.assertEquals(2.0, ds.getMaxValue(), 0); Assert.assertEquals(2.0, ds.getMaxValue(), 0);
} }
@Test
public void buildingSketchesAtIngestionTimeTwoValuesAndNumericalKey() throws Exception
{
Sequence<ResultRow> seq = helper.createIndexAndRunQueryOnSegment(
new File(
this.getClass().getClassLoader().getResource(
"tuple/array_of_doubles_build_data_two_values_and_key_as_number.tsv").getFile()),
String.join(
"\n",
"{",
" \"type\": \"string\",",
" \"parseSpec\": {",
" \"format\": \"tsv\",",
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
" \"dimensionsSpec\": {",
" \"dimensions\": [\"product\", {\"type\": \"long\", \"name\": \"key_num\"}],",
" \"dimensionExclusions\": [],",
" \"spatialDimensions\": []",
" },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"value1\", \"value2\"]",
" }",
"}"
),
String.join(
"\n",
"[",
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"key_num\", \"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<ResultRow> results = seq.toList();
Assert.assertEquals(1, results.size());
ResultRow row = results.get(0);
Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
Assert.assertEquals("estimate", 40.0, (double) row.get(1), 0);
Assert.assertEquals("union", 40.0, (double) row.get(3), 0);
Assert.assertEquals("intersection", 40.0, (double) row.get(4), 0);
Assert.assertEquals("anotb", 0, (double) row.get(5), 0);
Object meansObj = row.get(6); // 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.get(2); // 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 @Test
public void buildingSketchesAtIngestionTimeThreeValuesAndNulls() throws Exception public void buildingSketchesAtIngestionTimeThreeValuesAndNulls() throws Exception
{ {
@ -596,11 +703,11 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
" \"format\": \"tsv\",", " \"format\": \"tsv\",",
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},", " \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
" \"dimensionsSpec\": {", " \"dimensionsSpec\": {",
" \"dimensions\": [\"product\", \"key\"],", " \"dimensions\": [\"product\", \"key\", {\"type\": \"long\", \"name\": \"key_num\"}],",
" \"dimensionExclusions\": [],", " \"dimensionExclusions\": [],",
" \"spatialDimensions\": []", " \"spatialDimensions\": []",
" },", " },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"value\"]", " \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"value\"]",
" }", " }",
"}" "}"
), ),
@ -671,6 +778,268 @@ public class ArrayOfDoublesSketchAggregationTest extends InitializedNullHandling
Assert.assertEquals(1.0, ds.getMaxValue(), 0); Assert.assertEquals(1.0, ds.getMaxValue(), 0);
} }
@Test
public void buildingSketchesAtQueryTimeUseNumerical() throws Exception
{
Sequence<ResultRow> 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\", {\"type\": \"long\", \"name\": \"key_num\"}],",
" \"dimensionExclusions\": [],",
" \"spatialDimensions\": []",
" },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"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_num\", \"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<ResultRow> results = seq.toList();
Assert.assertEquals(1, results.size());
ResultRow row = results.get(0);
Assert.assertEquals("cnt", 40.0, new Double(row.get(1).toString()), 0);
Assert.assertEquals("sketch", 40.0, (double) row.get(0), 0);
Assert.assertEquals("estimate", 40.0, new Double(row.get(2).toString()), 0);
Assert.assertEquals("union", 40.0, new Double(row.get(4).toString()), 0);
Assert.assertEquals("intersection", 40.0, new Double(row.get(5).toString()), 0);
Assert.assertEquals("anotb", 0, new Double(row.get(6).toString()), 0);
Object obj = row.get(3); // 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 buildingSketchesAtQueryTimeTimeseries() throws Exception
{
Sequence<Result<TimeseriesResultValue>> seq = tsHelper.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\", {\"type\": \"long\", \"name\": \"key_num\"}],",
" \"dimensionExclusions\": [],",
" \"spatialDimensions\": []",
" },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"value\"]",
" }",
"}"
),
String.join(
"\n",
"[",
" {\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}",
"]"
),
0, // minTimestamp
Granularities.NONE,
40, // maxRowCount
String.join(
"\n",
"{",
" \"queryType\": \"timeseries\",",
" \"dataSource\": \"test_datasource\",",
" \"granularity\": \"ALL\",",
" \"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<Result<TimeseriesResultValue>> results = seq.toList();
Assert.assertEquals(1, results.size());
TimeseriesResultValue row = results.get(0).getValue();
Assert.assertEquals("cnt", 40.0, row.getDoubleMetric("cnt"), 0);
Assert.assertEquals("sketch", 40.0, row.getDoubleMetric("sketch"), 0);
Assert.assertEquals("estimate", 40.0, row.getDoubleMetric("estimate"), 0);
Assert.assertEquals("union", 40.0, row.getDoubleMetric("union"), 0);
Assert.assertEquals("intersection", 40.0, row.getDoubleMetric("intersection"), 0);
Assert.assertEquals("anotb", 0, row.getDoubleMetric("anotb"), 0);
Object obj = row.getMetric("quantiles-sketch"); // 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 buildingSketchesAtQueryTimeUsingNumericalTimeseries() throws Exception
{
Sequence<Result<TimeseriesResultValue>> seq = tsHelper.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\", {\"type\": \"long\", \"name\": \"key_num\"}],",
" \"dimensionExclusions\": [],",
" \"spatialDimensions\": []",
" },",
" \"columns\": [\"timestamp\", \"product\", \"key\", \"key_num\", \"value\"]",
" }",
"}"
),
String.join(
"\n",
"[",
" {\"type\": \"doubleSum\", \"name\": \"value\", \"fieldName\": \"value\"}",
"]"
),
0, // minTimestamp
Granularities.NONE,
40, // maxRowCount
String.join(
"\n",
"{",
" \"queryType\": \"timeseries\",",
" \"dataSource\": \"test_datasource\",",
" \"granularity\": \"ALL\",",
" \"aggregations\": [",
" {\"type\": \"arrayOfDoublesSketch\", \"name\": \"sketch\", \"fieldName\": \"key_num\", \"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<Result<TimeseriesResultValue>> results = seq.toList();
Assert.assertEquals(1, results.size());
TimeseriesResultValue row = results.get(0).getValue();
Assert.assertEquals("cnt", 40.0, row.getDoubleMetric("cnt"), 0);
Assert.assertEquals("sketch", 40.0, row.getDoubleMetric("sketch"), 0);
Assert.assertEquals("estimate", 40.0, row.getDoubleMetric("estimate"), 0);
Assert.assertEquals("union", 40.0, row.getDoubleMetric("union"), 0);
Assert.assertEquals("intersection", 40.0, row.getDoubleMetric("intersection"), 0);
Assert.assertEquals("anotb", 0, row.getDoubleMetric("anotb"), 0);
Object obj = row.getMetric("quantiles-sketch"); // 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. // Two buckets with statistically significant difference.
// See GenerateTestData class for details. // See GenerateTestData class for details.
@Test @Test

View File

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

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

View File

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