mirror of https://github.com/apache/druid.git
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:
parent
08fea7a46a
commit
b153cb2342
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
|
@ -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
|
|
Loading…
Reference in New Issue