mirror of https://github.com/apache/druid.git
fix moment sketch null handling (#9075)
This commit is contained in:
parent
7af85250cb
commit
c248e00984
|
@ -31,7 +31,6 @@ import org.apache.druid.query.aggregation.BufferAggregator;
|
||||||
import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper;
|
import org.apache.druid.query.aggregation.momentsketch.MomentSketchWrapper;
|
||||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
|
||||||
|
@ -110,11 +109,9 @@ public class MomentSketchAggregatorFactory extends AggregatorFactory
|
||||||
{
|
{
|
||||||
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
||||||
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
||||||
final ColumnValueSelector<Double> selector = metricFactory.makeColumnValueSelector(fieldName);
|
return new MomentSketchBuildAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress());
|
||||||
return new MomentSketchBuildAggregator(selector, k, getCompress());
|
|
||||||
} else {
|
} else {
|
||||||
final ColumnValueSelector<MomentSketchWrapper> selector = metricFactory.makeColumnValueSelector(fieldName);
|
return new MomentSketchMergeAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress());
|
||||||
return new MomentSketchMergeAggregator(selector, k, getCompress());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -123,11 +120,9 @@ public class MomentSketchAggregatorFactory extends AggregatorFactory
|
||||||
{
|
{
|
||||||
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
ColumnCapabilities cap = metricFactory.getColumnCapabilities(fieldName);
|
||||||
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
if (cap == null || ValueType.isNumeric(cap.getType())) {
|
||||||
final ColumnValueSelector<Double> selector = metricFactory.makeColumnValueSelector(fieldName);
|
return new MomentSketchBuildBufferAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress());
|
||||||
return new MomentSketchBuildBufferAggregator(selector, k, getCompress());
|
|
||||||
} else {
|
} else {
|
||||||
final ColumnValueSelector<MomentSketchWrapper> selector = metricFactory.makeColumnValueSelector(fieldName);
|
return new MomentSketchMergeBufferAggregator(metricFactory.makeColumnValueSelector(fieldName), k, getCompress());
|
||||||
return new MomentSketchMergeBufferAggregator(selector, k, getCompress());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -47,6 +47,9 @@ public class MomentSketchBuildAggregator implements Aggregator
|
||||||
@Override
|
@Override
|
||||||
public void aggregate()
|
public void aggregate()
|
||||||
{
|
{
|
||||||
|
if (valueSelector.isNull()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
momentsSketch.add(valueSelector.getDouble());
|
momentsSketch.add(valueSelector.getDouble());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -56,6 +56,9 @@ public class MomentSketchBuildBufferAggregator implements BufferAggregator
|
||||||
@Override
|
@Override
|
||||||
public synchronized void aggregate(final ByteBuffer buffer, final int position)
|
public synchronized void aggregate(final ByteBuffer buffer, final int position)
|
||||||
{
|
{
|
||||||
|
if (selector.isNull()) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
ByteBuffer mutationBuffer = buffer.duplicate();
|
ByteBuffer mutationBuffer = buffer.duplicate();
|
||||||
mutationBuffer.position(position);
|
mutationBuffer.position(position);
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,44 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.momentsketch.aggregator;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
public class MomentSketchAggregatorFactorySerdeTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void serializeDeserializeFactoryWithFieldName() throws Exception
|
||||||
|
{
|
||||||
|
ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
MomentSketchAggregatorFactory factory = new MomentSketchAggregatorFactory(
|
||||||
|
"name", "fieldName", 128, true
|
||||||
|
);
|
||||||
|
|
||||||
|
MomentSketchAggregatorFactory other = objectMapper.readValue(
|
||||||
|
objectMapper.writeValueAsString(factory),
|
||||||
|
MomentSketchAggregatorFactory.class
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(factory, other);
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,9 +20,8 @@
|
||||||
package org.apache.druid.query.aggregation.momentsketch.aggregator;
|
package org.apache.druid.query.aggregation.momentsketch.aggregator;
|
||||||
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.initialization.DruidModule;
|
import org.apache.druid.initialization.DruidModule;
|
||||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
|
||||||
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.aggregation.AggregationTestHelper;
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
@ -47,6 +46,7 @@ import java.util.List;
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
{
|
{
|
||||||
|
private final boolean hasNulls = !NullHandling.replaceWithDefault();
|
||||||
private final AggregationTestHelper helper;
|
private final AggregationTestHelper helper;
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
|
@ -70,23 +70,6 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
return constructors;
|
return constructors;
|
||||||
}
|
}
|
||||||
|
|
||||||
// this is to test Json properties and equals
|
|
||||||
@Test
|
|
||||||
public void serializeDeserializeFactoryWithFieldName() throws Exception
|
|
||||||
{
|
|
||||||
ObjectMapper objectMapper = new DefaultObjectMapper();
|
|
||||||
MomentSketchAggregatorFactory factory = new MomentSketchAggregatorFactory(
|
|
||||||
"name", "fieldName", 128, true
|
|
||||||
);
|
|
||||||
|
|
||||||
MomentSketchAggregatorFactory other = objectMapper.readValue(
|
|
||||||
objectMapper.writeValueAsString(factory),
|
|
||||||
MomentSketchAggregatorFactory.class
|
|
||||||
);
|
|
||||||
|
|
||||||
Assert.assertEquals(factory, other);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void buildingSketchesAtIngestionTime() throws Exception
|
public void buildingSketchesAtIngestionTime() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -104,11 +87,14 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
" \"dimensionExclusions\": [ \"sequenceNumber\"],",
|
" \"dimensionExclusions\": [ \"sequenceNumber\"],",
|
||||||
" \"spatialDimensions\": []",
|
" \"spatialDimensions\": []",
|
||||||
" },",
|
" },",
|
||||||
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
|
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\", \"valueWithNulls\"]",
|
||||||
" }",
|
" }",
|
||||||
"}"
|
"}"
|
||||||
),
|
),
|
||||||
"[{\"type\": \"momentSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 10, \"compress\": true}]",
|
"["
|
||||||
|
+ "{\"type\": \"momentSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 10, \"compress\": true},"
|
||||||
|
+ "{\"type\": \"momentSketch\", \"name\": \"sketchWithNulls\", \"fieldName\": \"valueWithNulls\", \"k\": 10, \"compress\": true}"
|
||||||
|
+ "]",
|
||||||
0,
|
0,
|
||||||
// minTimestamp
|
// minTimestamp
|
||||||
Granularities.NONE,
|
Granularities.NONE,
|
||||||
|
@ -122,12 +108,16 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
" \"granularity\": \"ALL\",",
|
" \"granularity\": \"ALL\",",
|
||||||
" \"dimensions\": [],",
|
" \"dimensions\": [],",
|
||||||
" \"aggregations\": [",
|
" \"aggregations\": [",
|
||||||
" {\"type\": \"momentSketchMerge\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"k\": 10, \"compress\": true}",
|
" {\"type\": \"momentSketchMerge\", \"name\": \"sketch\", \"fieldName\": \"sketch\", \"k\": 10, \"compress\": true},",
|
||||||
|
" {\"type\": \"momentSketchMerge\", \"name\": \"sketchWithNulls\", \"fieldName\": \"sketchWithNulls\", \"k\": 10, \"compress\": true}",
|
||||||
" ],",
|
" ],",
|
||||||
" \"postAggregations\": [",
|
" \"postAggregations\": [",
|
||||||
" {\"type\": \"momentSketchSolveQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
" {\"type\": \"momentSketchSolveQuantiles\", \"name\": \"quantiles\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
" {\"type\": \"momentSketchMin\", \"name\": \"min\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
" {\"type\": \"momentSketchMin\", \"name\": \"min\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
" {\"type\": \"momentSketchMax\", \"name\": \"max\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}}",
|
" {\"type\": \"momentSketchMax\", \"name\": \"max\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketch\"}},",
|
||||||
|
" {\"type\": \"momentSketchSolveQuantiles\", \"name\": \"quantilesWithNulls\", \"fractions\": [0, 0.5, 1], \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketchWithNulls\"}},",
|
||||||
|
" {\"type\": \"momentSketchMin\", \"name\": \"minWithNulls\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketchWithNulls\"}},",
|
||||||
|
" {\"type\": \"momentSketchMax\", \"name\": \"maxWithNulls\", \"field\": {\"type\": \"fieldAccess\", \"fieldName\": \"sketchWithNulls\"}}",
|
||||||
" ],",
|
" ],",
|
||||||
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
||||||
"}"
|
"}"
|
||||||
|
@ -136,19 +126,36 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
List<ResultRow> results = seq.toList();
|
List<ResultRow> results = seq.toList();
|
||||||
Assert.assertEquals(1, results.size());
|
Assert.assertEquals(1, results.size());
|
||||||
ResultRow row = results.get(0);
|
ResultRow row = results.get(0);
|
||||||
double[] quantilesArray = (double[]) row.get(1); // "quantiles"
|
MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
|
||||||
|
// 400 total products since this is pre-rollup
|
||||||
|
Assert.assertEquals(400.0, sketchObject.getPowerSums()[0], 1e-10);
|
||||||
|
|
||||||
|
MomentSketchWrapper sketchObjectWithNulls = (MomentSketchWrapper) row.get(1); // "sketchWithNulls"
|
||||||
|
// 23 null values, nulls at ingestion time are not replaced with default values for complex metrics inputs
|
||||||
|
Assert.assertEquals(377.0, sketchObjectWithNulls.getPowerSums()[0], 1e-10);
|
||||||
|
|
||||||
|
double[] quantilesArray = (double[]) row.get(2); // "quantiles"
|
||||||
Assert.assertEquals(0, quantilesArray[0], 0.05);
|
Assert.assertEquals(0, quantilesArray[0], 0.05);
|
||||||
Assert.assertEquals(.5, quantilesArray[1], 0.05);
|
Assert.assertEquals(.5, quantilesArray[1], 0.05);
|
||||||
Assert.assertEquals(1.0, quantilesArray[2], 0.05);
|
Assert.assertEquals(1.0, quantilesArray[2], 0.05);
|
||||||
|
|
||||||
Double minValue = (Double) row.get(2); // "min"
|
Double minValue = (Double) row.get(3); // "min"
|
||||||
Assert.assertEquals(0.0011, minValue, 0.0001);
|
Assert.assertEquals(0.0011, minValue, 0.0001);
|
||||||
|
|
||||||
Double maxValue = (Double) row.get(3); // "max"
|
Double maxValue = (Double) row.get(4); // "max"
|
||||||
Assert.assertEquals(0.9969, maxValue, 0.0001);
|
Assert.assertEquals(0.9969, maxValue, 0.0001);
|
||||||
|
|
||||||
MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
|
double[] quantilesArrayWithNulls = (double[]) row.get(5); // "quantilesWithNulls"
|
||||||
Assert.assertEquals(400.0, sketchObject.getPowerSums()[0], 1e-10);
|
Assert.assertEquals(5.0, quantilesArrayWithNulls[0], 0.05);
|
||||||
|
Assert.assertEquals(7.57, quantilesArrayWithNulls[1], 0.05);
|
||||||
|
Assert.assertEquals(10.0, quantilesArrayWithNulls[2], 0.05);
|
||||||
|
|
||||||
|
Double minValueWithNulls = (Double) row.get(6); // "minWithNulls"
|
||||||
|
Assert.assertEquals(5.0164, minValueWithNulls, 0.0001);
|
||||||
|
|
||||||
|
Double maxValueWithNulls = (Double) row.get(7); // "maxWithNulls"
|
||||||
|
Assert.assertEquals(9.9788, maxValueWithNulls, 0.0001);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -164,11 +171,11 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
" \"format\": \"tsv\",",
|
" \"format\": \"tsv\",",
|
||||||
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
" \"timestampSpec\": {\"column\": \"timestamp\", \"format\": \"yyyyMMddHH\"},",
|
||||||
" \"dimensionsSpec\": {",
|
" \"dimensionsSpec\": {",
|
||||||
" \"dimensions\": [ \"product\"],",
|
" \"dimensions\": [ \"product\", {\"name\":\"valueWithNulls\", \"type\":\"double\"}],",
|
||||||
" \"dimensionExclusions\": [\"sequenceNumber\"],",
|
" \"dimensionExclusions\": [\"sequenceNumber\"],",
|
||||||
" \"spatialDimensions\": []",
|
" \"spatialDimensions\": []",
|
||||||
" },",
|
" },",
|
||||||
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\"]",
|
" \"columns\": [\"timestamp\", \"sequenceNumber\", \"product\", \"value\", \"valueWithNulls\"]",
|
||||||
" }",
|
" }",
|
||||||
"}"
|
"}"
|
||||||
),
|
),
|
||||||
|
@ -184,7 +191,8 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
" \"granularity\": \"ALL\",",
|
" \"granularity\": \"ALL\",",
|
||||||
" \"dimensions\": [],",
|
" \"dimensions\": [],",
|
||||||
" \"aggregations\": [",
|
" \"aggregations\": [",
|
||||||
" {\"type\": \"momentSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 10}",
|
" {\"type\": \"momentSketch\", \"name\": \"sketch\", \"fieldName\": \"value\", \"k\": 10},",
|
||||||
|
" {\"type\": \"momentSketch\", \"name\": \"sketchWithNulls\", \"fieldName\": \"valueWithNulls\", \"k\": 10}",
|
||||||
" ],",
|
" ],",
|
||||||
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
" \"intervals\": [\"2016-01-01T00:00:00.000Z/2016-01-31T00:00:00.000Z\"]",
|
||||||
"}"
|
"}"
|
||||||
|
@ -196,8 +204,14 @@ public class MomentsSketchAggregatorTest extends InitializedNullHandlingTest
|
||||||
ResultRow row = results.get(0);
|
ResultRow row = results.get(0);
|
||||||
|
|
||||||
MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
|
MomentSketchWrapper sketchObject = (MomentSketchWrapper) row.get(0); // "sketch"
|
||||||
// 9 total products since we pre-sum the values.
|
// 385 total products since roll-up limited by valueWithNulls column
|
||||||
Assert.assertEquals(9.0, sketchObject.getPowerSums()[0], 1e-10);
|
Assert.assertEquals(385.0, sketchObject.getPowerSums()[0], 1e-10);
|
||||||
|
|
||||||
|
MomentSketchWrapper sketchObjectWithNulls = (MomentSketchWrapper) row.get(1); // "sketchWithNulls"
|
||||||
|
|
||||||
|
// in default mode, all 385 rows have a number value so will be computed, but only 377 rows have actual values in
|
||||||
|
// sql null mode
|
||||||
|
Assert.assertEquals(hasNulls ? 377.0 : 385.0, sketchObjectWithNulls.getPowerSums()[0], 1e-10);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1,400 +1,400 @@
|
||||||
2016010101 0 0 0.6529403005319299
|
2016010101 0 0 0.6529403005319299 6.4640
|
||||||
2016010101 1 0 0.9270214958987323
|
2016010101 1 0 0.9270214958987323 5.6748
|
||||||
2016010101 2 0 0.6383273609981486
|
2016010101 2 0 0.6383273609981486 9.0873
|
||||||
2016010101 3 0 0.8088289215633632
|
2016010101 3 0 0.8088289215633632 8.6046
|
||||||
2016010101 4 0 0.8163864917598281
|
2016010101 4 0 0.8163864917598281 5.2844
|
||||||
2016010101 5 0 0.38484848588530784
|
2016010101 5 0 0.38484848588530784 6.7631
|
||||||
2016010101 6 0 0.7690020468986823
|
2016010101 6 0 0.7690020468986823 9.4987
|
||||||
2016010101 7 0 0.6212078833139824
|
2016010101 7 0 0.6212078833139824
|
||||||
2016010101 8 0 0.4915825094949512
|
2016010101 8 0 0.4915825094949512
|
||||||
2016010101 9 0 0.688004059332008
|
2016010101 9 0 0.688004059332008 5.2246
|
||||||
2016010101 10 0 0.2536908275250508
|
2016010101 10 0 0.2536908275250508 5.8375
|
||||||
2016010101 11 0 0.6618435914290263
|
2016010101 11 0 0.6618435914290263 8.0502
|
||||||
2016010101 12 0 0.7892773595797635
|
2016010101 12 0 0.7892773595797635
|
||||||
2016010101 13 0 0.08857624134076048
|
2016010101 13 0 0.08857624134076048 9.3025
|
||||||
2016010101 14 0 0.11992633801904151
|
2016010101 14 0 0.11992633801904151 9.8775
|
||||||
2016010101 15 0 0.4959192800105586
|
2016010101 15 0 0.4959192800105586 5.3096
|
||||||
2016010101 16 0 0.5564893557708243
|
2016010101 16 0 0.5564893557708243 5.7811
|
||||||
2016010101 17 0 0.7755547456799993
|
2016010101 17 0 0.7755547456799993
|
||||||
2016010101 18 0 0.06420706406984311
|
2016010101 18 0 0.06420706406984311 7.5113
|
||||||
2016010101 19 0 0.23085639094262378
|
2016010101 19 0 0.23085639094262378 6.6375
|
||||||
2016010101 20 7 0.012013916725163498
|
2016010101 20 7 0.012013916725163498 5.5032
|
||||||
2016010101 21 7 0.34077219818209503
|
2016010101 21 7 0.34077219818209503 6.0330
|
||||||
2016010101 22 7 0.8445966884204918
|
2016010101 22 7 0.8445966884204918 6.9012
|
||||||
2016010101 23 7 0.6466142718287953
|
2016010101 23 7 0.6466142718287953 7.2324
|
||||||
2016010101 24 7 0.43959032391415487
|
2016010101 24 7 0.43959032391415487 8.5575
|
||||||
2016010101 25 7 0.7768829233737787
|
2016010101 25 7 0.7768829233737787
|
||||||
2016010101 26 7 0.5899544206136442
|
2016010101 26 7 0.5899544206136442 7.9103
|
||||||
2016010101 27 7 0.017782361911801825
|
2016010101 27 7 0.017782361911801825 9.5395
|
||||||
2016010101 28 7 0.5431916165782864
|
2016010101 28 7 0.5431916165782864 9.0030
|
||||||
2016010101 29 7 0.8218253174439416
|
2016010101 29 7 0.8218253174439416 8.2596
|
||||||
2016010101 30 7 0.6372788284951859
|
2016010101 30 7 0.6372788284951859 8.4348
|
||||||
2016010101 31 7 0.41403671834680933
|
2016010101 31 7 0.41403671834680933 7.0427
|
||||||
2016010101 32 7 0.042508330730374855
|
2016010101 32 7 0.042508330730374855 8.4631
|
||||||
2016010101 33 7 0.7416290691530969
|
2016010101 33 7 0.7416290691530969 5.8021
|
||||||
2016010101 34 7 0.6990557213726277
|
2016010101 34 7 0.6990557213726277 6.9269
|
||||||
2016010101 35 7 0.6302154208823348
|
2016010101 35 7 0.6302154208823348 8.6705
|
||||||
2016010101 36 7 0.021053567154993402
|
2016010101 36 7 0.021053567154993402 8.8914
|
||||||
2016010101 37 7 0.770280353784988
|
2016010101 37 7 0.770280353784988 7.3932
|
||||||
2016010101 38 7 0.08205576978448703
|
2016010101 38 7 0.08205576978448703 6.8809
|
||||||
2016010101 39 7 0.2049660800682488
|
2016010101 39 7 0.2049660800682488 5.9201
|
||||||
2016010101 40 5 0.08129304678049831
|
2016010101 40 5 0.08129304678049831
|
||||||
2016010101 41 5 0.17754747271638005
|
2016010101 41 5 0.17754747271638005 8.7760
|
||||||
2016010101 42 5 0.8441702357096768
|
2016010101 42 5 0.8441702357096768 6.2093
|
||||||
2016010101 43 5 0.9060464737257796
|
2016010101 43 5 0.9060464737257796 6.5394
|
||||||
2016010101 44 5 0.5970595512785409
|
2016010101 44 5 0.5970595512785409 9.9788
|
||||||
2016010101 45 5 0.843859346312315
|
2016010101 45 5 0.843859346312315 7.1848
|
||||||
2016010101 46 5 0.1649847892987305
|
2016010101 46 5 0.1649847892987305 8.4503
|
||||||
2016010101 47 5 0.5279903496999094
|
2016010101 47 5 0.5279903496999094 8.9682
|
||||||
2016010101 48 5 0.08758749830556767
|
2016010101 48 5 0.08758749830556767 6.4161
|
||||||
2016010101 49 5 0.6088480522002063
|
2016010101 49 5 0.6088480522002063 7.1579
|
||||||
2016010101 50 5 0.31079133043670004
|
2016010101 50 5 0.31079133043670004 9.3775
|
||||||
2016010101 51 5 0.43062105356651226
|
2016010101 51 5 0.43062105356651226 5.2508
|
||||||
2016010101 52 5 0.8542989852099488
|
2016010101 52 5 0.8542989852099488 7.4714
|
||||||
2016010101 53 5 0.42443162807834045
|
2016010101 53 5 0.42443162807834045 7.9648
|
||||||
2016010101 54 5 0.5020327054358468
|
2016010101 54 5 0.5020327054358468 7.4182
|
||||||
2016010101 55 5 0.36453920012074237
|
2016010101 55 5 0.36453920012074237 9.4505
|
||||||
2016010101 56 5 0.9884597580348689
|
2016010101 56 5 0.9884597580348689 6.1475
|
||||||
2016010101 57 5 0.3770559586575706
|
2016010101 57 5 0.3770559586575706 5.1045
|
||||||
2016010101 58 5 0.5989237303385875
|
2016010101 58 5 0.5989237303385875
|
||||||
2016010101 59 5 0.9926342802399872
|
2016010101 59 5 0.9926342802399872 7.7604
|
||||||
2016010101 60 4 0.7813961047849703
|
2016010101 60 4 0.7813961047849703 5.3715
|
||||||
2016010101 61 4 0.062171533805525425
|
2016010101 61 4 0.062171533805525425 6.2639
|
||||||
2016010101 62 4 0.5284977503473608
|
2016010101 62 4 0.5284977503473608 8.4169
|
||||||
2016010101 63 4 0.5924687065581794
|
2016010101 63 4 0.5924687065581794 9.3528
|
||||||
2016010101 64 4 0.06305234223879275
|
2016010101 64 4 0.06305234223879275 9.4684
|
||||||
2016010101 65 4 0.4959562731747129
|
2016010101 65 4 0.4959562731747129 9.9342
|
||||||
2016010101 66 4 0.6336733165353365
|
2016010101 66 4 0.6336733165353365 5.1156
|
||||||
2016010101 67 4 0.48860263540869875
|
2016010101 67 4 0.48860263540869875 8.3483
|
||||||
2016010101 68 4 0.9387610528974851
|
2016010101 68 4 0.9387610528974851 5.8623
|
||||||
2016010101 69 4 0.3391271652731308
|
2016010101 69 4 0.3391271652731308 6.8404
|
||||||
2016010101 70 4 0.5962837638971421
|
2016010101 70 4 0.5962837638971421 5.8733
|
||||||
2016010101 71 4 0.9190447294921896
|
2016010101 71 4 0.9190447294921896 5.6447
|
||||||
2016010101 72 4 0.33082943548872534
|
2016010101 72 4 0.33082943548872534 6.5562
|
||||||
2016010101 73 4 0.6236359023672029
|
2016010101 73 4 0.6236359023672029 9.6535
|
||||||
2016010101 74 4 0.27134427542016615
|
2016010101 74 4 0.27134427542016615 7.4968
|
||||||
2016010101 75 4 0.11665530238761901
|
2016010101 75 4 0.11665530238761901 8.7194
|
||||||
2016010101 76 4 0.10469260335277608
|
2016010101 76 4 0.10469260335277608 8.7135
|
||||||
2016010101 77 4 0.6824658847771211
|
2016010101 77 4 0.6824658847771211 5.5886
|
||||||
2016010101 78 4 0.6131047630496756
|
2016010101 78 4 0.6131047630496756 8.4490
|
||||||
2016010101 79 4 0.9838171536972515
|
2016010101 79 4 0.9838171536972515 9.2168
|
||||||
2016010101 80 4 0.7484669110852756
|
2016010101 80 4 0.7484669110852756 9.7527
|
||||||
2016010101 81 4 0.797620888697219
|
2016010101 81 4 0.797620888697219 8.2709
|
||||||
2016010101 82 4 0.7166673353657907
|
2016010101 82 4 0.7166673353657907 9.7910
|
||||||
2016010101 83 4 0.46968710353176557
|
2016010101 83 4 0.46968710353176557 7.6986
|
||||||
2016010101 84 4 0.3998491199643106
|
2016010101 84 4 0.3998491199643106 9.0208
|
||||||
2016010101 85 4 0.6314883585976869
|
2016010101 85 4 0.6314883585976869 8.5947
|
||||||
2016010101 86 4 0.8305617875577815
|
2016010101 86 4 0.8305617875577815 7.5846
|
||||||
2016010101 87 4 0.6867651870284084
|
2016010101 87 4 0.6867651870284084 7.4681
|
||||||
2016010101 88 4 0.9961677044887979
|
2016010101 88 4 0.9961677044887979 8.9175
|
||||||
2016010101 89 4 0.19745766301180412
|
2016010101 89 4 0.19745766301180412 8.3892
|
||||||
2016010101 90 4 0.2737652043079263
|
2016010101 90 4 0.2737652043079263 9.0234
|
||||||
2016010101 91 4 0.2954503444695358
|
2016010101 91 4 0.2954503444695358 6.1687
|
||||||
2016010101 92 4 0.6191902196833489
|
2016010101 92 4 0.6191902196833489 8.1738
|
||||||
2016010101 93 4 0.6828058006233482
|
2016010101 93 4 0.6828058006233482 7.5305
|
||||||
2016010101 94 4 0.7967115641510757
|
2016010101 94 4 0.7967115641510757 8.8036
|
||||||
2016010101 95 4 0.5485460823820962
|
2016010101 95 4 0.5485460823820962 9.0837
|
||||||
2016010101 96 4 0.4278132830938558
|
2016010101 96 4 0.4278132830938558
|
||||||
2016010101 97 4 0.32194908458166194
|
2016010101 97 4 0.32194908458166194 5.7173
|
||||||
2016010101 98 4 0.07094920295725238
|
2016010101 98 4 0.07094920295725238 6.8724
|
||||||
2016010101 99 4 0.4351839393889565
|
2016010101 99 4 0.4351839393889565 7.1156
|
||||||
2016010101 100 1 0.6160833396611648
|
2016010101 100 1 0.6160833396611648 5.9507
|
||||||
2016010101 101 1 0.4652667787803648
|
2016010101 101 1 0.4652667787803648 7.3727
|
||||||
2016010101 102 1 0.5026953463132913
|
2016010101 102 1 0.5026953463132913 5.9615
|
||||||
2016010101 103 1 0.4103237191034753
|
2016010101 103 1 0.4103237191034753 6.0366
|
||||||
2016010101 104 1 0.3298554666697301
|
2016010101 104 1 0.3298554666697301 8.3332
|
||||||
2016010101 105 1 0.16907537273919138
|
2016010101 105 1 0.16907537273919138 8.9892
|
||||||
2016010101 106 1 0.6945260598989513
|
2016010101 106 1 0.6945260598989513
|
||||||
2016010101 107 1 0.917138530496438
|
2016010101 107 1 0.917138530496438
|
||||||
2016010101 108 1 0.8810129148605083
|
2016010101 108 1 0.8810129148605083 6.9952
|
||||||
2016010101 109 1 0.11845626048380542
|
2016010101 109 1 0.11845626048380542 5.6466
|
||||||
2016010101 110 1 0.8848971155827816
|
2016010101 110 1 0.8848971155827816 9.2285
|
||||||
2016010101 111 1 0.9969103769603667
|
2016010101 111 1 0.9969103769603667 8.1092
|
||||||
2016010101 112 1 0.06274198529295416
|
2016010101 112 1 0.06274198529295416 7.7878
|
||||||
2016010101 113 1 0.2923616769686519
|
2016010101 113 1 0.2923616769686519 5.9326
|
||||||
2016010101 114 1 0.12621083638328634
|
2016010101 114 1 0.12621083638328634 7.1302
|
||||||
2016010101 115 1 0.9655188575577313
|
2016010101 115 1 0.9655188575577313
|
||||||
2016010101 116 1 0.6074995164352884
|
2016010101 116 1 0.6074995164352884 9.0669
|
||||||
2016010101 117 1 0.5501887988201414
|
2016010101 117 1 0.5501887988201414 5.2207
|
||||||
2016010101 118 1 0.9406914128003497
|
2016010101 118 1 0.9406914128003497 7.7406
|
||||||
2016010101 119 1 0.03264873659277656
|
2016010101 119 1 0.03264873659277656 6.2651
|
||||||
2016010101 120 6 0.004852543443656487
|
2016010101 120 6 0.004852543443656487 8.5527
|
||||||
2016010101 121 6 0.11161194329252788
|
2016010101 121 6 0.11161194329252788 9.1995
|
||||||
2016010101 122 6 0.9403527002796559
|
2016010101 122 6 0.9403527002796559 8.0145
|
||||||
2016010101 123 6 0.8951866979503953
|
2016010101 123 6 0.8951866979503953 9.5080
|
||||||
2016010101 124 6 0.07629846897033454
|
2016010101 124 6 0.07629846897033454 8.6304
|
||||||
2016010101 125 6 0.9898485014275873
|
2016010101 125 6 0.9898485014275873
|
||||||
2016010101 126 6 0.42827377712188075
|
2016010101 126 6 0.42827377712188075 9.9497
|
||||||
2016010101 127 6 0.4274796777951825
|
2016010101 127 6 0.4274796777951825 9.6071
|
||||||
2016010101 128 6 0.5569522946332676
|
2016010101 128 6 0.5569522946332676 6.2189
|
||||||
2016010101 129 6 0.028195121559112635
|
2016010101 129 6 0.028195121559112635 7.7296
|
||||||
2016010101 130 6 0.8599127909482382
|
2016010101 130 6 0.8599127909482382 5.9382
|
||||||
2016010101 131 6 0.3516112293128607
|
2016010101 131 6 0.3516112293128607
|
||||||
2016010101 132 6 0.3888868189342449
|
2016010101 132 6 0.3888868189342449 9.2290
|
||||||
2016010101 133 6 0.644589126160206
|
2016010101 133 6 0.644589126160206 5.0507
|
||||||
2016010101 134 6 0.7398741071492928
|
2016010101 134 6 0.7398741071492928 8.2090
|
||||||
2016010101 135 6 0.1998479248216123
|
2016010101 135 6 0.1998479248216123 8.0746
|
||||||
2016010101 136 6 0.8803215884594476
|
2016010101 136 6 0.8803215884594476
|
||||||
2016010101 137 6 0.7079531966558515
|
2016010101 137 6 0.7079531966558515 9.2607
|
||||||
2016010101 138 6 0.7904290564015343
|
2016010101 138 6 0.7904290564015343 5.1416
|
||||||
2016010101 139 6 0.475671788742007
|
2016010101 139 6 0.475671788742007 9.0617
|
||||||
2016010101 140 3 0.034708334899357096
|
2016010101 140 3 0.034708334899357096 5.8831
|
||||||
2016010101 141 3 0.4134637419532796
|
2016010101 141 3 0.4134637419532796 9.3853
|
||||||
2016010101 142 3 0.9757934592902832
|
2016010101 142 3 0.9757934592902832 7.6605
|
||||||
2016010101 143 3 0.37422347371609666
|
2016010101 143 3 0.37422347371609666 7.6535
|
||||||
2016010101 144 3 0.5904996168737154
|
2016010101 144 3 0.5904996168737154 7.1212
|
||||||
2016010101 145 3 0.5883259679727514
|
2016010101 145 3 0.5883259679727514 7.2821
|
||||||
2016010101 146 3 0.3380286015499171
|
2016010101 146 3 0.3380286015499171 6.0115
|
||||||
2016010101 147 3 0.42174393035143043
|
2016010101 147 3 0.42174393035143043 8.4855
|
||||||
2016010101 148 3 0.4764900074141757
|
2016010101 148 3 0.4764900074141757 9.0136
|
||||||
2016010101 149 3 0.01864239537224921
|
2016010101 149 3 0.01864239537224921 9.5688
|
||||||
2016010101 150 3 0.9124007087743986
|
2016010101 150 3 0.9124007087743986 5.0372
|
||||||
2016010101 151 3 0.8951275235699193
|
2016010101 151 3 0.8951275235699193 5.8328
|
||||||
2016010101 152 3 0.7037272142266654
|
2016010101 152 3 0.7037272142266654 9.6007
|
||||||
2016010101 153 3 0.5685506209266902
|
2016010101 153 3 0.5685506209266902 5.0164
|
||||||
2016010101 154 3 0.4104883958833594
|
2016010101 154 3 0.4104883958833594 6.5091
|
||||||
2016010101 155 3 0.7794005551450208
|
2016010101 155 3 0.7794005551450208 5.5946
|
||||||
2016010101 156 3 0.2879354697088996
|
2016010101 156 3 0.2879354697088996 8.5981
|
||||||
2016010101 157 3 0.5243215707259823
|
2016010101 157 3 0.5243215707259823 9.5059
|
||||||
2016010101 158 3 0.22238840286136063
|
2016010101 158 3 0.22238840286136063 8.6117
|
||||||
2016010101 159 3 0.11336472553284738
|
2016010101 159 3 0.11336472553284738 5.7849
|
||||||
2016010101 160 4 0.9800770037725316
|
2016010101 160 4 0.9800770037725316 6.4036
|
||||||
2016010101 161 4 0.7628237317889158
|
2016010101 161 4 0.7628237317889158 8.9102
|
||||||
2016010101 162 4 0.5355335935170453
|
2016010101 162 4 0.5355335935170453 7.7302
|
||||||
2016010101 163 4 0.9676939330565402
|
2016010101 163 4 0.9676939330565402 6.2399
|
||||||
2016010101 164 4 0.657825753108034
|
2016010101 164 4 0.657825753108034 9.4161
|
||||||
2016010101 165 4 0.9175328548944673
|
2016010101 165 4 0.9175328548944673 9.4530
|
||||||
2016010101 166 4 0.6834666043257283
|
2016010101 166 4 0.6834666043257283 7.0117
|
||||||
2016010101 167 4 0.08580759367942314
|
2016010101 167 4 0.08580759367942314 8.9389
|
||||||
2016010101 168 4 0.3134740602060899
|
2016010101 168 4 0.3134740602060899 7.4747
|
||||||
2016010101 169 4 0.3218818254752742
|
2016010101 169 4 0.3218818254752742 7.3339
|
||||||
2016010101 170 4 0.6119297354994999
|
2016010101 170 4 0.6119297354994999 5.5100
|
||||||
2016010101 171 4 0.07086832750773142
|
2016010101 171 4 0.07086832750773142 5.5108
|
||||||
2016010101 172 4 0.2700864307032772
|
2016010101 172 4 0.2700864307032772 7.0204
|
||||||
2016010101 173 4 0.7497315076673637
|
2016010101 173 4 0.7497315076673637 6.7076
|
||||||
2016010101 174 4 0.4959921300968493
|
2016010101 174 4 0.4959921300968493 9.5705
|
||||||
2016010101 175 4 0.09294825796093753
|
2016010101 175 4 0.09294825796093753 5.6014
|
||||||
2016010101 176 4 0.4954515904444161
|
2016010101 176 4 0.4954515904444161 9.8482
|
||||||
2016010101 177 4 0.8820366880191506
|
2016010101 177 4 0.8820366880191506 7.3854
|
||||||
2016010101 178 4 0.17978298283728522
|
2016010101 178 4 0.17978298283728522 6.1864
|
||||||
2016010101 179 4 0.05259679741524781
|
2016010101 179 4 0.05259679741524781 6.5391
|
||||||
2016010101 180 5 0.4711892966981096
|
2016010101 180 5 0.4711892966981096 6.9655
|
||||||
2016010101 181 5 0.5965662941715105
|
2016010101 181 5 0.5965662941715105 7.1904
|
||||||
2016010101 182 5 0.4775201668966973
|
2016010101 182 5 0.4775201668966973 6.1565
|
||||||
2016010101 183 5 0.05084576687030873
|
2016010101 183 5 0.05084576687030873 7.3423
|
||||||
2016010101 184 5 0.16680660677593928
|
2016010101 184 5 0.16680660677593928 6.8420
|
||||||
2016010101 185 5 0.9342287333653685
|
2016010101 185 5 0.9342287333653685
|
||||||
2016010101 186 5 0.8153161893769392
|
2016010101 186 5 0.8153161893769392 7.7256
|
||||||
2016010101 187 5 0.9362517669519288
|
2016010101 187 5 0.9362517669519288 9.3776
|
||||||
2016010101 188 5 0.10865218471840699
|
2016010101 188 5 0.10865218471840699 5.7953
|
||||||
2016010101 189 5 0.44665378915111065
|
2016010101 189 5 0.44665378915111065 6.9584
|
||||||
2016010101 190 5 0.8804454791937898
|
2016010101 190 5 0.8804454791937898 5.4257
|
||||||
2016010101 191 5 0.20666928346935398
|
2016010101 191 5 0.20666928346935398 6.5840
|
||||||
2016010101 192 5 0.7052479677101612
|
2016010101 192 5 0.7052479677101612 8.9173
|
||||||
2016010101 193 5 0.5006205470200923
|
2016010101 193 5 0.5006205470200923 7.7161
|
||||||
2016010101 194 5 0.23220501028575968
|
2016010101 194 5 0.23220501028575968 7.7501
|
||||||
2016010101 195 5 0.11776507130391467
|
2016010101 195 5 0.11776507130391467 6.6947
|
||||||
2016010101 196 5 0.592011744069295
|
2016010101 196 5 0.592011744069295 9.7699
|
||||||
2016010101 197 5 0.7089191450076786
|
2016010101 197 5 0.7089191450076786 6.4001
|
||||||
2016010101 198 5 0.7269340552231702
|
2016010101 198 5 0.7269340552231702
|
||||||
2016010101 199 5 0.7049554871226075
|
2016010101 199 5 0.7049554871226075 5.1937
|
||||||
2016010101 200 1 0.44078367400761076
|
2016010101 200 1 0.44078367400761076 6.1463
|
||||||
2016010101 201 1 0.7715264806037321
|
2016010101 201 1 0.7715264806037321 6.8473
|
||||||
2016010101 202 1 0.10151701902103971
|
2016010101 202 1 0.10151701902103971 5.9633
|
||||||
2016010101 203 1 0.661891806135609
|
2016010101 203 1 0.661891806135609 6.4671
|
||||||
2016010101 204 1 0.23095745116331567
|
2016010101 204 1 0.23095745116331567 6.7930
|
||||||
2016010101 205 1 0.46625278601359255
|
2016010101 205 1 0.46625278601359255 6.3031
|
||||||
2016010101 206 1 0.5912486124707177
|
2016010101 206 1 0.5912486124707177 5.1771
|
||||||
2016010101 207 1 0.963946871892115
|
2016010101 207 1 0.963946871892115 5.0794
|
||||||
2016010101 208 1 0.8172596270687692
|
2016010101 208 1 0.8172596270687692 6.2385
|
||||||
2016010101 209 1 0.05745699928199144
|
2016010101 209 1 0.05745699928199144 8.5527
|
||||||
2016010101 210 1 0.40612684342877337
|
2016010101 210 1 0.40612684342877337 8.1107
|
||||||
2016010101 211 1 0.6330844777969608
|
2016010101 211 1 0.6330844777969608 7.1172
|
||||||
2016010101 212 1 0.3148973406065705
|
2016010101 212 1 0.3148973406065705 9.1816
|
||||||
2016010101 213 1 0.23230462811318175
|
2016010101 213 1 0.23230462811318175 6.4461
|
||||||
2016010101 214 1 0.9960772952945196
|
2016010101 214 1 0.9960772952945196 9.5017
|
||||||
2016010101 215 1 0.4581376339786414
|
2016010101 215 1 0.4581376339786414 6.2413
|
||||||
2016010101 216 1 0.7181494575770677
|
2016010101 216 1 0.7181494575770677 6.9744
|
||||||
2016010101 217 1 0.04277917580280799
|
2016010101 217 1 0.04277917580280799 9.0779
|
||||||
2016010101 218 1 0.11137419446625674
|
2016010101 218 1 0.11137419446625674 8.5031
|
||||||
2016010101 219 1 0.014716278313423037
|
2016010101 219 1 0.014716278313423037 7.8452
|
||||||
2016010101 220 2 0.8988603727313186
|
2016010101 220 2 0.8988603727313186 7.2137
|
||||||
2016010101 221 2 0.8192124226306603
|
2016010101 221 2 0.8192124226306603 5.3630
|
||||||
2016010101 222 2 0.9304683598956597
|
2016010101 222 2 0.9304683598956597 7.2404
|
||||||
2016010101 223 2 0.4375546733938238
|
2016010101 223 2 0.4375546733938238 9.7816
|
||||||
2016010101 224 2 0.7676359685332207
|
2016010101 224 2 0.7676359685332207
|
||||||
2016010101 225 2 0.30977859822027964
|
2016010101 225 2 0.30977859822027964 6.9535
|
||||||
2016010101 226 2 0.008595955287459267
|
2016010101 226 2 0.008595955287459267 7.2172
|
||||||
2016010101 227 2 0.6790605343724216
|
2016010101 227 2 0.6790605343724216 8.9225
|
||||||
2016010101 228 2 0.36949588946147993
|
2016010101 228 2 0.36949588946147993 9.7846
|
||||||
2016010101 229 2 0.3826798435706562
|
2016010101 229 2 0.3826798435706562 6.8578
|
||||||
2016010101 230 2 0.13836513167087128
|
2016010101 230 2 0.13836513167087128 6.2374
|
||||||
2016010101 231 2 0.4451570472364902
|
2016010101 231 2 0.4451570472364902 5.0264
|
||||||
2016010101 232 2 0.8944067771338549
|
2016010101 232 2 0.8944067771338549 8.4037
|
||||||
2016010101 233 2 0.6068095655362902
|
2016010101 233 2 0.6068095655362902 6.1181
|
||||||
2016010101 234 2 0.7084870042917992
|
2016010101 234 2 0.7084870042917992 9.3126
|
||||||
2016010101 235 2 0.5867363290655241
|
2016010101 235 2 0.5867363290655241 8.8871
|
||||||
2016010101 236 2 0.6903863088381504
|
2016010101 236 2 0.6903863088381504 9.8005
|
||||||
2016010101 237 2 0.30984947936089124
|
2016010101 237 2 0.30984947936089124 6.0620
|
||||||
2016010101 238 2 0.31561088279452665
|
2016010101 238 2 0.31561088279452665 7.1647
|
||||||
2016010101 239 2 0.006286479849849758
|
2016010101 239 2 0.006286479849849758
|
||||||
2016010101 240 5 0.34397466439693725
|
2016010101 240 5 0.34397466439693725 7.5199
|
||||||
2016010101 241 5 0.052476003295899964
|
2016010101 241 5 0.052476003295899964 7.3817
|
||||||
2016010101 242 5 0.726106045184451
|
2016010101 242 5 0.726106045184451 7.6509
|
||||||
2016010101 243 5 0.01559115401009159
|
2016010101 243 5 0.01559115401009159 9.7002
|
||||||
2016010101 244 5 0.9219270739836661
|
2016010101 244 5 0.9219270739836661 8.0615
|
||||||
2016010101 245 5 0.5147917330760431
|
2016010101 245 5 0.5147917330760431 9.4082
|
||||||
2016010101 246 5 0.41919804470784205
|
2016010101 246 5 0.41919804470784205 5.7261
|
||||||
2016010101 247 5 0.4145101775865617
|
2016010101 247 5 0.4145101775865617 6.2074
|
||||||
2016010101 248 5 0.34153038022995796
|
2016010101 248 5 0.34153038022995796 8.7753
|
||||||
2016010101 249 5 0.9503817180587767
|
2016010101 249 5 0.9503817180587767 8.8932
|
||||||
2016010101 250 5 0.6958354849389804
|
2016010101 250 5 0.6958354849389804 9.1141
|
||||||
2016010101 251 5 0.46000811480536297
|
2016010101 251 5 0.46000811480536297 8.8439
|
||||||
2016010101 252 5 0.18379911670616378
|
2016010101 252 5 0.18379911670616378 8.2403
|
||||||
2016010101 253 5 0.20973108758556713
|
2016010101 253 5 0.20973108758556713 7.5995
|
||||||
2016010101 254 5 0.5979201603287885
|
2016010101 254 5 0.5979201603287885 6.0502
|
||||||
2016010101 255 5 0.5552419362393491
|
2016010101 255 5 0.5552419362393491 5.1349
|
||||||
2016010101 256 5 0.10996555307297629
|
2016010101 256 5 0.10996555307297629 8.9563
|
||||||
2016010101 257 5 0.3591453585622102
|
2016010101 257 5 0.3591453585622102 8.7323
|
||||||
2016010101 258 5 0.06098055111386691
|
2016010101 258 5 0.06098055111386691 7.9408
|
||||||
2016010101 259 5 0.5227270267924988
|
2016010101 259 5 0.5227270267924988 8.5808
|
||||||
2016010101 260 0 0.8492702312836989
|
2016010101 260 0 0.8492702312836989 7.9958
|
||||||
2016010101 261 0 0.5941242001151825
|
2016010101 261 0 0.5941242001151825 8.6743
|
||||||
2016010101 262 0 0.6840733026822607
|
2016010101 262 0 0.6840733026822607 7.1121
|
||||||
2016010101 263 0 0.8109777000249937
|
2016010101 263 0 0.8109777000249937 5.2360
|
||||||
2016010101 264 0 0.8599286045013937
|
2016010101 264 0 0.8599286045013937 7.7809
|
||||||
2016010101 265 0 0.7828806670746145
|
2016010101 265 0 0.7828806670746145 7.3934
|
||||||
2016010101 266 0 0.8102260971867188
|
2016010101 266 0 0.8102260971867188 5.6508
|
||||||
2016010101 267 0 0.38306094770114385
|
2016010101 267 0 0.38306094770114385 9.4577
|
||||||
2016010101 268 0 0.7093609268723879
|
2016010101 268 0 0.7093609268723879 8.8535
|
||||||
2016010101 269 0 0.4806583187577358
|
2016010101 269 0 0.4806583187577358 7.8049
|
||||||
2016010101 270 0 0.5766489331365172
|
2016010101 270 0 0.5766489331365172 9.4820
|
||||||
2016010101 271 0 0.7565067278238041
|
2016010101 271 0 0.7565067278238041 8.1346
|
||||||
2016010101 272 0 0.8262768908267573
|
2016010101 272 0 0.8262768908267573 5.4734
|
||||||
2016010101 273 0 0.7951015619138146
|
2016010101 273 0 0.7951015619138146 9.8669
|
||||||
2016010101 274 0 0.1938448910588796
|
2016010101 274 0 0.1938448910588796 6.4710
|
||||||
2016010101 275 0 0.8884608583839426
|
2016010101 275 0 0.8884608583839426 7.8987
|
||||||
2016010101 276 0 0.7046203516594505
|
2016010101 276 0 0.7046203516594505 5.1656
|
||||||
2016010101 277 0 0.5951074760704175
|
2016010101 277 0 0.5951074760704175 8.9059
|
||||||
2016010101 278 0 0.38207409719784036
|
2016010101 278 0 0.38207409719784036 6.5022
|
||||||
2016010101 279 0 0.2445271560830221
|
2016010101 279 0 0.2445271560830221 8.0526
|
||||||
2016010101 280 7 0.6032919624054952
|
2016010101 280 7 0.6032919624054952 9.5785
|
||||||
2016010101 281 7 0.1473220747987144
|
2016010101 281 7 0.1473220747987144 7.7168
|
||||||
2016010101 282 7 0.38396643099307604
|
2016010101 282 7 0.38396643099307604 8.0813
|
||||||
2016010101 283 7 0.4431561135554619
|
2016010101 283 7 0.4431561135554619 9.6805
|
||||||
2016010101 284 7 0.896578318093225
|
2016010101 284 7 0.896578318093225 9.3298
|
||||||
2016010101 285 7 0.6729206122043515
|
2016010101 285 7 0.6729206122043515 5.1569
|
||||||
2016010101 286 7 0.8498821349478478
|
2016010101 286 7 0.8498821349478478 8.8960
|
||||||
2016010101 287 7 0.48231924024179784
|
2016010101 287 7 0.48231924024179784 9.8923
|
||||||
2016010101 288 7 0.005379480238994816
|
2016010101 288 7 0.005379480238994816 6.2797
|
||||||
2016010101 289 7 0.8017936717647264
|
2016010101 289 7 0.8017936717647264 9.5651
|
||||||
2016010101 290 7 0.08193232952990348
|
2016010101 290 7 0.08193232952990348 9.1733
|
||||||
2016010101 291 7 0.3422943366454193
|
2016010101 291 7 0.3422943366454193 9.8168
|
||||||
2016010101 292 7 0.6081556855207957
|
2016010101 292 7 0.6081556855207957 6.5901
|
||||||
2016010101 293 7 0.641193222941943
|
2016010101 293 7 0.641193222941943 5.9771
|
||||||
2016010101 294 7 0.3716858024654186
|
2016010101 294 7 0.3716858024654186 7.8948
|
||||||
2016010101 295 7 0.0011169303830090849
|
2016010101 295 7 0.0011169303830090849 9.4816
|
||||||
2016010101 296 7 0.4698784438339285
|
2016010101 296 7 0.4698784438339285 8.1062
|
||||||
2016010101 297 7 0.958198841287214
|
2016010101 297 7 0.958198841287214 7.1534
|
||||||
2016010101 298 7 0.730945048929339
|
2016010101 298 7 0.730945048929339 6.3048
|
||||||
2016010101 299 7 0.1858601884405512
|
2016010101 299 7 0.1858601884405512 7.0620
|
||||||
2016010101 300 5 0.1020825694779407
|
2016010101 300 5 0.1020825694779407 9.9499
|
||||||
2016010101 301 5 0.5742385074938443
|
2016010101 301 5 0.5742385074938443 7.1200
|
||||||
2016010101 302 5 0.9846817584978909
|
2016010101 302 5 0.9846817584978909 8.1874
|
||||||
2016010101 303 5 0.3858694391491331
|
2016010101 303 5 0.3858694391491331 9.3786
|
||||||
2016010101 304 5 0.9822246873202894
|
2016010101 304 5 0.9822246873202894 9.2787
|
||||||
2016010101 305 5 0.39822015482143314
|
2016010101 305 5 0.39822015482143314 8.0036
|
||||||
2016010101 306 5 0.6575924137957005
|
2016010101 306 5 0.6575924137957005 8.2140
|
||||||
2016010101 307 5 0.02359557062746842
|
2016010101 307 5 0.02359557062746842 5.6017
|
||||||
2016010101 308 5 0.42059510563039115
|
2016010101 308 5 0.42059510563039115 8.4333
|
||||||
2016010101 309 5 0.5970764856116284
|
2016010101 309 5 0.5970764856116284 5.2788
|
||||||
2016010101 310 5 0.2817399870096221
|
2016010101 310 5 0.2817399870096221 9.5843
|
||||||
2016010101 311 5 0.5334091165258412
|
2016010101 311 5 0.5334091165258412 7.4521
|
||||||
2016010101 312 5 0.31199853410796585
|
2016010101 312 5 0.31199853410796585 5.9777
|
||||||
2016010101 313 5 0.3156991306990594
|
2016010101 313 5 0.3156991306990594 6.9610
|
||||||
2016010101 314 5 0.9560285139855889
|
2016010101 314 5 0.9560285139855889 5.8305
|
||||||
2016010101 315 5 0.7846951771498516
|
2016010101 315 5 0.7846951771498516 6.9552
|
||||||
2016010101 316 5 0.009731486767097897
|
2016010101 316 5 0.009731486767097897 6.1948
|
||||||
2016010101 317 5 0.22625857375026215
|
2016010101 317 5 0.22625857375026215 5.4469
|
||||||
2016010101 318 5 0.8580955944724618
|
2016010101 318 5 0.8580955944724618 6.5356
|
||||||
2016010101 319 5 0.9622008926137687
|
2016010101 319 5 0.9622008926137687 9.0020
|
||||||
2016010101 320 5 0.023872302930851297
|
2016010101 320 5 0.023872302930851297 9.1422
|
||||||
2016010101 321 5 0.3580981601151092
|
2016010101 321 5 0.3580981601151092 6.3018
|
||||||
2016010101 322 5 0.9120442264954038
|
2016010101 322 5 0.9120442264954038 8.0827
|
||||||
2016010101 323 5 0.5968491989965334
|
2016010101 323 5 0.5968491989965334 5.1635
|
||||||
2016010101 324 5 0.5028516120506729
|
2016010101 324 5 0.5028516120506729 8.0053
|
||||||
2016010101 325 5 0.30590552314314
|
2016010101 325 5 0.30590552314314 5.2736
|
||||||
2016010101 326 5 0.5566430714368423
|
2016010101 326 5 0.5566430714368423 5.1079
|
||||||
2016010101 327 5 0.6441099124064397
|
2016010101 327 5 0.6441099124064397 6.4961
|
||||||
2016010101 328 5 0.8765287851559298
|
2016010101 328 5 0.8765287851559298 8.6211
|
||||||
2016010101 329 5 0.38405928947408385
|
2016010101 329 5 0.38405928947408385 9.0091
|
||||||
2016010101 330 5 0.29654203975364
|
2016010101 330 5 0.29654203975364 6.4320
|
||||||
2016010101 331 5 0.3606921959261904
|
2016010101 331 5 0.3606921959261904 8.5929
|
||||||
2016010101 332 5 0.9617038824842609
|
2016010101 332 5 0.9617038824842609 5.9154
|
||||||
2016010101 333 5 0.3103700669261584
|
2016010101 333 5 0.3103700669261584 5.7548
|
||||||
2016010101 334 5 0.4935170174690311
|
2016010101 334 5 0.4935170174690311 9.8212
|
||||||
2016010101 335 5 0.34757561267296444
|
2016010101 335 5 0.34757561267296444 6.2293
|
||||||
2016010101 336 5 0.1236918485545484
|
2016010101 336 5 0.1236918485545484 7.7914
|
||||||
2016010101 337 5 0.24925258973306597
|
2016010101 337 5 0.24925258973306597 8.1718
|
||||||
2016010101 338 5 0.4104821367672965
|
2016010101 338 5 0.4104821367672965 6.1997
|
||||||
2016010101 339 5 0.3621850216936935
|
2016010101 339 5 0.3621850216936935 8.4722
|
||||||
2016010101 340 6 0.3816099229918041
|
2016010101 340 6 0.3816099229918041 8.0530
|
||||||
2016010101 341 6 0.9496667754823915
|
2016010101 341 6 0.9496667754823915 8.9414
|
||||||
2016010101 342 6 0.5594605720642025
|
2016010101 342 6 0.5594605720642025 8.1693
|
||||||
2016010101 343 6 0.8537860901562698
|
2016010101 343 6 0.8537860901562698 9.0562
|
||||||
2016010101 344 6 0.74787202967909
|
2016010101 344 6 0.74787202967909 5.2490
|
||||||
2016010101 345 6 0.29699361421249604
|
2016010101 345 6 0.29699361421249604 8.7462
|
||||||
2016010101 346 6 0.035943527086235605
|
2016010101 346 6 0.035943527086235605 8.6117
|
||||||
2016010101 347 6 0.20106098029261277
|
2016010101 347 6 0.20106098029261277 8.8491
|
||||||
2016010101 348 6 0.6589994525818863
|
2016010101 348 6 0.6589994525818863 7.2742
|
||||||
2016010101 349 6 0.3851541727199762
|
2016010101 349 6 0.3851541727199762 5.4651
|
||||||
2016010101 350 6 0.12262059605539744
|
2016010101 350 6 0.12262059605539744 5.6784
|
||||||
2016010101 351 6 0.33383436408012057
|
2016010101 351 6 0.33383436408012057 9.0930
|
||||||
2016010101 352 6 0.5087733967157267
|
2016010101 352 6 0.5087733967157267 6.8228
|
||||||
2016010101 353 6 0.34978350071897446
|
2016010101 353 6 0.34978350071897446 6.9314
|
||||||
2016010101 354 6 0.9171509423859847
|
2016010101 354 6 0.9171509423859847 6.5113
|
||||||
2016010101 355 6 0.6395164525815664
|
2016010101 355 6 0.6395164525815664 5.0802
|
||||||
2016010101 356 6 0.659637993918835
|
2016010101 356 6 0.659637993918835 8.8696
|
||||||
2016010101 357 6 0.5689746534857604
|
2016010101 357 6 0.5689746534857604 6.7891
|
||||||
2016010101 358 6 0.03266513163571427
|
2016010101 358 6 0.03266513163571427 9.0242
|
||||||
2016010101 359 6 0.5863675010868861
|
2016010101 359 6 0.5863675010868861 9.3277
|
||||||
2016010101 360 9 0.8665167898047901
|
2016010101 360 9 0.8665167898047901 7.7439
|
||||||
2016010101 361 9 0.7933960420424948
|
2016010101 361 9 0.7933960420424948
|
||||||
2016010101 362 9 0.8409667771425247
|
2016010101 362 9 0.8409667771425247 6.2808
|
||||||
2016010101 363 9 0.9544310598825743
|
2016010101 363 9 0.9544310598825743 9.5424
|
||||||
2016010101 364 9 0.36206869840549716
|
2016010101 364 9 0.36206869840549716
|
||||||
2016010101 365 9 0.253957983880155
|
2016010101 365 9 0.253957983880155 9.2830
|
||||||
2016010101 366 9 0.08496022679431525
|
2016010101 366 9 0.08496022679431525 7.7179
|
||||||
2016010101 367 9 0.5483782518766319
|
2016010101 367 9 0.5483782518766319 6.7984
|
||||||
2016010101 368 9 0.41440902281408365
|
2016010101 368 9 0.41440902281408365 7.7158
|
||||||
2016010101 369 9 0.2947889064970717
|
2016010101 369 9 0.2947889064970717 6.1741
|
||||||
2016010101 370 9 0.659477180019486
|
2016010101 370 9 0.659477180019486 7.2289
|
||||||
2016010101 371 9 0.9016744422830162
|
2016010101 371 9 0.9016744422830162 8.8920
|
||||||
2016010101 372 9 0.4692828259677926
|
2016010101 372 9 0.4692828259677926 5.0643
|
||||||
2016010101 373 9 0.4221974527778145
|
2016010101 373 9 0.4221974527778145 8.4176
|
||||||
2016010101 374 9 0.26318360778150285
|
2016010101 374 9 0.26318360778150285 5.4449
|
||||||
2016010101 375 9 0.10064081807071767
|
2016010101 375 9 0.10064081807071767 9.5148
|
||||||
2016010101 376 9 0.7781802619858804
|
2016010101 376 9 0.7781802619858804 6.3367
|
||||||
2016010101 377 9 0.529215767115243
|
2016010101 377 9 0.529215767115243 9.5807
|
||||||
2016010101 378 9 0.21094147073619007
|
2016010101 378 9 0.21094147073619007 5.9806
|
||||||
2016010101 379 9 0.18894985078463877
|
2016010101 379 9 0.18894985078463877 9.8089
|
||||||
2016010101 380 5 0.20683422198832369
|
2016010101 380 5 0.20683422198832369 7.8759
|
||||||
2016010101 381 5 0.9506923735546904
|
2016010101 381 5 0.9506923735546904 8.2207
|
||||||
2016010101 382 5 0.25734447316063913
|
2016010101 382 5 0.25734447316063913 7.5610
|
||||||
2016010101 383 5 0.6439025323539892
|
2016010101 383 5 0.6439025323539892 6.4487
|
||||||
2016010101 384 5 0.9099080819805052
|
2016010101 384 5 0.9099080819805052 9.4225
|
||||||
2016010101 385 5 0.9331714165375404
|
2016010101 385 5 0.9331714165375404
|
||||||
2016010101 386 5 0.24979840404324272
|
2016010101 386 5 0.24979840404324272
|
||||||
2016010101 387 5 0.40270120064812764
|
2016010101 387 5 0.40270120064812764 7.5996
|
||||||
2016010101 388 5 0.35895113537427137
|
2016010101 388 5 0.35895113537427137 5.4774
|
||||||
2016010101 389 5 0.44814114645480074
|
2016010101 389 5 0.44814114645480074 8.5523
|
||||||
2016010101 390 5 0.437368419580639
|
2016010101 390 5 0.437368419580639
|
||||||
2016010101 391 5 0.2777496228001308
|
2016010101 391 5 0.2777496228001308 7.1315
|
||||||
2016010101 392 5 0.09350862521048608
|
2016010101 392 5 0.09350862521048608 6.0913
|
||||||
2016010101 393 5 0.10366624548706516
|
2016010101 393 5 0.10366624548706516 5.9504
|
||||||
2016010101 394 5 0.8715309310993357
|
2016010101 394 5 0.8715309310993357 6.6814
|
||||||
2016010101 395 5 0.8953111125914557
|
2016010101 395 5 0.8953111125914557 7.4980
|
||||||
2016010101 396 5 0.9410866942183567
|
2016010101 396 5 0.9410866942183567 6.1849
|
||||||
2016010101 397 5 0.16367286942347592
|
2016010101 397 5 0.16367286942347592 6.3734
|
||||||
2016010101 398 5 0.6995415361957786
|
2016010101 398 5 0.6995415361957786 9.1507
|
||||||
2016010101 399 5 0.7170527361072194
|
2016010101 399 5 0.7170527361072194 8.5733
|
||||||
|
|
Can't render this file because it has a wrong number of fields in line 8.
|
|
@ -438,6 +438,7 @@ public class AggregationTestHelper implements Closeable
|
||||||
.setIndexSchema(
|
.setIndexSchema(
|
||||||
new IncrementalIndexSchema.Builder()
|
new IncrementalIndexSchema.Builder()
|
||||||
.withMinTimestamp(minTimestamp)
|
.withMinTimestamp(minTimestamp)
|
||||||
|
.withDimensionsSpec(parser.getParseSpec().getDimensionsSpec())
|
||||||
.withQueryGranularity(gran)
|
.withQueryGranularity(gran)
|
||||||
.withMetrics(metrics)
|
.withMetrics(metrics)
|
||||||
.withRollup(rollup)
|
.withRollup(rollup)
|
||||||
|
@ -458,6 +459,7 @@ public class AggregationTestHelper implements Closeable
|
||||||
.setIndexSchema(
|
.setIndexSchema(
|
||||||
new IncrementalIndexSchema.Builder()
|
new IncrementalIndexSchema.Builder()
|
||||||
.withMinTimestamp(minTimestamp)
|
.withMinTimestamp(minTimestamp)
|
||||||
|
.withDimensionsSpec(parser.getParseSpec().getDimensionsSpec())
|
||||||
.withQueryGranularity(gran)
|
.withQueryGranularity(gran)
|
||||||
.withMetrics(metrics)
|
.withMetrics(metrics)
|
||||||
.withRollup(rollup)
|
.withRollup(rollup)
|
||||||
|
|
Loading…
Reference in New Issue