From 089d8da5610453549fc9c305046f0b7397500b47 Mon Sep 17 00:00:00 2001 From: imply-cheddar <86940447+imply-cheddar@users.noreply.github.com> Date: Thu, 15 Dec 2022 11:04:39 +0900 Subject: [PATCH] Support Framing for Window Aggregations (#13514) * Support Framing for Window Aggregations This adds support for framing over ROWS for window aggregations. Still not implemented as yet: 1. RANGE frames 2. Multiple different frames in the same query 3. Frames on last/first functions --- ...stractParallelIndexSupervisorTaskTest.java | 13 +- .../org/apache/druid/query/DataSource.java | 6 +- .../query/operator/LimitedRowsAndColumns.java | 6 +- .../operator/NaivePartitioningOperator.java | 4 +- .../query/operator/window/Processor.java | 1 + .../window/WindowAggregateProcessor.java | 6 +- .../query/operator/window/WindowFrame.java | 123 + .../WindowFramedAggregateProcessor.java | 99 + .../ranking/WindowPercentileProcessor.java | 2 +- .../ranking/WindowRankingProcessorBase.java | 6 +- .../ranking/WindowRowNumberProcessor.java | 4 +- .../value/ShiftedColumnAccessorBase.java | 24 +- .../window/value/WindowOffsetProcessor.java | 8 +- .../value/WindowValueProcessorBase.java | 2 +- .../{frame => }/AppendableMapOfColumns.java | 5 +- .../rowsandcols/ArrayListRowsAndColumns.java | 4 +- .../MapOfColumnsRowsAndColumns.java | 5 +- .../query/rowsandcols/RowsAndColumns.java | 3 +- .../rowsandcols/column/ColumnAccessor.java | 2 +- .../column/ConstantObjectColumn.java | 2 +- .../rowsandcols/column/DoubleArrayColumn.java | 2 +- .../rowsandcols/column/IntArrayColumn.java | 2 +- .../column/NullColumnAccessor.java | 2 +- .../rowsandcols/column/ObjectArrayColumn.java | 4 +- .../column/ObjectColumnAccessorBase.java | 4 +- .../AppendableRowsAndColumns.java | 3 +- .../semantic/ColumnSelectorFactoryMaker.java | 39 + .../DefaultColumnSelectorFactoryMaker.java} | 74 +- .../DefaultFramedOnHeapAggregatable.java | 581 +++++ .../DefaultGroupPartitioner.java | 5 +- .../semantic/DefaultOnHeapAggregatable.java | 108 + .../DefaultSortedGroupPartitioner.java | 5 +- .../semantic/FramedOnHeapAggregatable.java | 57 + .../{ => semantic}/GroupPartitioner.java | 2 +- .../{ => semantic}/OnHeapAggregatable.java | 2 +- .../SortedGroupPartitioner.java | 4 +- .../NaivePartitioningOperatorTest.java | 2 +- .../query/operator/OperatorSequenceTest.java | 2 +- .../query/operator/SequenceOperatorTest.java | 2 +- .../operator/WindowProcessorOperatorTest.java | 2 +- .../operator/window/RowsAndColumnsHelper.java | 23 +- .../window/WindowAggregateProcessorTest.java | 2 +- .../WindowFramedAggregateProcessorTest.java | 94 + .../ranking/WindowCumeDistProcessorTest.java | 2 +- .../ranking/WindowDenseRankProcessorTest.java | 2 +- .../WindowPercentileProcessorTest.java | 2 +- .../ranking/WindowRankProcessorTest.java | 2 +- .../ranking/WindowRowNumberProcessorTest.java | 2 +- .../value/WindowFirstProcessorTest.java | 2 +- .../window/value/WindowLagProcessorTest.java | 2 +- .../window/value/WindowLastProcessorTest.java | 2 +- .../window/value/WindowLeadProcessorTest.java | 2 +- .../ArrayListRowsAndColumnsTest.java | 16 +- .../rowsandcols/AsOnlyTestRowsAndColumns.java | 63 + .../MapOfColumnsRowsAndColumnsTest.java | 8 +- .../query/rowsandcols/NoAsRowsAndColumns.java | 61 + .../rowsandcols/RowsAndColumnsTestBase.java | 191 +- .../column/NullColumnAccessorTest.java | 2 +- .../AppendableRowsAndColumnsTest.java | 63 + .../FramedOnHeapAggregatableTest.java | 459 ++++ .../semantic/OnHeapAggregatableTest.java | 72 + .../semantic/SemanticTestBase.java | 88 + .../semantic/SortedGroupPartitionerTest.java | 103 + .../druid/sql/calcite/rel/Windowing.java | 148 +- .../sql/calcite/CalciteWindowQueryTest.java | 39 +- .../calcite/tests/window/simpleSum.sqlTest | 31 +- .../window/wikipediaCumulativeOrdered.sqlTest | 2037 +++++++++-------- .../wikipediaFramedAggregations.sqlTest | 1008 ++++++++ .../window/wikipediaSimplePartition.sqlTest | 2019 ++++++++-------- 69 files changed, 5352 insertions(+), 2420 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java create mode 100644 processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/{frame => }/AppendableMapOfColumns.java (93%) rename processing/src/main/java/org/apache/druid/query/rowsandcols/{frame => }/MapOfColumnsRowsAndColumns.java (94%) rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/AppendableRowsAndColumns.java (92%) create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/ColumnSelectorFactoryMaker.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/{DefaultOnHeapAggregatable.java => semantic/DefaultColumnSelectorFactoryMaker.java} (76%) create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/DefaultGroupPartitioner.java (93%) create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultOnHeapAggregatable.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/DefaultSortedGroupPartitioner.java (94%) create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatable.java rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/GroupPartitioner.java (97%) rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/OnHeapAggregatable.java (97%) rename processing/src/main/java/org/apache/druid/query/rowsandcols/{ => semantic}/SortedGroupPartitioner.java (95%) create mode 100644 processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/AsOnlyTestRowsAndColumns.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatableTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticTestBase.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitionerTest.java create mode 100644 sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index d2255c31f0b..0fd90e8996f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -109,6 +109,7 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; import javax.annotation.Nullable; import java.io.File; @@ -201,6 +202,9 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule + public final TestName testName = new TestName(); + /** * Transient task failure rate emulated by the taskKiller in {@link SimpleThreadingTaskRunner}. * Per {@link SubTaskSpec}, there could be at most one task failure. @@ -239,7 +243,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase public void setUpAbstractParallelIndexSupervisorTaskTest() throws IOException { localDeepStorage = temporaryFolder.newFolder("localStorage"); - taskRunner = new SimpleThreadingTaskRunner(); + taskRunner = new SimpleThreadingTaskRunner(testName.getMethodName()); objectMapper = getObjectMapper(); indexingServiceClient = new LocalOverlordClient(objectMapper, taskRunner); intermediaryDataManager = new LocalIntermediaryDataManager( @@ -358,15 +362,14 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase public class SimpleThreadingTaskRunner { private final ConcurrentMap tasks = new ConcurrentHashMap<>(); - private final ListeningExecutorService service = MoreExecutors.listeningDecorator( - Execs.multiThreaded(5, "simple-threading-task-runner-%d") - ); + private final ListeningExecutorService service; private final ScheduledExecutorService taskKiller = Execs.scheduledSingleThreaded("simple-threading-task-killer"); private final Set killedSubtaskSpecs = new HashSet<>(); - SimpleThreadingTaskRunner() + SimpleThreadingTaskRunner(String threadNameBase) { + service = MoreExecutors.listeningDecorator(Execs.multiThreaded(5, threadNameBase + "-%d")); taskKiller.scheduleAtFixedRate( () -> { for (TaskContainer taskContainer : tasks.values()) { diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 6b26cffbc67..976b10ab7c0 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -73,12 +73,12 @@ public interface DataSource /** * Returns true if all servers have a full copy of this datasource. True for things like inline, lookup, etc, or * for queries of those. - * + *

* Currently this is coupled with joinability - if this returns true then the query engine expects there exists a * {@link org.apache.druid.segment.join.JoinableFactory} which might build a * {@link org.apache.druid.segment.join.Joinable} for this datasource directly. If a subquery 'inline' join is * required to join this datasource on the right hand side, then this value must be false for now. - * + *

* In the future, instead of directly using this method, the query planner and engine should consider * {@link org.apache.druid.segment.join.JoinableFactory#isDirectlyJoinable(DataSource)} when determining if the * right hand side is directly joinable, which would allow decoupling this property from joins. @@ -97,7 +97,7 @@ public interface DataSource /** * Returns a segment function on to how to segment should be modified. * - * @param query the input query + * @param query the input query * @param cpuTimeAcc the cpu time accumulator * @return the segment function */ diff --git a/processing/src/main/java/org/apache/druid/query/operator/LimitedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/operator/LimitedRowsAndColumns.java index d0521dbad0b..95b642f69c2 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/LimitedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/operator/LimitedRowsAndColumns.java @@ -81,9 +81,9 @@ public class LimitedRowsAndColumns implements RowsAndColumns } @Override - protected int getActualCell(int cell) + protected int getActualValue(int rowNum) { - int retVal = start + cell; + int retVal = start + rowNum; if (retVal >= end) { throw new ISE("Index out of bounds[%d] >= [%d], start[%s]", retVal, end, start); } @@ -91,7 +91,7 @@ public class LimitedRowsAndColumns implements RowsAndColumns } @Override - protected boolean outsideBounds(int cell) + protected boolean outsideBounds(int rowNum) { return false; } diff --git a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java index 8063a78b598..15b4344b4f4 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java +++ b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperator.java @@ -20,9 +20,9 @@ package org.apache.druid.query.operator; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.rowsandcols.DefaultSortedGroupPartitioner; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.rowsandcols.SortedGroupPartitioner; +import org.apache.druid.query.rowsandcols.semantic.DefaultSortedGroupPartitioner; +import org.apache.druid.query.rowsandcols.semantic.SortedGroupPartitioner; import java.util.Iterator; import java.util.List; diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java index 94e8c74a6b5..002a776fc59 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/Processor.java @@ -56,6 +56,7 @@ import org.apache.druid.query.rowsandcols.RowsAndColumns; @JsonSubTypes.Type(name = "last", value = WindowLastProcessor.class), @JsonSubTypes.Type(name = "offset", value = WindowOffsetProcessor.class), @JsonSubTypes.Type(name = "aggregate", value = WindowAggregateProcessor.class), + @JsonSubTypes.Type(name = "framedAgg", value = WindowFramedAggregateProcessor.class) }) public interface Processor { diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowAggregateProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowAggregateProcessor.java index 630d1145f77..9dcfe8631ed 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowAggregateProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowAggregateProcessor.java @@ -22,13 +22,13 @@ package org.apache.druid.query.operator.window; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; -import org.apache.druid.query.rowsandcols.DefaultOnHeapAggregatable; -import org.apache.druid.query.rowsandcols.OnHeapAggregatable; import org.apache.druid.query.rowsandcols.OnHeapCumulativeAggregatable; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ConstantObjectColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.DefaultOnHeapAggregatable; +import org.apache.druid.query.rowsandcols.semantic.OnHeapAggregatable; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java new file mode 100644 index 00000000000..e68760ac17e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java @@ -0,0 +1,123 @@ +/* + * 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.operator.window; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class WindowFrame +{ + @SuppressWarnings("unused") + public enum PeerType + { + ROWS, + RANGE + } + + // Will likely need to add the order by columns to also be able to deal with RANGE peer type. + private final PeerType peerType; + private final boolean lowerUnbounded; + private final int lowerOffset; + private final boolean upperUnbounded; + private final int upperOffset; + + @JsonCreator + public WindowFrame( + @JsonProperty("peerType") PeerType peerType, + @JsonProperty("lowUnbounded") boolean lowerUnbounded, + @JsonProperty("lowOffset") int lowerOffset, + @JsonProperty("uppUnbounded") boolean upperUnbounded, + @JsonProperty("uppOffset") int upperOffset + ) + { + this.peerType = peerType; + this.lowerUnbounded = lowerUnbounded; + this.lowerOffset = lowerOffset; + this.upperUnbounded = upperUnbounded; + this.upperOffset = upperOffset; + } + + @JsonProperty("peerType") + public PeerType getPeerType() + { + return peerType; + } + + @JsonProperty("lowUnbounded") + public boolean isLowerUnbounded() + { + return lowerUnbounded; + } + + @JsonProperty("lowOffset") + public int getLowerOffset() + { + return lowerOffset; + } + + @JsonProperty("uppUnbounded") + public boolean isUpperUnbounded() + { + return upperUnbounded; + } + + @JsonProperty("uppOffset") + public int getUpperOffset() + { + return upperOffset; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (!(o instanceof WindowFrame)) { + return false; + } + WindowFrame that = (WindowFrame) o; + return lowerUnbounded == that.lowerUnbounded + && lowerOffset == that.lowerOffset + && upperUnbounded == that.upperUnbounded + && upperOffset == that.upperOffset + && peerType == that.peerType; + } + + @Override + public int hashCode() + { + return Objects.hash(peerType, lowerUnbounded, lowerOffset, upperUnbounded, upperOffset); + } + + @Override + public String toString() + { + return "WindowFrame{" + + "peerType=" + peerType + + ", lowerUnbounded=" + lowerUnbounded + + ", lowerOffset=" + lowerOffset + + ", upperUnbounded=" + upperUnbounded + + ", upperOffset=" + upperOffset + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java new file mode 100644 index 00000000000..a67cb519407 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessor.java @@ -0,0 +1,99 @@ +/* + * 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.operator.window; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.DefaultFramedOnHeapAggregatable; +import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable; + +import javax.annotation.Nullable; +import java.util.Arrays; + +public class WindowFramedAggregateProcessor implements Processor +{ + @Nullable + private static T[] emptyToNull(T[] arr) + { + if (arr == null || arr.length == 0) { + return null; + } else { + return arr; + } + } + + private final WindowFrame frame; + private final AggregatorFactory[] aggregations; + + @JsonCreator + public WindowFramedAggregateProcessor( + @JsonProperty("frame") WindowFrame frame, + @JsonProperty("aggregations") AggregatorFactory[] aggregations + ) + { + this.frame = frame; + this.aggregations = emptyToNull(aggregations); + } + + @JsonProperty("frame") + public WindowFrame getFrame() + { + return frame; + } + + @JsonProperty("aggregations") + public AggregatorFactory[] getAggregations() + { + return aggregations; + } + + @Override + public RowsAndColumns process(RowsAndColumns inputPartition) + { + FramedOnHeapAggregatable agger = inputPartition.as(FramedOnHeapAggregatable.class); + if (agger == null) { + agger = new DefaultFramedOnHeapAggregatable(RowsAndColumns.expectAppendable(inputPartition)); + } + + return agger.aggregateAll(frame, aggregations); + } + + @Override + public boolean validateEquivalent(Processor otherProcessor) + { + if (otherProcessor instanceof WindowFramedAggregateProcessor) { + WindowFramedAggregateProcessor other = (WindowFramedAggregateProcessor) otherProcessor; + return frame.equals(other.frame) && Arrays.equals(aggregations, other.aggregations); + } + return false; + } + + + @Override + public String toString() + { + return "WindowFramedAggregateProcessor{" + + "frame=" + frame + + ", aggregations=" + Arrays.toString(aggregations) + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java index e8c9aec2ab4..541c1399e36 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessor.java @@ -23,9 +23,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import org.apache.druid.query.operator.window.Processor; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import java.util.Arrays; diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java index 105c586aa19..72ca3ad1056 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java @@ -21,11 +21,11 @@ package org.apache.druid.query.operator.window.ranking; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.query.operator.window.Processor; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; -import org.apache.druid.query.rowsandcols.DefaultSortedGroupPartitioner; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.rowsandcols.SortedGroupPartitioner; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.DefaultSortedGroupPartitioner; +import org.apache.druid.query.rowsandcols.semantic.SortedGroupPartitioner; import java.util.List; import java.util.function.Function; diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java index 6c998afa609..7821e3fd53b 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessor.java @@ -22,10 +22,10 @@ package org.apache.druid.query.operator.window.ranking; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.query.operator.window.Processor; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.segment.column.ColumnType; public class WindowRowNumberProcessor implements Processor @@ -105,7 +105,7 @@ public class WindowRowNumberProcessor implements Processor } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return Integer.compare(lhsRowNum, rhsRowNum); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/value/ShiftedColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/value/ShiftedColumnAccessorBase.java index 22288a197f2..72a6a232c6c 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/value/ShiftedColumnAccessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/value/ShiftedColumnAccessorBase.java @@ -46,7 +46,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public boolean isNull(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return true; } @@ -56,7 +56,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public Object getObject(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return null; } @@ -66,7 +66,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public double getDouble(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return 0.0D; } @@ -76,7 +76,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public float getFloat(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return 0.0F; } @@ -86,7 +86,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public long getLong(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return 0L; } @@ -96,7 +96,7 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor @Override public int getInt(int rowNum) { - final int actualCell = getActualCell(rowNum); + final int actualCell = getActualValue(rowNum); if (outsideBounds(actualCell)) { return 0; } @@ -104,10 +104,10 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { - int actualLhsCell = getActualCell(lhsRowNum); - int actualRhsCell = getActualCell(rhsRowNum); + int actualLhsCell = getActualValue(lhsRowNum); + int actualRhsCell = getActualValue(rhsRowNum); if (outsideBounds(actualLhsCell)) { if (outsideBounds(actualRhsCell)) { // Both are null @@ -119,12 +119,12 @@ public abstract class ShiftedColumnAccessorBase implements ColumnAccessor if (outsideBounds(actualRhsCell)) { return accessor.isNull(actualLhsCell) ? 0 : 1; } else { - return accessor.compareCells(actualLhsCell, actualRhsCell); + return accessor.compareRows(actualLhsCell, actualRhsCell); } } } - protected abstract int getActualCell(int cell); + protected abstract int getActualValue(int rowNum); - protected abstract boolean outsideBounds(int cell); + protected abstract boolean outsideBounds(int rowNum); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowOffsetProcessor.java b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowOffsetProcessor.java index 4128731be84..ffd44e33fe5 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowOffsetProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowOffsetProcessor.java @@ -55,15 +55,15 @@ public class WindowOffsetProcessor extends WindowValueProcessorBase new ShiftedColumnAccessorBase(column.toAccessor()) { @Override - protected int getActualCell(int cell) + protected int getActualValue(int rowNum) { - return cell + offset; + return rowNum + offset; } @Override - protected boolean outsideBounds(int actualCell) + protected boolean outsideBounds(int rowNum) { - return actualCell < 0 || actualCell >= numRows; + return rowNum < 0 || rowNum >= numRows; } })); } diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java index 5486f69b9d3..2e084ae983a 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/value/WindowValueProcessorBase.java @@ -22,9 +22,9 @@ package org.apache.druid.query.operator.window.value; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.operator.window.Processor; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import java.util.function.Function; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/frame/AppendableMapOfColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/frame/AppendableMapOfColumns.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java index 60a8d9fa531..61f6855cd01 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/frame/AppendableMapOfColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableMapOfColumns.java @@ -17,12 +17,11 @@ * under the License. */ -package org.apache.druid.query.rowsandcols.frame; +package org.apache.druid.query.rowsandcols; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; -import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import java.util.Collection; import java.util.LinkedHashMap; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 3162cdfadbf..c58e8cde30a 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -83,9 +83,9 @@ public class ArrayListRowsAndColumns implements RowsAndColumns return new ObjectColumnAccessorBase() { @Override - protected Object getVal(int cell) + protected Object getVal(int rowNum) { - return adapterForValue.apply(rows.get(cell)); + return adapterForValue.apply(rows.get(rowNum)); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/frame/MapOfColumnsRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java similarity index 94% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/frame/MapOfColumnsRowsAndColumns.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java index aee614b5142..31548c66bf6 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/frame/MapOfColumnsRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java @@ -17,13 +17,12 @@ * under the License. */ -package org.apache.druid.query.rowsandcols.frame; +package org.apache.druid.query.rowsandcols; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.rowsandcols.AppendableRowsAndColumns; -import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import java.util.Iterator; import java.util.Map; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java index c42f5c0a926..5d062a0ef97 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java @@ -20,7 +20,8 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.query.rowsandcols.column.Column; -import org.apache.druid.query.rowsandcols.frame.AppendableMapOfColumns; +import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; +import org.apache.druid.query.rowsandcols.semantic.OnHeapAggregatable; import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ColumnAccessor.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ColumnAccessor.java index acc6b5806d2..9f5959f1855 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ColumnAccessor.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ColumnAccessor.java @@ -100,5 +100,5 @@ public interface ColumnAccessor * @param rhsRowNum the cell id of the right-hand-side of the comparison * @return the result of the comparison of the two cells */ - int compareCells(int lhsRowNum, int rhsRowNum); + int compareRows(int lhsRowNum, int rhsRowNum); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java index 09aad34692d..e7c778dc8a5 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java @@ -88,7 +88,7 @@ public class ConstantObjectColumn implements Column } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return 0; } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java index c2d7547be12..860ec4d10f3 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java @@ -86,7 +86,7 @@ public class DoubleArrayColumn implements Column } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return Double.compare(lhsRowNum, rhsRowNum); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java index 10f70351efa..9e3745d153c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java @@ -86,7 +86,7 @@ public class IntArrayColumn implements Column } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return Integer.compare(vals[lhsRowNum], vals[rhsRowNum]); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessor.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessor.java index ea876c3e147..a51a861a336 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessor.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessor.java @@ -89,7 +89,7 @@ public class NullColumnAccessor implements ColumnAccessor } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return 0; } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java index fd850a0956d..25e5ed2841b 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectArrayColumn.java @@ -47,9 +47,9 @@ public class ObjectArrayColumn implements Column return new ObjectColumnAccessorBase() { @Override - protected Object getVal(int cell) + protected Object getVal(int rowNum) { - return objects[cell]; + return objects[rowNum]; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java index 8ef2aeae119..a657058f25e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ObjectColumnAccessorBase.java @@ -110,12 +110,12 @@ public abstract class ObjectColumnAccessorBase implements ColumnAccessor } @Override - public int compareCells(int lhsRowNum, int rhsRowNum) + public int compareRows(int lhsRowNum, int rhsRowNum) { return getComparator().compare(getVal(lhsRowNum), getVal(rhsRowNum)); } - protected abstract Object getVal(int cell); + protected abstract Object getVal(int rowNum); protected abstract Comparator getComparator(); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumns.java similarity index 92% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableRowsAndColumns.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumns.java index 55b197db612..8924ed1e30e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/AppendableRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumns.java @@ -17,8 +17,9 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; +import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; /** diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/ColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/ColumnSelectorFactoryMaker.java new file mode 100644 index 00000000000..14b5317244b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/ColumnSelectorFactoryMaker.java @@ -0,0 +1,39 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.ColumnSelectorFactory; + +import java.util.concurrent.atomic.AtomicInteger; + +public interface ColumnSelectorFactoryMaker +{ + static ColumnSelectorFactoryMaker fromRAC(RowsAndColumns rac) + { + ColumnSelectorFactoryMaker retVal = rac.as(ColumnSelectorFactoryMaker.class); + if (retVal == null) { + retVal = new DefaultColumnSelectorFactoryMaker(rac); + } + return retVal; + } + + ColumnSelectorFactory make(AtomicInteger rowIdProvider); +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java similarity index 76% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultOnHeapAggregatable.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java index 83aa50aee8d..ada5c18422e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultOnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java @@ -17,13 +17,12 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.aggregation.Aggregator; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.segment.BaseSingleValueDimensionSelector; @@ -38,7 +37,6 @@ import org.apache.druid.segment.serde.ComplexMetrics; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,81 +44,35 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -public class DefaultOnHeapAggregatable implements OnHeapAggregatable, OnHeapCumulativeAggregatable +public class DefaultColumnSelectorFactoryMaker implements ColumnSelectorFactoryMaker { private final RowsAndColumns rac; - public DefaultOnHeapAggregatable( - RowsAndColumns rac - ) + public DefaultColumnSelectorFactoryMaker(RowsAndColumns rac) { this.rac = rac; } @Override - public ArrayList aggregateAll( - List aggFactories - ) + public ColumnSelectorFactory make(AtomicInteger rowIdProvider) { - Aggregator[] aggs = new Aggregator[aggFactories.size()]; - - AtomicInteger currRow = new AtomicInteger(0); - int index = 0; - for (AggregatorFactory aggFactory : aggFactories) { - aggs[index++] = aggFactory.factorize(new ColumnAccessorBasedColumnSelectorFactory(currRow)); - } - - int numRows = rac.numRows(); - int rowId = currRow.get(); - while (rowId < numRows) { - for (Aggregator agg : aggs) { - agg.aggregate(); - } - rowId = currRow.incrementAndGet(); - } - - ArrayList retVal = new ArrayList<>(aggs.length); - for (Aggregator agg : aggs) { - retVal.add(agg.get()); - } - return retVal; + return new ColumnAccessorBasedColumnSelectorFactory(rowIdProvider, rac); } - @Override - public ArrayList aggregateCumulative(List aggFactories) - { - Aggregator[] aggs = new Aggregator[aggFactories.size()]; - ArrayList retVal = new ArrayList<>(aggFactories.size()); - - int numRows = rac.numRows(); - AtomicInteger currRow = new AtomicInteger(0); - int index = 0; - for (AggregatorFactory aggFactory : aggFactories) { - aggs[index++] = aggFactory.factorize(new ColumnAccessorBasedColumnSelectorFactory(currRow)); - retVal.add(new Object[numRows]); - } - - int rowId = currRow.get(); - while (rowId < numRows) { - for (int i = 0; i < aggs.length; ++i) { - aggs[i].aggregate(); - retVal.get(i)[rowId] = aggs[i].get(); - } - rowId = currRow.incrementAndGet(); - } - - return retVal; - } - - private class ColumnAccessorBasedColumnSelectorFactory implements ColumnSelectorFactory + public static class ColumnAccessorBasedColumnSelectorFactory implements ColumnSelectorFactory { private final Map accessorCache = new HashMap<>(); private final AtomicInteger cellIdSupplier; + private final RowsAndColumns rac; - public ColumnAccessorBasedColumnSelectorFactory(AtomicInteger cellIdSupplier) + public ColumnAccessorBasedColumnSelectorFactory( + AtomicInteger cellIdSupplier, + RowsAndColumns rac + ) { this.cellIdSupplier = cellIdSupplier; + this.rac = rac; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java new file mode 100644 index 00000000000..2b7798a99c6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java @@ -0,0 +1,581 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.ConstantObjectColumn; +import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicInteger; + +public class DefaultFramedOnHeapAggregatable implements FramedOnHeapAggregatable +{ + private final AppendableRowsAndColumns rac; + + public DefaultFramedOnHeapAggregatable( + AppendableRowsAndColumns rac + ) + { + this.rac = rac; + } + + @Override + public RowsAndColumns aggregateAll( + WindowFrame frame, + AggregatorFactory[] aggFactories + ) + { + if (frame.isLowerUnbounded() && frame.isUpperUnbounded()) { + return computeUnboundedAggregates(aggFactories); + } + + + if (frame.getPeerType() == WindowFrame.PeerType.ROWS) { + if (frame.isLowerUnbounded()) { + return computeCumulativeAggregates(aggFactories, frame.getUpperOffset()); + } else if (frame.isUpperUnbounded()) { + return computeReverseCumulativeAggregates(aggFactories, frame.getLowerOffset()); + } else { + final int numRows = rac.numRows(); + int lowerOffset = frame.getLowerOffset(); + int upperOffset = frame.getUpperOffset(); + + if (numRows < lowerOffset + upperOffset + 1) { + // In this case, there are not enough rows to completely build up the full window aperture before it needs to + // also start contracting the aperture because of the upper offset. So we use a method that specifically + // handles checks for both expanding and reducing the aperture on every iteration. + return aggregateWindowApertureInFlux(aggFactories, lowerOffset, upperOffset); + } else { + // In this case, there are 3 distinct phases that allow us to loop with less + // branches, so we have a method that specifically does that. + return aggregateWindowApertureWellBehaved(aggFactories, lowerOffset, upperOffset); + } + } + } else { + throw new UOE("RANGE peer groupings are unsupported"); + } + } + + private AppendableRowsAndColumns computeUnboundedAggregates(AggregatorFactory[] aggFactories) + { + Aggregator[] aggs = new Aggregator[aggFactories.length]; + + + AtomicInteger currRow = new AtomicInteger(0); + final ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactoryMaker.fromRAC(rac).make(currRow); + + for (int i = 0; i < aggFactories.length; i++) { + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + int numRows = rac.numRows(); + int rowId = currRow.get(); + while (rowId < numRows) { + for (Aggregator agg : aggs) { + agg.aggregate(); + } + rowId = currRow.incrementAndGet(); + } + + for (int i = 0; i < aggFactories.length; ++i) { + rac.addColumn( + aggFactories[i].getName(), + new ConstantObjectColumn(aggs[i].get(), numRows, aggFactories[i].getIntermediateType()) + ); + aggs[i].close(); + } + return rac; + } + + private AppendableRowsAndColumns computeCumulativeAggregates(AggregatorFactory[] aggFactories, int upperOffset) + { + int numRows = rac.numRows(); + if (upperOffset > numRows) { + return computeUnboundedAggregates(aggFactories); + } + + + // We store the results in an Object array for convenience. This is definitely sub-par from a memory management + // point of view as we should use native arrays when possible. This will be fine for now, but it probably makes + // sense to look at optimizing this in the future. That said, such an optimization might best come by having + // a specialized implementation of this interface against, say, a Frame object that can deal with arrays instead + // of trying to optimize this generic implementation. + Object[][] results = new Object[aggFactories.length][numRows]; + int resultStorageIndex = 0; + + AtomicInteger rowIdProvider = new AtomicInteger(0); + final ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactoryMaker.fromRAC(rac).make(rowIdProvider); + + AggregatorFactory[] combiningFactories = new AggregatorFactory[aggFactories.length]; + Aggregator[] aggs = new Aggregator[aggFactories.length]; + for (int i = 0; i < aggFactories.length; i++) { + combiningFactories[i] = aggFactories[i].getCombiningFactory(); + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + // If there is an upper offset, we accumulate those aggregations before starting to generate results + for (int i = 0; i < upperOffset; ++i) { + for (Aggregator agg : aggs) { + agg.aggregate(); + } + rowIdProvider.incrementAndGet(); + } + + // Prime the results + if (rowIdProvider.get() < numRows) { + for (int i = 0; i < aggs.length; i++) { + aggs[i].aggregate(); + results[i][resultStorageIndex] = aggs[i].get(); + aggs[i].close(); + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + ++resultStorageIndex; + rowIdProvider.incrementAndGet(); + } + + // From here out, we want to aggregate, peel off a row of results and then accumulate the aggregation + for (int rowId = rowIdProvider.get(); rowId < numRows; ++rowId) { + for (int i = 0; i < aggs.length; i++) { + aggs[i].aggregate(); + results[i][resultStorageIndex] = aggs[i].get(); + aggs[i].close(); + + // Use a combining aggregator to combine the result we just got with the result from the previous row + // This is a lot of hoops to jump through just to combine two values, but AggregatorFactory.combine + // allows for mutation of either of the arguments passed in, so it cannot be meaningfully used in this + // context. Instead, we have to jump through these hoops to make sure that we are generating a new object. + // It would've been nice if the AggregatorFactory interface had methods that were more usable for this, + // but it doesn't so :shrug: + final CumulativeColumnSelectorFactory combiningFactory = new CumulativeColumnSelectorFactory( + aggFactories[i], + results[i], + resultStorageIndex - 1 + ); + final Aggregator combiningAgg = combiningFactories[i].factorize(combiningFactory); + combiningAgg.aggregate(); + combiningFactory.increment(); + combiningAgg.aggregate(); + results[i][resultStorageIndex] = combiningAgg.get(); + combiningAgg.close(); + + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + ++resultStorageIndex; + rowIdProvider.incrementAndGet(); + } + + // If we haven't filled up all of the results yet, there are no more rows, so just point the rest of the results + // at the last result that we generated + for (Object[] resultArr : results) { + Arrays.fill(resultArr, resultStorageIndex, resultArr.length, resultArr[resultStorageIndex - 1]); + } + + return makeReturnRAC(aggFactories, results); + } + + private AppendableRowsAndColumns computeReverseCumulativeAggregates(AggregatorFactory[] aggFactories, int lowerOffset) + { + int numRows = rac.numRows(); + if (lowerOffset > numRows) { + return computeUnboundedAggregates(aggFactories); + } + + // We store the results in an Object array for convenience. This is definitely sub-par from a memory management + // point of view as we should use native arrays when possible. This will be fine for now, but it probably makes + // sense to look at optimizing this in the future. That said, such an optimization might best come by having + // a specialized implementation of this interface against, say, a Frame object that can deal with arrays instead + // of trying to optimize this generic implementation. + Object[][] results = new Object[aggFactories.length][numRows]; + int resultStorageIndex = numRows - 1; + + AtomicInteger rowIdProvider = new AtomicInteger(numRows - 1); + final ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactoryMaker.fromRAC(rac).make(rowIdProvider); + + AggregatorFactory[] combiningFactories = new AggregatorFactory[aggFactories.length]; + Aggregator[] aggs = new Aggregator[aggFactories.length]; + for (int i = 0; i < aggFactories.length; i++) { + combiningFactories[i] = aggFactories[i].getCombiningFactory(); + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + // If there is a lower offset, we accumulate those aggregations before starting to generate results + for (int i = 0; i < lowerOffset; ++i) { + for (Aggregator agg : aggs) { + agg.aggregate(); + } + rowIdProvider.decrementAndGet(); + } + + // Prime the results + if (rowIdProvider.get() >= 0) { + for (int i = 0; i < aggs.length; i++) { + aggs[i].aggregate(); + results[i][resultStorageIndex] = aggs[i].get(); + aggs[i].close(); + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + --resultStorageIndex; + rowIdProvider.decrementAndGet(); + } + + // From here out, we want to aggregate, peel off a row of results and then accumulate the aggregation + for (int rowId = rowIdProvider.get(); rowId >= 0; --rowId) { + for (int i = 0; i < aggs.length; i++) { + aggs[i].aggregate(); + results[i][resultStorageIndex] = aggs[i].get(); + aggs[i].close(); + + // Use a combining aggregator to combine the result we just got with the result from the previous row + // This is a lot of hoops to jump through just to combine two values, but AggregatorFactory.combine + // allows for mutation of either of the arguments passed in, so it cannot be meaningfully used in this + // context. Instead, we have to jump through these hoops to make sure that we are generating a new object. + // It would've been nice if the AggregatorFactory interface had methods that were more usable for this, + // but it doesn't so :shrug: + final CumulativeColumnSelectorFactory combiningFactory = new CumulativeColumnSelectorFactory( + aggFactories[i], + results[i], + resultStorageIndex + 1 + ); + final Aggregator combiningAgg = combiningFactories[i].factorize(combiningFactory); + combiningAgg.aggregate(); + combiningFactory.decrement(); + combiningAgg.aggregate(); + results[i][resultStorageIndex] = combiningAgg.get(); + combiningAgg.close(); + + aggs[i] = aggFactories[i].factorize(columnSelectorFactory); + } + + --resultStorageIndex; + rowIdProvider.decrementAndGet(); + } + + // If we haven't filled up all of the results yet, there are no more rows, so just point the rest of the results + // at the last result that we generated + for (Object[] resultArr : results) { + Arrays.fill(resultArr, 0, resultStorageIndex + 1, resultArr[resultStorageIndex + 1]); + } + + return makeReturnRAC(aggFactories, results); + } + + private AppendableRowsAndColumns aggregateWindowApertureWellBehaved( + AggregatorFactory[] aggFactories, + int lowerOffset, + int upperOffset + ) + { + /** + * There are 3 different phases of operation when we have more rows than our window size + * 1. Our window is not full, as we walk the rows we build up towards filling it + * 2. Our window is full, as we walk the rows we take a value off and add a new aggregation + * 3. We are nearing the end of the rows, we need to start shrinking the window aperture + */ + + int numRows = rac.numRows(); + int windowSize = lowerOffset + upperOffset + 1; + + // We store the results in an Object array for convenience. This is definitely sub-par from a memory management + // point of view as we should use native arrays when possible. This will be fine for now, but it probably makes + // sense to look at optimizing this in the future. That said, such an optimization might best come by having + // a specialized implementation of this interface against, say, a Frame object that can deal with arrays instead + // of trying to optimize this generic implementation. + Object[][] results = new Object[aggFactories.length][numRows]; + int resultStorageIndex = 0; + + AtomicInteger rowIdProvider = new AtomicInteger(0); + final ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactoryMaker.fromRAC(rac).make(rowIdProvider); + + // This is the number of aggregators to actually aggregate for the current row. + // Which also doubles as the nextIndex to roll through as we roll things in and out of the window + int nextIndex = lowerOffset + 1; + + Aggregator[][] aggregators = new Aggregator[aggFactories.length][windowSize]; + for (int i = 0; i < aggregators.length; i++) { + final AggregatorFactory aggFactory = aggFactories[i]; + // instantiate the aggregators that need to be read on the first row. + for (int j = 0; j < nextIndex; j++) { + aggregators[i][j] = aggFactory.factorize(columnSelectorFactory); + } + } + + // The first few rows will slowly build out the window to consume the upper-offset. The window will not + // be full until we have walked upperOffset number of rows, so phase 1 runs until we have consumed + // upperOffset number of rows. + for (int upperIndex = 0; upperIndex < upperOffset; ++upperIndex) { + for (Aggregator[] aggregator : aggregators) { + for (int j = 0; j < nextIndex; ++j) { + aggregator[j].aggregate(); + } + } + + for (int i = 0; i < aggFactories.length; ++i) { + aggregators[i][nextIndex] = aggFactories[i].factorize(columnSelectorFactory); + } + ++nextIndex; + rowIdProvider.incrementAndGet(); + } + + // End Phase 1, Enter Phase 2. At this point, nextIndex == windowSize, rowIdProvider is the same as + // upperOffset and the aggregators matrix is entirely non-null. We need to iterate until our window has all of + // the aggregators in it to fill up the final result set. + int endResultStorageIndex = numRows - windowSize; + for (; resultStorageIndex < endResultStorageIndex; ++resultStorageIndex) { + for (Aggregator[] aggregator : aggregators) { + for (Aggregator value : aggregator) { + value.aggregate(); + } + } + + if (nextIndex == windowSize) { + // Wrap back around and start pruning from the beginning of the window + nextIndex = 0; + } + + for (int i = 0; i < aggFactories.length; ++i) { + results[i][resultStorageIndex] = aggregators[i][nextIndex].get(); + aggregators[i][nextIndex].close(); + aggregators[i][nextIndex] = aggFactories[i].factorize(columnSelectorFactory); + } + + ++nextIndex; + rowIdProvider.incrementAndGet(); + } + + if (nextIndex == windowSize) { + nextIndex = 0; + } + + // End Phase 2, enter Phase 3. At this point, our window has enough aggregators in it to fill up our final + // result set. This means that for each new row that we complete, the window will "shrink" until we hit numRows, + // at which point we will collect anything yet remaining and be done. + + if (nextIndex != 0) { + // Start by organizing the aggregators so that we are 0-indexed from nextIndex. This trades off creating + // a new array of references in exchange for removing branches inside of the loop. It also makes the logic + // simpler to understand. + + Aggregator[][] reorganizedAggs = new Aggregator[aggFactories.length][windowSize]; + for (int i = 0; i < aggFactories.length; i++) { + System.arraycopy(aggregators[i], nextIndex, reorganizedAggs[i], 0, windowSize - nextIndex); + System.arraycopy(aggregators[i], 0, reorganizedAggs[i], windowSize - nextIndex, nextIndex); + } + aggregators = reorganizedAggs; + nextIndex = 0; + } + + for (int rowId = rowIdProvider.get(); rowId < numRows; ++rowId) { + for (Aggregator[] aggregator : aggregators) { + for (int j = nextIndex; j < aggregator.length; ++j) { + aggregator[j].aggregate(); + } + } + + for (int i = 0; i < aggFactories.length; ++i) { + results[i][resultStorageIndex] = aggregators[i][nextIndex].get(); + aggregators[i][nextIndex].close(); + aggregators[i][nextIndex] = null; + } + + ++nextIndex; + ++resultStorageIndex; + rowIdProvider.incrementAndGet(); + } + + // End Phase 3, anything left in the window needs to be collected and put into our results + for (; nextIndex < windowSize; ++nextIndex) { + for (int i = 0; i < aggFactories.length; ++i) { + results[i][resultStorageIndex] = aggregators[i][nextIndex].get(); + aggregators[i][nextIndex] = null; + } + ++resultStorageIndex; + } + + return makeReturnRAC(aggFactories, results); + } + + private AppendableRowsAndColumns aggregateWindowApertureInFlux( + AggregatorFactory[] aggFactories, + int lowerOffset, + int upperOffset + ) + { + int numRows = rac.numRows(); + int windowSize = numRows; + + // We store the results in an Object array for convenience. This is definitely sub-par from a memory management + // point of view as we should use native arrays when possible. This will be fine for now, but it probably makes + // sense to look at optimizing this in the future. That said, such an optimization might best come by having + // a specialized implementation of this interface against, say, a Frame object that can deal with arrays instead + // of trying to optimize this generic implementation. + Object[][] results = new Object[aggFactories.length][numRows]; + int resultStorageIndex = 0; + + AtomicInteger rowIdProvider = new AtomicInteger(0); + final ColumnSelectorFactory columnSelectorFactory = ColumnSelectorFactoryMaker.fromRAC(rac).make(rowIdProvider); + + Aggregator[][] aggregators = new Aggregator[aggFactories.length][windowSize]; + for (int i = 0; i < aggregators.length; i++) { + final AggregatorFactory aggFactory = aggFactories[i]; + for (int j = 0; j < aggregators[i].length; j++) { + aggregators[i][j] = aggFactory.factorize(columnSelectorFactory); + } + } + + // This is the index to stop at for the current window aperture + // The first row is used by all of the results for the lowerOffset num results, plus 1 for the "current row" + int stopIndex = Math.min(lowerOffset + 1, numRows); + + int startIndex = 0; + int rowId = rowIdProvider.get(); + while (rowId < numRows) { + for (Aggregator[] aggregator : aggregators) { + for (int j = startIndex; j < stopIndex; ++j) { + aggregator[j].aggregate(); + } + } + + if (rowId >= upperOffset) { + for (int i = 0; i < aggregators.length; ++i) { + results[i][resultStorageIndex] = aggregators[i][startIndex].get(); + aggregators[i][startIndex].close(); + aggregators[i][startIndex] = null; + } + + ++resultStorageIndex; + ++startIndex; + } + + if (stopIndex < windowSize) { + ++stopIndex; + } + rowId = rowIdProvider.incrementAndGet(); + } + + + for (; startIndex < windowSize; ++startIndex) { + for (int i = 0; i < aggregators.length; ++i) { + results[i][resultStorageIndex] = aggregators[i][startIndex].get(); + aggregators[i][startIndex].close(); + aggregators[i][startIndex] = null; + } + ++resultStorageIndex; + } + + return makeReturnRAC(aggFactories, results); + } + + private AppendableRowsAndColumns makeReturnRAC(AggregatorFactory[] aggFactories, Object[][] results) + { + for (int i = 0; i < aggFactories.length; ++i) { + rac.addColumn( + aggFactories[i].getName(), new ObjectArrayColumn(results[i], aggFactories[i].getIntermediateType()) + ); + } + return rac; + } + + private static class CumulativeColumnSelectorFactory implements ColumnSelectorFactory + { + private final ColumnCapabilitiesImpl columnCapabilities; + private final Object[] results; + + private int index; + + public CumulativeColumnSelectorFactory(AggregatorFactory factory, Object[] results, int initialIndex) + { + this.results = results; + this.index = initialIndex; + this.columnCapabilities = new ColumnCapabilitiesImpl() + .setHasBitmapIndexes(false) + .setDictionaryEncoded(false) + .setHasMultipleValues(false) + .setDictionaryValuesUnique(false) + .setFilterable(false) + .setType(factory.getIntermediateType()); + } + + public void increment() + { + ++index; + } + + public void decrement() + { + --index; + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + throw new UOE("combining factory shouldn't need dimensions, just columnValue, dim[%s]", dimensionSpec); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String columnName) + { + return new ObjectColumnSelector() + { + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Nullable + @Override + public Object getObject() + { + return results[index]; + } + + @Override + public Class classOfObject() + { + return results[index].getClass(); + } + }; + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return columnCapabilities; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultGroupPartitioner.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultGroupPartitioner.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultGroupPartitioner.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultGroupPartitioner.java index 96e559e358b..a8ffd93f36b 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultGroupPartitioner.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultGroupPartitioner.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; @@ -55,7 +56,7 @@ public class DefaultGroupPartitioner implements GroupPartitioner int prevGroupVal = 0; for (int i = 1; i < retVal.length; ++i) { if (retVal[i] == prevGroupVal) { - int comparison = accessor.compareCells(i - 1, i); + int comparison = accessor.compareRows(i - 1, i); if (comparison == 0) { retVal[i] = currGroup; continue; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultOnHeapAggregatable.java new file mode 100644 index 00000000000..9f5abbce6d9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultOnHeapAggregatable.java @@ -0,0 +1,108 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.rowsandcols.OnHeapCumulativeAggregatable; +import org.apache.druid.query.rowsandcols.RowsAndColumns; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +public class DefaultOnHeapAggregatable implements OnHeapAggregatable, OnHeapCumulativeAggregatable +{ + private final RowsAndColumns rac; + + public DefaultOnHeapAggregatable( + RowsAndColumns rac + ) + { + this.rac = rac; + } + + @Override + public ArrayList aggregateAll( + List aggFactories + ) + { + Aggregator[] aggs = new Aggregator[aggFactories.size()]; + + AtomicInteger currRow = new AtomicInteger(0); + int index = 0; + for (AggregatorFactory aggFactory : aggFactories) { + aggs[index++] = aggFactory.factorize(new DefaultColumnSelectorFactoryMaker.ColumnAccessorBasedColumnSelectorFactory( + currRow, + rac + )); + } + + int numRows = rac.numRows(); + int rowId = currRow.get(); + while (rowId < numRows) { + for (Aggregator agg : aggs) { + agg.aggregate(); + } + rowId = currRow.incrementAndGet(); + } + + ArrayList retVal = new ArrayList<>(aggs.length); + for (Aggregator agg : aggs) { + retVal.add(agg.get()); + agg.close(); + } + return retVal; + } + + @Override + public ArrayList aggregateCumulative(List aggFactories) + { + Aggregator[] aggs = new Aggregator[aggFactories.size()]; + ArrayList retVal = new ArrayList<>(aggFactories.size()); + + int numRows = rac.numRows(); + AtomicInteger currRow = new AtomicInteger(0); + int index = 0; + for (AggregatorFactory aggFactory : aggFactories) { + aggs[index++] = aggFactory.factorize(new DefaultColumnSelectorFactoryMaker.ColumnAccessorBasedColumnSelectorFactory( + currRow, + rac + )); + retVal.add(new Object[numRows]); + } + + int rowId = currRow.get(); + while (rowId < numRows) { + for (int i = 0; i < aggs.length; ++i) { + aggs[i].aggregate(); + retVal.get(i)[rowId] = aggs[i].get(); + } + rowId = currRow.incrementAndGet(); + } + + for (Aggregator agg : aggs) { + agg.close(); + } + + return retVal; + } + +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultSortedGroupPartitioner.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultSortedGroupPartitioner.java similarity index 94% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultSortedGroupPartitioner.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultSortedGroupPartitioner.java index 1470ac2da06..3fd08240678 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/DefaultSortedGroupPartitioner.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultSortedGroupPartitioner.java @@ -17,12 +17,13 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.operator.LimitedRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; @@ -61,7 +62,7 @@ public class DefaultSortedGroupPartitioner implements SortedGroupPartitioner int start = boundaries.getInt(i - 1); int end = boundaries.getInt(i); for (int j = start + 1; j < end; ++j) { - int comparison = accessor.compareCells(j - 1, j); + int comparison = accessor.compareRows(j - 1, j); if (comparison < 0) { newBoundaries.add(j); } else if (comparison > 0) { diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatable.java new file mode 100644 index 00000000000..deabe54338f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatable.java @@ -0,0 +1,57 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.rowsandcols.RowsAndColumns; + +/** + * A semantic interface used to aggregate a list of AggregatorFactories across a given set of data + *

+ * The aggregation specifically happens on-heap and should be used in places where it is known that the data + * set can be worked with entirely on-heap. There is support for frame definitions, frames aggregate certain + * subsets of rows in a rolling fashion like a windowed average. Frames are defined in terms of boundaries + * where a boundary could be based on rows or it could be based on "PEER" groupings. + *

+ * A peer grouping is defined as a set of rows that are the same based on the ORDER BY columns specified. As such + * peer-grouped values must also come with a set of ORDER BY columns. + */ +public interface FramedOnHeapAggregatable +{ + static FramedOnHeapAggregatable fromRAC(RowsAndColumns rac) + { + FramedOnHeapAggregatable retVal = rac.as(FramedOnHeapAggregatable.class); + if (retVal == null) { + retVal = new DefaultFramedOnHeapAggregatable(RowsAndColumns.expectAppendable(rac)); + } + return retVal; + } + + /** + * Aggregates the data according to the {@link WindowFrame} using the {@code AggregatorFactory} objects provided. + * + * @param frame window frame definition + * @param aggFactories definition of aggregations to be done + * @return a RowsAndColumns that contains columns representing the results of the aggregation + * from the AggregatorFactories + */ + RowsAndColumns aggregateAll(WindowFrame frame, AggregatorFactory[] aggFactories); +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/GroupPartitioner.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/GroupPartitioner.java similarity index 97% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/GroupPartitioner.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/GroupPartitioner.java index e3bae9d4284..c6b2d1be288 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/GroupPartitioner.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/GroupPartitioner.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; import java.util.List; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/OnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatable.java similarity index 97% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/OnHeapAggregatable.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatable.java index 9a707737a18..20421dfea2d 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/OnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatable.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; import org.apache.druid.query.aggregation.AggregatorFactory; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/SortedGroupPartitioner.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitioner.java similarity index 95% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/SortedGroupPartitioner.java rename to processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitioner.java index a60657937ae..b5e907716e5 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/SortedGroupPartitioner.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitioner.java @@ -17,7 +17,9 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.query.rowsandcols.semantic; + +import org.apache.druid.query.rowsandcols.RowsAndColumns; import java.util.ArrayList; import java.util.List; diff --git a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java index 5813f36630d..218fde166b8 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorTest.java @@ -23,9 +23,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java b/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java index 5d114b8bb2a..d4b0cdb1285 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/OperatorSequenceTest.java @@ -21,9 +21,9 @@ package org.apache.druid.query.operator; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java index 09f52562d4b..d980287972a 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/SequenceOperatorTest.java @@ -21,8 +21,8 @@ package org.apache.druid.query.operator; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java index 39ba5996d5d..a6f14332010 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowProcessorOperatorTest.java @@ -21,9 +21,9 @@ package org.apache.druid.query.operator; import com.google.common.collect.ImmutableMap; import org.apache.druid.query.operator.window.Processor; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java index 451ead6a939..7a5d24af031 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/RowsAndColumnsHelper.java @@ -96,6 +96,13 @@ public class RowsAndColumnsHelper return this; } + public RowsAndColumnsHelper expectColumn(String col, Object[] expectedVals, ColumnType type) + { + final ColumnHelper helper = columnHelper(col, expectedVals.length, type); + helper.setExpectation(expectedVals); + return this; + } + public ColumnHelper columnHelper(String column, int expectedSize, ColumnType expectedType) { ColumnHelper retVal = helpers.get(column); @@ -216,51 +223,43 @@ public class RowsAndColumnsHelper Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertNull(msg, accessor.getObject(i)); } + + Assert.assertEquals(msg + " is null?", expectedNulls[i], accessor.isNull(i)); if (expectedVal instanceof Float) { if (expectedNulls[i]) { - Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertEquals(msg, 0.0f, accessor.getFloat(i), 0.0); } else { - Assert.assertFalse(msg, accessor.isNull(i)); Assert.assertEquals(msg, (Float) expectedVal, accessor.getFloat(i), 0.0); } } else if (expectedVal instanceof Double) { if (expectedNulls[i]) { - Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertEquals(msg, 0.0d, accessor.getDouble(i), 0.0); } else { - Assert.assertFalse(msg, accessor.isNull(i)); Assert.assertEquals(msg, (Double) expectedVal, accessor.getDouble(i), 0.0); } } else if (expectedVal instanceof Integer) { if (expectedNulls[i]) { - Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertEquals(msg, 0, accessor.getInt(i)); } else { - Assert.assertFalse(msg, accessor.isNull(i)); Assert.assertEquals(msg, ((Integer) expectedVal).intValue(), accessor.getInt(i)); } } else if (expectedVal instanceof Long) { if (expectedNulls[i]) { - Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertEquals(msg, 0, accessor.getLong(i)); } else { - Assert.assertFalse(msg, accessor.isNull(i)); Assert.assertEquals(msg, ((Long) expectedVal).longValue(), accessor.getLong(i)); } } else { if (expectedNulls[i]) { - Assert.assertTrue(msg, accessor.isNull(i)); Assert.assertNull(msg, accessor.getObject(i)); // asserting null on the expected value is here for consistency in the tests. If it fails, it's most // likely indicative of something wrong with the test setup than the actual logic, we keep it for // sanity's sake to things consistent. - Assert.assertNull(msg, expectedVals[i]); + Assert.assertNull(msg, expectedVal); } else { final Object obj = accessor.getObject(i); - Assert.assertFalse(msg, accessor.isNull(i)); Assert.assertNotNull(msg, obj); - Assert.assertEquals(msg, expectedVals[i], obj); + Assert.assertEquals(msg, expectedVal, obj); } } } diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowAggregateProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowAggregateProcessorTest.java index 856dd0a3042..99581dbf176 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/WindowAggregateProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowAggregateProcessorTest.java @@ -25,12 +25,12 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.operator.window.ranking.WindowRowNumberProcessor; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java new file mode 100644 index 00000000000..43ab5400654 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java @@ -0,0 +1,94 @@ +/* + * 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.operator.window; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.rowsandcols.AsOnlyTestRowsAndColumns; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.NoAsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +@SuppressWarnings("unchecked") +public class WindowFramedAggregateProcessorTest +{ + @Test + public void testIsPassThruWhenRACReturnsSemanticInterface() + { + final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0); + final AggregatorFactory[] theAggs = { + new LongMaxAggregatorFactory("cummMax", "intCol"), + new DoubleSumAggregatorFactory("cummSum", "doubleCol") + }; + WindowFramedAggregateProcessor proc = new WindowFramedAggregateProcessor(theFrame, theAggs); + + final MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of( + "yay", new IntArrayColumn(new int[]{1, 2, 3}) + )); + + final RowsAndColumns processed = proc.process(new AsOnlyTestRowsAndColumns(theFrame, theAggs, rac) + { + @Nullable + @Override + public T as(Class clazz) + { + Assert.assertEquals(clazz, FramedOnHeapAggregatable.class); + return (T) (FramedOnHeapAggregatable) (frame, aggFactories) -> { + Assert.assertEquals(theFrame, frame); + Assert.assertArrayEquals(theAggs, aggFactories); + return rac; + }; + } + }); + + Assert.assertSame(rac, processed); + } + + @Test + public void testDoesStuffWhenNoSemanticInterfacesAvailable() + { + final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0); + final AggregatorFactory[] theAggs = { + new LongSumAggregatorFactory("sum", "intCol") + }; + WindowFramedAggregateProcessor proc = new WindowFramedAggregateProcessor(theFrame, theAggs); + + final MapOfColumnsRowsAndColumns rac = MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of( + "intCol", new IntArrayColumn(new int[]{1, 2, 3}) + )); + + final RowsAndColumns processed = proc.process(new NoAsRowsAndColumns(rac)); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{1, 2, 3}) + .expectColumn("sum", new int[]{1, 3, 6}) + .allColumnsRegistered() + .validate(processed); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java index 7b0bf144817..f5914e4f5db 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowCumeDistProcessorTest.java @@ -21,10 +21,10 @@ package org.apache.druid.query.operator.window.ranking; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Test; import java.util.Collections; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java index d61c40ca48e..e165f46f074 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowDenseRankProcessorTest.java @@ -21,10 +21,10 @@ package org.apache.druid.query.operator.window.ranking; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Test; import java.util.Collections; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java index cc6473f10a9..c38cd2a245c 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowPercentileProcessorTest.java @@ -22,12 +22,12 @@ package org.apache.druid.query.operator.window.ranking; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java index 3f004a6fd26..69989deeda3 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessorTest.java @@ -22,10 +22,10 @@ package org.apache.druid.query.operator.window.ranking; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Test; import java.util.Collections; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java index bc06de60e88..937fea7c360 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/ranking/WindowRowNumberProcessorTest.java @@ -21,12 +21,12 @@ package org.apache.druid.query.operator.window.ranking; import org.apache.druid.query.operator.window.Processor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java index 425ff903601..67242f05503 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowFirstProcessorTest.java @@ -21,12 +21,12 @@ package org.apache.druid.query.operator.window.value; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLagProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLagProcessorTest.java index 3079a6acea8..28c76b93f07 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLagProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLagProcessorTest.java @@ -21,12 +21,12 @@ package org.apache.druid.query.operator.window.value; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java index 2e6aabba497..5aa212b6acb 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLastProcessorTest.java @@ -21,12 +21,12 @@ package org.apache.druid.query.operator.window.value; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLeadProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLeadProcessorTest.java index fac4bf5e081..7b3c36e360e 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLeadProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/value/WindowLeadProcessorTest.java @@ -21,12 +21,12 @@ package org.apache.druid.query.operator.window.value; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.ColumnType; import org.junit.Test; diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java index 9d2d18870f5..dd492ae407f 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumnsTest.java @@ -21,17 +21,21 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.column.ColumnAccessor; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.apache.druid.segment.column.RowSignature; +import javax.annotation.Nonnull; import java.util.ArrayList; +import java.util.function.Function; -public class ArrayListRowsAndColumnsTest extends RowsAndColumnsTestBase> +public class ArrayListRowsAndColumnsTest extends RowsAndColumnsTestBase { - - @Override - public ArrayListRowsAndColumns makeRowsAndColumns(MapOfColumnsRowsAndColumns input) + public ArrayListRowsAndColumnsTest() { + super(ArrayListRowsAndColumns.class); + } + + @Nonnull + public static Function> MAKER = input -> { ArrayList rows = new ArrayList<>(input.numRows()); ArrayList cols = new ArrayList<>(input.getColumnNames()); @@ -62,5 +66,5 @@ public class ArrayListRowsAndColumnsTest extends RowsAndColumnsTestBase getColumnNames() + { + throw new UOE("not called"); + } + + @Override + public int numRows() + { + throw new UOE("not called"); + } + + @Override + public Column findColumn(String name) + { + throw new UOE("not called"); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumnsTest.java index 4e85c152f85..9ac32c999c7 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumnsTest.java @@ -21,18 +21,16 @@ package org.apache.druid.query.rowsandcols; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; import java.util.Collections; -public class MapOfColumnsRowsAndColumnsTest extends RowsAndColumnsTestBase +public class MapOfColumnsRowsAndColumnsTest extends RowsAndColumnsTestBase { - @Override - public MapOfColumnsRowsAndColumns makeRowsAndColumns(MapOfColumnsRowsAndColumns input) + public MapOfColumnsRowsAndColumnsTest() { - return input; + super(MapOfColumnsRowsAndColumns.class); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java new file mode 100644 index 00000000000..422c87c8b7c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/NoAsRowsAndColumns.java @@ -0,0 +1,61 @@ +/* + * 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.rowsandcols; + +import org.apache.druid.query.rowsandcols.column.Column; + +import javax.annotation.Nullable; +import java.util.Collection; + +public class NoAsRowsAndColumns implements RowsAndColumns +{ + private final RowsAndColumns rac; + + public NoAsRowsAndColumns(RowsAndColumns rac) + { + this.rac = rac; + } + + @Override + public Collection getColumnNames() + { + return rac.getColumnNames(); + } + + @Override + public int numRows() + { + return rac.numRows(); + } + + @Override + public Column findColumn(String name) + { + return rac.findColumn(name); + } + + @Nullable + @Override + public T as(Class clazz) + { + // Pretend like this doesn't implement any semantic interfaces + return null; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java index d0a4c30f0e6..d384989fbbf 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java @@ -19,170 +19,85 @@ package org.apache.druid.query.rowsandcols; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; -import org.apache.druid.query.aggregation.LongMinAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.operator.window.RowsAndColumnsHelper; -import org.apache.druid.query.rowsandcols.column.IntArrayColumn; -import org.apache.druid.query.rowsandcols.frame.AppendableMapOfColumns; -import org.apache.druid.query.rowsandcols.frame.MapOfColumnsRowsAndColumns; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; /** - * This base class is intended to serve as a common set of tests to validate specific RowsAndColumns implementations. + * This test base exists to enable testing of RowsAndColumns objects. When an implementation adds itself to this test + * it will automatically be tested against every semantic interface that also participates in this test suite (should + * be all of them). *

- * Different RowsAndColumns implementations will implement different of the semantic interfaces, this base class should - * test all of the possible semantic interfaces that can be implemented. By doing it this way, we can ensure that - * new RowsAndColumns implementations meet all of the corners cases and other issues that have been previously found. + * These test suites are combined a bit precariously, so there is work that the developer needs to do to make sure + * that things are wired up correctly. Specifically, a developer must register their RowsAndColumns implementation + * by adding an entry to the static {@link #getMakers()} method on this base class. The developer should *also* + * create a test class for their RowsAndColumns object that extends this class. By creating the test class that + * extends this class, there will be an extra validation done that ensures that the list of makers includes their + * RowsAndColumns class. *

- * It is expected that this base class is going to grow quite large. As it gets extra large, we could perhaps look - * into whether one of the JUnit test runners could allow us to further sub-divide the test functionality into - * semantic-interface-specific tests. The ultimate goal, however, should be that a new RowsAndColumns implementation - * can very simply take advantage of all of the tests by implementing the abstract - * {@link #makeRowsAndColumns(MapOfColumnsRowsAndColumns)} method and be done. - * - * @param + * The semantic interfaces, on the other hand, should all create a test that extends + * {@link org.apache.druid.query.rowsandcols.semantic.SemanticTestBase}. That test sets up a parameterized test, + * using the results of {@link #getMakers()} to do the parameterization. */ -public abstract class RowsAndColumnsTestBase +public abstract class RowsAndColumnsTestBase { static { NullHandling.initializeForTests(); } - public abstract T makeRowsAndColumns(MapOfColumnsRowsAndColumns input); + private final Class expectedClass; - @Test - public void testDefaultSortedGroupPartitioner() + private static final AtomicReference> MAKERS = new AtomicReference<>(); + + public static Iterable getMakers() { - T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), - "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) - ) - )); + Iterable retVal = MAKERS.get(); + if (retVal == null) { + retVal = Lists.newArrayList( + new Object[]{MapOfColumnsRowsAndColumns.class, Function.identity()}, + new Object[]{ArrayListRowsAndColumns.class, ArrayListRowsAndColumnsTest.MAKER} + ); + for (Object[] objects : retVal) { + Class aClazz = (Class) objects[0]; + final String expectedName = aClazz.getName() + "Test"; + try { + final Class testClass = Class.forName(expectedName); + if (!RowsAndColumnsTestBase.class.isAssignableFrom(testClass)) { + throw new ISE("testClass[%s] doesn't extend RowsAndColumnsTestBase, please extend it.", testClass); + } + } + catch (ClassNotFoundException e) { + throw new ISE("aClazz[%s] didn't have test class[%s], please make it", aClazz, expectedName); + } + } - validateSortedGroupPartitioner("default", new DefaultSortedGroupPartitioner(rac)); - - SortedGroupPartitioner specialized = rac.as(SortedGroupPartitioner.class); - if (specialized != null) { - validateSortedGroupPartitioner("specialized", specialized); + MAKERS.set(retVal); } + return retVal; } - private void validateSortedGroupPartitioner(String name, SortedGroupPartitioner parter) + public RowsAndColumnsTestBase( + Class expectedClass + ) { - - int[] expectedBounds = new int[]{0, 3, 5, 6, 9}; - - List expectations = Arrays.asList( - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{0, 0, 0}) - .expectColumn("unsorted", new int[]{3, 54, 21}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{1, 1}) - .expectColumn("unsorted", new int[]{1, 5}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{2}) - .expectColumn("unsorted", new int[]{54}) - .allColumnsRegistered(), - new RowsAndColumnsHelper() - .expectColumn("sorted", new int[]{4, 4, 4}) - .expectColumn("unsorted", new int[]{2, 3, 92}) - .allColumnsRegistered() - ); - - final List partCols = Collections.singletonList("sorted"); - Assert.assertArrayEquals(name, expectedBounds, parter.computeBoundaries(partCols)); - - final Iterator partedChunks = parter.partitionOnBoundaries(partCols).iterator(); - for (RowsAndColumnsHelper expectation : expectations) { - Assert.assertTrue(name, partedChunks.hasNext()); - expectation.validate(name, partedChunks.next()); - } - Assert.assertFalse(name, partedChunks.hasNext()); - - boolean exceptionThrown = false; - try { - parter.partitionOnBoundaries(Collections.singletonList("unsorted")); - } - catch (ISE ex) { - Assert.assertEquals("Pre-sorted data required, rows[1] and [2] were not in order", ex.getMessage()); - exceptionThrown = true; - } - Assert.assertTrue(exceptionThrown); + this.expectedClass = expectedClass; } @Test - public void testOnHeapAggregatable() + public void testInListOfMakers() { - T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "incremented", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), - "zeroesOut", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) - ) - )); - - validateOnHeapAggregatable("default", new DefaultOnHeapAggregatable(rac)); - - OnHeapAggregatable specialized = rac.as(OnHeapAggregatable.class); - if (specialized != null) { - validateOnHeapAggregatable("specialized", specialized); + boolean inList = false; + for (Object[] objs : getMakers()) { + if (expectedClass.equals(objs[0])) { + inList = true; + break; + } } - } - - private void validateOnHeapAggregatable(String name, OnHeapAggregatable agger) - { - final ArrayList results = agger.aggregateAll(Arrays.asList( - new LongSumAggregatorFactory("incremented", "incremented"), - new LongMaxAggregatorFactory("zeroesOutMax", "zeroesOut"), - new LongMinAggregatorFactory("zeroesOutMin", "zeroesOut") - )); - - Assert.assertEquals(name, 3, results.size()); - Assert.assertEquals(name, 55L, results.get(0)); - Assert.assertEquals(name, 82L, results.get(1)); - Assert.assertEquals(name, -90L, results.get(2)); - } - - @Test - public void testAppendableRowsAndColumns() - { - T rac = makeRowsAndColumns(MapOfColumnsRowsAndColumns.fromMap( - ImmutableMap.of( - "colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), - "colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) - ) - )); - - validateAppendableRowsAndColumns("default", new AppendableMapOfColumns(rac)); - - AppendableRowsAndColumns specialized = rac.as(AppendableRowsAndColumns.class); - if (specialized != null) { - validateAppendableRowsAndColumns("specialized", specialized); - } - } - - public void validateAppendableRowsAndColumns(String name, AppendableRowsAndColumns appender) - { - appender.addColumn("newCol", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); - - new RowsAndColumnsHelper() - .expectColumn("colA", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) - .expectColumn("colB", new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) - .expectColumn("newCol", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) - .allColumnsRegistered() - .validate(name, appender); + Assert.assertTrue(inList); } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessorTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessorTest.java index 89c286165dc..0eed413bb29 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/column/NullColumnAccessorTest.java @@ -39,7 +39,7 @@ public class NullColumnAccessorTest Assert.assertEquals(0.0, accessor.getFloat(i), 0); Assert.assertEquals(0.0, accessor.getDouble(i), 0); for (int j = 0; j < i; ++j) { - Assert.assertEquals(0, accessor.compareCells(j, i)); + Assert.assertEquals(0, accessor.compareRows(j, i)); } } } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java new file mode 100644 index 00000000000..57790ae0c0e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java @@ -0,0 +1,63 @@ +/* + * 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.rowsandcols.semantic; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Test; + +import java.util.function.Function; + +public class AppendableRowsAndColumnsTest extends SemanticTestBase +{ + public AppendableRowsAndColumnsTest( + String name, + Function fn + ) + { + super(name, fn); + } + + @Test + public void testAppendableRowsAndColumns() + { + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + "colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + ) + )); + + AppendableRowsAndColumns appender = RowsAndColumns.expectAppendable(rac); + + appender.addColumn("newCol", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10})); + + new RowsAndColumnsHelper() + .expectColumn("colA", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) + .expectColumn("colB", new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + .expectColumn("newCol", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) + .allColumnsRegistered() + .validate(appender); + } + +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java new file mode 100644 index 00000000000..0de6a069904 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java @@ -0,0 +1,459 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongMinAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; +import org.apache.druid.segment.column.ColumnType; +import org.junit.Test; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.function.Function; + +public class FramedOnHeapAggregatableTest extends SemanticTestBase +{ + + public FramedOnHeapAggregatableTest( + String name, + Function fn + ) + { + super(name, fn); + } + + @Test + public void testWindowedAggregationWindowSmallerThanRowsNoOffsets() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 0), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("maxFromInt", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowSmallerThanRows() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 1, false, 2), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{3, 6, 10, 14, 18, 22, 26, 30, 24, 17}) + .expectColumn("maxFromInt", new double[]{2, 3, 4, 5, 6, 7, 8, 9, 9, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowSmallerThanRowsOnlyUpper() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 2), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{3, 6, 9, 12, 15, 18, 21, 24, 17, 9}) + .expectColumn("maxFromInt", new double[]{2, 3, 4, 5, 6, 7, 8, 9, 9, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowSmallerThanRowsOnlyLower() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 2, false, 0), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{0, 1, 3, 6, 9, 12, 15, 18, 21, 24}) + .expectColumn("maxFromInt", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowLargerThanRows() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 5, false, 7), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{28, 36, 45, 45, 45, 45, 45, 44, 42, 39}) + .expectColumn("maxFromInt", new double[]{7, 8, 9, 9, 9, 9, 9, 9, 9, 9}) + .expectColumn("longMin", new long[]{0, 0, 0, 0, 0, 0, 1, 2, 3, 4}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationLowerLargerThanRows() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 5, false, 1), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2}) + .expectColumn("sumFromLong", new long[]{1, 3, 3}) + .expectColumn("maxFromInt", new double[]{1, 2, 2}) + .expectColumn("longMin", new long[]{0, 0, 0}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationLowerLargerThanRowsNoUpper() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 5, false, 0), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2}) + .expectColumn("sumFromLong", new long[]{0, 1, 3}) + .expectColumn("maxFromInt", new double[]{0, 1, 2}) + .expectColumn("longMin", new long[]{0, 0, 0}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationUpperLargerThanRows() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 1, false, 7), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2}) + .expectColumn("sumFromLong", new long[]{3, 3, 3}) + .expectColumn("maxFromInt", new double[]{2, 2, 2}) + .expectColumn("longMin", new long[]{0, 0, 1}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationUpperLargerThanRowsNoLower() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2}) + .expectColumn("sumFromLong", new long[]{3, 3, 2}) + .expectColumn("maxFromInt", new double[]{2, 2, 2}) + .expectColumn("longMin", new long[]{0, 1, 2}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowLargerThanRowsOnlyUpper() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{28, 36, 44, 42, 39, 35, 30, 24, 17, 9}) + .expectColumn("maxFromInt", new double[]{7, 8, 9, 9, 9, 9, 9, 9, 9, 9}) + .expectColumn("longMin", new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testWindowedAggregationWindowLargerThanRowsOnlyLower() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 5, false, 0), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new LongMinAggregatorFactory("longMin", "intCol"), + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("sumFromLong", new long[]{0, 1, 3, 6, 10, 15, 21, 27, 33, 39}) + .expectColumn("maxFromInt", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("longMin", new long[]{0, 0, 0, 0, 0, 0, 1, 2, 3, 4}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testUnboundedWindowedAggregation() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("objectCol", new ObjectArrayColumn( + new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, + ColumnType.STRING + ) + ); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("sumFromLong", "intCol"), + new LongSumAggregatorFactory("sumFromDouble", "doubleCol"), + new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), + new DoubleMaxAggregatorFactory("maxFromDouble", "doubleCol") + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("doubleCol", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("objectCol", new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, ColumnType.STRING) + .expectColumn("sumFromLong", new long[]{45, 45, 45, 45, 45, 45, 45, 45, 45, 45}) + .expectColumn("sumFromDouble", new long[]{45, 45, 45, 45, 45, 45, 45, 45, 45, 45}) + .expectColumn("maxFromInt", new double[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9}) + .expectColumn("maxFromDouble", new double[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testCumulativeAggregation() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("objectCol", new ObjectArrayColumn( + new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, + ColumnType.STRING + ) + ); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0), + new AggregatorFactory[]{ + new LongMaxAggregatorFactory("cummMax", "intCol"), + new DoubleSumAggregatorFactory("cummSum", "doubleCol") + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("doubleCol", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("objectCol", new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, ColumnType.STRING) + .expectColumn("cummMax", new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("cummSum", new double[]{0, 1, 3, 6, 10, 15, 21, 28, 36, 45}) + .allColumnsRegistered() + .validate(results); + } + + @Test + public void testReverseCumulativeAggregation() + { + Map map = new LinkedHashMap<>(); + map.put("intCol", new IntArrayColumn(new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("doubleCol", new DoubleArrayColumn(new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9})); + map.put("objectCol", new ObjectArrayColumn( + new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, + ColumnType.STRING + ) + ); + + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap(map)); + + FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); + + final RowsAndColumns results = agger.aggregateAll( + new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, true, 0), + new AggregatorFactory[]{ + new LongMaxAggregatorFactory("cummMax", "intCol"), + new DoubleSumAggregatorFactory("cummSum", "doubleCol") + } + ); + + new RowsAndColumnsHelper() + .expectColumn("intCol", new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("doubleCol", new double[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}) + .expectColumn("objectCol", new String[]{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}, ColumnType.STRING) + .expectColumn("cummMax", new long[]{9, 9, 9, 9, 9, 9, 9, 9, 9, 9}) + .expectColumn("cummSum", new double[]{45, 45, 44, 42, 39, 35, 30, 24, 17, 9}) + .allColumnsRegistered() + .validate(results); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatableTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatableTest.java new file mode 100644 index 00000000000..c8f69d4a98c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/OnHeapAggregatableTest.java @@ -0,0 +1,72 @@ +/* + * 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.rowsandcols.semantic; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongMinAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.function.Function; + +public class OnHeapAggregatableTest extends SemanticTestBase +{ + public OnHeapAggregatableTest( + String name, + Function fn + ) + { + super(name, fn); + } + + @Test + public void testOnHeapAggregatable() + { + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "incremented", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), + "zeroesOut", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + ) + )); + + OnHeapAggregatable agger = rac.as(OnHeapAggregatable.class); + if (agger == null) { + agger = new DefaultOnHeapAggregatable(rac); + } + + final ArrayList results = agger.aggregateAll(Arrays.asList( + new LongSumAggregatorFactory("incremented", "incremented"), + new LongMaxAggregatorFactory("zeroesOutMax", "zeroesOut"), + new LongMinAggregatorFactory("zeroesOutMin", "zeroesOut") + )); + + Assert.assertEquals(3, results.size()); + Assert.assertEquals(55L, results.get(0)); + Assert.assertEquals(82L, results.get(1)); + Assert.assertEquals(-90L, results.get(2)); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticTestBase.java new file mode 100644 index 00000000000..bc884dfaf37 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticTestBase.java @@ -0,0 +1,88 @@ +/* + * 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.rowsandcols.semantic; + +import com.google.common.collect.FluentIterable; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.NoAsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.function.Function; + +/** + * This base class exists to provide standard parameterization for Semantic interfaces. The idea is that the test + * will be fed a function that can be used to build a RowsAndColumns and then the test should do whatever it + * needs with the RowsAndColumns. By extending this base class, the test will end up running against every + * independent implementation of RowsAndColumns that has been registered with {@link RowsAndColumnsTestBase}. + */ +@RunWith(Parameterized.class) +public abstract class SemanticTestBase +{ + static { + NullHandling.initializeForTests(); + } + + @Parameterized.Parameters(name = "{0}") + public static Iterable parameterFeed() + { + return FluentIterable.from(RowsAndColumnsTestBase.getMakers()) + .transformAndConcat(input -> { + final String name = ((Class) input[0]).getSimpleName(); + return Arrays.asList( + new Object[]{name, input[1]}, + new Object[]{"NoAs-" + name, wrapNoAs(input[1])} + ); + }); + } + + private final String name; + private final Function fn; + + public SemanticTestBase( + String name, + Function fn + ) + { + this.name = name; + this.fn = fn; + } + + public RowsAndColumns make(MapOfColumnsRowsAndColumns rac) + { + try { + return fn.apply(rac); + } + catch (RuntimeException e) { + throw new RE(e, "using name[%s]", name); + } + } + + @SuppressWarnings("unchecked") + private static Function wrapNoAs(Object obj) + { + return ((Function) obj).andThen(NoAsRowsAndColumns::new); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitionerTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitionerTest.java new file mode 100644 index 00000000000..793cf13c2ac --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SortedGroupPartitionerTest.java @@ -0,0 +1,103 @@ +/* + * 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.rowsandcols.semantic; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; + +public class SortedGroupPartitionerTest extends SemanticTestBase +{ + public SortedGroupPartitionerTest( + String name, + Function fn + ) + { + super(name, fn); + } + + @Test + public void testDefaultSortedGroupPartitioner() + { + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "sorted", new IntArrayColumn(new int[]{0, 0, 0, 1, 1, 2, 4, 4, 4}), + "unsorted", new IntArrayColumn(new int[]{3, 54, 21, 1, 5, 54, 2, 3, 92}) + ) + )); + + SortedGroupPartitioner parter = rac.as(SortedGroupPartitioner.class); + if (parter == null) { + parter = new DefaultSortedGroupPartitioner(rac); + } + + int[] expectedBounds = new int[]{0, 3, 5, 6, 9}; + + List expectations = Arrays.asList( + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{0, 0, 0}) + .expectColumn("unsorted", new int[]{3, 54, 21}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{1, 1}) + .expectColumn("unsorted", new int[]{1, 5}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{2}) + .expectColumn("unsorted", new int[]{54}) + .allColumnsRegistered(), + new RowsAndColumnsHelper() + .expectColumn("sorted", new int[]{4, 4, 4}) + .expectColumn("unsorted", new int[]{2, 3, 92}) + .allColumnsRegistered() + ); + + final List partCols = Collections.singletonList("sorted"); + Assert.assertArrayEquals(expectedBounds, parter.computeBoundaries(partCols)); + + final Iterator partedChunks = parter.partitionOnBoundaries(partCols).iterator(); + for (RowsAndColumnsHelper expectation : expectations) { + Assert.assertTrue(partedChunks.hasNext()); + expectation.validate(partedChunks.next()); + } + Assert.assertFalse(partedChunks.hasNext()); + + boolean exceptionThrown = false; + try { + parter.partitionOnBoundaries(Collections.singletonList("unsorted")); + } + catch (ISE ex) { + Assert.assertEquals("Pre-sorted data required, rows[1] and [2] were not in order", ex.getMessage()); + exceptionThrown = true; + } + Assert.assertTrue(exceptionThrown); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 5cbffc6d2a5..891db3b70d0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -27,8 +27,10 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.core.Window; import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexWindowBound; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.operator.NaivePartitioningOperatorFactory; @@ -36,7 +38,8 @@ import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorFactory; import org.apache.druid.query.operator.window.ComposingProcessor; import org.apache.druid.query.operator.window.Processor; -import org.apache.druid.query.operator.window.WindowAggregateProcessor; +import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.operator.window.WindowFramedAggregateProcessor; import org.apache.druid.query.operator.window.ranking.WindowCumeDistProcessor; import org.apache.druid.query.operator.window.ranking.WindowDenseRankProcessor; import org.apache.druid.query.operator.window.ranking.WindowPercentileProcessor; @@ -71,11 +74,17 @@ public class Windowing .put("LEAD", (agg) -> new WindowOffsetProcessor(agg.getColumn(0), agg.getOutputName(), agg.getConstantInt(1))) .put("FIRST_VALUE", (agg) -> new WindowFirstProcessor(agg.getColumn(0), agg.getOutputName())) .put("LAST_VALUE", (agg) -> new WindowLastProcessor(agg.getColumn(0), agg.getOutputName())) - .put("CUME_DIST", (agg) -> new WindowCumeDistProcessor(agg.getOrderingColumns(), agg.getOutputName())) - .put("DENSE_RANK", (agg) -> new WindowDenseRankProcessor(agg.getOrderingColumns(), agg.getOutputName())) + .put("CUME_DIST", (agg) -> new WindowCumeDistProcessor(agg.getGroup().getOrderingColumns(), agg.getOutputName())) + .put( + "DENSE_RANK", + (agg) -> new WindowDenseRankProcessor(agg.getGroup().getOrderingColumns(), agg.getOutputName()) + ) .put("NTILE", (agg) -> new WindowPercentileProcessor(agg.getOutputName(), agg.getConstantInt(0))) - .put("PERCENT_RANK", (agg) -> new WindowRankProcessor(agg.getOrderingColumns(), agg.getOutputName(), true)) - .put("RANK", (agg) -> new WindowRankProcessor(agg.getOrderingColumns(), agg.getOutputName(), false)) + .put( + "PERCENT_RANK", + (agg) -> new WindowRankProcessor(agg.getGroup().getOrderingColumns(), agg.getOutputName(), true) + ) + .put("RANK", (agg) -> new WindowRankProcessor(agg.getGroup().getOrderingColumns(), agg.getOutputName(), false)) .put("ROW_NUMBER", (agg) -> new WindowRowNumberProcessor(agg.getOutputName())) .build(); private final List ops; @@ -90,43 +99,23 @@ public class Windowing { final Window window = Preconditions.checkNotNull(partialQuery.getWindow(), "window"); - // TODO(gianm): insert sorts and split the groups up at the rule stage; by this time, we assume there's one - // window and the dataset is already sorted appropriately. + // Right now, we assume that there is only a single grouping as our code cannot handle re-sorting and + // re-partitioning. As we relax that restriction, we will be able to plan multiple different groupings. if (window.groups.size() != 1) { plannerContext.setPlanningError("Multiple windows are not supported"); throw new CannotBuildQueryException(window); } - final Window.Group group = Iterables.getOnlyElement(window.groups); + final WindowGroup group = new WindowGroup(window, Iterables.getOnlyElement(window.groups), rowSignature); - // Window. - // TODO(gianm): Validate order-by keys instead of ignoring them. + // Presently, the order by keys are not validated to ensure that the incoming query has pre-sorted the data + // as required by the window query. This should be done. In order to do it, we will need to know what the + // sub-query that we are running against actually looks like in order to then validate that the data will + // come back in the order expected... - final List partitionColumns = new ArrayList<>(); - for (int groupKey : group.keys) { - partitionColumns.add(rowSignature.getColumnName(groupKey)); - } - - // Frame. - // TODO(gianm): Validate ROWS vs RANGE instead of ignoring it. - // TODO(gianm): Support various other kinds of frames. - if (!group.lowerBound.isUnbounded()) { - plannerContext.setPlanningError("Lower bound [%s] is not supported", group.upperBound); - throw new CannotBuildQueryException(window); - } - - final boolean cumulative; - if (group.upperBound.isUnbounded()) { - cumulative = false; - } else if (group.upperBound.isCurrentRow()) { - cumulative = true; - } else { - plannerContext.setPlanningError("Upper bound [%s] is not supported", group.upperBound); - throw new CannotBuildQueryException(window); - } // Aggregations. final String outputNamePrefix = Calcites.findUnusedPrefixForDigits("w", rowSignature.getColumnNames()); - final List aggregateCalls = group.getAggregateCalls(window); + final List aggregateCalls = group.getAggregateCalls(); final List processors = new ArrayList<>(); final List aggregations = new ArrayList<>(); @@ -149,7 +138,7 @@ public class Windowing Collections.emptyList(), aggName, aggCall, - false // TODO: finalize in a separate operator + false // Windowed aggregations don't currently finalize. This means that sketches won't work as expected. ); if (aggregation == null @@ -178,11 +167,12 @@ public class Windowing } if (!aggregations.isEmpty()) { - if (cumulative) { - processors.add(new WindowAggregateProcessor(null, aggregations)); - } else { - processors.add(new WindowAggregateProcessor(aggregations, null)); - } + processors.add( + new WindowFramedAggregateProcessor( + group.getWindowFrame(), + aggregations.toArray(new AggregatorFactory[0]) + ) + ); } if (processors.isEmpty()) { @@ -190,7 +180,7 @@ public class Windowing } final List ops = Arrays.asList( - new NaivePartitioningOperatorFactory(partitionColumns), + new NaivePartitioningOperatorFactory(group.getPartitionColumns()), new WindowOperatorFactory( processors.size() == 1 ? processors.get(0) : new ComposingProcessor(processors.toArray(new Processor[0])) @@ -229,6 +219,68 @@ public class Windowing Processor make(WindowAggregate agg); } + private static class WindowGroup + { + private final Window window; + private final RowSignature sig; + private final Window.Group group; + + public WindowGroup(Window window, Window.Group group, RowSignature sig) + { + this.window = window; + this.sig = sig; + this.group = group; + } + + public ArrayList getPartitionColumns() + { + final ArrayList retVal = new ArrayList<>(); + for (int groupKey : group.keys) { + retVal.add(sig.getColumnName(groupKey)); + } + return retVal; + } + + public ArrayList getOrderingColumns() + { + final List fields = group.orderKeys.getFieldCollations(); + ArrayList retVal = new ArrayList<>(fields.size()); + for (RelFieldCollation field : fields) { + retVal.add(sig.getColumnName(field.getFieldIndex())); + } + return retVal; + } + + public List getAggregateCalls() + { + return group.getAggregateCalls(window); + } + + public WindowFrame getWindowFrame() + { + return new WindowFrame( + WindowFrame.PeerType.ROWS, + group.lowerBound.isUnbounded(), + figureOutOffset(group.lowerBound), + group.upperBound.isUnbounded(), + figureOutOffset(group.upperBound) + ); + } + + private int figureOutOffset(RexWindowBound bound) + { + if (bound.isUnbounded() || bound.isCurrentRow()) { + return 0; + } + return getConstant(((RexInputRef) bound.getOffset()).getIndex()); + } + + private int getConstant(int refIndex) + { + return ((Number) window.constants.get(refIndex - sig.size()).getValue()).intValue(); + } + } + private static class WindowAggregate { private final String outputName; @@ -237,7 +289,7 @@ public class Windowing private final PlannerContext context; private final Project project; private final List constants; - private final Window.Group group; + private final WindowGroup group; private WindowAggregate( String outputName, @@ -246,7 +298,7 @@ public class Windowing PlannerContext context, Project project, List constants, - Window.Group group + WindowGroup group ) { this.outputName = outputName; @@ -267,14 +319,9 @@ public class Windowing return outputName; } - public ArrayList getOrderingColumns() + public WindowGroup getGroup() { - final List fields = group.orderKeys.getFieldCollations(); - ArrayList retVal = new ArrayList<>(fields.size()); - for (RelFieldCollation field : fields) { - retVal.add(sig.getColumnName(field.getFieldIndex())); - } - return retVal; + return group; } public String getColumn(int argPosition) @@ -286,8 +333,7 @@ public class Windowing public RexLiteral getConstantArgument(int argPosition) { - final Integer constantIndex = call.getArgList().get(argPosition) - sig.size(); - return constants.get(constantIndex); + return constants.get(call.getArgList().get(argPosition) - sig.size()); } public int getConstantInt(int argPosition) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index 0a12eb8c88e..7db16454e84 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -24,9 +24,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.google.common.collect.ImmutableMap; -import junitparams.JUnitParamsRunner; -import junitparams.Parameters; -import junitparams.naming.TestCaseName; import org.apache.druid.common.config.NullHandling; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; @@ -36,10 +33,10 @@ import org.apache.druid.query.operator.OperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; @@ -48,11 +45,12 @@ import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Objects; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -import java.util.regex.Pattern; -@RunWith(JUnitParamsRunner.class) +/** + * These tests are file-based, look in resources -> calcite/tests/window for the set of test specifications. + */ +@RunWith(Parameterized.class) public class CalciteWindowQueryTest extends BaseCalciteQueryTest { @@ -66,10 +64,8 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest private static final ObjectMapper YAML_JACKSON = new DefaultObjectMapper(new YAMLFactory(), "tests"); - private static final AtomicLong EXPECTED_TESTS = new AtomicLong(); - private static final AtomicLong TEST_COUNTER = new AtomicLong(); - - public Object parametersForWindowQueryTest() throws Exception + @Parameterized.Parameters(name = "{0}") + public static Object parametersForWindowQueryTest() throws Exception { final URL windowFolderUrl = ClassLoader.getSystemResource("calcite/tests/window"); File windowFolder = new File(windowFolderUrl.toURI()); @@ -77,31 +73,25 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest final File[] listedFiles = windowFolder.listFiles( pathname -> pathname.getName().toLowerCase(Locale.ROOT).endsWith(".sqltest") ); - EXPECTED_TESTS.set(listedFiles.length); - - Pattern matcher = Pattern.compile(".*"); return Arrays .stream(Objects.requireNonNull(listedFiles)) .map(File::getName) - .filter(matcher.asPredicate()) .toArray(); } - @AfterClass - public static void testRanAllTests() + private final String filename; + + public CalciteWindowQueryTest( + String filename + ) { - // This validation exists to catch issues with the filter Pattern accidentally getting checked in. It validates - // that we ran all of the tests from the directory. If this is failing, most likely, the filter Pattern in - // parametersForWindowQueryTest accidentally got checked in as something other than ".*" - Assert.assertEquals(EXPECTED_TESTS.get(), TEST_COUNTER.get()); + this.filename = filename; } @Test - @Parameters(method = "parametersForWindowQueryTest") @SuppressWarnings("unchecked") - @TestCaseName("{0}") - public void windowQueryTest(String filename) throws IOException + public void windowQueryTest() throws IOException { final Function stringManipulator; if (NullHandling.sqlCompatible()) { @@ -110,7 +100,6 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest stringManipulator = Function.identity(); } - TEST_COUNTER.incrementAndGet(); final URL systemResource = ClassLoader.getSystemResource("calcite/tests/window/" + filename); final Object objectFromYaml = YAML_JACKSON.readValue(systemResource.openStream(), Object.class); diff --git a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest index d10e710c5d4..d4affc6ec56 100644 --- a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest @@ -1,25 +1,26 @@ type: "operatorValidation" -sql: " - SELECT - FLOOR(__time TO DAY) t, - SUM(cnt) c, - SUM(SUM(cnt)) OVER (ORDER BY FLOOR(__time TO DAY)) cc - FROM foo - GROUP BY FLOOR(__time TO DAY)" +sql: | + SELECT + FLOOR(__time TO DAY) t, + SUM(cnt) c, + SUM(SUM(cnt)) OVER (ORDER BY FLOOR(__time TO DAY)) cc + FROM foo + GROUP BY FLOOR(__time TO DAY) expectedOperators: - { type: "naivePartition", partitionColumns: [ ] } - type: "window" processor: - type: "aggregate" - cumulativeAggregations: + type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: false, uppOffset: 0 } + aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } expectedResults: - - [946684800000, 1, 1] - - [946771200000, 1, 2] - - [946857600000, 1, 3] - - [978307200000, 1, 4] - - [978393600000, 1, 5] - - [978480000000, 1, 6] + - [ 946684800000, 1, 1 ] + - [ 946771200000, 1, 2 ] + - [ 946857600000, 1, 3 ] + - [ 978307200000, 1, 4 ] + - [ 978393600000, 1, 5 ] + - [ 978480000000, 1, 6 ] diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest index e87e1db391e..4e5bd48365e 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest @@ -1,1028 +1,1029 @@ type: "operatorValidation" -sql: " - SELECT - countryIsoCode, - FLOOR(__time TO HOUR) t, - SUM(delta) delta, - SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) totalDelta, - LAG(FLOOR(__time TO HOUR), 2) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) laggardTime, - LEAD(FLOOR(__time TO HOUR), 1) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) leadTime, - FIRST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS firstDelay, - LAST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS lastDelay, - NTILE(3) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayNTile, - RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRank, - PERCENT_RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRankPercent, - DENSE_RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRankDense, - CUME_DIST() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayCumeDist - FROM wikipedia - GROUP BY 1, 2 - ORDER BY 1, 3" +sql: | + SELECT + countryIsoCode, + FLOOR(__time TO HOUR) t, + SUM(delta) delta, + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) totalDelta, + LAG(FLOOR(__time TO HOUR), 2) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) laggardTime, + LEAD(FLOOR(__time TO HOUR), 1) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) leadTime, + FIRST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS firstDelay, + LAST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS lastDelay, + NTILE(3) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayNTile, + RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRank, + PERCENT_RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRankPercent, + DENSE_RANK() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayRankDense, + CUME_DIST() OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta)) AS delayCumeDist + FROM wikipedia + GROUP BY 1, 2 + ORDER BY 1, 3 expectedOperators: - - { type: "naivePartition", partitionColumns: ["d0"] } + - { type: "naivePartition", partitionColumns: [ "d0" ] } - type: "window" processor: type: "composing" processors: - - {"type":"offset", "inputColumn":"d1", "outputColumn":"w1", "offset":-2} - - {"type":"offset", "inputColumn":"d1", "outputColumn":"w2", "offset":1} - - {"type":"first", "inputColumn":"a0", "outputColumn":"w3"} - - {"type":"last", "inputColumn":"a0", "outputColumn":"w4"} - - {"type":"percentile", "outputColumn":"w5", "numBuckets":3} - - {"type":"rank", "group":["a0"], "outputColumn":"w6", "asPercent":false} - - {"type":"rank", "group":["a0"], "outputColumn":"w7", "asPercent":true} - - {"type":"denseRank", "group":["a0"], "outputColumn":"w8"} - - {"type":"cumeDist", "group":["a0"], "outputColumn":"w9"} - - type: "aggregate" - cumulativeAggregations: - - {"type":"longSum", "name":"w0", "fieldName":"a0"} + - { "type": "offset", "inputColumn": "d1", "outputColumn": "w1", "offset": -2 } + - { "type": "offset", "inputColumn": "d1", "outputColumn": "w2", "offset": 1 } + - { "type": "first", "inputColumn": "a0", "outputColumn": "w3" } + - { "type": "last", "inputColumn": "a0", "outputColumn": "w4" } + - { "type": "percentile", "outputColumn": "w5", "numBuckets": 3 } + - { "type": "rank", "group": [ "a0" ], "outputColumn": "w6", "asPercent": false } + - { "type": "rank", "group": [ "a0" ], "outputColumn": "w7", "asPercent": true } + - { "type": "denseRank", "group": [ "a0" ], "outputColumn": "w8" } + - { "type": "cumeDist", "group": [ "a0" ], "outputColumn": "w9" } + - type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: false, uppOffset: 0 } + aggregations: + - { "type": "longSum", "name": "w0", "fieldName": "a0" } expectedResults: - - ["", 1442016000000, 29873, 29873, null, 1442030400000, 29873, 787370, 1, 1, 0.0, 1, 0.041666666666666664] - - ["", 1442030400000, 166672, 196545, null, 1442019600000, 29873, 787370, 1, 2, 0.043478260869565216, 2, 0.08333333333333333] - - ["", 1442019600000, 173892, 370437, 1442016000000, 1442037600000, 29873, 787370, 1, 3, 0.08695652173913043, 3, 0.125] - - ["", 1442037600000, 200605, 571042, 1442030400000, 1442026800000, 29873, 787370, 1, 4, 0.13043478260869565, 4, 0.16666666666666666] - - ["", 1442026800000, 252626, 823668, 1442019600000, 1442098800000, 29873, 787370, 1, 5, 0.17391304347826086, 5, 0.20833333333333334] - - ["", 1442098800000, 276159, 1099827, 1442037600000, 1442055600000, 29873, 787370, 1, 6, 0.21739130434782608, 6, 0.25] - - ["", 1442055600000, 283958, 1383785, 1442026800000, 1442088000000, 29873, 787370, 1, 7, 0.2608695652173913, 7, 0.2916666666666667] - - ["", 1442088000000, 303872, 1687657, 1442098800000, 1442048400000, 29873, 787370, 1, 8, 0.30434782608695654, 8, 0.3333333333333333] - - ["", 1442048400000, 308316, 1995973, 1442055600000, 1442044800000, 29873, 787370, 2, 9, 0.34782608695652173, 9, 0.375] - - ["", 1442044800000, 316002, 2311975, 1442088000000, 1442034000000, 29873, 787370, 2, 10, 0.391304347826087, 10, 0.4166666666666667] - - ["", 1442034000000, 330957, 2642932, 1442048400000, 1442066400000, 29873, 787370, 2, 11, 0.43478260869565216, 11, 0.4583333333333333] - - ["", 1442066400000, 351584, 2994516, 1442044800000, 1442070000000, 29873, 787370, 2, 12, 0.4782608695652174, 12, 0.5] - - ["", 1442070000000, 358515, 3353031, 1442034000000, 1442084400000, 29873, 787370, 2, 13, 0.5217391304347826, 13, 0.5416666666666666] - - ["", 1442084400000, 372569, 3725600, 1442066400000, 1442095200000, 29873, 787370, 2, 14, 0.5652173913043478, 14, 0.5833333333333334] - - ["", 1442095200000, 374501, 4100101, 1442070000000, 1442073600000, 29873, 787370, 2, 15, 0.6086956521739131, 15, 0.625] - - ["", 1442073600000, 375394, 4475495, 1442084400000, 1442062800000, 29873, 787370, 2, 16, 0.6521739130434783, 16, 0.6666666666666666] - - ["", 1442062800000, 389465, 4864960, 1442095200000, 1442077200000, 29873, 787370, 3, 17, 0.6956521739130435, 17, 0.7083333333333334] - - ["", 1442077200000, 392483, 5257443, 1442073600000, 1442023200000, 29873, 787370, 3, 18, 0.7391304347826086, 18, 0.75] - - ["", 1442023200000, 399636, 5657079, 1442062800000, 1442080800000, 29873, 787370, 3, 19, 0.782608695652174, 19, 0.7916666666666666] - - ["", 1442080800000, 453077, 6110156, 1442077200000, 1442059200000, 29873, 787370, 3, 20, 0.8260869565217391, 20, 0.8333333333333334] - - ["", 1442059200000, 459297, 6569453, 1442023200000, 1442091600000, 29873, 787370, 3, 21, 0.8695652173913043, 21, 0.875] - - ["", 1442091600000, 514427, 7083880, 1442080800000, 1442041200000, 29873, 787370, 3, 22, 0.9130434782608695, 22, 0.9166666666666666] - - ["", 1442041200000, 543450, 7627330, 1442059200000, 1442052000000, 29873, 787370, 3, 23, 0.9565217391304348, 23, 0.9583333333333334] - - ["", 1442052000000, 787370, 8414700, 1442091600000, null, 29873, 787370, 3, 24, 1.0, 24, 1.0] - - ["AE", 1442059200000, -11, -11, null, 1442044800000, -11, 6323, 1, 1, 0.0, 1, 0.125] - - ["AE", 1442044800000, -7, -18, null, 1442052000000, -11, 6323, 1, 2, 0.14285714285714285, 2, 0.25] - - ["AE", 1442052000000, -3, -21, 1442059200000, 1442048400000, -11, 6323, 1, 3, 0.2857142857142857, 3, 0.375] - - ["AE", 1442048400000, 39, 18, 1442044800000, 1442080800000, -11, 6323, 2, 4, 0.42857142857142855, 4, 0.5] - - ["AE", 1442080800000, 42, 60, 1442052000000, 1442070000000, -11, 6323, 2, 5, 0.5714285714285714, 5, 0.625] - - ["AE", 1442070000000, 46, 106, 1442048400000, 1442030400000, -11, 6323, 2, 6, 0.7142857142857143, 6, 0.75] - - ["AE", 1442030400000, 118, 224, 1442080800000, 1442077200000, -11, 6323, 3, 7, 0.8571428571428571, 7, 0.875] - - ["AE", 1442077200000, 6323, 6547, 1442070000000, null, -11, 6323, 3, 8, 1.0, 8, 1.0] - - ["AL", 1442077200000, 26, 26, null, 1442091600000, 26, 54, 1, 1, 0.0, 1, 0.5] - - ["AL", 1442091600000, 54, 80, null, null, 26, 54, 2, 2, 1.0, 2, 1.0] - - ["AO", 1442041200000, -26, -26, null, 1442052000000, -26, 722, 1, 1, 0.0, 1, 0.25] - - ["AO", 1442052000000, -18, -44, null, 1442088000000, -26, 722, 1, 2, 0.3333333333333333, 2, 0.5] - - ["AO", 1442088000000, 62, 18, 1442041200000, 1442098800000, -26, 722, 2, 3, 0.6666666666666666, 3, 0.75] - - ["AO", 1442098800000, 722, 740, 1442052000000, null, -26, 722, 3, 4, 1.0, 4, 1.0] - - ["AR", 1442077200000, -591, -591, null, 1442055600000, -591, 2514, 1, 1, 0.0, 1, 0.058823529411764705] - - ["AR", 1442055600000, -54, -645, null, 1442084400000, -591, 2514, 1, 2, 0.0625, 2, 0.11764705882352941] - - ["AR", 1442084400000, -5, -650, 1442077200000, 1442030400000, -591, 2514, 1, 3, 0.125, 3, 0.17647058823529413] - - ["AR", 1442030400000, -3, -653, 1442055600000, 1442066400000, -591, 2514, 1, 4, 0.1875, 4, 0.23529411764705882] - - ["AR", 1442066400000, 0, -653, 1442084400000, 1442019600000, -591, 2514, 1, 5, 0.25, 5, 0.29411764705882354] - - ["AR", 1442019600000, 1, -652, 1442030400000, 1442080800000, -591, 2514, 1, 6, 0.3125, 6, 0.4117647058823529] - - ["AR", 1442080800000, 1, -651, 1442066400000, 1442062800000, -591, 2514, 2, 6, 0.3125, 6, 0.4117647058823529] - - ["AR", 1442062800000, 29, -622, 1442019600000, 1442098800000, -591, 2514, 2, 8, 0.4375, 7, 0.47058823529411764] - - ["AR", 1442098800000, 64, -558, 1442080800000, 1442037600000, -591, 2514, 2, 9, 0.5, 8, 0.5294117647058824] - - ["AR", 1442037600000, 81, -477, 1442062800000, 1442059200000, -591, 2514, 2, 10, 0.5625, 9, 0.5882352941176471] - - ["AR", 1442059200000, 210, -267, 1442098800000, 1442034000000, -591, 2514, 2, 11, 0.625, 10, 0.6470588235294118] - - ["AR", 1442034000000, 212, -55, 1442037600000, 1442091600000, -591, 2514, 2, 12, 0.6875, 11, 0.7058823529411765] - - ["AR", 1442091600000, 340, 285, 1442059200000, 1442070000000, -591, 2514, 3, 13, 0.75, 12, 0.7647058823529411] - - ["AR", 1442070000000, 377, 662, 1442034000000, 1442095200000, -591, 2514, 3, 14, 0.8125, 13, 0.8235294117647058] - - ["AR", 1442095200000, 630, 1292, 1442091600000, 1442026800000, -591, 2514, 3, 15, 0.875, 14, 0.8823529411764706] - - ["AR", 1442026800000, 644, 1936, 1442070000000, 1442023200000, -591, 2514, 3, 16, 0.9375, 15, 0.9411764705882353] - - ["AR", 1442023200000, 2514, 4450, 1442095200000, null, -591, 2514, 3, 17, 1.0, 16, 1.0] - - ["AT", 1442062800000, -155, -155, null, 1442084400000, -155, 7050, 1, 1, 0.0, 1, 0.14285714285714285] - - ["AT", 1442084400000, -2, -157, null, 1442066400000, -155, 7050, 1, 2, 0.16666666666666666, 2, 0.2857142857142857] - - ["AT", 1442066400000, 0, -157, 1442062800000, 1442091600000, -155, 7050, 1, 3, 0.3333333333333333, 3, 0.42857142857142855] - - ["AT", 1442091600000, 89, -68, 1442084400000, 1442070000000, -155, 7050, 2, 4, 0.5, 4, 0.5714285714285714] - - ["AT", 1442070000000, 272, 204, 1442066400000, 1442052000000, -155, 7050, 2, 5, 0.6666666666666666, 5, 0.7142857142857143] - - ["AT", 1442052000000, 4793, 4997, 1442091600000, 1442088000000, -155, 7050, 3, 6, 0.8333333333333334, 6, 0.8571428571428571] - - ["AT", 1442088000000, 7050, 12047, 1442070000000, null, -155, 7050, 3, 7, 1.0, 7, 1.0] - - ["AU", 1442052000000, -643, -643, null, 1442030400000, -643, 1138, 1, 1, 0.0, 1, 0.05263157894736842] - - ["AU", 1442030400000, -377, -1020, null, 1442066400000, -643, 1138, 1, 2, 0.05555555555555555, 2, 0.10526315789473684] - - ["AU", 1442066400000, -21, -1041, 1442052000000, 1442070000000, -643, 1138, 1, 3, 0.1111111111111111, 3, 0.15789473684210525] - - ["AU", 1442070000000, -12, -1053, 1442030400000, 1442016000000, -643, 1138, 1, 4, 0.16666666666666666, 4, 0.21052631578947367] - - ["AU", 1442016000000, 0, -1053, 1442066400000, 1442077200000, -643, 1138, 1, 5, 0.2222222222222222, 5, 0.2631578947368421] - - ["AU", 1442077200000, 1, -1052, 1442070000000, 1442037600000, -643, 1138, 1, 6, 0.2777777777777778, 6, 0.3157894736842105] - - ["AU", 1442037600000, 3, -1049, 1442016000000, 1442059200000, -643, 1138, 1, 7, 0.3333333333333333, 7, 0.3684210526315789] - - ["AU", 1442059200000, 38, -1011, 1442077200000, 1442023200000, -643, 1138, 2, 8, 0.3888888888888889, 8, 0.42105263157894735] - - ["AU", 1442023200000, 52, -959, 1442037600000, 1442048400000, -643, 1138, 2, 9, 0.4444444444444444, 9, 0.47368421052631576] - - ["AU", 1442048400000, 135, -824, 1442059200000, 1442055600000, -643, 1138, 2, 10, 0.5, 10, 0.5263157894736842] - - ["AU", 1442055600000, 182, -642, 1442023200000, 1442026800000, -643, 1138, 2, 11, 0.5555555555555556, 11, 0.5789473684210527] - - ["AU", 1442026800000, 188, -454, 1442048400000, 1442041200000, -643, 1138, 2, 12, 0.6111111111111112, 12, 0.631578947368421] - - ["AU", 1442041200000, 194, -260, 1442055600000, 1442019600000, -643, 1138, 2, 13, 0.6666666666666666, 13, 0.6842105263157895] - - ["AU", 1442019600000, 253, -7, 1442026800000, 1442034000000, -643, 1138, 3, 14, 0.7222222222222222, 14, 0.7368421052631579] - - ["AU", 1442034000000, 283, 276, 1442041200000, 1442044800000, -643, 1138, 3, 15, 0.7777777777777778, 15, 0.7894736842105263] - - ["AU", 1442044800000, 373, 649, 1442019600000, 1442095200000, -643, 1138, 3, 16, 0.8333333333333334, 16, 0.8421052631578947] - - ["AU", 1442095200000, 395, 1044, 1442034000000, 1442098800000, -643, 1138, 3, 17, 0.8888888888888888, 17, 0.8947368421052632] - - ["AU", 1442098800000, 518, 1562, 1442044800000, 1442091600000, -643, 1138, 3, 18, 0.9444444444444444, 18, 0.9473684210526315] - - ["AU", 1442091600000, 1138, 2700, 1442095200000, null, -643, 1138, 3, 19, 1.0, 19, 1.0] - - ["BA", 1442055600000, -202, -202, null, 1442048400000, -202, 38, 1, 1, 0.0, 1, 0.25] - - ["BA", 1442048400000, -13, -215, null, 1442084400000, -202, 38, 1, 2, 0.3333333333333333, 2, 0.5] - - ["BA", 1442084400000, -1, -216, 1442055600000, 1442052000000, -202, 38, 2, 3, 0.6666666666666666, 3, 0.75] - - ["BA", 1442052000000, 38, -178, 1442048400000, null, -202, 38, 3, 4, 1.0, 4, 1.0] - - ["BD", 1442091600000, -2, -2, null, 1442019600000, -2, 854, 1, 1, 0.0, 1, 0.16666666666666666] - - ["BD", 1442019600000, 0, -2, null, 1442077200000, -2, 854, 1, 2, 0.2, 2, 0.3333333333333333] - - ["BD", 1442077200000, 75, 73, 1442091600000, 1442066400000, -2, 854, 2, 3, 0.4, 3, 0.5] - - ["BD", 1442066400000, 76, 149, 1442019600000, 1442073600000, -2, 854, 2, 4, 0.6, 4, 0.6666666666666666] - - ["BD", 1442073600000, 103, 252, 1442077200000, 1442041200000, -2, 854, 3, 5, 0.8, 5, 0.8333333333333334] - - ["BD", 1442041200000, 854, 1106, 1442066400000, null, -2, 854, 3, 6, 1.0, 6, 1.0] - - ["BE", 1442030400000, -103, -103, null, 1442052000000, -103, 233, 1, 1, 0.0, 1, 0.08333333333333333] - - ["BE", 1442052000000, -1, -104, null, 1442080800000, -103, 233, 1, 2, 0.09090909090909091, 2, 0.16666666666666666] - - ["BE", 1442080800000, 1, -103, 1442030400000, 1442098800000, -103, 233, 1, 3, 0.18181818181818182, 3, 0.25] - - ["BE", 1442098800000, 9, -94, 1442052000000, 1442073600000, -103, 233, 1, 4, 0.2727272727272727, 4, 0.3333333333333333] - - ["BE", 1442073600000, 19, -75, 1442080800000, 1442048400000, -103, 233, 2, 5, 0.36363636363636365, 5, 0.4166666666666667] - - ["BE", 1442048400000, 59, -16, 1442098800000, 1442088000000, -103, 233, 2, 6, 0.45454545454545453, 6, 0.5] - - ["BE", 1442088000000, 67, 51, 1442073600000, 1442062800000, -103, 233, 2, 7, 0.5454545454545454, 7, 0.5833333333333334] - - ["BE", 1442062800000, 91, 142, 1442048400000, 1442091600000, -103, 233, 2, 8, 0.6363636363636364, 8, 0.6666666666666666] - - ["BE", 1442091600000, 101, 243, 1442088000000, 1442066400000, -103, 233, 3, 9, 0.7272727272727273, 9, 0.75] - - ["BE", 1442066400000, 136, 379, 1442062800000, 1442084400000, -103, 233, 3, 10, 0.8181818181818182, 10, 0.8333333333333334] - - ["BE", 1442084400000, 183, 562, 1442091600000, 1442055600000, -103, 233, 3, 11, 0.9090909090909091, 11, 0.9166666666666666] - - ["BE", 1442055600000, 233, 795, 1442066400000, null, -103, 233, 3, 12, 1.0, 12, 1.0] - - ["BG", 1442041200000, 9, 9, null, 1442070000000, 9, 18936, 1, 1, 0.0, 1, 0.2] - - ["BG", 1442070000000, 55, 64, null, 1442059200000, 9, 18936, 1, 2, 0.25, 2, 0.4] - - ["BG", 1442059200000, 191, 255, 1442041200000, 1442084400000, 9, 18936, 2, 3, 0.5, 3, 0.6] - - ["BG", 1442084400000, 401, 656, 1442070000000, 1442052000000, 9, 18936, 2, 4, 0.75, 4, 0.8] - - ["BG", 1442052000000, 18936, 19592, 1442059200000, null, 9, 18936, 3, 5, 1.0, 5, 1.0] - - ["BH", 1442052000000, 44, 44, null, null, 44, 44, 1, 1, 0.0, 1, 1.0] - - ["BO", 1442095200000, -4, -4, null, 1442080800000, -4, 4, 1, 1, 0.0, 1, 0.3333333333333333] - - ["BO", 1442080800000, 4, 0, null, 1442088000000, -4, 4, 2, 2, 0.5, 2, 1.0] - - ["BO", 1442088000000, 4, 4, 1442095200000, null, -4, 4, 3, 2, 0.5, 2, 1.0] - - ["BR", 1442098800000, -645, -645, null, 1442080800000, -645, 2253, 1, 1, 0.0, 1, 0.043478260869565216] - - ["BR", 1442080800000, -267, -912, null, 1442016000000, -645, 2253, 1, 2, 0.045454545454545456, 2, 0.08695652173913043] - - ["BR", 1442016000000, -248, -1160, 1442098800000, 1442041200000, -645, 2253, 1, 3, 0.09090909090909091, 3, 0.13043478260869565] - - ["BR", 1442041200000, 3, -1157, 1442080800000, 1442091600000, -645, 2253, 1, 4, 0.13636363636363635, 4, 0.17391304347826086] - - ["BR", 1442091600000, 11, -1146, 1442016000000, 1442034000000, -645, 2253, 1, 5, 0.18181818181818182, 5, 0.21739130434782608] - - ["BR", 1442034000000, 21, -1125, 1442041200000, 1442030400000, -645, 2253, 1, 6, 0.22727272727272727, 6, 0.2608695652173913] - - ["BR", 1442030400000, 30, -1095, 1442091600000, 1442026800000, -645, 2253, 1, 7, 0.2727272727272727, 7, 0.30434782608695654] - - ["BR", 1442026800000, 51, -1044, 1442034000000, 1442044800000, -645, 2253, 1, 8, 0.3181818181818182, 8, 0.34782608695652173] - - ["BR", 1442044800000, 71, -973, 1442030400000, 1442059200000, -645, 2253, 2, 9, 0.36363636363636365, 9, 0.391304347826087] - - ["BR", 1442059200000, 73, -900, 1442026800000, 1442062800000, -645, 2253, 2, 10, 0.4090909090909091, 10, 0.43478260869565216] - - ["BR", 1442062800000, 93, -807, 1442044800000, 1442088000000, -645, 2253, 2, 11, 0.45454545454545453, 11, 0.4782608695652174] - - ["BR", 1442088000000, 215, -592, 1442059200000, 1442052000000, -645, 2253, 2, 12, 0.5, 12, 0.5217391304347826] - - ["BR", 1442052000000, 232, -360, 1442062800000, 1442055600000, -645, 2253, 2, 13, 0.5454545454545454, 13, 0.5652173913043478] - - ["BR", 1442055600000, 242, -118, 1442088000000, 1442037600000, -645, 2253, 2, 14, 0.5909090909090909, 14, 0.6086956521739131] - - ["BR", 1442037600000, 267, 149, 1442052000000, 1442019600000, -645, 2253, 2, 15, 0.6363636363636364, 15, 0.6521739130434783] - - ["BR", 1442019600000, 372, 521, 1442055600000, 1442084400000, -645, 2253, 2, 16, 0.6818181818181818, 16, 0.6956521739130435] - - ["BR", 1442084400000, 492, 1013, 1442037600000, 1442070000000, -645, 2253, 3, 17, 0.7272727272727273, 17, 0.7391304347826086] - - ["BR", 1442070000000, 536, 1549, 1442019600000, 1442095200000, -645, 2253, 3, 18, 0.7727272727272727, 18, 0.782608695652174] - - ["BR", 1442095200000, 748, 2297, 1442084400000, 1442023200000, -645, 2253, 3, 19, 0.8181818181818182, 19, 0.8260869565217391] - - ["BR", 1442023200000, 879, 3176, 1442070000000, 1442066400000, -645, 2253, 3, 20, 0.8636363636363636, 20, 0.8695652173913043] - - ["BR", 1442066400000, 1034, 4210, 1442095200000, 1442073600000, -645, 2253, 3, 21, 0.9090909090909091, 21, 0.9130434782608695] - - ["BR", 1442073600000, 2087, 6297, 1442023200000, 1442077200000, -645, 2253, 3, 22, 0.9545454545454546, 22, 0.9565217391304348] - - ["BR", 1442077200000, 2253, 8550, 1442066400000, null, -645, 2253, 3, 23, 1.0, 23, 1.0] - - ["BY", 1442055600000, 1, 1, null, 1442084400000, 1, 1464, 1, 1, 0.0, 1, 0.2857142857142857] - - ["BY", 1442084400000, 1, 2, null, 1442080800000, 1, 1464, 1, 1, 0.0, 1, 0.2857142857142857] - - ["BY", 1442080800000, 28, 30, 1442055600000, 1442077200000, 1, 1464, 1, 3, 0.3333333333333333, 2, 0.42857142857142855] - - ["BY", 1442077200000, 30, 60, 1442084400000, 1442088000000, 1, 1464, 2, 4, 0.5, 3, 0.5714285714285714] - - ["BY", 1442088000000, 33, 93, 1442080800000, 1442073600000, 1, 1464, 2, 5, 0.6666666666666666, 4, 0.7142857142857143] - - ["BY", 1442073600000, 596, 689, 1442077200000, 1442059200000, 1, 1464, 3, 6, 0.8333333333333334, 5, 0.8571428571428571] - - ["BY", 1442059200000, 1464, 2153, 1442088000000, null, 1, 1464, 3, 7, 1.0, 6, 1.0] - - ["CA", 1442016000000, -371, -371, null, 1442062800000, -371, 2858, 1, 1, 0.0, 1, 0.045454545454545456] - - ["CA", 1442062800000, -367, -738, null, 1442095200000, -371, 2858, 1, 2, 0.047619047619047616, 2, 0.09090909090909091] - - ["CA", 1442095200000, -361, -1099, 1442016000000, 1442077200000, -371, 2858, 1, 3, 0.09523809523809523, 3, 0.13636363636363635] - - ["CA", 1442077200000, -282, -1381, 1442062800000, 1442037600000, -371, 2858, 1, 4, 0.14285714285714285, 4, 0.18181818181818182] - - ["CA", 1442037600000, -132, -1513, 1442095200000, 1442030400000, -371, 2858, 1, 5, 0.19047619047619047, 5, 0.22727272727272727] - - ["CA", 1442030400000, -47, -1560, 1442077200000, 1442044800000, -371, 2858, 1, 6, 0.23809523809523808, 6, 0.2727272727272727] - - ["CA", 1442044800000, 1, -1559, 1442037600000, 1442041200000, -371, 2858, 1, 7, 0.2857142857142857, 7, 0.3181818181818182] - - ["CA", 1442041200000, 5, -1554, 1442030400000, 1442088000000, -371, 2858, 1, 8, 0.3333333333333333, 8, 0.36363636363636365] - - ["CA", 1442088000000, 35, -1519, 1442044800000, 1442052000000, -371, 2858, 2, 9, 0.38095238095238093, 9, 0.4090909090909091] - - ["CA", 1442052000000, 38, -1481, 1442041200000, 1442084400000, -371, 2858, 2, 10, 0.42857142857142855, 10, 0.45454545454545453] - - ["CA", 1442084400000, 44, -1437, 1442088000000, 1442073600000, -371, 2858, 2, 11, 0.47619047619047616, 11, 0.5] - - ["CA", 1442073600000, 86, -1351, 1442052000000, 1442098800000, -371, 2858, 2, 12, 0.5238095238095238, 12, 0.5454545454545454] - - ["CA", 1442098800000, 164, -1187, 1442084400000, 1442034000000, -371, 2858, 2, 13, 0.5714285714285714, 13, 0.5909090909090909] - - ["CA", 1442034000000, 178, -1009, 1442073600000, 1442070000000, -371, 2858, 2, 14, 0.6190476190476191, 14, 0.6363636363636364] - - ["CA", 1442070000000, 185, -824, 1442098800000, 1442023200000, -371, 2858, 2, 15, 0.6666666666666666, 15, 0.6818181818181818] - - ["CA", 1442023200000, 286, -538, 1442034000000, 1442066400000, -371, 2858, 3, 16, 0.7142857142857143, 16, 0.7272727272727273] - - ["CA", 1442066400000, 307, -231, 1442070000000, 1442080800000, -371, 2858, 3, 17, 0.7619047619047619, 17, 0.7727272727272727] - - ["CA", 1442080800000, 481, 250, 1442023200000, 1442059200000, -371, 2858, 3, 18, 0.8095238095238095, 18, 0.8181818181818182] - - ["CA", 1442059200000, 1036, 1286, 1442066400000, 1442019600000, -371, 2858, 3, 19, 0.8571428571428571, 19, 0.8636363636363636] - - ["CA", 1442019600000, 2184, 3470, 1442080800000, 1442026800000, -371, 2858, 3, 20, 0.9047619047619048, 20, 0.9090909090909091] - - ["CA", 1442026800000, 2216, 5686, 1442059200000, 1442091600000, -371, 2858, 3, 21, 0.9523809523809523, 21, 0.9545454545454546] - - ["CA", 1442091600000, 2858, 8544, 1442019600000, null, -371, 2858, 3, 22, 1.0, 22, 1.0] - - ["CH", 1442044800000, -54, -54, null, 1442055600000, -54, 360, 1, 1, 0.0, 1, 0.08333333333333333] - - ["CH", 1442055600000, 0, -54, null, 1442077200000, -54, 360, 1, 2, 0.09090909090909091, 2, 0.16666666666666666] - - ["CH", 1442077200000, 6, -48, 1442044800000, 1442070000000, -54, 360, 1, 3, 0.18181818181818182, 3, 0.25] - - ["CH", 1442070000000, 11, -37, 1442055600000, 1442084400000, -54, 360, 1, 4, 0.2727272727272727, 4, 0.3333333333333333] - - ["CH", 1442084400000, 13, -24, 1442077200000, 1442062800000, -54, 360, 2, 5, 0.36363636363636365, 5, 0.4166666666666667] - - ["CH", 1442062800000, 22, -2, 1442070000000, 1442048400000, -54, 360, 2, 6, 0.45454545454545453, 6, 0.5] - - ["CH", 1442048400000, 24, 22, 1442084400000, 1442052000000, -54, 360, 2, 7, 0.5454545454545454, 7, 0.5833333333333334] - - ["CH", 1442052000000, 47, 69, 1442062800000, 1442037600000, -54, 360, 2, 8, 0.6363636363636364, 8, 0.6666666666666666] - - ["CH", 1442037600000, 59, 128, 1442048400000, 1442091600000, -54, 360, 3, 9, 0.7272727272727273, 9, 0.75] - - ["CH", 1442091600000, 67, 195, 1442052000000, 1442041200000, -54, 360, 3, 10, 0.8181818181818182, 10, 0.8333333333333334] - - ["CH", 1442041200000, 198, 393, 1442037600000, 1442073600000, -54, 360, 3, 11, 0.9090909090909091, 11, 0.9166666666666666] - - ["CH", 1442073600000, 360, 753, 1442091600000, null, -54, 360, 3, 12, 1.0, 12, 1.0] - - ["CL", 1442019600000, -370, -370, null, 1442095200000, -370, 390, 1, 1, 0.0, 1, 0.05] - - ["CL", 1442095200000, -276, -646, null, 1442066400000, -370, 390, 1, 2, 0.05263157894736842, 2, 0.1] - - ["CL", 1442066400000, -41, -687, 1442019600000, 1442077200000, -370, 390, 1, 3, 0.10526315789473684, 3, 0.15] - - ["CL", 1442077200000, -15, -702, 1442095200000, 1442059200000, -370, 390, 1, 4, 0.15789473684210525, 4, 0.2] - - ["CL", 1442059200000, -12, -714, 1442066400000, 1442034000000, -370, 390, 1, 5, 0.21052631578947367, 5, 0.25] - - ["CL", 1442034000000, -1, -715, 1442077200000, 1442041200000, -370, 390, 1, 6, 0.2631578947368421, 6, 0.35] - - ["CL", 1442041200000, -1, -716, 1442059200000, 1442037600000, -370, 390, 1, 6, 0.2631578947368421, 6, 0.35] - - ["CL", 1442037600000, 2, -714, 1442034000000, 1442098800000, -370, 390, 2, 8, 0.3684210526315789, 7, 0.4] - - ["CL", 1442098800000, 9, -705, 1442041200000, 1442070000000, -370, 390, 2, 9, 0.42105263157894735, 8, 0.45] - - ["CL", 1442070000000, 13, -692, 1442037600000, 1442023200000, -370, 390, 2, 10, 0.47368421052631576, 9, 0.5] - - ["CL", 1442023200000, 15, -677, 1442098800000, 1442062800000, -370, 390, 2, 11, 0.5263157894736842, 10, 0.55] - - ["CL", 1442062800000, 17, -660, 1442070000000, 1442080800000, -370, 390, 2, 12, 0.5789473684210527, 11, 0.65] - - ["CL", 1442080800000, 17, -643, 1442023200000, 1442091600000, -370, 390, 2, 12, 0.5789473684210527, 11, 0.65] - - ["CL", 1442091600000, 20, -623, 1442062800000, 1442030400000, -370, 390, 2, 14, 0.6842105263157895, 12, 0.7] - - ["CL", 1442030400000, 40, -583, 1442080800000, 1442084400000, -370, 390, 3, 15, 0.7368421052631579, 13, 0.75] - - ["CL", 1442084400000, 126, -457, 1442091600000, 1442073600000, -370, 390, 3, 16, 0.7894736842105263, 14, 0.8] - - ["CL", 1442073600000, 153, -304, 1442030400000, 1442016000000, -370, 390, 3, 17, 0.8421052631578947, 15, 0.85] - - ["CL", 1442016000000, 161, -143, 1442084400000, 1442088000000, -370, 390, 3, 18, 0.8947368421052632, 16, 0.9] - - ["CL", 1442088000000, 286, 143, 1442073600000, 1442052000000, -370, 390, 3, 19, 0.9473684210526315, 17, 0.95] - - ["CL", 1442052000000, 390, 533, 1442016000000, null, -370, 390, 3, 20, 1.0, 18, 1.0] - - ["CN", 1442066400000, -15, -15, null, 1442023200000, -15, 293, 1, 1, 0.0, 1, 0.1] - - ["CN", 1442023200000, -13, -28, null, 1442080800000, -15, 293, 1, 2, 0.1111111111111111, 2, 0.2] - - ["CN", 1442080800000, -10, -38, 1442066400000, 1442084400000, -15, 293, 1, 3, 0.2222222222222222, 3, 0.3] - - ["CN", 1442084400000, -1, -39, 1442023200000, 1442052000000, -15, 293, 1, 4, 0.3333333333333333, 4, 0.4] - - ["CN", 1442052000000, 0, -39, 1442080800000, 1442059200000, -15, 293, 2, 5, 0.4444444444444444, 5, 0.5] - - ["CN", 1442059200000, 8, -31, 1442084400000, 1442055600000, -15, 293, 2, 6, 0.5555555555555556, 6, 0.6] - - ["CN", 1442055600000, 69, 38, 1442052000000, 1442037600000, -15, 293, 2, 7, 0.6666666666666666, 7, 0.7] - - ["CN", 1442037600000, 98, 136, 1442059200000, 1442026800000, -15, 293, 3, 8, 0.7777777777777778, 8, 0.8] - - ["CN", 1442026800000, 154, 290, 1442055600000, 1442048400000, -15, 293, 3, 9, 0.8888888888888888, 9, 0.9] - - ["CN", 1442048400000, 293, 583, 1442037600000, null, -15, 293, 3, 10, 1.0, 10, 1.0] - - ["CO", 1442070000000, -45, -45, null, 1442023200000, -45, 39860, 1, 1, 0.0, 1, 0.06666666666666667] - - ["CO", 1442023200000, 9, -36, null, 1442019600000, -45, 39860, 1, 2, 0.07142857142857142, 2, 0.13333333333333333] - - ["CO", 1442019600000, 12, -24, 1442070000000, 1442016000000, -45, 39860, 1, 3, 0.14285714285714285, 3, 0.2] - - ["CO", 1442016000000, 16, -8, 1442023200000, 1442080800000, -45, 39860, 1, 4, 0.21428571428571427, 4, 0.26666666666666666] - - ["CO", 1442080800000, 25, 17, 1442019600000, 1442084400000, -45, 39860, 1, 5, 0.2857142857142857, 5, 0.3333333333333333] - - ["CO", 1442084400000, 51, 68, 1442016000000, 1442098800000, -45, 39860, 2, 6, 0.35714285714285715, 6, 0.4] - - ["CO", 1442098800000, 83, 151, 1442080800000, 1442066400000, -45, 39860, 2, 7, 0.42857142857142855, 7, 0.4666666666666667] - - ["CO", 1442066400000, 288, 439, 1442084400000, 1442095200000, -45, 39860, 2, 8, 0.5, 8, 0.5333333333333333] - - ["CO", 1442095200000, 290, 729, 1442098800000, 1442091600000, -45, 39860, 2, 9, 0.5714285714285714, 9, 0.6] - - ["CO", 1442091600000, 377, 1106, 1442066400000, 1442030400000, -45, 39860, 2, 10, 0.6428571428571429, 10, 0.6666666666666666] - - ["CO", 1442030400000, 441, 1547, 1442095200000, 1442059200000, -45, 39860, 3, 11, 0.7142857142857143, 11, 0.7333333333333333] - - ["CO", 1442059200000, 473, 2020, 1442091600000, 1442077200000, -45, 39860, 3, 12, 0.7857142857142857, 12, 0.8] - - ["CO", 1442077200000, 581, 2601, 1442030400000, 1442088000000, -45, 39860, 3, 13, 0.8571428571428571, 13, 0.8666666666666667] - - ["CO", 1442088000000, 17150, 19751, 1442059200000, 1442073600000, -45, 39860, 3, 14, 0.9285714285714286, 14, 0.9333333333333333] - - ["CO", 1442073600000, 39860, 59611, 1442077200000, null, -45, 39860, 3, 15, 1.0, 15, 1.0] - - ["CR", 1442041200000, 51, 51, null, 1442019600000, 51, 2497, 1, 1, 0.0, 1, 0.125] - - ["CR", 1442019600000, 62, 113, null, 1442023200000, 51, 2497, 1, 2, 0.14285714285714285, 2, 0.375] - - ["CR", 1442023200000, 62, 175, 1442041200000, 1442088000000, 51, 2497, 1, 2, 0.14285714285714285, 2, 0.375] - - ["CR", 1442088000000, 72, 247, 1442019600000, 1442026800000, 51, 2497, 2, 4, 0.42857142857142855, 3, 0.5] - - ["CR", 1442026800000, 140, 387, 1442023200000, 1442048400000, 51, 2497, 2, 5, 0.5714285714285714, 4, 0.625] - - ["CR", 1442048400000, 163, 550, 1442088000000, 1442044800000, 51, 2497, 2, 6, 0.7142857142857143, 5, 0.75] - - ["CR", 1442044800000, 194, 744, 1442026800000, 1442030400000, 51, 2497, 3, 7, 0.8571428571428571, 6, 0.875] - - ["CR", 1442030400000, 2497, 3241, 1442048400000, null, 51, 2497, 3, 8, 1.0, 7, 1.0] - - ["CZ", 1442080800000, -28, -28, null, 1442026800000, -28, 2051, 1, 1, 0.0, 1, 0.09090909090909091] - - ["CZ", 1442026800000, -19, -47, null, 1442062800000, -28, 2051, 1, 2, 0.1, 2, 0.18181818181818182] - - ["CZ", 1442062800000, 0, -47, 1442080800000, 1442098800000, -28, 2051, 1, 3, 0.2, 3, 0.2727272727272727] - - ["CZ", 1442098800000, 2, -45, 1442026800000, 1442037600000, -28, 2051, 1, 4, 0.3, 4, 0.36363636363636365] - - ["CZ", 1442037600000, 18, -27, 1442062800000, 1442059200000, -28, 2051, 2, 5, 0.4, 5, 0.45454545454545453] - - ["CZ", 1442059200000, 21, -6, 1442098800000, 1442034000000, -28, 2051, 2, 6, 0.5, 6, 0.5454545454545454] - - ["CZ", 1442034000000, 78, 72, 1442037600000, 1442077200000, -28, 2051, 2, 7, 0.6, 7, 0.6363636363636364] - - ["CZ", 1442077200000, 115, 187, 1442059200000, 1442070000000, -28, 2051, 2, 8, 0.7, 8, 0.7272727272727273] - - ["CZ", 1442070000000, 168, 355, 1442034000000, 1442055600000, -28, 2051, 3, 9, 0.8, 9, 0.8181818181818182] - - ["CZ", 1442055600000, 1073, 1428, 1442077200000, 1442073600000, -28, 2051, 3, 10, 0.9, 10, 0.9090909090909091] - - ["CZ", 1442073600000, 2051, 3479, 1442070000000, null, -28, 2051, 3, 11, 1.0, 11, 1.0] - - ["DE", 1442084400000, -125, -125, null, 1442019600000, -125, 6075, 1, 1, 0.0, 1, 0.043478260869565216] - - ["DE", 1442019600000, 0, -125, null, 1442023200000, -125, 6075, 1, 2, 0.045454545454545456, 2, 0.08695652173913043] - - ["DE", 1442023200000, 64, -61, 1442084400000, 1442016000000, -125, 6075, 1, 3, 0.09090909090909091, 3, 0.13043478260869565] - - ["DE", 1442016000000, 167, 106, 1442019600000, 1442088000000, -125, 6075, 1, 4, 0.13636363636363635, 4, 0.17391304347826086] - - ["DE", 1442088000000, 190, 296, 1442023200000, 1442041200000, -125, 6075, 1, 5, 0.18181818181818182, 5, 0.21739130434782608] - - ["DE", 1442041200000, 197, 493, 1442016000000, 1442062800000, -125, 6075, 1, 6, 0.22727272727272727, 6, 0.2608695652173913] - - ["DE", 1442062800000, 283, 776, 1442088000000, 1442059200000, -125, 6075, 1, 7, 0.2727272727272727, 7, 0.30434782608695654] - - ["DE", 1442059200000, 289, 1065, 1442041200000, 1442098800000, -125, 6075, 1, 8, 0.3181818181818182, 8, 0.34782608695652173] - - ["DE", 1442098800000, 329, 1394, 1442062800000, 1442034000000, -125, 6075, 2, 9, 0.36363636363636365, 9, 0.391304347826087] - - ["DE", 1442034000000, 358, 1752, 1442059200000, 1442030400000, -125, 6075, 2, 10, 0.4090909090909091, 10, 0.43478260869565216] - - ["DE", 1442030400000, 373, 2125, 1442098800000, 1442037600000, -125, 6075, 2, 11, 0.45454545454545453, 11, 0.4782608695652174] - - ["DE", 1442037600000, 544, 2669, 1442034000000, 1442048400000, -125, 6075, 2, 12, 0.5, 12, 0.5217391304347826] - - ["DE", 1442048400000, 811, 3480, 1442030400000, 1442044800000, -125, 6075, 2, 13, 0.5454545454545454, 13, 0.5652173913043478] - - ["DE", 1442044800000, 979, 4459, 1442037600000, 1442095200000, -125, 6075, 2, 14, 0.5909090909090909, 14, 0.6086956521739131] - - ["DE", 1442095200000, 1007, 5466, 1442048400000, 1442080800000, -125, 6075, 2, 15, 0.6363636363636364, 15, 0.6521739130434783] - - ["DE", 1442080800000, 1133, 6599, 1442044800000, 1442055600000, -125, 6075, 2, 16, 0.6818181818181818, 16, 0.6956521739130435] - - ["DE", 1442055600000, 1523, 8122, 1442095200000, 1442066400000, -125, 6075, 3, 17, 0.7272727272727273, 17, 0.7391304347826086] - - ["DE", 1442066400000, 1577, 9699, 1442080800000, 1442052000000, -125, 6075, 3, 18, 0.7727272727272727, 18, 0.782608695652174] - - ["DE", 1442052000000, 1600, 11299, 1442055600000, 1442070000000, -125, 6075, 3, 19, 0.8181818181818182, 19, 0.8260869565217391] - - ["DE", 1442070000000, 1666, 12965, 1442066400000, 1442077200000, -125, 6075, 3, 20, 0.8636363636363636, 20, 0.8695652173913043] - - ["DE", 1442077200000, 2188, 15153, 1442052000000, 1442091600000, -125, 6075, 3, 21, 0.9090909090909091, 21, 0.9130434782608695] - - ["DE", 1442091600000, 4355, 19508, 1442070000000, 1442073600000, -125, 6075, 3, 22, 0.9545454545454546, 22, 0.9565217391304348] - - ["DE", 1442073600000, 6075, 25583, 1442077200000, null, -125, 6075, 3, 23, 1.0, 23, 1.0] - - ["DK", 1442084400000, -97, -97, null, 1442077200000, -97, 416, 1, 1, 0.0, 1, 0.08333333333333333] - - ["DK", 1442077200000, -9, -106, null, 1442048400000, -97, 416, 1, 2, 0.09090909090909091, 2, 0.16666666666666666] - - ["DK", 1442048400000, -5, -111, 1442084400000, 1442059200000, -97, 416, 1, 3, 0.18181818181818182, 3, 0.25] - - ["DK", 1442059200000, 0, -111, 1442077200000, 1442095200000, -97, 416, 1, 4, 0.2727272727272727, 4, 0.4166666666666667] - - ["DK", 1442095200000, 0, -111, 1442048400000, 1442062800000, -97, 416, 2, 4, 0.2727272727272727, 4, 0.4166666666666667] - - ["DK", 1442062800000, 1, -110, 1442059200000, 1442037600000, -97, 416, 2, 6, 0.45454545454545453, 5, 0.5] - - ["DK", 1442037600000, 10, -100, 1442095200000, 1442044800000, -97, 416, 2, 7, 0.5454545454545454, 6, 0.5833333333333334] - - ["DK", 1442044800000, 36, -64, 1442062800000, 1442055600000, -97, 416, 2, 8, 0.6363636363636364, 7, 0.6666666666666666] - - ["DK", 1442055600000, 42, -22, 1442037600000, 1442080800000, -97, 416, 3, 9, 0.7272727272727273, 8, 0.75] - - ["DK", 1442080800000, 61, 39, 1442044800000, 1442091600000, -97, 416, 3, 10, 0.8181818181818182, 9, 0.8333333333333334] - - ["DK", 1442091600000, 139, 178, 1442055600000, 1442066400000, -97, 416, 3, 11, 0.9090909090909091, 10, 0.9166666666666666] - - ["DK", 1442066400000, 416, 594, 1442080800000, null, -97, 416, 3, 12, 1.0, 11, 1.0] - - ["DO", 1442023200000, 8, 8, null, 1442084400000, 8, 200, 1, 1, 0.0, 1, 0.4] - - ["DO", 1442084400000, 8, 16, null, 1442095200000, 8, 200, 1, 1, 0.0, 1, 0.4] - - ["DO", 1442095200000, 13, 29, 1442023200000, 1442066400000, 8, 200, 2, 3, 0.5, 2, 0.6] - - ["DO", 1442066400000, 35, 64, 1442084400000, 1442073600000, 8, 200, 2, 4, 0.75, 3, 0.8] - - ["DO", 1442073600000, 200, 264, 1442095200000, null, 8, 200, 3, 5, 1.0, 4, 1.0] - - ["DZ", 1442077200000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0] - - ["EC", 1442077200000, -366, -366, null, 1442023200000, -366, 568, 1, 1, 0.0, 1, 0.16666666666666666] - - ["EC", 1442023200000, -9, -375, null, 1442030400000, -366, 568, 1, 2, 0.2, 2, 0.3333333333333333] - - ["EC", 1442030400000, 0, -375, 1442077200000, 1442095200000, -366, 568, 2, 3, 0.4, 3, 0.5] - - ["EC", 1442095200000, 10, -365, 1442023200000, 1442019600000, -366, 568, 2, 4, 0.6, 4, 0.6666666666666666] - - ["EC", 1442019600000, 29, -336, 1442030400000, 1442084400000, -366, 568, 3, 5, 0.8, 5, 0.8333333333333334] - - ["EC", 1442084400000, 568, 232, 1442095200000, null, -366, 568, 3, 6, 1.0, 6, 1.0] - - ["EE", 1442044800000, -19, -19, null, 1442041200000, -19, 37, 1, 1, 0.0, 1, 0.5] - - ["EE", 1442041200000, 37, 18, null, null, -19, 37, 2, 2, 1.0, 2, 1.0] - - ["EG", 1442073600000, 1, 1, null, 1442055600000, 1, 112, 1, 1, 0.0, 1, 0.2] - - ["EG", 1442055600000, 14, 15, null, 1442026800000, 1, 112, 1, 2, 0.25, 2, 0.4] - - ["EG", 1442026800000, 16, 31, 1442073600000, 1442091600000, 1, 112, 2, 3, 0.5, 3, 0.6] - - ["EG", 1442091600000, 27, 58, 1442055600000, 1442062800000, 1, 112, 2, 4, 0.75, 4, 0.8] - - ["EG", 1442062800000, 112, 170, 1442026800000, null, 1, 112, 3, 5, 1.0, 5, 1.0] - - ["ES", 1442044800000, -169, -169, null, 1442088000000, -169, 2506, 1, 1, 0.0, 1, 0.05] - - ["ES", 1442088000000, -130, -299, null, 1442062800000, -169, 2506, 1, 2, 0.05263157894736842, 2, 0.1] - - ["ES", 1442062800000, -71, -370, 1442044800000, 1442034000000, -169, 2506, 1, 3, 0.10526315789473684, 3, 0.15] - - ["ES", 1442034000000, -52, -422, 1442088000000, 1442023200000, -169, 2506, 1, 4, 0.15789473684210525, 4, 0.2] - - ["ES", 1442023200000, -5, -427, 1442062800000, 1442052000000, -169, 2506, 1, 5, 0.21052631578947367, 5, 0.25] - - ["ES", 1442052000000, -4, -431, 1442034000000, 1442037600000, -169, 2506, 1, 6, 0.2631578947368421, 6, 0.3] - - ["ES", 1442037600000, 3, -428, 1442023200000, 1442070000000, -169, 2506, 1, 7, 0.3157894736842105, 7, 0.35] - - ["ES", 1442070000000, 61, -367, 1442052000000, 1442019600000, -169, 2506, 2, 8, 0.3684210526315789, 8, 0.4] - - ["ES", 1442019600000, 103, -264, 1442037600000, 1442041200000, -169, 2506, 2, 9, 0.42105263157894735, 9, 0.45] - - ["ES", 1442041200000, 118, -146, 1442070000000, 1442073600000, -169, 2506, 2, 10, 0.47368421052631576, 10, 0.5] - - ["ES", 1442073600000, 154, 8, 1442019600000, 1442048400000, -169, 2506, 2, 11, 0.5263157894736842, 11, 0.55] - - ["ES", 1442048400000, 158, 166, 1442041200000, 1442084400000, -169, 2506, 2, 12, 0.5789473684210527, 12, 0.6] - - ["ES", 1442084400000, 337, 503, 1442073600000, 1442098800000, -169, 2506, 2, 13, 0.631578947368421, 13, 0.65] - - ["ES", 1442098800000, 458, 961, 1442048400000, 1442066400000, -169, 2506, 2, 14, 0.6842105263157895, 14, 0.7] - - ["ES", 1442066400000, 461, 1422, 1442084400000, 1442055600000, -169, 2506, 3, 15, 0.7368421052631579, 15, 0.75] - - ["ES", 1442055600000, 495, 1917, 1442098800000, 1442091600000, -169, 2506, 3, 16, 0.7894736842105263, 16, 0.8] - - ["ES", 1442091600000, 700, 2617, 1442066400000, 1442059200000, -169, 2506, 3, 17, 0.8421052631578947, 17, 0.85] - - ["ES", 1442059200000, 1086, 3703, 1442055600000, 1442077200000, -169, 2506, 3, 18, 0.8947368421052632, 18, 0.9] - - ["ES", 1442077200000, 1240, 4943, 1442091600000, 1442095200000, -169, 2506, 3, 19, 0.9473684210526315, 19, 0.95] - - ["ES", 1442095200000, 2506, 7449, 1442059200000, null, -169, 2506, 3, 20, 1.0, 20, 1.0] - - ["FI", 1442073600000, -1, -1, null, 1442048400000, -1, 1491, 1, 1, 0.0, 1, 0.08333333333333333] - - ["FI", 1442048400000, 12, 11, null, 1442037600000, -1, 1491, 1, 2, 0.09090909090909091, 2, 0.16666666666666666] - - ["FI", 1442037600000, 14, 25, 1442073600000, 1442062800000, -1, 1491, 1, 3, 0.18181818181818182, 3, 0.25] - - ["FI", 1442062800000, 19, 44, 1442048400000, 1442095200000, -1, 1491, 1, 4, 0.2727272727272727, 4, 0.3333333333333333] - - ["FI", 1442095200000, 69, 113, 1442037600000, 1442080800000, -1, 1491, 2, 5, 0.36363636363636365, 5, 0.4166666666666667] - - ["FI", 1442080800000, 104, 217, 1442062800000, 1442066400000, -1, 1491, 2, 6, 0.45454545454545453, 6, 0.5] - - ["FI", 1442066400000, 183, 400, 1442095200000, 1442052000000, -1, 1491, 2, 7, 0.5454545454545454, 7, 0.5833333333333334] - - ["FI", 1442052000000, 186, 586, 1442080800000, 1442077200000, -1, 1491, 2, 8, 0.6363636363636364, 8, 0.6666666666666666] - - ["FI", 1442077200000, 200, 786, 1442066400000, 1442059200000, -1, 1491, 3, 9, 0.7272727272727273, 9, 0.75] - - ["FI", 1442059200000, 407, 1193, 1442052000000, 1442084400000, -1, 1491, 3, 10, 0.8181818181818182, 10, 0.8333333333333334] - - ["FI", 1442084400000, 895, 2088, 1442077200000, 1442030400000, -1, 1491, 3, 11, 0.9090909090909091, 11, 0.9166666666666666] - - ["FI", 1442030400000, 1491, 3579, 1442059200000, null, -1, 1491, 3, 12, 1.0, 12, 1.0] - - ["FR", 1442077200000, -444, -444, null, 1442016000000, -444, 6643, 1, 1, 0.0, 1, 0.043478260869565216] - - ["FR", 1442016000000, -1, -445, null, 1442026800000, -444, 6643, 1, 2, 0.045454545454545456, 2, 0.08695652173913043] - - ["FR", 1442026800000, 86, -359, 1442077200000, 1442095200000, -444, 6643, 1, 3, 0.09090909090909091, 3, 0.13043478260869565] - - ["FR", 1442095200000, 87, -272, 1442016000000, 1442098800000, -444, 6643, 1, 4, 0.13636363636363635, 4, 0.17391304347826086] - - ["FR", 1442098800000, 136, -136, 1442026800000, 1442044800000, -444, 6643, 1, 5, 0.18181818181818182, 5, 0.21739130434782608] - - ["FR", 1442044800000, 172, 36, 1442095200000, 1442055600000, -444, 6643, 1, 6, 0.22727272727272727, 6, 0.2608695652173913] - - ["FR", 1442055600000, 463, 499, 1442098800000, 1442070000000, -444, 6643, 1, 7, 0.2727272727272727, 7, 0.30434782608695654] - - ["FR", 1442070000000, 474, 973, 1442044800000, 1442034000000, -444, 6643, 1, 8, 0.3181818181818182, 8, 0.34782608695652173] - - ["FR", 1442034000000, 476, 1449, 1442055600000, 1442080800000, -444, 6643, 2, 9, 0.36363636363636365, 9, 0.391304347826087] - - ["FR", 1442080800000, 557, 2006, 1442070000000, 1442019600000, -444, 6643, 2, 10, 0.4090909090909091, 10, 0.43478260869565216] - - ["FR", 1442019600000, 585, 2591, 1442034000000, 1442041200000, -444, 6643, 2, 11, 0.45454545454545453, 11, 0.4782608695652174] - - ["FR", 1442041200000, 604, 3195, 1442080800000, 1442023200000, -444, 6643, 2, 12, 0.5, 12, 0.5217391304347826] - - ["FR", 1442023200000, 628, 3823, 1442019600000, 1442052000000, -444, 6643, 2, 13, 0.5454545454545454, 13, 0.5652173913043478] - - ["FR", 1442052000000, 637, 4460, 1442041200000, 1442091600000, -444, 6643, 2, 14, 0.5909090909090909, 14, 0.6086956521739131] - - ["FR", 1442091600000, 741, 5201, 1442023200000, 1442088000000, -444, 6643, 2, 15, 0.6363636363636364, 15, 0.6521739130434783] - - ["FR", 1442088000000, 1872, 7073, 1442052000000, 1442066400000, -444, 6643, 2, 16, 0.6818181818181818, 16, 0.6956521739130435] - - ["FR", 1442066400000, 2516, 9589, 1442091600000, 1442048400000, -444, 6643, 3, 17, 0.7272727272727273, 17, 0.7391304347826086] - - ["FR", 1442048400000, 3027, 12616, 1442088000000, 1442073600000, -444, 6643, 3, 18, 0.7727272727272727, 18, 0.782608695652174] - - ["FR", 1442073600000, 3522, 16138, 1442066400000, 1442037600000, -444, 6643, 3, 19, 0.8181818181818182, 19, 0.8260869565217391] - - ["FR", 1442037600000, 4174, 20312, 1442048400000, 1442059200000, -444, 6643, 3, 20, 0.8636363636363636, 20, 0.8695652173913043] - - ["FR", 1442059200000, 4650, 24962, 1442073600000, 1442062800000, -444, 6643, 3, 21, 0.9090909090909091, 21, 0.9130434782608695] - - ["FR", 1442062800000, 5676, 30638, 1442037600000, 1442084400000, -444, 6643, 3, 22, 0.9545454545454546, 22, 0.9565217391304348] - - ["FR", 1442084400000, 6643, 37281, 1442059200000, null, -444, 6643, 3, 23, 1.0, 23, 1.0] - - ["GB", 1442016000000, -44, -44, null, 1442034000000, -44, 16111, 1, 1, 0.0, 1, 0.041666666666666664] - - ["GB", 1442034000000, -12, -56, null, 1442044800000, -44, 16111, 1, 2, 0.043478260869565216, 2, 0.08333333333333333] - - ["GB", 1442044800000, 32, -24, 1442016000000, 1442041200000, -44, 16111, 1, 3, 0.08695652173913043, 3, 0.125] - - ["GB", 1442041200000, 42, 18, 1442034000000, 1442098800000, -44, 16111, 1, 4, 0.13043478260869565, 4, 0.16666666666666666] - - ["GB", 1442098800000, 49, 67, 1442044800000, 1442019600000, -44, 16111, 1, 5, 0.17391304347826086, 5, 0.20833333333333334] - - ["GB", 1442019600000, 54, 121, 1442041200000, 1442052000000, -44, 16111, 1, 6, 0.21739130434782608, 6, 0.25] - - ["GB", 1442052000000, 168, 289, 1442098800000, 1442095200000, -44, 16111, 1, 7, 0.2608695652173913, 7, 0.2916666666666667] - - ["GB", 1442095200000, 238, 527, 1442019600000, 1442026800000, -44, 16111, 1, 8, 0.30434782608695654, 8, 0.3333333333333333] - - ["GB", 1442026800000, 339, 866, 1442052000000, 1442070000000, -44, 16111, 2, 9, 0.34782608695652173, 9, 0.375] - - ["GB", 1442070000000, 374, 1240, 1442095200000, 1442084400000, -44, 16111, 2, 10, 0.391304347826087, 10, 0.4166666666666667] - - ["GB", 1442084400000, 384, 1624, 1442026800000, 1442055600000, -44, 16111, 2, 11, 0.43478260869565216, 11, 0.4583333333333333] - - ["GB", 1442055600000, 453, 2077, 1442070000000, 1442037600000, -44, 16111, 2, 12, 0.4782608695652174, 12, 0.5] - - ["GB", 1442037600000, 544, 2621, 1442084400000, 1442073600000, -44, 16111, 2, 13, 0.5217391304347826, 13, 0.5416666666666666] - - ["GB", 1442073600000, 648, 3269, 1442055600000, 1442066400000, -44, 16111, 2, 14, 0.5652173913043478, 14, 0.5833333333333334] - - ["GB", 1442066400000, 671, 3940, 1442037600000, 1442048400000, -44, 16111, 2, 15, 0.6086956521739131, 15, 0.625] - - ["GB", 1442048400000, 740, 4680, 1442073600000, 1442091600000, -44, 16111, 2, 16, 0.6521739130434783, 16, 0.6666666666666666] - - ["GB", 1442091600000, 811, 5491, 1442066400000, 1442077200000, -44, 16111, 3, 17, 0.6956521739130435, 17, 0.7083333333333334] - - ["GB", 1442077200000, 1135, 6626, 1442048400000, 1442080800000, -44, 16111, 3, 18, 0.7391304347826086, 18, 0.75] - - ["GB", 1442080800000, 1444, 8070, 1442091600000, 1442088000000, -44, 16111, 3, 19, 0.782608695652174, 19, 0.7916666666666666] - - ["GB", 1442088000000, 1593, 9663, 1442077200000, 1442023200000, -44, 16111, 3, 20, 0.8260869565217391, 20, 0.8333333333333334] - - ["GB", 1442023200000, 1816, 11479, 1442080800000, 1442030400000, -44, 16111, 3, 21, 0.8695652173913043, 21, 0.875] - - ["GB", 1442030400000, 2524, 14003, 1442088000000, 1442062800000, -44, 16111, 3, 22, 0.9130434782608695, 22, 0.9166666666666666] - - ["GB", 1442062800000, 5743, 19746, 1442023200000, 1442059200000, -44, 16111, 3, 23, 0.9565217391304348, 23, 0.9583333333333334] - - ["GB", 1442059200000, 16111, 35857, 1442030400000, null, -44, 16111, 3, 24, 1.0, 24, 1.0] - - ["GE", 1442052000000, -108, -108, null, 1442080800000, -108, 16, 1, 1, 0.0, 1, 0.25] - - ["GE", 1442080800000, -27, -135, null, 1442044800000, -108, 16, 1, 2, 0.3333333333333333, 2, 0.5] - - ["GE", 1442044800000, -21, -156, 1442052000000, 1442062800000, -108, 16, 2, 3, 0.6666666666666666, 3, 0.75] - - ["GE", 1442062800000, 16, -140, 1442080800000, null, -108, 16, 3, 4, 1.0, 4, 1.0] - - ["GH", 1442088000000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0] - - ["GR", 1442073600000, -314, -314, null, 1442048400000, -314, 179, 1, 1, 0.0, 1, 0.1] - - ["GR", 1442048400000, -26, -340, null, 1442034000000, -314, 179, 1, 2, 0.1111111111111111, 2, 0.2] - - ["GR", 1442034000000, 0, -340, 1442073600000, 1442070000000, -314, 179, 1, 3, 0.2222222222222222, 3, 0.3] - - ["GR", 1442070000000, 2, -338, 1442048400000, 1442041200000, -314, 179, 1, 4, 0.3333333333333333, 4, 0.4] - - ["GR", 1442041200000, 7, -331, 1442034000000, 1442062800000, -314, 179, 2, 5, 0.4444444444444444, 5, 0.5] - - ["GR", 1442062800000, 8, -323, 1442070000000, 1442019600000, -314, 179, 2, 6, 0.5555555555555556, 6, 0.6] - - ["GR", 1442019600000, 82, -241, 1442041200000, 1442080800000, -314, 179, 2, 7, 0.6666666666666666, 7, 0.7] - - ["GR", 1442080800000, 88, -153, 1442062800000, 1442091600000, -314, 179, 3, 8, 0.7777777777777778, 8, 0.8] - - ["GR", 1442091600000, 123, -30, 1442019600000, 1442084400000, -314, 179, 3, 9, 0.8888888888888888, 9, 0.9] - - ["GR", 1442084400000, 179, 149, 1442080800000, null, -314, 179, 3, 10, 1.0, 10, 1.0] - - ["GT", 1442023200000, -167, -167, null, 1442098800000, -167, 173, 1, 1, 0.0, 1, 0.3333333333333333] - - ["GT", 1442098800000, 1, -166, null, 1442026800000, -167, 173, 2, 2, 0.5, 2, 0.6666666666666666] - - ["GT", 1442026800000, 173, 7, 1442023200000, null, -167, 173, 3, 3, 1.0, 3, 1.0] - - ["HK", 1442026800000, -211, -211, null, 1442019600000, -211, 5545, 1, 1, 0.0, 1, 0.05263157894736842] - - ["HK", 1442019600000, -113, -324, null, 1442041200000, -211, 5545, 1, 2, 0.05555555555555555, 2, 0.10526315789473684] - - ["HK", 1442041200000, -15, -339, 1442026800000, 1442091600000, -211, 5545, 1, 3, 0.1111111111111111, 3, 0.15789473684210525] - - ["HK", 1442091600000, -3, -342, 1442019600000, 1442095200000, -211, 5545, 1, 4, 0.16666666666666666, 4, 0.21052631578947367] - - ["HK", 1442095200000, -1, -343, 1442041200000, 1442080800000, -211, 5545, 1, 5, 0.2222222222222222, 5, 0.2631578947368421] - - ["HK", 1442080800000, 0, -343, 1442091600000, 1442048400000, -211, 5545, 1, 6, 0.2777777777777778, 6, 0.3157894736842105] - - ["HK", 1442048400000, 1, -342, 1442095200000, 1442062800000, -211, 5545, 1, 7, 0.3333333333333333, 7, 0.42105263157894735] - - ["HK", 1442062800000, 1, -341, 1442080800000, 1442059200000, -211, 5545, 2, 7, 0.3333333333333333, 7, 0.42105263157894735] - - ["HK", 1442059200000, 2, -339, 1442048400000, 1442052000000, -211, 5545, 2, 9, 0.4444444444444444, 8, 0.47368421052631576] - - ["HK", 1442052000000, 15, -324, 1442062800000, 1442044800000, -211, 5545, 2, 10, 0.5, 9, 0.5263157894736842] - - ["HK", 1442044800000, 21, -303, 1442059200000, 1442066400000, -211, 5545, 2, 11, 0.5555555555555556, 10, 0.5789473684210527] - - ["HK", 1442066400000, 39, -264, 1442052000000, 1442030400000, -211, 5545, 2, 12, 0.6111111111111112, 11, 0.631578947368421] - - ["HK", 1442030400000, 157, -107, 1442044800000, 1442070000000, -211, 5545, 2, 13, 0.6666666666666666, 12, 0.6842105263157895] - - ["HK", 1442070000000, 314, 207, 1442066400000, 1442037600000, -211, 5545, 3, 14, 0.7222222222222222, 13, 0.7368421052631579] - - ["HK", 1442037600000, 636, 843, 1442030400000, 1442055600000, -211, 5545, 3, 15, 0.7777777777777778, 14, 0.7894736842105263] - - ["HK", 1442055600000, 804, 1647, 1442070000000, 1442034000000, -211, 5545, 3, 16, 0.8333333333333334, 15, 0.8421052631578947] - - ["HK", 1442034000000, 1137, 2784, 1442037600000, 1442023200000, -211, 5545, 3, 17, 0.8888888888888888, 16, 0.8947368421052632] - - ["HK", 1442023200000, 2414, 5198, 1442055600000, 1442073600000, -211, 5545, 3, 18, 0.9444444444444444, 17, 0.9473684210526315] - - ["HK", 1442073600000, 5545, 10743, 1442034000000, null, -211, 5545, 3, 19, 1.0, 18, 1.0] - - ["HN", 1442026800000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0] - - ["HR", 1442084400000, -10, -10, null, 1442073600000, -10, 220, 1, 1, 0.0, 1, 0.16666666666666666] - - ["HR", 1442073600000, 0, -10, null, 1442070000000, -10, 220, 1, 2, 0.2, 2, 0.3333333333333333] - - ["HR", 1442070000000, 32, 22, 1442084400000, 1442077200000, -10, 220, 2, 3, 0.4, 3, 0.5] - - ["HR", 1442077200000, 58, 80, 1442073600000, 1442088000000, -10, 220, 2, 4, 0.6, 4, 0.6666666666666666] - - ["HR", 1442088000000, 82, 162, 1442070000000, 1442080800000, -10, 220, 3, 5, 0.8, 5, 0.8333333333333334] - - ["HR", 1442080800000, 220, 382, 1442077200000, null, -10, 220, 3, 6, 1.0, 6, 1.0] - - ["HU", 1442088000000, -71, -71, null, 1442091600000, -71, 547, 1, 1, 0.0, 1, 0.07692307692307693] - - ["HU", 1442091600000, -5, -76, null, 1442055600000, -71, 547, 1, 2, 0.08333333333333333, 2, 0.15384615384615385] - - ["HU", 1442055600000, -2, -78, 1442088000000, 1442019600000, -71, 547, 1, 3, 0.16666666666666666, 3, 0.23076923076923078] - - ["HU", 1442019600000, 46, -32, 1442091600000, 1442062800000, -71, 547, 1, 4, 0.25, 4, 0.3076923076923077] - - ["HU", 1442062800000, 50, 18, 1442055600000, 1442041200000, -71, 547, 1, 5, 0.3333333333333333, 5, 0.38461538461538464] - - ["HU", 1442041200000, 91, 109, 1442019600000, 1442098800000, -71, 547, 2, 6, 0.4166666666666667, 6, 0.46153846153846156] - - ["HU", 1442098800000, 110, 219, 1442062800000, 1442084400000, -71, 547, 2, 7, 0.5, 7, 0.5384615384615384] - - ["HU", 1442084400000, 141, 360, 1442041200000, 1442037600000, -71, 547, 2, 8, 0.5833333333333334, 8, 0.6153846153846154] - - ["HU", 1442037600000, 197, 557, 1442098800000, 1442080800000, -71, 547, 2, 9, 0.6666666666666666, 9, 0.6923076923076923] - - ["HU", 1442080800000, 242, 799, 1442084400000, 1442095200000, -71, 547, 3, 10, 0.75, 10, 0.7692307692307693] - - ["HU", 1442095200000, 271, 1070, 1442037600000, 1442048400000, -71, 547, 3, 11, 0.8333333333333334, 11, 0.8461538461538461] - - ["HU", 1442048400000, 499, 1569, 1442080800000, 1442044800000, -71, 547, 3, 12, 0.9166666666666666, 12, 0.9230769230769231] - - ["HU", 1442044800000, 547, 2116, 1442095200000, null, -71, 547, 3, 13, 1.0, 13, 1.0] - - ["ID", 1442026800000, -416, -416, null, 1442044800000, -416, 279, 1, 1, 0.0, 1, 0.07692307692307693] - - ["ID", 1442044800000, -388, -804, null, 1442041200000, -416, 279, 1, 2, 0.08333333333333333, 2, 0.15384615384615385] - - ["ID", 1442041200000, 2, -802, 1442026800000, 1442098800000, -416, 279, 1, 3, 0.16666666666666666, 3, 0.23076923076923078] - - ["ID", 1442098800000, 13, -789, 1442044800000, 1442037600000, -416, 279, 1, 4, 0.25, 4, 0.3076923076923077] - - ["ID", 1442037600000, 14, -775, 1442041200000, 1442055600000, -416, 279, 1, 5, 0.3333333333333333, 5, 0.38461538461538464] - - ["ID", 1442055600000, 16, -759, 1442098800000, 1442059200000, -416, 279, 2, 6, 0.4166666666666667, 6, 0.46153846153846156] - - ["ID", 1442059200000, 17, -742, 1442037600000, 1442034000000, -416, 279, 2, 7, 0.5, 7, 0.5384615384615384] - - ["ID", 1442034000000, 19, -723, 1442055600000, 1442095200000, -416, 279, 2, 8, 0.5833333333333334, 8, 0.6153846153846154] - - ["ID", 1442095200000, 20, -703, 1442059200000, 1442091600000, -416, 279, 2, 9, 0.6666666666666666, 9, 0.6923076923076923] - - ["ID", 1442091600000, 21, -682, 1442034000000, 1442070000000, -416, 279, 3, 10, 0.75, 10, 0.7692307692307693] - - ["ID", 1442070000000, 42, -640, 1442095200000, 1442023200000, -416, 279, 3, 11, 0.8333333333333334, 11, 0.8461538461538461] - - ["ID", 1442023200000, 106, -534, 1442091600000, 1442030400000, -416, 279, 3, 12, 0.9166666666666666, 12, 0.9230769230769231] - - ["ID", 1442030400000, 279, -255, 1442070000000, null, -416, 279, 3, 13, 1.0, 13, 1.0] - - ["IE", 1442070000000, -100, -100, null, 1442091600000, -100, 1062, 1, 1, 0.0, 1, 0.125] - - ["IE", 1442091600000, -71, -171, null, 1442026800000, -100, 1062, 1, 2, 0.14285714285714285, 2, 0.25] - - ["IE", 1442026800000, 1, -170, 1442070000000, 1442030400000, -100, 1062, 1, 3, 0.2857142857142857, 3, 0.5] - - ["IE", 1442030400000, 1, -169, 1442091600000, 1442048400000, -100, 1062, 2, 3, 0.2857142857142857, 3, 0.5] - - ["IE", 1442048400000, 27, -142, 1442026800000, 1442077200000, -100, 1062, 2, 5, 0.5714285714285714, 4, 0.625] - - ["IE", 1442077200000, 403, 261, 1442030400000, 1442084400000, -100, 1062, 2, 6, 0.7142857142857143, 5, 0.75] - - ["IE", 1442084400000, 819, 1080, 1442048400000, 1442066400000, -100, 1062, 3, 7, 0.8571428571428571, 6, 0.875] - - ["IE", 1442066400000, 1062, 2142, 1442077200000, null, -100, 1062, 3, 8, 1.0, 7, 1.0] - - ["IL", 1442095200000, 0, 0, null, 1442066400000, 0, 2745, 1, 1, 0.0, 1, 0.0625] - - ["IL", 1442066400000, 3, 3, null, 1442098800000, 0, 2745, 1, 2, 0.06666666666666667, 2, 0.1875] - - ["IL", 1442098800000, 3, 6, 1442095200000, 1442055600000, 0, 2745, 1, 2, 0.06666666666666667, 2, 0.1875] - - ["IL", 1442055600000, 4, 10, 1442066400000, 1442048400000, 0, 2745, 1, 4, 0.2, 3, 0.25] - - ["IL", 1442048400000, 25, 35, 1442098800000, 1442073600000, 0, 2745, 1, 5, 0.26666666666666666, 4, 0.3125] - - ["IL", 1442073600000, 31, 66, 1442055600000, 1442041200000, 0, 2745, 1, 6, 0.3333333333333333, 5, 0.375] - - ["IL", 1442041200000, 35, 101, 1442048400000, 1442070000000, 0, 2745, 2, 7, 0.4, 6, 0.4375] - - ["IL", 1442070000000, 49, 150, 1442073600000, 1442080800000, 0, 2745, 2, 8, 0.4666666666666667, 7, 0.5] - - ["IL", 1442080800000, 88, 238, 1442041200000, 1442062800000, 0, 2745, 2, 9, 0.5333333333333333, 8, 0.5625] - - ["IL", 1442062800000, 180, 418, 1442070000000, 1442077200000, 0, 2745, 2, 10, 0.6, 9, 0.625] - - ["IL", 1442077200000, 187, 605, 1442080800000, 1442044800000, 0, 2745, 2, 11, 0.6666666666666666, 10, 0.6875] - - ["IL", 1442044800000, 218, 823, 1442062800000, 1442091600000, 0, 2745, 3, 12, 0.7333333333333333, 11, 0.75] - - ["IL", 1442091600000, 707, 1530, 1442077200000, 1442084400000, 0, 2745, 3, 13, 0.8, 12, 0.8125] - - ["IL", 1442084400000, 1137, 2667, 1442044800000, 1442059200000, 0, 2745, 3, 14, 0.8666666666666667, 13, 0.875] - - ["IL", 1442059200000, 1205, 3872, 1442091600000, 1442052000000, 0, 2745, 3, 15, 0.9333333333333333, 14, 0.9375] - - ["IL", 1442052000000, 2745, 6617, 1442084400000, null, 0, 2745, 3, 16, 1.0, 15, 1.0] - - ["IN", 1442023200000, -142, -142, null, 1442080800000, -142, 12091, 1, 1, 0.0, 1, 0.045454545454545456] - - ["IN", 1442080800000, 0, -142, null, 1442016000000, -142, 12091, 1, 2, 0.047619047619047616, 2, 0.09090909090909091] - - ["IN", 1442016000000, 1, -141, 1442023200000, 1442095200000, -142, 12091, 1, 3, 0.09523809523809523, 3, 0.13636363636363635] - - ["IN", 1442095200000, 4, -137, 1442080800000, 1442019600000, -142, 12091, 1, 4, 0.14285714285714285, 4, 0.18181818181818182] - - ["IN", 1442019600000, 38, -99, 1442016000000, 1442041200000, -142, 12091, 1, 5, 0.19047619047619047, 5, 0.22727272727272727] - - ["IN", 1442041200000, 80, -19, 1442095200000, 1442066400000, -142, 12091, 1, 6, 0.23809523809523808, 6, 0.2727272727272727] - - ["IN", 1442066400000, 116, 97, 1442019600000, 1442088000000, -142, 12091, 1, 7, 0.2857142857142857, 7, 0.3181818181818182] - - ["IN", 1442088000000, 121, 218, 1442041200000, 1442037600000, -142, 12091, 1, 8, 0.3333333333333333, 8, 0.36363636363636365] - - ["IN", 1442037600000, 135, 353, 1442066400000, 1442055600000, -142, 12091, 2, 9, 0.38095238095238093, 9, 0.4090909090909091] - - ["IN", 1442055600000, 166, 519, 1442088000000, 1442084400000, -142, 12091, 2, 10, 0.42857142857142855, 10, 0.45454545454545453] - - ["IN", 1442084400000, 187, 706, 1442037600000, 1442048400000, -142, 12091, 2, 11, 0.47619047619047616, 11, 0.5] - - ["IN", 1442048400000, 262, 968, 1442055600000, 1442052000000, -142, 12091, 2, 12, 0.5238095238095238, 12, 0.5454545454545454] - - ["IN", 1442052000000, 534, 1502, 1442084400000, 1442059200000, -142, 12091, 2, 13, 0.5714285714285714, 13, 0.5909090909090909] - - ["IN", 1442059200000, 708, 2210, 1442048400000, 1442026800000, -142, 12091, 2, 14, 0.6190476190476191, 14, 0.6363636363636364] - - ["IN", 1442026800000, 974, 3184, 1442052000000, 1442073600000, -142, 12091, 2, 15, 0.6666666666666666, 15, 0.6818181818181818] - - ["IN", 1442073600000, 1170, 4354, 1442059200000, 1442034000000, -142, 12091, 3, 16, 0.7142857142857143, 16, 0.7272727272727273] - - ["IN", 1442034000000, 1350, 5704, 1442026800000, 1442030400000, -142, 12091, 3, 17, 0.7619047619047619, 17, 0.7727272727272727] - - ["IN", 1442030400000, 1448, 7152, 1442073600000, 1442062800000, -142, 12091, 3, 18, 0.8095238095238095, 18, 0.8181818181818182] - - ["IN", 1442062800000, 1547, 8699, 1442034000000, 1442044800000, -142, 12091, 3, 19, 0.8571428571428571, 19, 0.8636363636363636] - - ["IN", 1442044800000, 2677, 11376, 1442030400000, 1442077200000, -142, 12091, 3, 20, 0.9047619047619048, 20, 0.9090909090909091] - - ["IN", 1442077200000, 5699, 17075, 1442062800000, 1442070000000, -142, 12091, 3, 21, 0.9523809523809523, 21, 0.9545454545454546] - - ["IN", 1442070000000, 12091, 29166, 1442044800000, null, -142, 12091, 3, 22, 1.0, 22, 1.0] - - ["IQ", 1442095200000, -2, -2, null, 1442041200000, -2, 6, 1, 1, 0.0, 1, 0.25] - - ["IQ", 1442041200000, -1, -3, null, 1442052000000, -2, 6, 1, 2, 0.3333333333333333, 2, 0.5] - - ["IQ", 1442052000000, 0, -3, 1442095200000, 1442044800000, -2, 6, 2, 3, 0.6666666666666666, 3, 0.75] - - ["IQ", 1442044800000, 6, 3, 1442041200000, null, -2, 6, 3, 4, 1.0, 4, 1.0] - - ["IR", 1442073600000, -193, -193, null, 1442055600000, -193, 1455, 1, 1, 0.0, 1, 0.07692307692307693] - - ["IR", 1442055600000, -124, -317, null, 1442041200000, -193, 1455, 1, 2, 0.08333333333333333, 2, 0.15384615384615385] - - ["IR", 1442041200000, -79, -396, 1442073600000, 1442077200000, -193, 1455, 1, 3, 0.16666666666666666, 3, 0.23076923076923078] - - ["IR", 1442077200000, -34, -430, 1442055600000, 1442034000000, -193, 1455, 1, 4, 0.25, 4, 0.3076923076923077] - - ["IR", 1442034000000, -8, -438, 1442041200000, 1442026800000, -193, 1455, 1, 5, 0.3333333333333333, 5, 0.38461538461538464] - - ["IR", 1442026800000, 0, -438, 1442077200000, 1442091600000, -193, 1455, 2, 6, 0.4166666666666667, 6, 0.5384615384615384] - - ["IR", 1442091600000, 0, -438, 1442034000000, 1442080800000, -193, 1455, 2, 6, 0.4166666666666667, 6, 0.5384615384615384] - - ["IR", 1442080800000, 131, -307, 1442026800000, 1442052000000, -193, 1455, 2, 8, 0.5833333333333334, 7, 0.6153846153846154] - - ["IR", 1442052000000, 155, -152, 1442091600000, 1442044800000, -193, 1455, 2, 9, 0.6666666666666666, 8, 0.6923076923076923] - - ["IR", 1442044800000, 306, 154, 1442080800000, 1442030400000, -193, 1455, 3, 10, 0.75, 9, 0.7692307692307693] - - ["IR", 1442030400000, 375, 529, 1442052000000, 1442088000000, -193, 1455, 3, 11, 0.8333333333333334, 10, 0.8461538461538461] - - ["IR", 1442088000000, 714, 1243, 1442044800000, 1442059200000, -193, 1455, 3, 12, 0.9166666666666666, 11, 0.9230769230769231] - - ["IR", 1442059200000, 1455, 2698, 1442030400000, null, -193, 1455, 3, 13, 1.0, 12, 1.0] - - ["IT", 1442030400000, -17, -17, null, 1442037600000, -17, 6240, 1, 1, 0.0, 1, 0.041666666666666664] - - ["IT", 1442037600000, -9, -26, null, 1442016000000, -17, 6240, 1, 2, 0.043478260869565216, 2, 0.08333333333333333] - - ["IT", 1442016000000, 0, -26, 1442030400000, 1442041200000, -17, 6240, 1, 3, 0.08695652173913043, 3, 0.125] - - ["IT", 1442041200000, 20, -6, 1442037600000, 1442070000000, -17, 6240, 1, 4, 0.13043478260869565, 4, 0.16666666666666666] - - ["IT", 1442070000000, 81, 75, 1442016000000, 1442023200000, -17, 6240, 1, 5, 0.17391304347826086, 5, 0.20833333333333334] - - ["IT", 1442023200000, 111, 186, 1442041200000, 1442019600000, -17, 6240, 1, 6, 0.21739130434782608, 6, 0.25] - - ["IT", 1442019600000, 183, 369, 1442070000000, 1442026800000, -17, 6240, 1, 7, 0.2608695652173913, 7, 0.2916666666666667] - - ["IT", 1442026800000, 222, 591, 1442023200000, 1442091600000, -17, 6240, 1, 8, 0.30434782608695654, 8, 0.3333333333333333] - - ["IT", 1442091600000, 351, 942, 1442019600000, 1442059200000, -17, 6240, 2, 9, 0.34782608695652173, 9, 0.375] - - ["IT", 1442059200000, 542, 1484, 1442026800000, 1442098800000, -17, 6240, 2, 10, 0.391304347826087, 10, 0.4166666666666667] - - ["IT", 1442098800000, 565, 2049, 1442091600000, 1442048400000, -17, 6240, 2, 11, 0.43478260869565216, 11, 0.4583333333333333] - - ["IT", 1442048400000, 676, 2725, 1442059200000, 1442034000000, -17, 6240, 2, 12, 0.4782608695652174, 12, 0.5] - - ["IT", 1442034000000, 1006, 3731, 1442098800000, 1442044800000, -17, 6240, 2, 13, 0.5217391304347826, 13, 0.5416666666666666] - - ["IT", 1442044800000, 1483, 5214, 1442048400000, 1442052000000, -17, 6240, 2, 14, 0.5652173913043478, 14, 0.5833333333333334] - - ["IT", 1442052000000, 1880, 7094, 1442034000000, 1442062800000, -17, 6240, 2, 15, 0.6086956521739131, 15, 0.625] - - ["IT", 1442062800000, 1938, 9032, 1442044800000, 1442077200000, -17, 6240, 2, 16, 0.6521739130434783, 16, 0.6666666666666666] - - ["IT", 1442077200000, 2188, 11220, 1442052000000, 1442073600000, -17, 6240, 3, 17, 0.6956521739130435, 17, 0.7083333333333334] - - ["IT", 1442073600000, 2586, 13806, 1442062800000, 1442084400000, -17, 6240, 3, 18, 0.7391304347826086, 18, 0.75] - - ["IT", 1442084400000, 2660, 16466, 1442077200000, 1442095200000, -17, 6240, 3, 19, 0.782608695652174, 19, 0.7916666666666666] - - ["IT", 1442095200000, 2940, 19406, 1442073600000, 1442088000000, -17, 6240, 3, 20, 0.8260869565217391, 20, 0.8333333333333334] - - ["IT", 1442088000000, 3746, 23152, 1442084400000, 1442066400000, -17, 6240, 3, 21, 0.8695652173913043, 21, 0.875] - - ["IT", 1442066400000, 4155, 27307, 1442095200000, 1442080800000, -17, 6240, 3, 22, 0.9130434782608695, 22, 0.9166666666666666] - - ["IT", 1442080800000, 5544, 32851, 1442088000000, 1442055600000, -17, 6240, 3, 23, 0.9565217391304348, 23, 0.9583333333333334] - - ["IT", 1442055600000, 6240, 39091, 1442066400000, null, -17, 6240, 3, 24, 1.0, 24, 1.0] - - ["JM", 1442070000000, 30, 30, null, null, 30, 30, 1, 1, 0.0, 1, 1.0] - - ["JO", 1442055600000, -2, -2, null, 1442059200000, -2, 4, 1, 1, 0.0, 1, 0.3333333333333333] - - ["JO", 1442059200000, 0, -2, null, 1442080800000, -2, 4, 2, 2, 0.5, 2, 0.6666666666666666] - - ["JO", 1442080800000, 4, 2, 1442055600000, null, -2, 4, 3, 3, 1.0, 3, 1.0] - - ["JP", 1442016000000, -113, -113, null, 1442059200000, -113, 2789, 1, 1, 0.0, 1, 0.041666666666666664] - - ["JP", 1442059200000, -85, -198, null, 1442098800000, -113, 2789, 1, 2, 0.043478260869565216, 2, 0.08333333333333333] - - ["JP", 1442098800000, -6, -204, 1442016000000, 1442095200000, -113, 2789, 1, 3, 0.08695652173913043, 3, 0.125] - - ["JP", 1442095200000, 0, -204, 1442059200000, 1442084400000, -113, 2789, 1, 4, 0.13043478260869565, 4, 0.16666666666666666] - - ["JP", 1442084400000, 13, -191, 1442098800000, 1442077200000, -113, 2789, 1, 5, 0.17391304347826086, 5, 0.20833333333333334] - - ["JP", 1442077200000, 51, -140, 1442095200000, 1442088000000, -113, 2789, 1, 6, 0.21739130434782608, 6, 0.25] - - ["JP", 1442088000000, 57, -83, 1442084400000, 1442070000000, -113, 2789, 1, 7, 0.2608695652173913, 7, 0.2916666666666667] - - ["JP", 1442070000000, 79, -4, 1442077200000, 1442066400000, -113, 2789, 1, 8, 0.30434782608695654, 8, 0.3333333333333333] - - ["JP", 1442066400000, 167, 163, 1442088000000, 1442091600000, -113, 2789, 2, 9, 0.34782608695652173, 9, 0.375] - - ["JP", 1442091600000, 228, 391, 1442070000000, 1442080800000, -113, 2789, 2, 10, 0.391304347826087, 10, 0.4166666666666667] - - ["JP", 1442080800000, 420, 811, 1442066400000, 1442062800000, -113, 2789, 2, 11, 0.43478260869565216, 11, 0.4583333333333333] - - ["JP", 1442062800000, 803, 1614, 1442091600000, 1442030400000, -113, 2789, 2, 12, 0.4782608695652174, 12, 0.5] - - ["JP", 1442030400000, 805, 2419, 1442080800000, 1442034000000, -113, 2789, 2, 13, 0.5217391304347826, 13, 0.5416666666666666] - - ["JP", 1442034000000, 910, 3329, 1442062800000, 1442055600000, -113, 2789, 2, 14, 0.5652173913043478, 14, 0.5833333333333334] - - ["JP", 1442055600000, 998, 4327, 1442030400000, 1442026800000, -113, 2789, 2, 15, 0.6086956521739131, 15, 0.625] - - ["JP", 1442026800000, 1035, 5362, 1442034000000, 1442073600000, -113, 2789, 2, 16, 0.6521739130434783, 16, 0.6666666666666666] - - ["JP", 1442073600000, 1162, 6524, 1442055600000, 1442041200000, -113, 2789, 3, 17, 0.6956521739130435, 17, 0.7083333333333334] - - ["JP", 1442041200000, 1373, 7897, 1442026800000, 1442044800000, -113, 2789, 3, 18, 0.7391304347826086, 18, 0.75] - - ["JP", 1442044800000, 1569, 9466, 1442073600000, 1442023200000, -113, 2789, 3, 19, 0.782608695652174, 19, 0.7916666666666666] - - ["JP", 1442023200000, 1959, 11425, 1442041200000, 1442048400000, -113, 2789, 3, 20, 0.8260869565217391, 20, 0.8333333333333334] - - ["JP", 1442048400000, 1981, 13406, 1442044800000, 1442019600000, -113, 2789, 3, 21, 0.8695652173913043, 21, 0.875] - - ["JP", 1442019600000, 2002, 15408, 1442023200000, 1442037600000, -113, 2789, 3, 22, 0.9130434782608695, 22, 0.9166666666666666] - - ["JP", 1442037600000, 2181, 17589, 1442048400000, 1442052000000, -113, 2789, 3, 23, 0.9565217391304348, 23, 0.9583333333333334] - - ["JP", 1442052000000, 2789, 20378, 1442019600000, null, -113, 2789, 3, 24, 1.0, 24, 1.0] - - ["KE", 1442044800000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0] - - ["KG", 1442073600000, 6, 6, null, null, 6, 6, 1, 1, 0.0, 1, 1.0] - - ["KR", 1442048400000, -374, -374, null, 1442026800000, -374, 3640, 1, 1, 0.0, 1, 0.045454545454545456] - - ["KR", 1442026800000, -179, -553, null, 1442077200000, -374, 3640, 1, 2, 0.047619047619047616, 2, 0.09090909090909091] - - ["KR", 1442077200000, -40, -593, 1442048400000, 1442098800000, -374, 3640, 1, 3, 0.09523809523809523, 3, 0.13636363636363635] - - ["KR", 1442098800000, -36, -629, 1442026800000, 1442080800000, -374, 3640, 1, 4, 0.14285714285714285, 4, 0.18181818181818182] - - ["KR", 1442080800000, -33, -662, 1442077200000, 1442052000000, -374, 3640, 1, 5, 0.19047619047619047, 5, 0.22727272727272727] - - ["KR", 1442052000000, -3, -665, 1442098800000, 1442041200000, -374, 3640, 1, 6, 0.23809523809523808, 6, 0.2727272727272727] - - ["KR", 1442041200000, 20, -645, 1442080800000, 1442037600000, -374, 3640, 1, 7, 0.2857142857142857, 7, 0.3181818181818182] - - ["KR", 1442037600000, 26, -619, 1442052000000, 1442059200000, -374, 3640, 1, 8, 0.3333333333333333, 8, 0.36363636363636365] - - ["KR", 1442059200000, 208, -411, 1442041200000, 1442070000000, -374, 3640, 2, 9, 0.38095238095238093, 9, 0.4090909090909091] - - ["KR", 1442070000000, 222, -189, 1442037600000, 1442084400000, -374, 3640, 2, 10, 0.42857142857142855, 10, 0.45454545454545453] - - ["KR", 1442084400000, 314, 125, 1442059200000, 1442023200000, -374, 3640, 2, 11, 0.47619047619047616, 11, 0.5] - - ["KR", 1442023200000, 319, 444, 1442070000000, 1442034000000, -374, 3640, 2, 12, 0.5238095238095238, 12, 0.5454545454545454] - - ["KR", 1442034000000, 434, 878, 1442084400000, 1442019600000, -374, 3640, 2, 13, 0.5714285714285714, 13, 0.5909090909090909] - - ["KR", 1442019600000, 445, 1323, 1442023200000, 1442088000000, -374, 3640, 2, 14, 0.6190476190476191, 14, 0.6363636363636364] - - ["KR", 1442088000000, 524, 1847, 1442034000000, 1442095200000, -374, 3640, 2, 15, 0.6666666666666666, 15, 0.6818181818181818] - - ["KR", 1442095200000, 827, 2674, 1442019600000, 1442044800000, -374, 3640, 3, 16, 0.7142857142857143, 16, 0.7272727272727273] - - ["KR", 1442044800000, 829, 3503, 1442088000000, 1442016000000, -374, 3640, 3, 17, 0.7619047619047619, 17, 0.7727272727272727] - - ["KR", 1442016000000, 1024, 4527, 1442095200000, 1442030400000, -374, 3640, 3, 18, 0.8095238095238095, 18, 0.8181818181818182] - - ["KR", 1442030400000, 1035, 5562, 1442044800000, 1442062800000, -374, 3640, 3, 19, 0.8571428571428571, 19, 0.8636363636363636] - - ["KR", 1442062800000, 1096, 6658, 1442016000000, 1442066400000, -374, 3640, 3, 20, 0.9047619047619048, 20, 0.9090909090909091] - - ["KR", 1442066400000, 3299, 9957, 1442030400000, 1442055600000, -374, 3640, 3, 21, 0.9523809523809523, 21, 0.9545454545454546] - - ["KR", 1442055600000, 3640, 13597, 1442062800000, null, -374, 3640, 3, 22, 1.0, 22, 1.0] - - ["KW", 1442080800000, -33, -33, null, 1442055600000, -33, 1815, 1, 1, 0.0, 1, 0.25] - - ["KW", 1442055600000, -2, -35, null, 1442077200000, -33, 1815, 1, 2, 0.3333333333333333, 2, 0.75] - - ["KW", 1442077200000, -2, -37, 1442080800000, 1442070000000, -33, 1815, 2, 2, 0.3333333333333333, 2, 0.75] - - ["KW", 1442070000000, 1815, 1778, 1442055600000, null, -33, 1815, 3, 4, 1.0, 3, 1.0] - - ["KZ", 1442077200000, -317, -317, null, 1442084400000, -317, 439, 1, 1, 0.0, 1, 0.09090909090909091] - - ["KZ", 1442084400000, -22, -339, null, 1442062800000, -317, 439, 1, 2, 0.1, 2, 0.18181818181818182] - - ["KZ", 1442062800000, 0, -339, 1442077200000, 1442066400000, -317, 439, 1, 3, 0.2, 3, 0.36363636363636365] - - ["KZ", 1442066400000, 0, -339, 1442084400000, 1442059200000, -317, 439, 1, 3, 0.2, 3, 0.36363636363636365] - - ["KZ", 1442059200000, 33, -306, 1442062800000, 1442055600000, -317, 439, 2, 5, 0.4, 4, 0.45454545454545453] - - ["KZ", 1442055600000, 63, -243, 1442066400000, 1442095200000, -317, 439, 2, 6, 0.5, 5, 0.5454545454545454] - - ["KZ", 1442095200000, 91, -152, 1442059200000, 1442034000000, -317, 439, 2, 7, 0.6, 6, 0.6363636363636364] - - ["KZ", 1442034000000, 161, 9, 1442055600000, 1442044800000, -317, 439, 2, 8, 0.7, 7, 0.7272727272727273] - - ["KZ", 1442044800000, 401, 410, 1442095200000, 1442052000000, -317, 439, 3, 9, 0.8, 8, 0.8181818181818182] - - ["KZ", 1442052000000, 412, 822, 1442034000000, 1442048400000, -317, 439, 3, 10, 0.9, 9, 0.9090909090909091] - - ["KZ", 1442048400000, 439, 1261, 1442044800000, null, -317, 439, 3, 11, 1.0, 10, 1.0] - - ["LB", 1442055600000, -67, -67, null, null, -67, -67, 1, 1, 0.0, 1, 1.0] - - ["LK", 1442084400000, -3, -3, null, 1442048400000, -3, 79, 1, 1, 0.0, 1, 0.25] - - ["LK", 1442048400000, 8, 5, null, 1442052000000, -3, 79, 1, 2, 0.3333333333333333, 2, 0.5] - - ["LK", 1442052000000, 47, 52, 1442084400000, 1442026800000, -3, 79, 2, 3, 0.6666666666666666, 3, 0.75] - - ["LK", 1442026800000, 79, 131, 1442048400000, null, -3, 79, 3, 4, 1.0, 4, 1.0] - - ["LT", 1442098800000, -24, -24, null, 1442080800000, -24, 12, 1, 1, 0.0, 1, 0.5] - - ["LT", 1442080800000, 12, -12, null, null, -24, 12, 2, 2, 1.0, 2, 1.0] - - ["LU", 1442066400000, 0, 0, null, 1442095200000, 0, 525, 1, 1, 0.0, 1, 0.25] - - ["LU", 1442095200000, 2, 2, null, 1442059200000, 0, 525, 1, 2, 0.3333333333333333, 2, 0.5] - - ["LU", 1442059200000, 79, 81, 1442066400000, 1442077200000, 0, 525, 2, 3, 0.6666666666666666, 3, 0.75] - - ["LU", 1442077200000, 525, 606, 1442095200000, null, 0, 525, 3, 4, 1.0, 4, 1.0] - - ["LV", 1442095200000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0] - - ["MA", 1442059200000, -56, -56, null, 1442019600000, -56, 250, 1, 1, 0.0, 1, 0.14285714285714285] - - ["MA", 1442019600000, -1, -57, null, 1442062800000, -56, 250, 1, 2, 0.16666666666666666, 2, 0.2857142857142857] - - ["MA", 1442062800000, 0, -57, 1442059200000, 1442080800000, -56, 250, 1, 3, 0.3333333333333333, 3, 0.42857142857142855] - - ["MA", 1442080800000, 5, -52, 1442019600000, 1442098800000, -56, 250, 2, 4, 0.5, 4, 0.5714285714285714] - - ["MA", 1442098800000, 8, -44, 1442062800000, 1442055600000, -56, 250, 2, 5, 0.6666666666666666, 5, 0.7142857142857143] - - ["MA", 1442055600000, 23, -21, 1442080800000, 1442077200000, -56, 250, 3, 6, 0.8333333333333334, 6, 0.8571428571428571] - - ["MA", 1442077200000, 250, 229, 1442098800000, null, -56, 250, 3, 7, 1.0, 7, 1.0] - - ["MD", 1442077200000, 6916, 6916, null, null, 6916, 6916, 1, 1, 0.0, 1, 1.0] - - ["ME", 1442073600000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0] - - ["MH", 1442052000000, 40, 40, null, null, 40, 40, 1, 1, 0.0, 1, 1.0] - - ["MK", 1442077200000, -72, -72, null, null, -72, -72, 1, 1, 0.0, 1, 1.0] - - ["MM", 1442070000000, 3, 3, null, 1442073600000, 3, 25, 1, 1, 0.0, 1, 0.5] - - ["MM", 1442073600000, 25, 28, null, null, 3, 25, 2, 2, 1.0, 2, 1.0] - - ["MO", 1442070000000, 18, 18, null, 1442034000000, 18, 30, 1, 1, 0.0, 1, 0.5] - - ["MO", 1442034000000, 30, 48, null, null, 18, 30, 2, 2, 1.0, 2, 1.0] - - ["MR", 1442080800000, 10, 10, null, null, 10, 10, 1, 1, 0.0, 1, 1.0] - - ["MT", 1442048400000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0] - - ["MV", 1442073600000, -3, -3, null, null, -3, -3, 1, 1, 0.0, 1, 1.0] - - ["MX", 1442095200000, -456, -456, null, 1442080800000, -456, 3874, 1, 1, 0.0, 1, 0.058823529411764705] - - ["MX", 1442080800000, -376, -832, null, 1442041200000, -456, 3874, 1, 2, 0.0625, 2, 0.11764705882352941] - - ["MX", 1442041200000, -294, -1126, 1442095200000, 1442016000000, -456, 3874, 1, 3, 0.125, 3, 0.17647058823529413] - - ["MX", 1442016000000, -67, -1193, 1442080800000, 1442073600000, -456, 3874, 1, 4, 0.1875, 4, 0.23529411764705882] - - ["MX", 1442073600000, -21, -1214, 1442041200000, 1442066400000, -456, 3874, 1, 5, 0.25, 5, 0.29411764705882354] - - ["MX", 1442066400000, -1, -1215, 1442016000000, 1442070000000, -456, 3874, 1, 6, 0.3125, 6, 0.4117647058823529] - - ["MX", 1442070000000, -1, -1216, 1442073600000, 1442037600000, -456, 3874, 2, 6, 0.3125, 6, 0.4117647058823529] - - ["MX", 1442037600000, 4, -1212, 1442066400000, 1442098800000, -456, 3874, 2, 8, 0.4375, 7, 0.47058823529411764] - - ["MX", 1442098800000, 28, -1184, 1442070000000, 1442030400000, -456, 3874, 2, 9, 0.5, 8, 0.5294117647058824] - - ["MX", 1442030400000, 373, -811, 1442037600000, 1442088000000, -456, 3874, 2, 10, 0.5625, 9, 0.5882352941176471] - - ["MX", 1442088000000, 494, -317, 1442098800000, 1442023200000, -456, 3874, 2, 11, 0.625, 10, 0.6470588235294118] - - ["MX", 1442023200000, 549, 232, 1442030400000, 1442091600000, -456, 3874, 2, 12, 0.6875, 11, 0.7058823529411765] - - ["MX", 1442091600000, 799, 1031, 1442088000000, 1442034000000, -456, 3874, 3, 13, 0.75, 12, 0.7647058823529411] - - ["MX", 1442034000000, 944, 1975, 1442023200000, 1442084400000, -456, 3874, 3, 14, 0.8125, 13, 0.8235294117647058] - - ["MX", 1442084400000, 981, 2956, 1442091600000, 1442026800000, -456, 3874, 3, 15, 0.875, 14, 0.8823529411764706] - - ["MX", 1442026800000, 3642, 6598, 1442034000000, 1442077200000, -456, 3874, 3, 16, 0.9375, 15, 0.9411764705882353] - - ["MX", 1442077200000, 3874, 10472, 1442084400000, null, -456, 3874, 3, 17, 1.0, 16, 1.0] - - ["MY", 1442044800000, -127, -127, null, 1442077200000, -127, 1028, 1, 1, 0.0, 1, 0.08333333333333333] - - ["MY", 1442077200000, -10, -137, null, 1442019600000, -127, 1028, 1, 2, 0.09090909090909091, 2, 0.16666666666666666] - - ["MY", 1442019600000, -7, -144, 1442044800000, 1442030400000, -127, 1028, 1, 3, 0.18181818181818182, 3, 0.25] - - ["MY", 1442030400000, -3, -147, 1442077200000, 1442059200000, -127, 1028, 1, 4, 0.2727272727272727, 4, 0.3333333333333333] - - ["MY", 1442059200000, 0, -147, 1442019600000, 1442055600000, -127, 1028, 2, 5, 0.36363636363636365, 5, 0.4166666666666667] - - ["MY", 1442055600000, 1, -146, 1442030400000, 1442066400000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666] - - ["MY", 1442066400000, 1, -145, 1442059200000, 1442073600000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666] - - ["MY", 1442073600000, 1, -144, 1442055600000, 1442048400000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666] - - ["MY", 1442048400000, 649, 505, 1442066400000, 1442098800000, -127, 1028, 3, 9, 0.7272727272727273, 7, 0.75] - - ["MY", 1442098800000, 739, 1244, 1442073600000, 1442041200000, -127, 1028, 3, 10, 0.8181818181818182, 8, 0.8333333333333334] - - ["MY", 1442041200000, 935, 2179, 1442048400000, 1442034000000, -127, 1028, 3, 11, 0.9090909090909091, 9, 0.9166666666666666] - - ["MY", 1442034000000, 1028, 3207, 1442098800000, null, -127, 1028, 3, 12, 1.0, 10, 1.0] - - ["NG", 1442070000000, 6, 6, null, 1442052000000, 6, 208, 1, 1, 0.0, 1, 0.5] - - ["NG", 1442052000000, 208, 214, null, null, 6, 208, 2, 2, 1.0, 2, 1.0] - - ["NL", 1442070000000, -84, -84, null, 1442062800000, -84, 8947, 1, 1, 0.0, 1, 0.058823529411764705] - - ["NL", 1442062800000, -30, -114, null, 1442034000000, -84, 8947, 1, 2, 0.0625, 2, 0.11764705882352941] - - ["NL", 1442034000000, 0, -114, 1442070000000, 1442098800000, -84, 8947, 1, 3, 0.125, 3, 0.17647058823529413] - - ["NL", 1442098800000, 4, -110, 1442062800000, 1442088000000, -84, 8947, 1, 4, 0.1875, 4, 0.23529411764705882] - - ["NL", 1442088000000, 12, -98, 1442034000000, 1442044800000, -84, 8947, 1, 5, 0.25, 5, 0.29411764705882354] - - ["NL", 1442044800000, 16, -82, 1442098800000, 1442091600000, -84, 8947, 1, 6, 0.3125, 6, 0.35294117647058826] - - ["NL", 1442091600000, 19, -63, 1442088000000, 1442052000000, -84, 8947, 2, 7, 0.375, 7, 0.4117647058823529] - - ["NL", 1442052000000, 53, -10, 1442044800000, 1442066400000, -84, 8947, 2, 8, 0.4375, 8, 0.47058823529411764] - - ["NL", 1442066400000, 61, 51, 1442091600000, 1442095200000, -84, 8947, 2, 9, 0.5, 9, 0.5294117647058824] - - ["NL", 1442095200000, 70, 121, 1442052000000, 1442055600000, -84, 8947, 2, 10, 0.5625, 10, 0.5882352941176471] - - ["NL", 1442055600000, 105, 226, 1442066400000, 1442073600000, -84, 8947, 2, 11, 0.625, 11, 0.6470588235294118] - - ["NL", 1442073600000, 166, 392, 1442095200000, 1442059200000, -84, 8947, 2, 12, 0.6875, 12, 0.7058823529411765] - - ["NL", 1442059200000, 206, 598, 1442055600000, 1442084400000, -84, 8947, 3, 13, 0.75, 13, 0.7647058823529411] - - ["NL", 1442084400000, 436, 1034, 1442073600000, 1442077200000, -84, 8947, 3, 14, 0.8125, 14, 0.8235294117647058] - - ["NL", 1442077200000, 878, 1912, 1442059200000, 1442048400000, -84, 8947, 3, 15, 0.875, 15, 0.8823529411764706] - - ["NL", 1442048400000, 1303, 3215, 1442084400000, 1442080800000, -84, 8947, 3, 16, 0.9375, 16, 0.9411764705882353] - - ["NL", 1442080800000, 8947, 12162, 1442077200000, null, -84, 8947, 3, 17, 1.0, 17, 1.0] - - ["NO", 1442048400000, -447, -447, null, 1442095200000, -447, 447, 1, 1, 0.0, 1, 0.09090909090909091] - - ["NO", 1442095200000, -1, -448, null, 1442098800000, -447, 447, 1, 2, 0.1, 2, 0.18181818181818182] - - ["NO", 1442098800000, 2, -446, 1442048400000, 1442088000000, -447, 447, 1, 3, 0.2, 3, 0.2727272727272727] - - ["NO", 1442088000000, 15, -431, 1442095200000, 1442091600000, -447, 447, 1, 4, 0.3, 4, 0.45454545454545453] - - ["NO", 1442091600000, 15, -416, 1442098800000, 1442055600000, -447, 447, 2, 4, 0.3, 4, 0.45454545454545453] - - ["NO", 1442055600000, 29, -387, 1442088000000, 1442080800000, -447, 447, 2, 6, 0.5, 5, 0.5454545454545454] - - ["NO", 1442080800000, 31, -356, 1442091600000, 1442019600000, -447, 447, 2, 7, 0.6, 6, 0.6363636363636364] - - ["NO", 1442019600000, 48, -308, 1442055600000, 1442066400000, -447, 447, 2, 8, 0.7, 7, 0.7272727272727273] - - ["NO", 1442066400000, 71, -237, 1442080800000, 1442073600000, -447, 447, 3, 9, 0.8, 8, 0.8181818181818182] - - ["NO", 1442073600000, 222, -15, 1442019600000, 1442052000000, -447, 447, 3, 10, 0.9, 9, 0.9090909090909091] - - ["NO", 1442052000000, 447, 432, 1442066400000, null, -447, 447, 3, 11, 1.0, 10, 1.0] - - ["NP", 1442048400000, 61, 61, null, null, 61, 61, 1, 1, 0.0, 1, 1.0] - - ["NZ", 1442084400000, -52, -52, null, 1442095200000, -52, 635, 1, 1, 0.0, 1, 0.1111111111111111] - - ["NZ", 1442095200000, -4, -56, null, 1442098800000, -52, 635, 1, 2, 0.125, 2, 0.2222222222222222] - - ["NZ", 1442098800000, -2, -58, 1442084400000, 1442019600000, -52, 635, 1, 3, 0.25, 3, 0.3333333333333333] - - ["NZ", 1442019600000, 28, -30, 1442095200000, 1442037600000, -52, 635, 2, 4, 0.375, 4, 0.4444444444444444] - - ["NZ", 1442037600000, 66, 36, 1442098800000, 1442048400000, -52, 635, 2, 5, 0.5, 5, 0.5555555555555556] - - ["NZ", 1442048400000, 189, 225, 1442019600000, 1442088000000, -52, 635, 2, 6, 0.625, 6, 0.6666666666666666] - - ["NZ", 1442088000000, 405, 630, 1442037600000, 1442059200000, -52, 635, 3, 7, 0.75, 7, 0.7777777777777778] - - ["NZ", 1442059200000, 428, 1058, 1442048400000, 1442026800000, -52, 635, 3, 8, 0.875, 8, 0.8888888888888888] - - ["NZ", 1442026800000, 635, 1693, 1442088000000, null, -52, 635, 3, 9, 1.0, 9, 1.0] - - ["OM", 1442052000000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0] - - ["PA", 1442026800000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0] - - ["PE", 1442077200000, -163, -163, null, 1442084400000, -163, 1861, 1, 1, 0.0, 1, 0.1111111111111111] - - ["PE", 1442084400000, -68, -231, null, 1442095200000, -163, 1861, 1, 2, 0.125, 2, 0.2222222222222222] - - ["PE", 1442095200000, -19, -250, 1442077200000, 1442026800000, -163, 1861, 1, 3, 0.25, 3, 0.3333333333333333] - - ["PE", 1442026800000, -12, -262, 1442084400000, 1442062800000, -163, 1861, 2, 4, 0.375, 4, 0.5555555555555556] - - ["PE", 1442062800000, -12, -274, 1442095200000, 1442080800000, -163, 1861, 2, 4, 0.375, 4, 0.5555555555555556] - - ["PE", 1442080800000, -2, -276, 1442026800000, 1442023200000, -163, 1861, 2, 6, 0.625, 5, 0.6666666666666666] - - ["PE", 1442023200000, 26, -250, 1442062800000, 1442019600000, -163, 1861, 3, 7, 0.75, 6, 0.7777777777777778] - - ["PE", 1442019600000, 523, 273, 1442080800000, 1442098800000, -163, 1861, 3, 8, 0.875, 7, 0.8888888888888888] - - ["PE", 1442098800000, 1861, 2134, 1442023200000, null, -163, 1861, 3, 9, 1.0, 8, 1.0] - - ["PH", 1442073600000, -227, -227, null, 1442041200000, -227, 1969, 1, 1, 0.0, 1, 0.047619047619047616] - - ["PH", 1442041200000, 0, -227, null, 1442077200000, -227, 1969, 1, 2, 0.05, 2, 0.09523809523809523] - - ["PH", 1442077200000, 2, -225, 1442073600000, 1442019600000, -227, 1969, 1, 3, 0.1, 3, 0.14285714285714285] - - ["PH", 1442019600000, 6, -219, 1442041200000, 1442098800000, -227, 1969, 1, 4, 0.15, 4, 0.19047619047619047] - - ["PH", 1442098800000, 8, -211, 1442077200000, 1442037600000, -227, 1969, 1, 5, 0.2, 5, 0.23809523809523808] - - ["PH", 1442037600000, 17, -194, 1442019600000, 1442052000000, -227, 1969, 1, 6, 0.25, 6, 0.2857142857142857] - - ["PH", 1442052000000, 22, -172, 1442098800000, 1442030400000, -227, 1969, 1, 7, 0.3, 7, 0.3333333333333333] - - ["PH", 1442030400000, 26, -146, 1442037600000, 1442080800000, -227, 1969, 2, 8, 0.35, 8, 0.38095238095238093] - - ["PH", 1442080800000, 32, -114, 1442052000000, 1442070000000, -227, 1969, 2, 9, 0.4, 9, 0.42857142857142855] - - ["PH", 1442070000000, 34, -80, 1442030400000, 1442084400000, -227, 1969, 2, 10, 0.45, 10, 0.47619047619047616] - - ["PH", 1442084400000, 39, -41, 1442080800000, 1442044800000, -227, 1969, 2, 11, 0.5, 11, 0.5238095238095238] - - ["PH", 1442044800000, 55, 14, 1442070000000, 1442034000000, -227, 1969, 2, 12, 0.55, 12, 0.5714285714285714] - - ["PH", 1442034000000, 59, 73, 1442084400000, 1442048400000, -227, 1969, 2, 13, 0.6, 13, 0.6190476190476191] - - ["PH", 1442048400000, 62, 135, 1442044800000, 1442062800000, -227, 1969, 2, 14, 0.65, 14, 0.6666666666666666] - - ["PH", 1442062800000, 171, 306, 1442034000000, 1442059200000, -227, 1969, 3, 15, 0.7, 15, 0.7142857142857143] - - ["PH", 1442059200000, 273, 579, 1442048400000, 1442023200000, -227, 1969, 3, 16, 0.75, 16, 0.7619047619047619] - - ["PH", 1442023200000, 459, 1038, 1442062800000, 1442091600000, -227, 1969, 3, 17, 0.8, 17, 0.8095238095238095] - - ["PH", 1442091600000, 816, 1854, 1442059200000, 1442026800000, -227, 1969, 3, 18, 0.85, 18, 0.8571428571428571] - - ["PH", 1442026800000, 910, 2764, 1442023200000, 1442066400000, -227, 1969, 3, 19, 0.9, 19, 0.9047619047619048] - - ["PH", 1442066400000, 1880, 4644, 1442091600000, 1442055600000, -227, 1969, 3, 20, 0.95, 20, 0.9523809523809523] - - ["PH", 1442055600000, 1969, 6613, 1442026800000, null, -227, 1969, 3, 21, 1.0, 21, 1.0] - - ["PK", 1442048400000, 15, 15, null, 1442062800000, 15, 335, 1, 1, 0.0, 1, 0.14285714285714285] - - ["PK", 1442062800000, 23, 38, null, 1442041200000, 15, 335, 1, 2, 0.16666666666666666, 2, 0.2857142857142857] - - ["PK", 1442041200000, 24, 62, 1442048400000, 1442070000000, 15, 335, 1, 3, 0.3333333333333333, 3, 0.42857142857142855] - - ["PK", 1442070000000, 43, 105, 1442062800000, 1442037600000, 15, 335, 2, 4, 0.5, 4, 0.5714285714285714] - - ["PK", 1442037600000, 100, 205, 1442041200000, 1442026800000, 15, 335, 2, 5, 0.6666666666666666, 5, 0.7142857142857143] - - ["PK", 1442026800000, 101, 306, 1442070000000, 1442019600000, 15, 335, 3, 6, 0.8333333333333334, 6, 0.8571428571428571] - - ["PK", 1442019600000, 335, 641, 1442037600000, null, 15, 335, 3, 7, 1.0, 7, 1.0] - - ["PL", 1442098800000, -9, -9, null, 1442080800000, -9, 4171, 1, 1, 0.0, 1, 0.05555555555555555] - - ["PL", 1442080800000, 7, -2, null, 1442084400000, -9, 4171, 1, 2, 0.058823529411764705, 2, 0.1111111111111111] - - ["PL", 1442084400000, 13, 11, 1442098800000, 1442073600000, -9, 4171, 1, 3, 0.11764705882352941, 3, 0.16666666666666666] - - ["PL", 1442073600000, 30, 41, 1442080800000, 1442066400000, -9, 4171, 1, 4, 0.17647058823529413, 4, 0.2222222222222222] - - ["PL", 1442066400000, 34, 75, 1442084400000, 1442037600000, -9, 4171, 1, 5, 0.23529411764705882, 5, 0.2777777777777778] - - ["PL", 1442037600000, 95, 170, 1442073600000, 1442070000000, -9, 4171, 1, 6, 0.29411764705882354, 6, 0.3333333333333333] - - ["PL", 1442070000000, 146, 316, 1442066400000, 1442059200000, -9, 4171, 2, 7, 0.35294117647058826, 7, 0.3888888888888889] - - ["PL", 1442059200000, 199, 515, 1442037600000, 1442041200000, -9, 4171, 2, 8, 0.4117647058823529, 8, 0.4444444444444444] - - ["PL", 1442041200000, 281, 796, 1442070000000, 1442044800000, -9, 4171, 2, 9, 0.47058823529411764, 9, 0.5] - - ["PL", 1442044800000, 319, 1115, 1442059200000, 1442077200000, -9, 4171, 2, 10, 0.5294117647058824, 10, 0.5555555555555556] - - ["PL", 1442077200000, 324, 1439, 1442041200000, 1442052000000, -9, 4171, 2, 11, 0.5882352941176471, 11, 0.6111111111111112] - - ["PL", 1442052000000, 330, 1769, 1442044800000, 1442088000000, -9, 4171, 2, 12, 0.6470588235294118, 12, 0.6666666666666666] - - ["PL", 1442088000000, 346, 2115, 1442077200000, 1442048400000, -9, 4171, 3, 13, 0.7058823529411765, 13, 0.7222222222222222] - - ["PL", 1442048400000, 366, 2481, 1442052000000, 1442055600000, -9, 4171, 3, 14, 0.7647058823529411, 14, 0.7777777777777778] - - ["PL", 1442055600000, 410, 2891, 1442088000000, 1442091600000, -9, 4171, 3, 15, 0.8235294117647058, 15, 0.8333333333333334] - - ["PL", 1442091600000, 902, 3793, 1442048400000, 1442095200000, -9, 4171, 3, 16, 0.8823529411764706, 16, 0.8888888888888888] - - ["PL", 1442095200000, 1851, 5644, 1442055600000, 1442062800000, -9, 4171, 3, 17, 0.9411764705882353, 17, 0.9444444444444444] - - ["PL", 1442062800000, 4171, 9815, 1442091600000, null, -9, 4171, 3, 18, 1.0, 18, 1.0] - - ["PR", 1442059200000, -35, -35, null, 1442030400000, -35, 29, 1, 1, 0.0, 1, 0.2] - - ["PR", 1442030400000, 2, -33, null, 1442077200000, -35, 29, 1, 2, 0.25, 2, 0.4] - - ["PR", 1442077200000, 5, -28, 1442059200000, 1442026800000, -35, 29, 2, 3, 0.5, 3, 0.6] - - ["PR", 1442026800000, 22, -6, 1442030400000, 1442095200000, -35, 29, 2, 4, 0.75, 4, 0.8] - - ["PR", 1442095200000, 29, 23, 1442077200000, null, -35, 29, 3, 5, 1.0, 5, 1.0] - - ["PT", 1442080800000, -79, -79, null, 1442077200000, -79, 3470, 1, 1, 0.0, 1, 0.1] - - ["PT", 1442077200000, -75, -154, null, 1442098800000, -79, 3470, 1, 2, 0.1111111111111111, 2, 0.2] - - ["PT", 1442098800000, 2, -152, 1442080800000, 1442044800000, -79, 3470, 1, 3, 0.2222222222222222, 3, 0.3] - - ["PT", 1442044800000, 11, -141, 1442077200000, 1442066400000, -79, 3470, 1, 4, 0.3333333333333333, 4, 0.4] - - ["PT", 1442066400000, 12, -129, 1442098800000, 1442095200000, -79, 3470, 2, 5, 0.4444444444444444, 5, 0.5] - - ["PT", 1442095200000, 19, -110, 1442044800000, 1442052000000, -79, 3470, 2, 6, 0.5555555555555556, 6, 0.6] - - ["PT", 1442052000000, 102, -8, 1442066400000, 1442019600000, -79, 3470, 2, 7, 0.6666666666666666, 7, 0.7] - - ["PT", 1442019600000, 172, 164, 1442095200000, 1442088000000, -79, 3470, 3, 8, 0.7777777777777778, 8, 0.8] - - ["PT", 1442088000000, 403, 567, 1442052000000, 1442070000000, -79, 3470, 3, 9, 0.8888888888888888, 9, 0.9] - - ["PT", 1442070000000, 3470, 4037, 1442019600000, null, -79, 3470, 3, 10, 1.0, 10, 1.0] - - ["PY", 1442019600000, 1, 1, null, 1442080800000, 1, 628, 1, 1, 0.0, 1, 0.3333333333333333] - - ["PY", 1442080800000, 5, 6, null, 1442084400000, 1, 628, 2, 2, 0.5, 2, 0.6666666666666666] - - ["PY", 1442084400000, 628, 634, 1442019600000, null, 1, 628, 3, 3, 1.0, 3, 1.0] - - ["QA", 1442041200000, 13, 13, null, null, 13, 13, 1, 1, 0.0, 1, 1.0] - - ["RO", 1442070000000, -29, -29, null, 1442091600000, -29, 845, 1, 1, 0.0, 1, 0.1] - - ["RO", 1442091600000, 0, -29, null, 1442073600000, -29, 845, 1, 2, 0.1111111111111111, 2, 0.2] - - ["RO", 1442073600000, 15, -14, 1442070000000, 1442055600000, -29, 845, 1, 3, 0.2222222222222222, 3, 0.3] - - ["RO", 1442055600000, 26, 12, 1442091600000, 1442034000000, -29, 845, 1, 4, 0.3333333333333333, 4, 0.4] - - ["RO", 1442034000000, 68, 80, 1442073600000, 1442044800000, -29, 845, 2, 5, 0.4444444444444444, 5, 0.5] - - ["RO", 1442044800000, 284, 364, 1442055600000, 1442052000000, -29, 845, 2, 6, 0.5555555555555556, 6, 0.6] - - ["RO", 1442052000000, 319, 683, 1442034000000, 1442062800000, -29, 845, 2, 7, 0.6666666666666666, 7, 0.7] - - ["RO", 1442062800000, 541, 1224, 1442044800000, 1442095200000, -29, 845, 3, 8, 0.7777777777777778, 8, 0.8] - - ["RO", 1442095200000, 824, 2048, 1442052000000, 1442041200000, -29, 845, 3, 9, 0.8888888888888888, 9, 0.9] - - ["RO", 1442041200000, 845, 2893, 1442062800000, null, -29, 845, 3, 10, 1.0, 10, 1.0] - - ["RS", 1442091600000, -15, -15, null, 1442066400000, -15, 813, 1, 1, 0.0, 1, 0.14285714285714285] - - ["RS", 1442066400000, 0, -15, null, 1442080800000, -15, 813, 1, 2, 0.16666666666666666, 2, 0.42857142857142855] - - ["RS", 1442080800000, 0, -15, 1442091600000, 1442019600000, -15, 813, 1, 2, 0.16666666666666666, 2, 0.42857142857142855] - - ["RS", 1442019600000, 6, -9, 1442066400000, 1442062800000, -15, 813, 2, 4, 0.5, 3, 0.5714285714285714] - - ["RS", 1442062800000, 13, 4, 1442080800000, 1442084400000, -15, 813, 2, 5, 0.6666666666666666, 4, 0.7142857142857143] - - ["RS", 1442084400000, 89, 93, 1442019600000, 1442073600000, -15, 813, 3, 6, 0.8333333333333334, 5, 0.8571428571428571] - - ["RS", 1442073600000, 813, 906, 1442062800000, null, -15, 813, 3, 7, 1.0, 6, 1.0] - - ["RU", 1442037600000, -324, -324, null, 1442026800000, -324, 12098, 1, 1, 0.0, 1, 0.043478260869565216] - - ["RU", 1442026800000, 0, -324, null, 1442030400000, -324, 12098, 1, 2, 0.045454545454545456, 2, 0.08695652173913043] - - ["RU", 1442030400000, 76, -248, 1442037600000, 1442062800000, -324, 12098, 1, 3, 0.09090909090909091, 3, 0.13043478260869565] - - ["RU", 1442062800000, 168, -80, 1442026800000, 1442023200000, -324, 12098, 1, 4, 0.13636363636363635, 4, 0.17391304347826086] - - ["RU", 1442023200000, 299, 219, 1442030400000, 1442095200000, -324, 12098, 1, 5, 0.18181818181818182, 5, 0.21739130434782608] - - ["RU", 1442095200000, 435, 654, 1442062800000, 1442055600000, -324, 12098, 1, 6, 0.22727272727272727, 6, 0.2608695652173913] - - ["RU", 1442055600000, 499, 1153, 1442023200000, 1442041200000, -324, 12098, 1, 7, 0.2727272727272727, 7, 0.30434782608695654] - - ["RU", 1442041200000, 580, 1733, 1442095200000, 1442080800000, -324, 12098, 1, 8, 0.3181818181818182, 8, 0.34782608695652173] - - ["RU", 1442080800000, 655, 2388, 1442055600000, 1442034000000, -324, 12098, 2, 9, 0.36363636363636365, 9, 0.391304347826087] - - ["RU", 1442034000000, 658, 3046, 1442041200000, 1442048400000, -324, 12098, 2, 10, 0.4090909090909091, 10, 0.43478260869565216] - - ["RU", 1442048400000, 1027, 4073, 1442080800000, 1442077200000, -324, 12098, 2, 11, 0.45454545454545453, 11, 0.4782608695652174] - - ["RU", 1442077200000, 1162, 5235, 1442034000000, 1442052000000, -324, 12098, 2, 12, 0.5, 12, 0.5217391304347826] - - ["RU", 1442052000000, 1214, 6449, 1442048400000, 1442073600000, -324, 12098, 2, 13, 0.5454545454545454, 13, 0.5652173913043478] - - ["RU", 1442073600000, 1618, 8067, 1442077200000, 1442066400000, -324, 12098, 2, 14, 0.5909090909090909, 14, 0.6086956521739131] - - ["RU", 1442066400000, 2047, 10114, 1442052000000, 1442019600000, -324, 12098, 2, 15, 0.6363636363636364, 15, 0.6521739130434783] - - ["RU", 1442019600000, 2214, 12328, 1442073600000, 1442044800000, -324, 12098, 2, 16, 0.6818181818181818, 16, 0.6956521739130435] - - ["RU", 1442044800000, 2564, 14892, 1442066400000, 1442088000000, -324, 12098, 3, 17, 0.7272727272727273, 17, 0.7391304347826086] - - ["RU", 1442088000000, 2596, 17488, 1442019600000, 1442091600000, -324, 12098, 3, 18, 0.7727272727272727, 18, 0.782608695652174] - - ["RU", 1442091600000, 3449, 20937, 1442044800000, 1442059200000, -324, 12098, 3, 19, 0.8181818181818182, 19, 0.8260869565217391] - - ["RU", 1442059200000, 3902, 24839, 1442088000000, 1442070000000, -324, 12098, 3, 20, 0.8636363636363636, 20, 0.8695652173913043] - - ["RU", 1442070000000, 4706, 29545, 1442091600000, 1442084400000, -324, 12098, 3, 21, 0.9090909090909091, 21, 0.9130434782608695] - - ["RU", 1442084400000, 6461, 36006, 1442059200000, 1442098800000, -324, 12098, 3, 22, 0.9545454545454546, 22, 0.9565217391304348] - - ["RU", 1442098800000, 12098, 48104, 1442070000000, null, -324, 12098, 3, 23, 1.0, 23, 1.0] - - ["SA", 1442037600000, -97, -97, null, 1442077200000, -97, 1276, 1, 1, 0.0, 1, 0.125] - - ["SA", 1442077200000, -50, -147, null, 1442059200000, -97, 1276, 1, 2, 0.14285714285714285, 2, 0.25] - - ["SA", 1442059200000, 0, -147, 1442037600000, 1442073600000, -97, 1276, 1, 3, 0.2857142857142857, 3, 0.375] - - ["SA", 1442073600000, 2, -145, 1442077200000, 1442055600000, -97, 1276, 2, 4, 0.42857142857142855, 4, 0.5] - - ["SA", 1442055600000, 11, -134, 1442059200000, 1442048400000, -97, 1276, 2, 5, 0.5714285714285714, 5, 0.625] - - ["SA", 1442048400000, 14, -120, 1442073600000, 1442084400000, -97, 1276, 2, 6, 0.7142857142857143, 6, 0.75] - - ["SA", 1442084400000, 458, 338, 1442055600000, 1442066400000, -97, 1276, 3, 7, 0.8571428571428571, 7, 0.875] - - ["SA", 1442066400000, 1276, 1614, 1442048400000, null, -97, 1276, 3, 8, 1.0, 8, 1.0] - - ["SE", 1442048400000, -145, -145, null, 1442055600000, -145, 1476, 1, 1, 0.0, 1, 0.06666666666666667] - - ["SE", 1442055600000, -5, -150, null, 1442091600000, -145, 1476, 1, 2, 0.07142857142857142, 2, 0.13333333333333333] - - ["SE", 1442091600000, -1, -151, 1442048400000, 1442098800000, -145, 1476, 1, 3, 0.14285714285714285, 3, 0.2] - - ["SE", 1442098800000, 0, -151, 1442055600000, 1442052000000, -145, 1476, 1, 4, 0.21428571428571427, 4, 0.26666666666666666] - - ["SE", 1442052000000, 1, -150, 1442091600000, 1442023200000, -145, 1476, 1, 5, 0.2857142857142857, 5, 0.3333333333333333] - - ["SE", 1442023200000, 3, -147, 1442098800000, 1442066400000, -145, 1476, 2, 6, 0.35714285714285715, 6, 0.4] - - ["SE", 1442066400000, 14, -133, 1442052000000, 1442030400000, -145, 1476, 2, 7, 0.42857142857142855, 7, 0.4666666666666667] - - ["SE", 1442030400000, 30, -103, 1442023200000, 1442084400000, -145, 1476, 2, 8, 0.5, 8, 0.5333333333333333] - - ["SE", 1442084400000, 37, -66, 1442066400000, 1442095200000, -145, 1476, 2, 9, 0.5714285714285714, 9, 0.6] - - ["SE", 1442095200000, 61, -5, 1442030400000, 1442070000000, -145, 1476, 2, 10, 0.6428571428571429, 10, 0.6666666666666666] - - ["SE", 1442070000000, 78, 73, 1442084400000, 1442080800000, -145, 1476, 3, 11, 0.7142857142857143, 11, 0.7333333333333333] - - ["SE", 1442080800000, 89, 162, 1442095200000, 1442041200000, -145, 1476, 3, 12, 0.7857142857142857, 12, 0.8] - - ["SE", 1442041200000, 91, 253, 1442070000000, 1442019600000, -145, 1476, 3, 13, 0.8571428571428571, 13, 0.8666666666666667] - - ["SE", 1442019600000, 109, 362, 1442080800000, 1442059200000, -145, 1476, 3, 14, 0.9285714285714286, 14, 0.9333333333333333] - - ["SE", 1442059200000, 1476, 1838, 1442041200000, null, -145, 1476, 3, 15, 1.0, 15, 1.0] - - ["SG", 1442066400000, 0, 0, null, 1442030400000, 0, 2758, 1, 1, 0.0, 1, 0.125] - - ["SG", 1442030400000, 1, 1, null, 1442037600000, 0, 2758, 1, 2, 0.14285714285714285, 2, 0.25] - - ["SG", 1442037600000, 3, 4, 1442066400000, 1442048400000, 0, 2758, 1, 3, 0.2857142857142857, 3, 0.375] - - ["SG", 1442048400000, 52, 56, 1442030400000, 1442041200000, 0, 2758, 2, 4, 0.42857142857142855, 4, 0.5] - - ["SG", 1442041200000, 59, 115, 1442037600000, 1442044800000, 0, 2758, 2, 5, 0.5714285714285714, 5, 0.625] - - ["SG", 1442044800000, 77, 192, 1442048400000, 1442062800000, 0, 2758, 2, 6, 0.7142857142857143, 6, 0.75] - - ["SG", 1442062800000, 388, 580, 1442041200000, 1442026800000, 0, 2758, 3, 7, 0.8571428571428571, 7, 0.875] - - ["SG", 1442026800000, 2758, 3338, 1442044800000, null, 0, 2758, 3, 8, 1.0, 8, 1.0] - - ["SI", 1442080800000, -45, -45, null, 1442091600000, -45, 9, 1, 1, 0.0, 1, 0.5] - - ["SI", 1442091600000, 9, -36, null, null, -45, 9, 2, 2, 1.0, 2, 1.0] - - ["SK", 1442084400000, -92, -92, null, 1442037600000, -92, 446, 1, 1, 0.0, 1, 0.16666666666666666] - - ["SK", 1442037600000, -1, -93, null, 1442062800000, -92, 446, 1, 2, 0.2, 2, 0.3333333333333333] - - ["SK", 1442062800000, 6, -87, 1442084400000, 1442098800000, -92, 446, 2, 3, 0.4, 3, 0.5] - - ["SK", 1442098800000, 7, -80, 1442037600000, 1442052000000, -92, 446, 2, 4, 0.6, 4, 0.6666666666666666] - - ["SK", 1442052000000, 13, -67, 1442062800000, 1442073600000, -92, 446, 3, 5, 0.8, 5, 0.8333333333333334] - - ["SK", 1442073600000, 446, 379, 1442098800000, null, -92, 446, 3, 6, 1.0, 6, 1.0] - - ["SV", 1442019600000, -1, -1, null, 1442088000000, -1, 106, 1, 1, 0.0, 1, 0.3333333333333333] - - ["SV", 1442088000000, 9, 8, null, 1442084400000, -1, 106, 2, 2, 0.5, 2, 0.6666666666666666] - - ["SV", 1442084400000, 106, 114, 1442019600000, null, -1, 106, 3, 3, 1.0, 3, 1.0] - - ["TH", 1442062800000, -46, -46, null, 1442066400000, -46, 110, 1, 1, 0.0, 1, 0.1111111111111111] - - ["TH", 1442066400000, -34, -80, null, 1442052000000, -46, 110, 1, 2, 0.125, 2, 0.2222222222222222] - - ["TH", 1442052000000, -22, -102, 1442062800000, 1442034000000, -46, 110, 1, 3, 0.25, 3, 0.3333333333333333] - - ["TH", 1442034000000, 0, -102, 1442066400000, 1442055600000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666] - - ["TH", 1442055600000, 0, -102, 1442052000000, 1442070000000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666] - - ["TH", 1442070000000, 0, -102, 1442034000000, 1442041200000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666] - - ["TH", 1442041200000, 3, -99, 1442055600000, 1442084400000, -46, 110, 3, 7, 0.75, 5, 0.7777777777777778] - - ["TH", 1442084400000, 13, -86, 1442070000000, 1442044800000, -46, 110, 3, 8, 0.875, 6, 0.8888888888888888] - - ["TH", 1442044800000, 110, 24, 1442041200000, null, -46, 110, 3, 9, 1.0, 7, 1.0] - - ["TJ", 1442048400000, 1471, 1471, null, null, 1471, 1471, 1, 1, 0.0, 1, 1.0] - - ["TN", 1442098800000, -9, -9, null, null, -9, -9, 1, 1, 0.0, 1, 1.0] - - ["TR", 1442095200000, -29, -29, null, 1442080800000, -29, 3048, 1, 1, 0.0, 1, 0.06666666666666667] - - ["TR", 1442080800000, -1, -30, null, 1442041200000, -29, 3048, 1, 2, 0.07142857142857142, 2, 0.13333333333333333] - - ["TR", 1442041200000, 1, -29, 1442095200000, 1442044800000, -29, 3048, 1, 3, 0.14285714285714285, 3, 0.2] - - ["TR", 1442044800000, 41, 12, 1442080800000, 1442052000000, -29, 3048, 1, 4, 0.21428571428571427, 4, 0.3333333333333333] - - ["TR", 1442052000000, 41, 53, 1442041200000, 1442066400000, -29, 3048, 1, 4, 0.21428571428571427, 4, 0.3333333333333333] - - ["TR", 1442066400000, 85, 138, 1442044800000, 1442048400000, -29, 3048, 2, 6, 0.35714285714285715, 5, 0.4] - - ["TR", 1442048400000, 88, 226, 1442052000000, 1442077200000, -29, 3048, 2, 7, 0.42857142857142855, 6, 0.4666666666666667] - - ["TR", 1442077200000, 89, 315, 1442066400000, 1442084400000, -29, 3048, 2, 8, 0.5, 7, 0.5333333333333333] - - ["TR", 1442084400000, 170, 485, 1442048400000, 1442070000000, -29, 3048, 2, 9, 0.5714285714285714, 8, 0.6] - - ["TR", 1442070000000, 236, 721, 1442077200000, 1442055600000, -29, 3048, 2, 10, 0.6428571428571429, 9, 0.6666666666666666] - - ["TR", 1442055600000, 299, 1020, 1442084400000, 1442023200000, -29, 3048, 3, 11, 0.7142857142857143, 10, 0.7333333333333333] - - ["TR", 1442023200000, 306, 1326, 1442070000000, 1442062800000, -29, 3048, 3, 12, 0.7857142857142857, 11, 0.8] - - ["TR", 1442062800000, 315, 1641, 1442055600000, 1442088000000, -29, 3048, 3, 13, 0.8571428571428571, 12, 0.8666666666666667] - - ["TR", 1442088000000, 2389, 4030, 1442023200000, 1442091600000, -29, 3048, 3, 14, 0.9285714285714286, 13, 0.9333333333333333] - - ["TR", 1442091600000, 3048, 7078, 1442062800000, null, -29, 3048, 3, 15, 1.0, 14, 1.0] - - ["TT", 1442088000000, 9, 9, null, null, 9, 9, 1, 1, 0.0, 1, 1.0] - - ["TW", 1442062800000, -272, -272, null, 1442059200000, -272, 772, 1, 1, 0.0, 1, 0.045454545454545456] - - ["TW", 1442059200000, -157, -429, null, 1442095200000, -272, 772, 1, 2, 0.047619047619047616, 2, 0.09090909090909091] - - ["TW", 1442095200000, -77, -506, 1442062800000, 1442098800000, -272, 772, 1, 3, 0.09523809523809523, 3, 0.13636363636363635] - - ["TW", 1442098800000, -60, -566, 1442059200000, 1442019600000, -272, 772, 1, 4, 0.14285714285714285, 4, 0.18181818181818182] - - ["TW", 1442019600000, 0, -566, 1442095200000, 1442030400000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182] - - ["TW", 1442030400000, 0, -566, 1442098800000, 1442084400000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182] - - ["TW", 1442084400000, 0, -566, 1442019600000, 1442044800000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182] - - ["TW", 1442044800000, 24, -542, 1442030400000, 1442052000000, -272, 772, 1, 8, 0.3333333333333333, 6, 0.45454545454545453] - - ["TW", 1442052000000, 24, -518, 1442084400000, 1442080800000, -272, 772, 2, 8, 0.3333333333333333, 6, 0.45454545454545453] - - ["TW", 1442080800000, 24, -494, 1442044800000, 1442055600000, -272, 772, 2, 8, 0.3333333333333333, 6, 0.45454545454545453] - - ["TW", 1442055600000, 48, -446, 1442052000000, 1442048400000, -272, 772, 2, 11, 0.47619047619047616, 7, 0.5] - - ["TW", 1442048400000, 75, -371, 1442080800000, 1442016000000, -272, 772, 2, 12, 0.5238095238095238, 8, 0.5454545454545454] - - ["TW", 1442016000000, 92, -279, 1442055600000, 1442023200000, -272, 772, 2, 13, 0.5714285714285714, 9, 0.5909090909090909] - - ["TW", 1442023200000, 97, -182, 1442048400000, 1442034000000, -272, 772, 2, 14, 0.6190476190476191, 10, 0.6363636363636364] - - ["TW", 1442034000000, 143, -39, 1442016000000, 1442037600000, -272, 772, 2, 15, 0.6666666666666666, 11, 0.6818181818181818] - - ["TW", 1442037600000, 266, 227, 1442023200000, 1442041200000, -272, 772, 3, 16, 0.7142857142857143, 12, 0.7272727272727273] - - ["TW", 1442041200000, 366, 593, 1442034000000, 1442070000000, -272, 772, 3, 17, 0.7619047619047619, 13, 0.7727272727272727] - - ["TW", 1442070000000, 485, 1078, 1442037600000, 1442077200000, -272, 772, 3, 18, 0.8095238095238095, 14, 0.8181818181818182] - - ["TW", 1442077200000, 502, 1580, 1442041200000, 1442066400000, -272, 772, 3, 19, 0.8571428571428571, 15, 0.8636363636363636] - - ["TW", 1442066400000, 624, 2204, 1442070000000, 1442026800000, -272, 772, 3, 20, 0.9047619047619048, 16, 0.9090909090909091] - - ["TW", 1442026800000, 680, 2884, 1442077200000, 1442073600000, -272, 772, 3, 21, 0.9523809523809523, 17, 0.9545454545454546] - - ["TW", 1442073600000, 772, 3656, 1442066400000, null, -272, 772, 3, 22, 1.0, 18, 1.0] - - ["UA", 1442091600000, -388, -388, null, 1442077200000, -388, 14202, 1, 1, 0.0, 1, 0.05263157894736842] - - ["UA", 1442077200000, -181, -569, null, 1442095200000, -388, 14202, 1, 2, 0.05555555555555555, 2, 0.10526315789473684] - - ["UA", 1442095200000, -30, -599, 1442091600000, 1442088000000, -388, 14202, 1, 3, 0.1111111111111111, 3, 0.15789473684210525] - - ["UA", 1442088000000, -21, -620, 1442077200000, 1442059200000, -388, 14202, 1, 4, 0.16666666666666666, 4, 0.21052631578947367] - - ["UA", 1442059200000, -2, -622, 1442095200000, 1442037600000, -388, 14202, 1, 5, 0.2222222222222222, 5, 0.2631578947368421] - - ["UA", 1442037600000, -1, -623, 1442088000000, 1442080800000, -388, 14202, 1, 6, 0.2777777777777778, 6, 0.3684210526315789] - - ["UA", 1442080800000, -1, -624, 1442059200000, 1442048400000, -388, 14202, 1, 6, 0.2777777777777778, 6, 0.3684210526315789] - - ["UA", 1442048400000, 2, -622, 1442037600000, 1442084400000, -388, 14202, 2, 8, 0.3888888888888889, 7, 0.42105263157894735] - - ["UA", 1442084400000, 5, -617, 1442080800000, 1442098800000, -388, 14202, 2, 9, 0.4444444444444444, 8, 0.47368421052631576] - - ["UA", 1442098800000, 38, -579, 1442048400000, 1442041200000, -388, 14202, 2, 10, 0.5, 9, 0.5263157894736842] - - ["UA", 1442041200000, 74, -505, 1442084400000, 1442044800000, -388, 14202, 2, 11, 0.5555555555555556, 10, 0.5789473684210527] - - ["UA", 1442044800000, 280, -225, 1442098800000, 1442066400000, -388, 14202, 2, 12, 0.6111111111111112, 11, 0.631578947368421] - - ["UA", 1442066400000, 296, 71, 1442041200000, 1442052000000, -388, 14202, 2, 13, 0.6666666666666666, 12, 0.6842105263157895] - - ["UA", 1442052000000, 410, 481, 1442044800000, 1442062800000, -388, 14202, 3, 14, 0.7222222222222222, 13, 0.7368421052631579] - - ["UA", 1442062800000, 773, 1254, 1442066400000, 1442070000000, -388, 14202, 3, 15, 0.7777777777777778, 14, 0.7894736842105263] - - ["UA", 1442070000000, 1733, 2987, 1442052000000, 1442034000000, -388, 14202, 3, 16, 0.8333333333333334, 15, 0.8421052631578947] - - ["UA", 1442034000000, 3468, 6455, 1442062800000, 1442073600000, -388, 14202, 3, 17, 0.8888888888888888, 16, 0.8947368421052632] - - ["UA", 1442073600000, 4241, 10696, 1442070000000, 1442055600000, -388, 14202, 3, 18, 0.9444444444444444, 17, 0.9473684210526315] - - ["UA", 1442055600000, 14202, 24898, 1442034000000, null, -388, 14202, 3, 19, 1.0, 18, 1.0] - - ["UG", 1442070000000, 1, 1, null, null, 1, 1, 1, 1, 0.0, 1, 1.0] - - ["US", 1442048400000, -466, -466, null, 1442052000000, -466, 4001, 1, 1, 0.0, 1, 0.041666666666666664] - - ["US", 1442052000000, -2, -468, null, 1442016000000, -466, 4001, 1, 2, 0.043478260869565216, 2, 0.08333333333333333] - - ["US", 1442016000000, 0, -468, 1442048400000, 1442059200000, -466, 4001, 1, 3, 0.08695652173913043, 3, 0.125] - - ["US", 1442059200000, 11, -457, 1442052000000, 1442062800000, -466, 4001, 1, 4, 0.13043478260869565, 4, 0.16666666666666666] - - ["US", 1442062800000, 47, -410, 1442016000000, 1442044800000, -466, 4001, 1, 5, 0.17391304347826086, 5, 0.20833333333333334] - - ["US", 1442044800000, 139, -271, 1442059200000, 1442055600000, -466, 4001, 1, 6, 0.21739130434782608, 6, 0.25] - - ["US", 1442055600000, 156, -115, 1442062800000, 1442095200000, -466, 4001, 1, 7, 0.2608695652173913, 7, 0.2916666666666667] - - ["US", 1442095200000, 416, 301, 1442044800000, 1442066400000, -466, 4001, 1, 8, 0.30434782608695654, 8, 0.3333333333333333] - - ["US", 1442066400000, 772, 1073, 1442055600000, 1442019600000, -466, 4001, 2, 9, 0.34782608695652173, 9, 0.375] - - ["US", 1442019600000, 1043, 2116, 1442095200000, 1442073600000, -466, 4001, 2, 10, 0.391304347826087, 10, 0.4166666666666667] - - ["US", 1442073600000, 1100, 3216, 1442066400000, 1442026800000, -466, 4001, 2, 11, 0.43478260869565216, 11, 0.4583333333333333] - - ["US", 1442026800000, 1512, 4728, 1442019600000, 1442088000000, -466, 4001, 2, 12, 0.4782608695652174, 12, 0.5] - - ["US", 1442088000000, 1691, 6419, 1442073600000, 1442041200000, -466, 4001, 2, 13, 0.5217391304347826, 13, 0.5416666666666666] - - ["US", 1442041200000, 1999, 8418, 1442026800000, 1442030400000, -466, 4001, 2, 14, 0.5652173913043478, 14, 0.5833333333333334] - - ["US", 1442030400000, 2023, 10441, 1442088000000, 1442077200000, -466, 4001, 2, 15, 0.6086956521739131, 15, 0.625] - - ["US", 1442077200000, 2168, 12609, 1442041200000, 1442091600000, -466, 4001, 2, 16, 0.6521739130434783, 16, 0.6666666666666666] - - ["US", 1442091600000, 2502, 15111, 1442030400000, 1442084400000, -466, 4001, 3, 17, 0.6956521739130435, 17, 0.7083333333333334] - - ["US", 1442084400000, 2523, 17634, 1442077200000, 1442023200000, -466, 4001, 3, 18, 0.7391304347826086, 18, 0.75] - - ["US", 1442023200000, 2844, 20478, 1442091600000, 1442070000000, -466, 4001, 3, 19, 0.782608695652174, 19, 0.7916666666666666] - - ["US", 1442070000000, 3505, 23983, 1442084400000, 1442098800000, -466, 4001, 3, 20, 0.8260869565217391, 20, 0.8333333333333334] - - ["US", 1442098800000, 3575, 27558, 1442023200000, 1442034000000, -466, 4001, 3, 21, 0.8695652173913043, 21, 0.875] - - ["US", 1442034000000, 3648, 31206, 1442070000000, 1442037600000, -466, 4001, 3, 22, 0.9130434782608695, 22, 0.9166666666666666] - - ["US", 1442037600000, 3675, 34881, 1442098800000, 1442080800000, -466, 4001, 3, 23, 0.9565217391304348, 23, 0.9583333333333334] - - ["US", 1442080800000, 4001, 38882, 1442034000000, null, -466, 4001, 3, 24, 1.0, 24, 1.0] - - ["UY", 1442073600000, -42, -42, null, 1442037600000, -42, 517, 1, 1, 0.0, 1, 0.14285714285714285] - - ["UY", 1442037600000, 1, -41, null, 1442077200000, -42, 517, 1, 2, 0.16666666666666666, 2, 0.2857142857142857] - - ["UY", 1442077200000, 23, -18, 1442073600000, 1442026800000, -42, 517, 1, 3, 0.3333333333333333, 3, 0.42857142857142855] - - ["UY", 1442026800000, 76, 58, 1442037600000, 1442019600000, -42, 517, 2, 4, 0.5, 4, 0.5714285714285714] - - ["UY", 1442019600000, 77, 135, 1442077200000, 1442070000000, -42, 517, 2, 5, 0.6666666666666666, 5, 0.7142857142857143] - - ["UY", 1442070000000, 284, 419, 1442026800000, 1442023200000, -42, 517, 3, 6, 0.8333333333333334, 6, 0.8571428571428571] - - ["UY", 1442023200000, 517, 936, 1442019600000, null, -42, 517, 3, 7, 1.0, 7, 1.0] - - ["UZ", 1442044800000, 1369, 1369, null, null, 1369, 1369, 1, 1, 0.0, 1, 1.0] - - ["VE", 1442026800000, -17, -17, null, 1442034000000, -17, 420, 1, 1, 0.0, 1, 0.1] - - ["VE", 1442034000000, -2, -19, null, 1442098800000, -17, 420, 1, 2, 0.1111111111111111, 2, 0.2] - - ["VE", 1442098800000, 9, -10, 1442026800000, 1442066400000, -17, 420, 1, 3, 0.2222222222222222, 3, 0.3] - - ["VE", 1442066400000, 18, 8, 1442034000000, 1442095200000, -17, 420, 1, 4, 0.3333333333333333, 4, 0.4] - - ["VE", 1442095200000, 35, 43, 1442098800000, 1442030400000, -17, 420, 2, 5, 0.4444444444444444, 5, 0.5] - - ["VE", 1442030400000, 51, 94, 1442066400000, 1442084400000, -17, 420, 2, 6, 0.5555555555555556, 6, 0.6] - - ["VE", 1442084400000, 60, 154, 1442095200000, 1442023200000, -17, 420, 2, 7, 0.6666666666666666, 7, 0.7] - - ["VE", 1442023200000, 115, 269, 1442030400000, 1442077200000, -17, 420, 3, 8, 0.7777777777777778, 8, 0.8] - - ["VE", 1442077200000, 412, 681, 1442084400000, 1442070000000, -17, 420, 3, 9, 0.8888888888888888, 9, 0.9] - - ["VE", 1442070000000, 420, 1101, 1442023200000, null, -17, 420, 3, 10, 1.0, 10, 1.0] - - ["VG", 1442062800000, -238, -238, null, null, -238, -238, 1, 1, 0.0, 1, 1.0] - - ["VN", 1442034000000, -29, -29, null, 1442048400000, -29, 811, 1, 1, 0.0, 1, 0.07692307692307693] - - ["VN", 1442048400000, -15, -44, null, 1442037600000, -29, 811, 1, 2, 0.08333333333333333, 2, 0.15384615384615385] - - ["VN", 1442037600000, -11, -55, 1442034000000, 1442084400000, -29, 811, 1, 3, 0.16666666666666666, 3, 0.23076923076923078] - - ["VN", 1442084400000, -10, -65, 1442048400000, 1442023200000, -29, 811, 1, 4, 0.25, 4, 0.3076923076923077] - - ["VN", 1442023200000, -9, -74, 1442037600000, 1442041200000, -29, 811, 1, 5, 0.3333333333333333, 5, 0.38461538461538464] - - ["VN", 1442041200000, 0, -74, 1442084400000, 1442059200000, -29, 811, 2, 6, 0.4166666666666667, 6, 0.46153846153846156] - - ["VN", 1442059200000, 8, -66, 1442023200000, 1442055600000, -29, 811, 2, 7, 0.5, 7, 0.5384615384615384] - - ["VN", 1442055600000, 37, -29, 1442041200000, 1442026800000, -29, 811, 2, 8, 0.5833333333333334, 8, 0.6153846153846154] - - ["VN", 1442026800000, 63, 34, 1442059200000, 1442052000000, -29, 811, 2, 9, 0.6666666666666666, 9, 0.6923076923076923] - - ["VN", 1442052000000, 90, 124, 1442055600000, 1442062800000, -29, 811, 3, 10, 0.75, 10, 0.7692307692307693] - - ["VN", 1442062800000, 146, 270, 1442026800000, 1442070000000, -29, 811, 3, 11, 0.8333333333333334, 11, 0.8461538461538461] - - ["VN", 1442070000000, 479, 749, 1442052000000, 1442066400000, -29, 811, 3, 12, 0.9166666666666666, 12, 0.9230769230769231] - - ["VN", 1442066400000, 811, 1560, 1442062800000, null, -29, 811, 3, 13, 1.0, 13, 1.0] - - ["ZA", 1442034000000, -3, -3, null, 1442070000000, -3, 79, 1, 1, 0.0, 1, 0.2] - - ["ZA", 1442070000000, 0, -3, null, 1442091600000, -3, 79, 1, 2, 0.25, 2, 0.4] - - ["ZA", 1442091600000, 1, -2, 1442034000000, 1442059200000, -3, 79, 2, 3, 0.5, 3, 0.6] - - ["ZA", 1442059200000, 50, 48, 1442070000000, 1442048400000, -3, 79, 2, 4, 0.75, 4, 0.8] - - ["ZA", 1442048400000, 79, 127, 1442091600000, null, -3, 79, 3, 5, 1.0, 5, 1.0] - - ["ZM", 1442041200000, 133, 133, null, null, 133, 133, 1, 1, 0.0, 1, 1.0] - - ["ZW", 1442044800000, 0, 0, null, 1442048400000, 0, 254, 1, 1, 0.0, 1, 0.5] - - ["ZW", 1442048400000, 254, 254, null, null, 0, 254, 2, 2, 1.0, 2, 1.0] \ No newline at end of file + - [ "", 1442016000000, 29873, 29873, null, 1442030400000, 29873, 787370, 1, 1, 0.0, 1, 0.041666666666666664 ] + - [ "", 1442030400000, 166672, 196545, null, 1442019600000, 29873, 787370, 1, 2, 0.043478260869565216, 2, 0.08333333333333333 ] + - [ "", 1442019600000, 173892, 370437, 1442016000000, 1442037600000, 29873, 787370, 1, 3, 0.08695652173913043, 3, 0.125 ] + - [ "", 1442037600000, 200605, 571042, 1442030400000, 1442026800000, 29873, 787370, 1, 4, 0.13043478260869565, 4, 0.16666666666666666 ] + - [ "", 1442026800000, 252626, 823668, 1442019600000, 1442098800000, 29873, 787370, 1, 5, 0.17391304347826086, 5, 0.20833333333333334 ] + - [ "", 1442098800000, 276159, 1099827, 1442037600000, 1442055600000, 29873, 787370, 1, 6, 0.21739130434782608, 6, 0.25 ] + - [ "", 1442055600000, 283958, 1383785, 1442026800000, 1442088000000, 29873, 787370, 1, 7, 0.2608695652173913, 7, 0.2916666666666667 ] + - [ "", 1442088000000, 303872, 1687657, 1442098800000, 1442048400000, 29873, 787370, 1, 8, 0.30434782608695654, 8, 0.3333333333333333 ] + - [ "", 1442048400000, 308316, 1995973, 1442055600000, 1442044800000, 29873, 787370, 2, 9, 0.34782608695652173, 9, 0.375 ] + - [ "", 1442044800000, 316002, 2311975, 1442088000000, 1442034000000, 29873, 787370, 2, 10, 0.391304347826087, 10, 0.4166666666666667 ] + - [ "", 1442034000000, 330957, 2642932, 1442048400000, 1442066400000, 29873, 787370, 2, 11, 0.43478260869565216, 11, 0.4583333333333333 ] + - [ "", 1442066400000, 351584, 2994516, 1442044800000, 1442070000000, 29873, 787370, 2, 12, 0.4782608695652174, 12, 0.5 ] + - [ "", 1442070000000, 358515, 3353031, 1442034000000, 1442084400000, 29873, 787370, 2, 13, 0.5217391304347826, 13, 0.5416666666666666 ] + - [ "", 1442084400000, 372569, 3725600, 1442066400000, 1442095200000, 29873, 787370, 2, 14, 0.5652173913043478, 14, 0.5833333333333334 ] + - [ "", 1442095200000, 374501, 4100101, 1442070000000, 1442073600000, 29873, 787370, 2, 15, 0.6086956521739131, 15, 0.625 ] + - [ "", 1442073600000, 375394, 4475495, 1442084400000, 1442062800000, 29873, 787370, 2, 16, 0.6521739130434783, 16, 0.6666666666666666 ] + - [ "", 1442062800000, 389465, 4864960, 1442095200000, 1442077200000, 29873, 787370, 3, 17, 0.6956521739130435, 17, 0.7083333333333334 ] + - [ "", 1442077200000, 392483, 5257443, 1442073600000, 1442023200000, 29873, 787370, 3, 18, 0.7391304347826086, 18, 0.75 ] + - [ "", 1442023200000, 399636, 5657079, 1442062800000, 1442080800000, 29873, 787370, 3, 19, 0.782608695652174, 19, 0.7916666666666666 ] + - [ "", 1442080800000, 453077, 6110156, 1442077200000, 1442059200000, 29873, 787370, 3, 20, 0.8260869565217391, 20, 0.8333333333333334 ] + - [ "", 1442059200000, 459297, 6569453, 1442023200000, 1442091600000, 29873, 787370, 3, 21, 0.8695652173913043, 21, 0.875 ] + - [ "", 1442091600000, 514427, 7083880, 1442080800000, 1442041200000, 29873, 787370, 3, 22, 0.9130434782608695, 22, 0.9166666666666666 ] + - [ "", 1442041200000, 543450, 7627330, 1442059200000, 1442052000000, 29873, 787370, 3, 23, 0.9565217391304348, 23, 0.9583333333333334 ] + - [ "", 1442052000000, 787370, 8414700, 1442091600000, null, 29873, 787370, 3, 24, 1.0, 24, 1.0 ] + - [ "AE", 1442059200000, -11, -11, null, 1442044800000, -11, 6323, 1, 1, 0.0, 1, 0.125 ] + - [ "AE", 1442044800000, -7, -18, null, 1442052000000, -11, 6323, 1, 2, 0.14285714285714285, 2, 0.25 ] + - [ "AE", 1442052000000, -3, -21, 1442059200000, 1442048400000, -11, 6323, 1, 3, 0.2857142857142857, 3, 0.375 ] + - [ "AE", 1442048400000, 39, 18, 1442044800000, 1442080800000, -11, 6323, 2, 4, 0.42857142857142855, 4, 0.5 ] + - [ "AE", 1442080800000, 42, 60, 1442052000000, 1442070000000, -11, 6323, 2, 5, 0.5714285714285714, 5, 0.625 ] + - [ "AE", 1442070000000, 46, 106, 1442048400000, 1442030400000, -11, 6323, 2, 6, 0.7142857142857143, 6, 0.75 ] + - [ "AE", 1442030400000, 118, 224, 1442080800000, 1442077200000, -11, 6323, 3, 7, 0.8571428571428571, 7, 0.875 ] + - [ "AE", 1442077200000, 6323, 6547, 1442070000000, null, -11, 6323, 3, 8, 1.0, 8, 1.0 ] + - [ "AL", 1442077200000, 26, 26, null, 1442091600000, 26, 54, 1, 1, 0.0, 1, 0.5 ] + - [ "AL", 1442091600000, 54, 80, null, null, 26, 54, 2, 2, 1.0, 2, 1.0 ] + - [ "AO", 1442041200000, -26, -26, null, 1442052000000, -26, 722, 1, 1, 0.0, 1, 0.25 ] + - [ "AO", 1442052000000, -18, -44, null, 1442088000000, -26, 722, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "AO", 1442088000000, 62, 18, 1442041200000, 1442098800000, -26, 722, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "AO", 1442098800000, 722, 740, 1442052000000, null, -26, 722, 3, 4, 1.0, 4, 1.0 ] + - [ "AR", 1442077200000, -591, -591, null, 1442055600000, -591, 2514, 1, 1, 0.0, 1, 0.058823529411764705 ] + - [ "AR", 1442055600000, -54, -645, null, 1442084400000, -591, 2514, 1, 2, 0.0625, 2, 0.11764705882352941 ] + - [ "AR", 1442084400000, -5, -650, 1442077200000, 1442030400000, -591, 2514, 1, 3, 0.125, 3, 0.17647058823529413 ] + - [ "AR", 1442030400000, -3, -653, 1442055600000, 1442066400000, -591, 2514, 1, 4, 0.1875, 4, 0.23529411764705882 ] + - [ "AR", 1442066400000, 0, -653, 1442084400000, 1442019600000, -591, 2514, 1, 5, 0.25, 5, 0.29411764705882354 ] + - [ "AR", 1442019600000, 1, -652, 1442030400000, 1442080800000, -591, 2514, 1, 6, 0.3125, 6, 0.4117647058823529 ] + - [ "AR", 1442080800000, 1, -651, 1442066400000, 1442062800000, -591, 2514, 2, 6, 0.3125, 6, 0.4117647058823529 ] + - [ "AR", 1442062800000, 29, -622, 1442019600000, 1442098800000, -591, 2514, 2, 8, 0.4375, 7, 0.47058823529411764 ] + - [ "AR", 1442098800000, 64, -558, 1442080800000, 1442037600000, -591, 2514, 2, 9, 0.5, 8, 0.5294117647058824 ] + - [ "AR", 1442037600000, 81, -477, 1442062800000, 1442059200000, -591, 2514, 2, 10, 0.5625, 9, 0.5882352941176471 ] + - [ "AR", 1442059200000, 210, -267, 1442098800000, 1442034000000, -591, 2514, 2, 11, 0.625, 10, 0.6470588235294118 ] + - [ "AR", 1442034000000, 212, -55, 1442037600000, 1442091600000, -591, 2514, 2, 12, 0.6875, 11, 0.7058823529411765 ] + - [ "AR", 1442091600000, 340, 285, 1442059200000, 1442070000000, -591, 2514, 3, 13, 0.75, 12, 0.7647058823529411 ] + - [ "AR", 1442070000000, 377, 662, 1442034000000, 1442095200000, -591, 2514, 3, 14, 0.8125, 13, 0.8235294117647058 ] + - [ "AR", 1442095200000, 630, 1292, 1442091600000, 1442026800000, -591, 2514, 3, 15, 0.875, 14, 0.8823529411764706 ] + - [ "AR", 1442026800000, 644, 1936, 1442070000000, 1442023200000, -591, 2514, 3, 16, 0.9375, 15, 0.9411764705882353 ] + - [ "AR", 1442023200000, 2514, 4450, 1442095200000, null, -591, 2514, 3, 17, 1.0, 16, 1.0 ] + - [ "AT", 1442062800000, -155, -155, null, 1442084400000, -155, 7050, 1, 1, 0.0, 1, 0.14285714285714285 ] + - [ "AT", 1442084400000, -2, -157, null, 1442066400000, -155, 7050, 1, 2, 0.16666666666666666, 2, 0.2857142857142857 ] + - [ "AT", 1442066400000, 0, -157, 1442062800000, 1442091600000, -155, 7050, 1, 3, 0.3333333333333333, 3, 0.42857142857142855 ] + - [ "AT", 1442091600000, 89, -68, 1442084400000, 1442070000000, -155, 7050, 2, 4, 0.5, 4, 0.5714285714285714 ] + - [ "AT", 1442070000000, 272, 204, 1442066400000, 1442052000000, -155, 7050, 2, 5, 0.6666666666666666, 5, 0.7142857142857143 ] + - [ "AT", 1442052000000, 4793, 4997, 1442091600000, 1442088000000, -155, 7050, 3, 6, 0.8333333333333334, 6, 0.8571428571428571 ] + - [ "AT", 1442088000000, 7050, 12047, 1442070000000, null, -155, 7050, 3, 7, 1.0, 7, 1.0 ] + - [ "AU", 1442052000000, -643, -643, null, 1442030400000, -643, 1138, 1, 1, 0.0, 1, 0.05263157894736842 ] + - [ "AU", 1442030400000, -377, -1020, null, 1442066400000, -643, 1138, 1, 2, 0.05555555555555555, 2, 0.10526315789473684 ] + - [ "AU", 1442066400000, -21, -1041, 1442052000000, 1442070000000, -643, 1138, 1, 3, 0.1111111111111111, 3, 0.15789473684210525 ] + - [ "AU", 1442070000000, -12, -1053, 1442030400000, 1442016000000, -643, 1138, 1, 4, 0.16666666666666666, 4, 0.21052631578947367 ] + - [ "AU", 1442016000000, 0, -1053, 1442066400000, 1442077200000, -643, 1138, 1, 5, 0.2222222222222222, 5, 0.2631578947368421 ] + - [ "AU", 1442077200000, 1, -1052, 1442070000000, 1442037600000, -643, 1138, 1, 6, 0.2777777777777778, 6, 0.3157894736842105 ] + - [ "AU", 1442037600000, 3, -1049, 1442016000000, 1442059200000, -643, 1138, 1, 7, 0.3333333333333333, 7, 0.3684210526315789 ] + - [ "AU", 1442059200000, 38, -1011, 1442077200000, 1442023200000, -643, 1138, 2, 8, 0.3888888888888889, 8, 0.42105263157894735 ] + - [ "AU", 1442023200000, 52, -959, 1442037600000, 1442048400000, -643, 1138, 2, 9, 0.4444444444444444, 9, 0.47368421052631576 ] + - [ "AU", 1442048400000, 135, -824, 1442059200000, 1442055600000, -643, 1138, 2, 10, 0.5, 10, 0.5263157894736842 ] + - [ "AU", 1442055600000, 182, -642, 1442023200000, 1442026800000, -643, 1138, 2, 11, 0.5555555555555556, 11, 0.5789473684210527 ] + - [ "AU", 1442026800000, 188, -454, 1442048400000, 1442041200000, -643, 1138, 2, 12, 0.6111111111111112, 12, 0.631578947368421 ] + - [ "AU", 1442041200000, 194, -260, 1442055600000, 1442019600000, -643, 1138, 2, 13, 0.6666666666666666, 13, 0.6842105263157895 ] + - [ "AU", 1442019600000, 253, -7, 1442026800000, 1442034000000, -643, 1138, 3, 14, 0.7222222222222222, 14, 0.7368421052631579 ] + - [ "AU", 1442034000000, 283, 276, 1442041200000, 1442044800000, -643, 1138, 3, 15, 0.7777777777777778, 15, 0.7894736842105263 ] + - [ "AU", 1442044800000, 373, 649, 1442019600000, 1442095200000, -643, 1138, 3, 16, 0.8333333333333334, 16, 0.8421052631578947 ] + - [ "AU", 1442095200000, 395, 1044, 1442034000000, 1442098800000, -643, 1138, 3, 17, 0.8888888888888888, 17, 0.8947368421052632 ] + - [ "AU", 1442098800000, 518, 1562, 1442044800000, 1442091600000, -643, 1138, 3, 18, 0.9444444444444444, 18, 0.9473684210526315 ] + - [ "AU", 1442091600000, 1138, 2700, 1442095200000, null, -643, 1138, 3, 19, 1.0, 19, 1.0 ] + - [ "BA", 1442055600000, -202, -202, null, 1442048400000, -202, 38, 1, 1, 0.0, 1, 0.25 ] + - [ "BA", 1442048400000, -13, -215, null, 1442084400000, -202, 38, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "BA", 1442084400000, -1, -216, 1442055600000, 1442052000000, -202, 38, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "BA", 1442052000000, 38, -178, 1442048400000, null, -202, 38, 3, 4, 1.0, 4, 1.0 ] + - [ "BD", 1442091600000, -2, -2, null, 1442019600000, -2, 854, 1, 1, 0.0, 1, 0.16666666666666666 ] + - [ "BD", 1442019600000, 0, -2, null, 1442077200000, -2, 854, 1, 2, 0.2, 2, 0.3333333333333333 ] + - [ "BD", 1442077200000, 75, 73, 1442091600000, 1442066400000, -2, 854, 2, 3, 0.4, 3, 0.5 ] + - [ "BD", 1442066400000, 76, 149, 1442019600000, 1442073600000, -2, 854, 2, 4, 0.6, 4, 0.6666666666666666 ] + - [ "BD", 1442073600000, 103, 252, 1442077200000, 1442041200000, -2, 854, 3, 5, 0.8, 5, 0.8333333333333334 ] + - [ "BD", 1442041200000, 854, 1106, 1442066400000, null, -2, 854, 3, 6, 1.0, 6, 1.0 ] + - [ "BE", 1442030400000, -103, -103, null, 1442052000000, -103, 233, 1, 1, 0.0, 1, 0.08333333333333333 ] + - [ "BE", 1442052000000, -1, -104, null, 1442080800000, -103, 233, 1, 2, 0.09090909090909091, 2, 0.16666666666666666 ] + - [ "BE", 1442080800000, 1, -103, 1442030400000, 1442098800000, -103, 233, 1, 3, 0.18181818181818182, 3, 0.25 ] + - [ "BE", 1442098800000, 9, -94, 1442052000000, 1442073600000, -103, 233, 1, 4, 0.2727272727272727, 4, 0.3333333333333333 ] + - [ "BE", 1442073600000, 19, -75, 1442080800000, 1442048400000, -103, 233, 2, 5, 0.36363636363636365, 5, 0.4166666666666667 ] + - [ "BE", 1442048400000, 59, -16, 1442098800000, 1442088000000, -103, 233, 2, 6, 0.45454545454545453, 6, 0.5 ] + - [ "BE", 1442088000000, 67, 51, 1442073600000, 1442062800000, -103, 233, 2, 7, 0.5454545454545454, 7, 0.5833333333333334 ] + - [ "BE", 1442062800000, 91, 142, 1442048400000, 1442091600000, -103, 233, 2, 8, 0.6363636363636364, 8, 0.6666666666666666 ] + - [ "BE", 1442091600000, 101, 243, 1442088000000, 1442066400000, -103, 233, 3, 9, 0.7272727272727273, 9, 0.75 ] + - [ "BE", 1442066400000, 136, 379, 1442062800000, 1442084400000, -103, 233, 3, 10, 0.8181818181818182, 10, 0.8333333333333334 ] + - [ "BE", 1442084400000, 183, 562, 1442091600000, 1442055600000, -103, 233, 3, 11, 0.9090909090909091, 11, 0.9166666666666666 ] + - [ "BE", 1442055600000, 233, 795, 1442066400000, null, -103, 233, 3, 12, 1.0, 12, 1.0 ] + - [ "BG", 1442041200000, 9, 9, null, 1442070000000, 9, 18936, 1, 1, 0.0, 1, 0.2 ] + - [ "BG", 1442070000000, 55, 64, null, 1442059200000, 9, 18936, 1, 2, 0.25, 2, 0.4 ] + - [ "BG", 1442059200000, 191, 255, 1442041200000, 1442084400000, 9, 18936, 2, 3, 0.5, 3, 0.6 ] + - [ "BG", 1442084400000, 401, 656, 1442070000000, 1442052000000, 9, 18936, 2, 4, 0.75, 4, 0.8 ] + - [ "BG", 1442052000000, 18936, 19592, 1442059200000, null, 9, 18936, 3, 5, 1.0, 5, 1.0 ] + - [ "BH", 1442052000000, 44, 44, null, null, 44, 44, 1, 1, 0.0, 1, 1.0 ] + - [ "BO", 1442095200000, -4, -4, null, 1442080800000, -4, 4, 1, 1, 0.0, 1, 0.3333333333333333 ] + - [ "BO", 1442080800000, 4, 0, null, 1442088000000, -4, 4, 2, 2, 0.5, 2, 1.0 ] + - [ "BO", 1442088000000, 4, 4, 1442095200000, null, -4, 4, 3, 2, 0.5, 2, 1.0 ] + - [ "BR", 1442098800000, -645, -645, null, 1442080800000, -645, 2253, 1, 1, 0.0, 1, 0.043478260869565216 ] + - [ "BR", 1442080800000, -267, -912, null, 1442016000000, -645, 2253, 1, 2, 0.045454545454545456, 2, 0.08695652173913043 ] + - [ "BR", 1442016000000, -248, -1160, 1442098800000, 1442041200000, -645, 2253, 1, 3, 0.09090909090909091, 3, 0.13043478260869565 ] + - [ "BR", 1442041200000, 3, -1157, 1442080800000, 1442091600000, -645, 2253, 1, 4, 0.13636363636363635, 4, 0.17391304347826086 ] + - [ "BR", 1442091600000, 11, -1146, 1442016000000, 1442034000000, -645, 2253, 1, 5, 0.18181818181818182, 5, 0.21739130434782608 ] + - [ "BR", 1442034000000, 21, -1125, 1442041200000, 1442030400000, -645, 2253, 1, 6, 0.22727272727272727, 6, 0.2608695652173913 ] + - [ "BR", 1442030400000, 30, -1095, 1442091600000, 1442026800000, -645, 2253, 1, 7, 0.2727272727272727, 7, 0.30434782608695654 ] + - [ "BR", 1442026800000, 51, -1044, 1442034000000, 1442044800000, -645, 2253, 1, 8, 0.3181818181818182, 8, 0.34782608695652173 ] + - [ "BR", 1442044800000, 71, -973, 1442030400000, 1442059200000, -645, 2253, 2, 9, 0.36363636363636365, 9, 0.391304347826087 ] + - [ "BR", 1442059200000, 73, -900, 1442026800000, 1442062800000, -645, 2253, 2, 10, 0.4090909090909091, 10, 0.43478260869565216 ] + - [ "BR", 1442062800000, 93, -807, 1442044800000, 1442088000000, -645, 2253, 2, 11, 0.45454545454545453, 11, 0.4782608695652174 ] + - [ "BR", 1442088000000, 215, -592, 1442059200000, 1442052000000, -645, 2253, 2, 12, 0.5, 12, 0.5217391304347826 ] + - [ "BR", 1442052000000, 232, -360, 1442062800000, 1442055600000, -645, 2253, 2, 13, 0.5454545454545454, 13, 0.5652173913043478 ] + - [ "BR", 1442055600000, 242, -118, 1442088000000, 1442037600000, -645, 2253, 2, 14, 0.5909090909090909, 14, 0.6086956521739131 ] + - [ "BR", 1442037600000, 267, 149, 1442052000000, 1442019600000, -645, 2253, 2, 15, 0.6363636363636364, 15, 0.6521739130434783 ] + - [ "BR", 1442019600000, 372, 521, 1442055600000, 1442084400000, -645, 2253, 2, 16, 0.6818181818181818, 16, 0.6956521739130435 ] + - [ "BR", 1442084400000, 492, 1013, 1442037600000, 1442070000000, -645, 2253, 3, 17, 0.7272727272727273, 17, 0.7391304347826086 ] + - [ "BR", 1442070000000, 536, 1549, 1442019600000, 1442095200000, -645, 2253, 3, 18, 0.7727272727272727, 18, 0.782608695652174 ] + - [ "BR", 1442095200000, 748, 2297, 1442084400000, 1442023200000, -645, 2253, 3, 19, 0.8181818181818182, 19, 0.8260869565217391 ] + - [ "BR", 1442023200000, 879, 3176, 1442070000000, 1442066400000, -645, 2253, 3, 20, 0.8636363636363636, 20, 0.8695652173913043 ] + - [ "BR", 1442066400000, 1034, 4210, 1442095200000, 1442073600000, -645, 2253, 3, 21, 0.9090909090909091, 21, 0.9130434782608695 ] + - [ "BR", 1442073600000, 2087, 6297, 1442023200000, 1442077200000, -645, 2253, 3, 22, 0.9545454545454546, 22, 0.9565217391304348 ] + - [ "BR", 1442077200000, 2253, 8550, 1442066400000, null, -645, 2253, 3, 23, 1.0, 23, 1.0 ] + - [ "BY", 1442055600000, 1, 1, null, 1442084400000, 1, 1464, 1, 1, 0.0, 1, 0.2857142857142857 ] + - [ "BY", 1442084400000, 1, 2, null, 1442080800000, 1, 1464, 1, 1, 0.0, 1, 0.2857142857142857 ] + - [ "BY", 1442080800000, 28, 30, 1442055600000, 1442077200000, 1, 1464, 1, 3, 0.3333333333333333, 2, 0.42857142857142855 ] + - [ "BY", 1442077200000, 30, 60, 1442084400000, 1442088000000, 1, 1464, 2, 4, 0.5, 3, 0.5714285714285714 ] + - [ "BY", 1442088000000, 33, 93, 1442080800000, 1442073600000, 1, 1464, 2, 5, 0.6666666666666666, 4, 0.7142857142857143 ] + - [ "BY", 1442073600000, 596, 689, 1442077200000, 1442059200000, 1, 1464, 3, 6, 0.8333333333333334, 5, 0.8571428571428571 ] + - [ "BY", 1442059200000, 1464, 2153, 1442088000000, null, 1, 1464, 3, 7, 1.0, 6, 1.0 ] + - [ "CA", 1442016000000, -371, -371, null, 1442062800000, -371, 2858, 1, 1, 0.0, 1, 0.045454545454545456 ] + - [ "CA", 1442062800000, -367, -738, null, 1442095200000, -371, 2858, 1, 2, 0.047619047619047616, 2, 0.09090909090909091 ] + - [ "CA", 1442095200000, -361, -1099, 1442016000000, 1442077200000, -371, 2858, 1, 3, 0.09523809523809523, 3, 0.13636363636363635 ] + - [ "CA", 1442077200000, -282, -1381, 1442062800000, 1442037600000, -371, 2858, 1, 4, 0.14285714285714285, 4, 0.18181818181818182 ] + - [ "CA", 1442037600000, -132, -1513, 1442095200000, 1442030400000, -371, 2858, 1, 5, 0.19047619047619047, 5, 0.22727272727272727 ] + - [ "CA", 1442030400000, -47, -1560, 1442077200000, 1442044800000, -371, 2858, 1, 6, 0.23809523809523808, 6, 0.2727272727272727 ] + - [ "CA", 1442044800000, 1, -1559, 1442037600000, 1442041200000, -371, 2858, 1, 7, 0.2857142857142857, 7, 0.3181818181818182 ] + - [ "CA", 1442041200000, 5, -1554, 1442030400000, 1442088000000, -371, 2858, 1, 8, 0.3333333333333333, 8, 0.36363636363636365 ] + - [ "CA", 1442088000000, 35, -1519, 1442044800000, 1442052000000, -371, 2858, 2, 9, 0.38095238095238093, 9, 0.4090909090909091 ] + - [ "CA", 1442052000000, 38, -1481, 1442041200000, 1442084400000, -371, 2858, 2, 10, 0.42857142857142855, 10, 0.45454545454545453 ] + - [ "CA", 1442084400000, 44, -1437, 1442088000000, 1442073600000, -371, 2858, 2, 11, 0.47619047619047616, 11, 0.5 ] + - [ "CA", 1442073600000, 86, -1351, 1442052000000, 1442098800000, -371, 2858, 2, 12, 0.5238095238095238, 12, 0.5454545454545454 ] + - [ "CA", 1442098800000, 164, -1187, 1442084400000, 1442034000000, -371, 2858, 2, 13, 0.5714285714285714, 13, 0.5909090909090909 ] + - [ "CA", 1442034000000, 178, -1009, 1442073600000, 1442070000000, -371, 2858, 2, 14, 0.6190476190476191, 14, 0.6363636363636364 ] + - [ "CA", 1442070000000, 185, -824, 1442098800000, 1442023200000, -371, 2858, 2, 15, 0.6666666666666666, 15, 0.6818181818181818 ] + - [ "CA", 1442023200000, 286, -538, 1442034000000, 1442066400000, -371, 2858, 3, 16, 0.7142857142857143, 16, 0.7272727272727273 ] + - [ "CA", 1442066400000, 307, -231, 1442070000000, 1442080800000, -371, 2858, 3, 17, 0.7619047619047619, 17, 0.7727272727272727 ] + - [ "CA", 1442080800000, 481, 250, 1442023200000, 1442059200000, -371, 2858, 3, 18, 0.8095238095238095, 18, 0.8181818181818182 ] + - [ "CA", 1442059200000, 1036, 1286, 1442066400000, 1442019600000, -371, 2858, 3, 19, 0.8571428571428571, 19, 0.8636363636363636 ] + - [ "CA", 1442019600000, 2184, 3470, 1442080800000, 1442026800000, -371, 2858, 3, 20, 0.9047619047619048, 20, 0.9090909090909091 ] + - [ "CA", 1442026800000, 2216, 5686, 1442059200000, 1442091600000, -371, 2858, 3, 21, 0.9523809523809523, 21, 0.9545454545454546 ] + - [ "CA", 1442091600000, 2858, 8544, 1442019600000, null, -371, 2858, 3, 22, 1.0, 22, 1.0 ] + - [ "CH", 1442044800000, -54, -54, null, 1442055600000, -54, 360, 1, 1, 0.0, 1, 0.08333333333333333 ] + - [ "CH", 1442055600000, 0, -54, null, 1442077200000, -54, 360, 1, 2, 0.09090909090909091, 2, 0.16666666666666666 ] + - [ "CH", 1442077200000, 6, -48, 1442044800000, 1442070000000, -54, 360, 1, 3, 0.18181818181818182, 3, 0.25 ] + - [ "CH", 1442070000000, 11, -37, 1442055600000, 1442084400000, -54, 360, 1, 4, 0.2727272727272727, 4, 0.3333333333333333 ] + - [ "CH", 1442084400000, 13, -24, 1442077200000, 1442062800000, -54, 360, 2, 5, 0.36363636363636365, 5, 0.4166666666666667 ] + - [ "CH", 1442062800000, 22, -2, 1442070000000, 1442048400000, -54, 360, 2, 6, 0.45454545454545453, 6, 0.5 ] + - [ "CH", 1442048400000, 24, 22, 1442084400000, 1442052000000, -54, 360, 2, 7, 0.5454545454545454, 7, 0.5833333333333334 ] + - [ "CH", 1442052000000, 47, 69, 1442062800000, 1442037600000, -54, 360, 2, 8, 0.6363636363636364, 8, 0.6666666666666666 ] + - [ "CH", 1442037600000, 59, 128, 1442048400000, 1442091600000, -54, 360, 3, 9, 0.7272727272727273, 9, 0.75 ] + - [ "CH", 1442091600000, 67, 195, 1442052000000, 1442041200000, -54, 360, 3, 10, 0.8181818181818182, 10, 0.8333333333333334 ] + - [ "CH", 1442041200000, 198, 393, 1442037600000, 1442073600000, -54, 360, 3, 11, 0.9090909090909091, 11, 0.9166666666666666 ] + - [ "CH", 1442073600000, 360, 753, 1442091600000, null, -54, 360, 3, 12, 1.0, 12, 1.0 ] + - [ "CL", 1442019600000, -370, -370, null, 1442095200000, -370, 390, 1, 1, 0.0, 1, 0.05 ] + - [ "CL", 1442095200000, -276, -646, null, 1442066400000, -370, 390, 1, 2, 0.05263157894736842, 2, 0.1 ] + - [ "CL", 1442066400000, -41, -687, 1442019600000, 1442077200000, -370, 390, 1, 3, 0.10526315789473684, 3, 0.15 ] + - [ "CL", 1442077200000, -15, -702, 1442095200000, 1442059200000, -370, 390, 1, 4, 0.15789473684210525, 4, 0.2 ] + - [ "CL", 1442059200000, -12, -714, 1442066400000, 1442034000000, -370, 390, 1, 5, 0.21052631578947367, 5, 0.25 ] + - [ "CL", 1442034000000, -1, -715, 1442077200000, 1442041200000, -370, 390, 1, 6, 0.2631578947368421, 6, 0.35 ] + - [ "CL", 1442041200000, -1, -716, 1442059200000, 1442037600000, -370, 390, 1, 6, 0.2631578947368421, 6, 0.35 ] + - [ "CL", 1442037600000, 2, -714, 1442034000000, 1442098800000, -370, 390, 2, 8, 0.3684210526315789, 7, 0.4 ] + - [ "CL", 1442098800000, 9, -705, 1442041200000, 1442070000000, -370, 390, 2, 9, 0.42105263157894735, 8, 0.45 ] + - [ "CL", 1442070000000, 13, -692, 1442037600000, 1442023200000, -370, 390, 2, 10, 0.47368421052631576, 9, 0.5 ] + - [ "CL", 1442023200000, 15, -677, 1442098800000, 1442062800000, -370, 390, 2, 11, 0.5263157894736842, 10, 0.55 ] + - [ "CL", 1442062800000, 17, -660, 1442070000000, 1442080800000, -370, 390, 2, 12, 0.5789473684210527, 11, 0.65 ] + - [ "CL", 1442080800000, 17, -643, 1442023200000, 1442091600000, -370, 390, 2, 12, 0.5789473684210527, 11, 0.65 ] + - [ "CL", 1442091600000, 20, -623, 1442062800000, 1442030400000, -370, 390, 2, 14, 0.6842105263157895, 12, 0.7 ] + - [ "CL", 1442030400000, 40, -583, 1442080800000, 1442084400000, -370, 390, 3, 15, 0.7368421052631579, 13, 0.75 ] + - [ "CL", 1442084400000, 126, -457, 1442091600000, 1442073600000, -370, 390, 3, 16, 0.7894736842105263, 14, 0.8 ] + - [ "CL", 1442073600000, 153, -304, 1442030400000, 1442016000000, -370, 390, 3, 17, 0.8421052631578947, 15, 0.85 ] + - [ "CL", 1442016000000, 161, -143, 1442084400000, 1442088000000, -370, 390, 3, 18, 0.8947368421052632, 16, 0.9 ] + - [ "CL", 1442088000000, 286, 143, 1442073600000, 1442052000000, -370, 390, 3, 19, 0.9473684210526315, 17, 0.95 ] + - [ "CL", 1442052000000, 390, 533, 1442016000000, null, -370, 390, 3, 20, 1.0, 18, 1.0 ] + - [ "CN", 1442066400000, -15, -15, null, 1442023200000, -15, 293, 1, 1, 0.0, 1, 0.1 ] + - [ "CN", 1442023200000, -13, -28, null, 1442080800000, -15, 293, 1, 2, 0.1111111111111111, 2, 0.2 ] + - [ "CN", 1442080800000, -10, -38, 1442066400000, 1442084400000, -15, 293, 1, 3, 0.2222222222222222, 3, 0.3 ] + - [ "CN", 1442084400000, -1, -39, 1442023200000, 1442052000000, -15, 293, 1, 4, 0.3333333333333333, 4, 0.4 ] + - [ "CN", 1442052000000, 0, -39, 1442080800000, 1442059200000, -15, 293, 2, 5, 0.4444444444444444, 5, 0.5 ] + - [ "CN", 1442059200000, 8, -31, 1442084400000, 1442055600000, -15, 293, 2, 6, 0.5555555555555556, 6, 0.6 ] + - [ "CN", 1442055600000, 69, 38, 1442052000000, 1442037600000, -15, 293, 2, 7, 0.6666666666666666, 7, 0.7 ] + - [ "CN", 1442037600000, 98, 136, 1442059200000, 1442026800000, -15, 293, 3, 8, 0.7777777777777778, 8, 0.8 ] + - [ "CN", 1442026800000, 154, 290, 1442055600000, 1442048400000, -15, 293, 3, 9, 0.8888888888888888, 9, 0.9 ] + - [ "CN", 1442048400000, 293, 583, 1442037600000, null, -15, 293, 3, 10, 1.0, 10, 1.0 ] + - [ "CO", 1442070000000, -45, -45, null, 1442023200000, -45, 39860, 1, 1, 0.0, 1, 0.06666666666666667 ] + - [ "CO", 1442023200000, 9, -36, null, 1442019600000, -45, 39860, 1, 2, 0.07142857142857142, 2, 0.13333333333333333 ] + - [ "CO", 1442019600000, 12, -24, 1442070000000, 1442016000000, -45, 39860, 1, 3, 0.14285714285714285, 3, 0.2 ] + - [ "CO", 1442016000000, 16, -8, 1442023200000, 1442080800000, -45, 39860, 1, 4, 0.21428571428571427, 4, 0.26666666666666666 ] + - [ "CO", 1442080800000, 25, 17, 1442019600000, 1442084400000, -45, 39860, 1, 5, 0.2857142857142857, 5, 0.3333333333333333 ] + - [ "CO", 1442084400000, 51, 68, 1442016000000, 1442098800000, -45, 39860, 2, 6, 0.35714285714285715, 6, 0.4 ] + - [ "CO", 1442098800000, 83, 151, 1442080800000, 1442066400000, -45, 39860, 2, 7, 0.42857142857142855, 7, 0.4666666666666667 ] + - [ "CO", 1442066400000, 288, 439, 1442084400000, 1442095200000, -45, 39860, 2, 8, 0.5, 8, 0.5333333333333333 ] + - [ "CO", 1442095200000, 290, 729, 1442098800000, 1442091600000, -45, 39860, 2, 9, 0.5714285714285714, 9, 0.6 ] + - [ "CO", 1442091600000, 377, 1106, 1442066400000, 1442030400000, -45, 39860, 2, 10, 0.6428571428571429, 10, 0.6666666666666666 ] + - [ "CO", 1442030400000, 441, 1547, 1442095200000, 1442059200000, -45, 39860, 3, 11, 0.7142857142857143, 11, 0.7333333333333333 ] + - [ "CO", 1442059200000, 473, 2020, 1442091600000, 1442077200000, -45, 39860, 3, 12, 0.7857142857142857, 12, 0.8 ] + - [ "CO", 1442077200000, 581, 2601, 1442030400000, 1442088000000, -45, 39860, 3, 13, 0.8571428571428571, 13, 0.8666666666666667 ] + - [ "CO", 1442088000000, 17150, 19751, 1442059200000, 1442073600000, -45, 39860, 3, 14, 0.9285714285714286, 14, 0.9333333333333333 ] + - [ "CO", 1442073600000, 39860, 59611, 1442077200000, null, -45, 39860, 3, 15, 1.0, 15, 1.0 ] + - [ "CR", 1442041200000, 51, 51, null, 1442019600000, 51, 2497, 1, 1, 0.0, 1, 0.125 ] + - [ "CR", 1442019600000, 62, 113, null, 1442023200000, 51, 2497, 1, 2, 0.14285714285714285, 2, 0.375 ] + - [ "CR", 1442023200000, 62, 175, 1442041200000, 1442088000000, 51, 2497, 1, 2, 0.14285714285714285, 2, 0.375 ] + - [ "CR", 1442088000000, 72, 247, 1442019600000, 1442026800000, 51, 2497, 2, 4, 0.42857142857142855, 3, 0.5 ] + - [ "CR", 1442026800000, 140, 387, 1442023200000, 1442048400000, 51, 2497, 2, 5, 0.5714285714285714, 4, 0.625 ] + - [ "CR", 1442048400000, 163, 550, 1442088000000, 1442044800000, 51, 2497, 2, 6, 0.7142857142857143, 5, 0.75 ] + - [ "CR", 1442044800000, 194, 744, 1442026800000, 1442030400000, 51, 2497, 3, 7, 0.8571428571428571, 6, 0.875 ] + - [ "CR", 1442030400000, 2497, 3241, 1442048400000, null, 51, 2497, 3, 8, 1.0, 7, 1.0 ] + - [ "CZ", 1442080800000, -28, -28, null, 1442026800000, -28, 2051, 1, 1, 0.0, 1, 0.09090909090909091 ] + - [ "CZ", 1442026800000, -19, -47, null, 1442062800000, -28, 2051, 1, 2, 0.1, 2, 0.18181818181818182 ] + - [ "CZ", 1442062800000, 0, -47, 1442080800000, 1442098800000, -28, 2051, 1, 3, 0.2, 3, 0.2727272727272727 ] + - [ "CZ", 1442098800000, 2, -45, 1442026800000, 1442037600000, -28, 2051, 1, 4, 0.3, 4, 0.36363636363636365 ] + - [ "CZ", 1442037600000, 18, -27, 1442062800000, 1442059200000, -28, 2051, 2, 5, 0.4, 5, 0.45454545454545453 ] + - [ "CZ", 1442059200000, 21, -6, 1442098800000, 1442034000000, -28, 2051, 2, 6, 0.5, 6, 0.5454545454545454 ] + - [ "CZ", 1442034000000, 78, 72, 1442037600000, 1442077200000, -28, 2051, 2, 7, 0.6, 7, 0.6363636363636364 ] + - [ "CZ", 1442077200000, 115, 187, 1442059200000, 1442070000000, -28, 2051, 2, 8, 0.7, 8, 0.7272727272727273 ] + - [ "CZ", 1442070000000, 168, 355, 1442034000000, 1442055600000, -28, 2051, 3, 9, 0.8, 9, 0.8181818181818182 ] + - [ "CZ", 1442055600000, 1073, 1428, 1442077200000, 1442073600000, -28, 2051, 3, 10, 0.9, 10, 0.9090909090909091 ] + - [ "CZ", 1442073600000, 2051, 3479, 1442070000000, null, -28, 2051, 3, 11, 1.0, 11, 1.0 ] + - [ "DE", 1442084400000, -125, -125, null, 1442019600000, -125, 6075, 1, 1, 0.0, 1, 0.043478260869565216 ] + - [ "DE", 1442019600000, 0, -125, null, 1442023200000, -125, 6075, 1, 2, 0.045454545454545456, 2, 0.08695652173913043 ] + - [ "DE", 1442023200000, 64, -61, 1442084400000, 1442016000000, -125, 6075, 1, 3, 0.09090909090909091, 3, 0.13043478260869565 ] + - [ "DE", 1442016000000, 167, 106, 1442019600000, 1442088000000, -125, 6075, 1, 4, 0.13636363636363635, 4, 0.17391304347826086 ] + - [ "DE", 1442088000000, 190, 296, 1442023200000, 1442041200000, -125, 6075, 1, 5, 0.18181818181818182, 5, 0.21739130434782608 ] + - [ "DE", 1442041200000, 197, 493, 1442016000000, 1442062800000, -125, 6075, 1, 6, 0.22727272727272727, 6, 0.2608695652173913 ] + - [ "DE", 1442062800000, 283, 776, 1442088000000, 1442059200000, -125, 6075, 1, 7, 0.2727272727272727, 7, 0.30434782608695654 ] + - [ "DE", 1442059200000, 289, 1065, 1442041200000, 1442098800000, -125, 6075, 1, 8, 0.3181818181818182, 8, 0.34782608695652173 ] + - [ "DE", 1442098800000, 329, 1394, 1442062800000, 1442034000000, -125, 6075, 2, 9, 0.36363636363636365, 9, 0.391304347826087 ] + - [ "DE", 1442034000000, 358, 1752, 1442059200000, 1442030400000, -125, 6075, 2, 10, 0.4090909090909091, 10, 0.43478260869565216 ] + - [ "DE", 1442030400000, 373, 2125, 1442098800000, 1442037600000, -125, 6075, 2, 11, 0.45454545454545453, 11, 0.4782608695652174 ] + - [ "DE", 1442037600000, 544, 2669, 1442034000000, 1442048400000, -125, 6075, 2, 12, 0.5, 12, 0.5217391304347826 ] + - [ "DE", 1442048400000, 811, 3480, 1442030400000, 1442044800000, -125, 6075, 2, 13, 0.5454545454545454, 13, 0.5652173913043478 ] + - [ "DE", 1442044800000, 979, 4459, 1442037600000, 1442095200000, -125, 6075, 2, 14, 0.5909090909090909, 14, 0.6086956521739131 ] + - [ "DE", 1442095200000, 1007, 5466, 1442048400000, 1442080800000, -125, 6075, 2, 15, 0.6363636363636364, 15, 0.6521739130434783 ] + - [ "DE", 1442080800000, 1133, 6599, 1442044800000, 1442055600000, -125, 6075, 2, 16, 0.6818181818181818, 16, 0.6956521739130435 ] + - [ "DE", 1442055600000, 1523, 8122, 1442095200000, 1442066400000, -125, 6075, 3, 17, 0.7272727272727273, 17, 0.7391304347826086 ] + - [ "DE", 1442066400000, 1577, 9699, 1442080800000, 1442052000000, -125, 6075, 3, 18, 0.7727272727272727, 18, 0.782608695652174 ] + - [ "DE", 1442052000000, 1600, 11299, 1442055600000, 1442070000000, -125, 6075, 3, 19, 0.8181818181818182, 19, 0.8260869565217391 ] + - [ "DE", 1442070000000, 1666, 12965, 1442066400000, 1442077200000, -125, 6075, 3, 20, 0.8636363636363636, 20, 0.8695652173913043 ] + - [ "DE", 1442077200000, 2188, 15153, 1442052000000, 1442091600000, -125, 6075, 3, 21, 0.9090909090909091, 21, 0.9130434782608695 ] + - [ "DE", 1442091600000, 4355, 19508, 1442070000000, 1442073600000, -125, 6075, 3, 22, 0.9545454545454546, 22, 0.9565217391304348 ] + - [ "DE", 1442073600000, 6075, 25583, 1442077200000, null, -125, 6075, 3, 23, 1.0, 23, 1.0 ] + - [ "DK", 1442084400000, -97, -97, null, 1442077200000, -97, 416, 1, 1, 0.0, 1, 0.08333333333333333 ] + - [ "DK", 1442077200000, -9, -106, null, 1442048400000, -97, 416, 1, 2, 0.09090909090909091, 2, 0.16666666666666666 ] + - [ "DK", 1442048400000, -5, -111, 1442084400000, 1442059200000, -97, 416, 1, 3, 0.18181818181818182, 3, 0.25 ] + - [ "DK", 1442059200000, 0, -111, 1442077200000, 1442095200000, -97, 416, 1, 4, 0.2727272727272727, 4, 0.4166666666666667 ] + - [ "DK", 1442095200000, 0, -111, 1442048400000, 1442062800000, -97, 416, 2, 4, 0.2727272727272727, 4, 0.4166666666666667 ] + - [ "DK", 1442062800000, 1, -110, 1442059200000, 1442037600000, -97, 416, 2, 6, 0.45454545454545453, 5, 0.5 ] + - [ "DK", 1442037600000, 10, -100, 1442095200000, 1442044800000, -97, 416, 2, 7, 0.5454545454545454, 6, 0.5833333333333334 ] + - [ "DK", 1442044800000, 36, -64, 1442062800000, 1442055600000, -97, 416, 2, 8, 0.6363636363636364, 7, 0.6666666666666666 ] + - [ "DK", 1442055600000, 42, -22, 1442037600000, 1442080800000, -97, 416, 3, 9, 0.7272727272727273, 8, 0.75 ] + - [ "DK", 1442080800000, 61, 39, 1442044800000, 1442091600000, -97, 416, 3, 10, 0.8181818181818182, 9, 0.8333333333333334 ] + - [ "DK", 1442091600000, 139, 178, 1442055600000, 1442066400000, -97, 416, 3, 11, 0.9090909090909091, 10, 0.9166666666666666 ] + - [ "DK", 1442066400000, 416, 594, 1442080800000, null, -97, 416, 3, 12, 1.0, 11, 1.0 ] + - [ "DO", 1442023200000, 8, 8, null, 1442084400000, 8, 200, 1, 1, 0.0, 1, 0.4 ] + - [ "DO", 1442084400000, 8, 16, null, 1442095200000, 8, 200, 1, 1, 0.0, 1, 0.4 ] + - [ "DO", 1442095200000, 13, 29, 1442023200000, 1442066400000, 8, 200, 2, 3, 0.5, 2, 0.6 ] + - [ "DO", 1442066400000, 35, 64, 1442084400000, 1442073600000, 8, 200, 2, 4, 0.75, 3, 0.8 ] + - [ "DO", 1442073600000, 200, 264, 1442095200000, null, 8, 200, 3, 5, 1.0, 4, 1.0 ] + - [ "DZ", 1442077200000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0 ] + - [ "EC", 1442077200000, -366, -366, null, 1442023200000, -366, 568, 1, 1, 0.0, 1, 0.16666666666666666 ] + - [ "EC", 1442023200000, -9, -375, null, 1442030400000, -366, 568, 1, 2, 0.2, 2, 0.3333333333333333 ] + - [ "EC", 1442030400000, 0, -375, 1442077200000, 1442095200000, -366, 568, 2, 3, 0.4, 3, 0.5 ] + - [ "EC", 1442095200000, 10, -365, 1442023200000, 1442019600000, -366, 568, 2, 4, 0.6, 4, 0.6666666666666666 ] + - [ "EC", 1442019600000, 29, -336, 1442030400000, 1442084400000, -366, 568, 3, 5, 0.8, 5, 0.8333333333333334 ] + - [ "EC", 1442084400000, 568, 232, 1442095200000, null, -366, 568, 3, 6, 1.0, 6, 1.0 ] + - [ "EE", 1442044800000, -19, -19, null, 1442041200000, -19, 37, 1, 1, 0.0, 1, 0.5 ] + - [ "EE", 1442041200000, 37, 18, null, null, -19, 37, 2, 2, 1.0, 2, 1.0 ] + - [ "EG", 1442073600000, 1, 1, null, 1442055600000, 1, 112, 1, 1, 0.0, 1, 0.2 ] + - [ "EG", 1442055600000, 14, 15, null, 1442026800000, 1, 112, 1, 2, 0.25, 2, 0.4 ] + - [ "EG", 1442026800000, 16, 31, 1442073600000, 1442091600000, 1, 112, 2, 3, 0.5, 3, 0.6 ] + - [ "EG", 1442091600000, 27, 58, 1442055600000, 1442062800000, 1, 112, 2, 4, 0.75, 4, 0.8 ] + - [ "EG", 1442062800000, 112, 170, 1442026800000, null, 1, 112, 3, 5, 1.0, 5, 1.0 ] + - [ "ES", 1442044800000, -169, -169, null, 1442088000000, -169, 2506, 1, 1, 0.0, 1, 0.05 ] + - [ "ES", 1442088000000, -130, -299, null, 1442062800000, -169, 2506, 1, 2, 0.05263157894736842, 2, 0.1 ] + - [ "ES", 1442062800000, -71, -370, 1442044800000, 1442034000000, -169, 2506, 1, 3, 0.10526315789473684, 3, 0.15 ] + - [ "ES", 1442034000000, -52, -422, 1442088000000, 1442023200000, -169, 2506, 1, 4, 0.15789473684210525, 4, 0.2 ] + - [ "ES", 1442023200000, -5, -427, 1442062800000, 1442052000000, -169, 2506, 1, 5, 0.21052631578947367, 5, 0.25 ] + - [ "ES", 1442052000000, -4, -431, 1442034000000, 1442037600000, -169, 2506, 1, 6, 0.2631578947368421, 6, 0.3 ] + - [ "ES", 1442037600000, 3, -428, 1442023200000, 1442070000000, -169, 2506, 1, 7, 0.3157894736842105, 7, 0.35 ] + - [ "ES", 1442070000000, 61, -367, 1442052000000, 1442019600000, -169, 2506, 2, 8, 0.3684210526315789, 8, 0.4 ] + - [ "ES", 1442019600000, 103, -264, 1442037600000, 1442041200000, -169, 2506, 2, 9, 0.42105263157894735, 9, 0.45 ] + - [ "ES", 1442041200000, 118, -146, 1442070000000, 1442073600000, -169, 2506, 2, 10, 0.47368421052631576, 10, 0.5 ] + - [ "ES", 1442073600000, 154, 8, 1442019600000, 1442048400000, -169, 2506, 2, 11, 0.5263157894736842, 11, 0.55 ] + - [ "ES", 1442048400000, 158, 166, 1442041200000, 1442084400000, -169, 2506, 2, 12, 0.5789473684210527, 12, 0.6 ] + - [ "ES", 1442084400000, 337, 503, 1442073600000, 1442098800000, -169, 2506, 2, 13, 0.631578947368421, 13, 0.65 ] + - [ "ES", 1442098800000, 458, 961, 1442048400000, 1442066400000, -169, 2506, 2, 14, 0.6842105263157895, 14, 0.7 ] + - [ "ES", 1442066400000, 461, 1422, 1442084400000, 1442055600000, -169, 2506, 3, 15, 0.7368421052631579, 15, 0.75 ] + - [ "ES", 1442055600000, 495, 1917, 1442098800000, 1442091600000, -169, 2506, 3, 16, 0.7894736842105263, 16, 0.8 ] + - [ "ES", 1442091600000, 700, 2617, 1442066400000, 1442059200000, -169, 2506, 3, 17, 0.8421052631578947, 17, 0.85 ] + - [ "ES", 1442059200000, 1086, 3703, 1442055600000, 1442077200000, -169, 2506, 3, 18, 0.8947368421052632, 18, 0.9 ] + - [ "ES", 1442077200000, 1240, 4943, 1442091600000, 1442095200000, -169, 2506, 3, 19, 0.9473684210526315, 19, 0.95 ] + - [ "ES", 1442095200000, 2506, 7449, 1442059200000, null, -169, 2506, 3, 20, 1.0, 20, 1.0 ] + - [ "FI", 1442073600000, -1, -1, null, 1442048400000, -1, 1491, 1, 1, 0.0, 1, 0.08333333333333333 ] + - [ "FI", 1442048400000, 12, 11, null, 1442037600000, -1, 1491, 1, 2, 0.09090909090909091, 2, 0.16666666666666666 ] + - [ "FI", 1442037600000, 14, 25, 1442073600000, 1442062800000, -1, 1491, 1, 3, 0.18181818181818182, 3, 0.25 ] + - [ "FI", 1442062800000, 19, 44, 1442048400000, 1442095200000, -1, 1491, 1, 4, 0.2727272727272727, 4, 0.3333333333333333 ] + - [ "FI", 1442095200000, 69, 113, 1442037600000, 1442080800000, -1, 1491, 2, 5, 0.36363636363636365, 5, 0.4166666666666667 ] + - [ "FI", 1442080800000, 104, 217, 1442062800000, 1442066400000, -1, 1491, 2, 6, 0.45454545454545453, 6, 0.5 ] + - [ "FI", 1442066400000, 183, 400, 1442095200000, 1442052000000, -1, 1491, 2, 7, 0.5454545454545454, 7, 0.5833333333333334 ] + - [ "FI", 1442052000000, 186, 586, 1442080800000, 1442077200000, -1, 1491, 2, 8, 0.6363636363636364, 8, 0.6666666666666666 ] + - [ "FI", 1442077200000, 200, 786, 1442066400000, 1442059200000, -1, 1491, 3, 9, 0.7272727272727273, 9, 0.75 ] + - [ "FI", 1442059200000, 407, 1193, 1442052000000, 1442084400000, -1, 1491, 3, 10, 0.8181818181818182, 10, 0.8333333333333334 ] + - [ "FI", 1442084400000, 895, 2088, 1442077200000, 1442030400000, -1, 1491, 3, 11, 0.9090909090909091, 11, 0.9166666666666666 ] + - [ "FI", 1442030400000, 1491, 3579, 1442059200000, null, -1, 1491, 3, 12, 1.0, 12, 1.0 ] + - [ "FR", 1442077200000, -444, -444, null, 1442016000000, -444, 6643, 1, 1, 0.0, 1, 0.043478260869565216 ] + - [ "FR", 1442016000000, -1, -445, null, 1442026800000, -444, 6643, 1, 2, 0.045454545454545456, 2, 0.08695652173913043 ] + - [ "FR", 1442026800000, 86, -359, 1442077200000, 1442095200000, -444, 6643, 1, 3, 0.09090909090909091, 3, 0.13043478260869565 ] + - [ "FR", 1442095200000, 87, -272, 1442016000000, 1442098800000, -444, 6643, 1, 4, 0.13636363636363635, 4, 0.17391304347826086 ] + - [ "FR", 1442098800000, 136, -136, 1442026800000, 1442044800000, -444, 6643, 1, 5, 0.18181818181818182, 5, 0.21739130434782608 ] + - [ "FR", 1442044800000, 172, 36, 1442095200000, 1442055600000, -444, 6643, 1, 6, 0.22727272727272727, 6, 0.2608695652173913 ] + - [ "FR", 1442055600000, 463, 499, 1442098800000, 1442070000000, -444, 6643, 1, 7, 0.2727272727272727, 7, 0.30434782608695654 ] + - [ "FR", 1442070000000, 474, 973, 1442044800000, 1442034000000, -444, 6643, 1, 8, 0.3181818181818182, 8, 0.34782608695652173 ] + - [ "FR", 1442034000000, 476, 1449, 1442055600000, 1442080800000, -444, 6643, 2, 9, 0.36363636363636365, 9, 0.391304347826087 ] + - [ "FR", 1442080800000, 557, 2006, 1442070000000, 1442019600000, -444, 6643, 2, 10, 0.4090909090909091, 10, 0.43478260869565216 ] + - [ "FR", 1442019600000, 585, 2591, 1442034000000, 1442041200000, -444, 6643, 2, 11, 0.45454545454545453, 11, 0.4782608695652174 ] + - [ "FR", 1442041200000, 604, 3195, 1442080800000, 1442023200000, -444, 6643, 2, 12, 0.5, 12, 0.5217391304347826 ] + - [ "FR", 1442023200000, 628, 3823, 1442019600000, 1442052000000, -444, 6643, 2, 13, 0.5454545454545454, 13, 0.5652173913043478 ] + - [ "FR", 1442052000000, 637, 4460, 1442041200000, 1442091600000, -444, 6643, 2, 14, 0.5909090909090909, 14, 0.6086956521739131 ] + - [ "FR", 1442091600000, 741, 5201, 1442023200000, 1442088000000, -444, 6643, 2, 15, 0.6363636363636364, 15, 0.6521739130434783 ] + - [ "FR", 1442088000000, 1872, 7073, 1442052000000, 1442066400000, -444, 6643, 2, 16, 0.6818181818181818, 16, 0.6956521739130435 ] + - [ "FR", 1442066400000, 2516, 9589, 1442091600000, 1442048400000, -444, 6643, 3, 17, 0.7272727272727273, 17, 0.7391304347826086 ] + - [ "FR", 1442048400000, 3027, 12616, 1442088000000, 1442073600000, -444, 6643, 3, 18, 0.7727272727272727, 18, 0.782608695652174 ] + - [ "FR", 1442073600000, 3522, 16138, 1442066400000, 1442037600000, -444, 6643, 3, 19, 0.8181818181818182, 19, 0.8260869565217391 ] + - [ "FR", 1442037600000, 4174, 20312, 1442048400000, 1442059200000, -444, 6643, 3, 20, 0.8636363636363636, 20, 0.8695652173913043 ] + - [ "FR", 1442059200000, 4650, 24962, 1442073600000, 1442062800000, -444, 6643, 3, 21, 0.9090909090909091, 21, 0.9130434782608695 ] + - [ "FR", 1442062800000, 5676, 30638, 1442037600000, 1442084400000, -444, 6643, 3, 22, 0.9545454545454546, 22, 0.9565217391304348 ] + - [ "FR", 1442084400000, 6643, 37281, 1442059200000, null, -444, 6643, 3, 23, 1.0, 23, 1.0 ] + - [ "GB", 1442016000000, -44, -44, null, 1442034000000, -44, 16111, 1, 1, 0.0, 1, 0.041666666666666664 ] + - [ "GB", 1442034000000, -12, -56, null, 1442044800000, -44, 16111, 1, 2, 0.043478260869565216, 2, 0.08333333333333333 ] + - [ "GB", 1442044800000, 32, -24, 1442016000000, 1442041200000, -44, 16111, 1, 3, 0.08695652173913043, 3, 0.125 ] + - [ "GB", 1442041200000, 42, 18, 1442034000000, 1442098800000, -44, 16111, 1, 4, 0.13043478260869565, 4, 0.16666666666666666 ] + - [ "GB", 1442098800000, 49, 67, 1442044800000, 1442019600000, -44, 16111, 1, 5, 0.17391304347826086, 5, 0.20833333333333334 ] + - [ "GB", 1442019600000, 54, 121, 1442041200000, 1442052000000, -44, 16111, 1, 6, 0.21739130434782608, 6, 0.25 ] + - [ "GB", 1442052000000, 168, 289, 1442098800000, 1442095200000, -44, 16111, 1, 7, 0.2608695652173913, 7, 0.2916666666666667 ] + - [ "GB", 1442095200000, 238, 527, 1442019600000, 1442026800000, -44, 16111, 1, 8, 0.30434782608695654, 8, 0.3333333333333333 ] + - [ "GB", 1442026800000, 339, 866, 1442052000000, 1442070000000, -44, 16111, 2, 9, 0.34782608695652173, 9, 0.375 ] + - [ "GB", 1442070000000, 374, 1240, 1442095200000, 1442084400000, -44, 16111, 2, 10, 0.391304347826087, 10, 0.4166666666666667 ] + - [ "GB", 1442084400000, 384, 1624, 1442026800000, 1442055600000, -44, 16111, 2, 11, 0.43478260869565216, 11, 0.4583333333333333 ] + - [ "GB", 1442055600000, 453, 2077, 1442070000000, 1442037600000, -44, 16111, 2, 12, 0.4782608695652174, 12, 0.5 ] + - [ "GB", 1442037600000, 544, 2621, 1442084400000, 1442073600000, -44, 16111, 2, 13, 0.5217391304347826, 13, 0.5416666666666666 ] + - [ "GB", 1442073600000, 648, 3269, 1442055600000, 1442066400000, -44, 16111, 2, 14, 0.5652173913043478, 14, 0.5833333333333334 ] + - [ "GB", 1442066400000, 671, 3940, 1442037600000, 1442048400000, -44, 16111, 2, 15, 0.6086956521739131, 15, 0.625 ] + - [ "GB", 1442048400000, 740, 4680, 1442073600000, 1442091600000, -44, 16111, 2, 16, 0.6521739130434783, 16, 0.6666666666666666 ] + - [ "GB", 1442091600000, 811, 5491, 1442066400000, 1442077200000, -44, 16111, 3, 17, 0.6956521739130435, 17, 0.7083333333333334 ] + - [ "GB", 1442077200000, 1135, 6626, 1442048400000, 1442080800000, -44, 16111, 3, 18, 0.7391304347826086, 18, 0.75 ] + - [ "GB", 1442080800000, 1444, 8070, 1442091600000, 1442088000000, -44, 16111, 3, 19, 0.782608695652174, 19, 0.7916666666666666 ] + - [ "GB", 1442088000000, 1593, 9663, 1442077200000, 1442023200000, -44, 16111, 3, 20, 0.8260869565217391, 20, 0.8333333333333334 ] + - [ "GB", 1442023200000, 1816, 11479, 1442080800000, 1442030400000, -44, 16111, 3, 21, 0.8695652173913043, 21, 0.875 ] + - [ "GB", 1442030400000, 2524, 14003, 1442088000000, 1442062800000, -44, 16111, 3, 22, 0.9130434782608695, 22, 0.9166666666666666 ] + - [ "GB", 1442062800000, 5743, 19746, 1442023200000, 1442059200000, -44, 16111, 3, 23, 0.9565217391304348, 23, 0.9583333333333334 ] + - [ "GB", 1442059200000, 16111, 35857, 1442030400000, null, -44, 16111, 3, 24, 1.0, 24, 1.0 ] + - [ "GE", 1442052000000, -108, -108, null, 1442080800000, -108, 16, 1, 1, 0.0, 1, 0.25 ] + - [ "GE", 1442080800000, -27, -135, null, 1442044800000, -108, 16, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "GE", 1442044800000, -21, -156, 1442052000000, 1442062800000, -108, 16, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "GE", 1442062800000, 16, -140, 1442080800000, null, -108, 16, 3, 4, 1.0, 4, 1.0 ] + - [ "GH", 1442088000000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0 ] + - [ "GR", 1442073600000, -314, -314, null, 1442048400000, -314, 179, 1, 1, 0.0, 1, 0.1 ] + - [ "GR", 1442048400000, -26, -340, null, 1442034000000, -314, 179, 1, 2, 0.1111111111111111, 2, 0.2 ] + - [ "GR", 1442034000000, 0, -340, 1442073600000, 1442070000000, -314, 179, 1, 3, 0.2222222222222222, 3, 0.3 ] + - [ "GR", 1442070000000, 2, -338, 1442048400000, 1442041200000, -314, 179, 1, 4, 0.3333333333333333, 4, 0.4 ] + - [ "GR", 1442041200000, 7, -331, 1442034000000, 1442062800000, -314, 179, 2, 5, 0.4444444444444444, 5, 0.5 ] + - [ "GR", 1442062800000, 8, -323, 1442070000000, 1442019600000, -314, 179, 2, 6, 0.5555555555555556, 6, 0.6 ] + - [ "GR", 1442019600000, 82, -241, 1442041200000, 1442080800000, -314, 179, 2, 7, 0.6666666666666666, 7, 0.7 ] + - [ "GR", 1442080800000, 88, -153, 1442062800000, 1442091600000, -314, 179, 3, 8, 0.7777777777777778, 8, 0.8 ] + - [ "GR", 1442091600000, 123, -30, 1442019600000, 1442084400000, -314, 179, 3, 9, 0.8888888888888888, 9, 0.9 ] + - [ "GR", 1442084400000, 179, 149, 1442080800000, null, -314, 179, 3, 10, 1.0, 10, 1.0 ] + - [ "GT", 1442023200000, -167, -167, null, 1442098800000, -167, 173, 1, 1, 0.0, 1, 0.3333333333333333 ] + - [ "GT", 1442098800000, 1, -166, null, 1442026800000, -167, 173, 2, 2, 0.5, 2, 0.6666666666666666 ] + - [ "GT", 1442026800000, 173, 7, 1442023200000, null, -167, 173, 3, 3, 1.0, 3, 1.0 ] + - [ "HK", 1442026800000, -211, -211, null, 1442019600000, -211, 5545, 1, 1, 0.0, 1, 0.05263157894736842 ] + - [ "HK", 1442019600000, -113, -324, null, 1442041200000, -211, 5545, 1, 2, 0.05555555555555555, 2, 0.10526315789473684 ] + - [ "HK", 1442041200000, -15, -339, 1442026800000, 1442091600000, -211, 5545, 1, 3, 0.1111111111111111, 3, 0.15789473684210525 ] + - [ "HK", 1442091600000, -3, -342, 1442019600000, 1442095200000, -211, 5545, 1, 4, 0.16666666666666666, 4, 0.21052631578947367 ] + - [ "HK", 1442095200000, -1, -343, 1442041200000, 1442080800000, -211, 5545, 1, 5, 0.2222222222222222, 5, 0.2631578947368421 ] + - [ "HK", 1442080800000, 0, -343, 1442091600000, 1442048400000, -211, 5545, 1, 6, 0.2777777777777778, 6, 0.3157894736842105 ] + - [ "HK", 1442048400000, 1, -342, 1442095200000, 1442062800000, -211, 5545, 1, 7, 0.3333333333333333, 7, 0.42105263157894735 ] + - [ "HK", 1442062800000, 1, -341, 1442080800000, 1442059200000, -211, 5545, 2, 7, 0.3333333333333333, 7, 0.42105263157894735 ] + - [ "HK", 1442059200000, 2, -339, 1442048400000, 1442052000000, -211, 5545, 2, 9, 0.4444444444444444, 8, 0.47368421052631576 ] + - [ "HK", 1442052000000, 15, -324, 1442062800000, 1442044800000, -211, 5545, 2, 10, 0.5, 9, 0.5263157894736842 ] + - [ "HK", 1442044800000, 21, -303, 1442059200000, 1442066400000, -211, 5545, 2, 11, 0.5555555555555556, 10, 0.5789473684210527 ] + - [ "HK", 1442066400000, 39, -264, 1442052000000, 1442030400000, -211, 5545, 2, 12, 0.6111111111111112, 11, 0.631578947368421 ] + - [ "HK", 1442030400000, 157, -107, 1442044800000, 1442070000000, -211, 5545, 2, 13, 0.6666666666666666, 12, 0.6842105263157895 ] + - [ "HK", 1442070000000, 314, 207, 1442066400000, 1442037600000, -211, 5545, 3, 14, 0.7222222222222222, 13, 0.7368421052631579 ] + - [ "HK", 1442037600000, 636, 843, 1442030400000, 1442055600000, -211, 5545, 3, 15, 0.7777777777777778, 14, 0.7894736842105263 ] + - [ "HK", 1442055600000, 804, 1647, 1442070000000, 1442034000000, -211, 5545, 3, 16, 0.8333333333333334, 15, 0.8421052631578947 ] + - [ "HK", 1442034000000, 1137, 2784, 1442037600000, 1442023200000, -211, 5545, 3, 17, 0.8888888888888888, 16, 0.8947368421052632 ] + - [ "HK", 1442023200000, 2414, 5198, 1442055600000, 1442073600000, -211, 5545, 3, 18, 0.9444444444444444, 17, 0.9473684210526315 ] + - [ "HK", 1442073600000, 5545, 10743, 1442034000000, null, -211, 5545, 3, 19, 1.0, 18, 1.0 ] + - [ "HN", 1442026800000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0 ] + - [ "HR", 1442084400000, -10, -10, null, 1442073600000, -10, 220, 1, 1, 0.0, 1, 0.16666666666666666 ] + - [ "HR", 1442073600000, 0, -10, null, 1442070000000, -10, 220, 1, 2, 0.2, 2, 0.3333333333333333 ] + - [ "HR", 1442070000000, 32, 22, 1442084400000, 1442077200000, -10, 220, 2, 3, 0.4, 3, 0.5 ] + - [ "HR", 1442077200000, 58, 80, 1442073600000, 1442088000000, -10, 220, 2, 4, 0.6, 4, 0.6666666666666666 ] + - [ "HR", 1442088000000, 82, 162, 1442070000000, 1442080800000, -10, 220, 3, 5, 0.8, 5, 0.8333333333333334 ] + - [ "HR", 1442080800000, 220, 382, 1442077200000, null, -10, 220, 3, 6, 1.0, 6, 1.0 ] + - [ "HU", 1442088000000, -71, -71, null, 1442091600000, -71, 547, 1, 1, 0.0, 1, 0.07692307692307693 ] + - [ "HU", 1442091600000, -5, -76, null, 1442055600000, -71, 547, 1, 2, 0.08333333333333333, 2, 0.15384615384615385 ] + - [ "HU", 1442055600000, -2, -78, 1442088000000, 1442019600000, -71, 547, 1, 3, 0.16666666666666666, 3, 0.23076923076923078 ] + - [ "HU", 1442019600000, 46, -32, 1442091600000, 1442062800000, -71, 547, 1, 4, 0.25, 4, 0.3076923076923077 ] + - [ "HU", 1442062800000, 50, 18, 1442055600000, 1442041200000, -71, 547, 1, 5, 0.3333333333333333, 5, 0.38461538461538464 ] + - [ "HU", 1442041200000, 91, 109, 1442019600000, 1442098800000, -71, 547, 2, 6, 0.4166666666666667, 6, 0.46153846153846156 ] + - [ "HU", 1442098800000, 110, 219, 1442062800000, 1442084400000, -71, 547, 2, 7, 0.5, 7, 0.5384615384615384 ] + - [ "HU", 1442084400000, 141, 360, 1442041200000, 1442037600000, -71, 547, 2, 8, 0.5833333333333334, 8, 0.6153846153846154 ] + - [ "HU", 1442037600000, 197, 557, 1442098800000, 1442080800000, -71, 547, 2, 9, 0.6666666666666666, 9, 0.6923076923076923 ] + - [ "HU", 1442080800000, 242, 799, 1442084400000, 1442095200000, -71, 547, 3, 10, 0.75, 10, 0.7692307692307693 ] + - [ "HU", 1442095200000, 271, 1070, 1442037600000, 1442048400000, -71, 547, 3, 11, 0.8333333333333334, 11, 0.8461538461538461 ] + - [ "HU", 1442048400000, 499, 1569, 1442080800000, 1442044800000, -71, 547, 3, 12, 0.9166666666666666, 12, 0.9230769230769231 ] + - [ "HU", 1442044800000, 547, 2116, 1442095200000, null, -71, 547, 3, 13, 1.0, 13, 1.0 ] + - [ "ID", 1442026800000, -416, -416, null, 1442044800000, -416, 279, 1, 1, 0.0, 1, 0.07692307692307693 ] + - [ "ID", 1442044800000, -388, -804, null, 1442041200000, -416, 279, 1, 2, 0.08333333333333333, 2, 0.15384615384615385 ] + - [ "ID", 1442041200000, 2, -802, 1442026800000, 1442098800000, -416, 279, 1, 3, 0.16666666666666666, 3, 0.23076923076923078 ] + - [ "ID", 1442098800000, 13, -789, 1442044800000, 1442037600000, -416, 279, 1, 4, 0.25, 4, 0.3076923076923077 ] + - [ "ID", 1442037600000, 14, -775, 1442041200000, 1442055600000, -416, 279, 1, 5, 0.3333333333333333, 5, 0.38461538461538464 ] + - [ "ID", 1442055600000, 16, -759, 1442098800000, 1442059200000, -416, 279, 2, 6, 0.4166666666666667, 6, 0.46153846153846156 ] + - [ "ID", 1442059200000, 17, -742, 1442037600000, 1442034000000, -416, 279, 2, 7, 0.5, 7, 0.5384615384615384 ] + - [ "ID", 1442034000000, 19, -723, 1442055600000, 1442095200000, -416, 279, 2, 8, 0.5833333333333334, 8, 0.6153846153846154 ] + - [ "ID", 1442095200000, 20, -703, 1442059200000, 1442091600000, -416, 279, 2, 9, 0.6666666666666666, 9, 0.6923076923076923 ] + - [ "ID", 1442091600000, 21, -682, 1442034000000, 1442070000000, -416, 279, 3, 10, 0.75, 10, 0.7692307692307693 ] + - [ "ID", 1442070000000, 42, -640, 1442095200000, 1442023200000, -416, 279, 3, 11, 0.8333333333333334, 11, 0.8461538461538461 ] + - [ "ID", 1442023200000, 106, -534, 1442091600000, 1442030400000, -416, 279, 3, 12, 0.9166666666666666, 12, 0.9230769230769231 ] + - [ "ID", 1442030400000, 279, -255, 1442070000000, null, -416, 279, 3, 13, 1.0, 13, 1.0 ] + - [ "IE", 1442070000000, -100, -100, null, 1442091600000, -100, 1062, 1, 1, 0.0, 1, 0.125 ] + - [ "IE", 1442091600000, -71, -171, null, 1442026800000, -100, 1062, 1, 2, 0.14285714285714285, 2, 0.25 ] + - [ "IE", 1442026800000, 1, -170, 1442070000000, 1442030400000, -100, 1062, 1, 3, 0.2857142857142857, 3, 0.5 ] + - [ "IE", 1442030400000, 1, -169, 1442091600000, 1442048400000, -100, 1062, 2, 3, 0.2857142857142857, 3, 0.5 ] + - [ "IE", 1442048400000, 27, -142, 1442026800000, 1442077200000, -100, 1062, 2, 5, 0.5714285714285714, 4, 0.625 ] + - [ "IE", 1442077200000, 403, 261, 1442030400000, 1442084400000, -100, 1062, 2, 6, 0.7142857142857143, 5, 0.75 ] + - [ "IE", 1442084400000, 819, 1080, 1442048400000, 1442066400000, -100, 1062, 3, 7, 0.8571428571428571, 6, 0.875 ] + - [ "IE", 1442066400000, 1062, 2142, 1442077200000, null, -100, 1062, 3, 8, 1.0, 7, 1.0 ] + - [ "IL", 1442095200000, 0, 0, null, 1442066400000, 0, 2745, 1, 1, 0.0, 1, 0.0625 ] + - [ "IL", 1442066400000, 3, 3, null, 1442098800000, 0, 2745, 1, 2, 0.06666666666666667, 2, 0.1875 ] + - [ "IL", 1442098800000, 3, 6, 1442095200000, 1442055600000, 0, 2745, 1, 2, 0.06666666666666667, 2, 0.1875 ] + - [ "IL", 1442055600000, 4, 10, 1442066400000, 1442048400000, 0, 2745, 1, 4, 0.2, 3, 0.25 ] + - [ "IL", 1442048400000, 25, 35, 1442098800000, 1442073600000, 0, 2745, 1, 5, 0.26666666666666666, 4, 0.3125 ] + - [ "IL", 1442073600000, 31, 66, 1442055600000, 1442041200000, 0, 2745, 1, 6, 0.3333333333333333, 5, 0.375 ] + - [ "IL", 1442041200000, 35, 101, 1442048400000, 1442070000000, 0, 2745, 2, 7, 0.4, 6, 0.4375 ] + - [ "IL", 1442070000000, 49, 150, 1442073600000, 1442080800000, 0, 2745, 2, 8, 0.4666666666666667, 7, 0.5 ] + - [ "IL", 1442080800000, 88, 238, 1442041200000, 1442062800000, 0, 2745, 2, 9, 0.5333333333333333, 8, 0.5625 ] + - [ "IL", 1442062800000, 180, 418, 1442070000000, 1442077200000, 0, 2745, 2, 10, 0.6, 9, 0.625 ] + - [ "IL", 1442077200000, 187, 605, 1442080800000, 1442044800000, 0, 2745, 2, 11, 0.6666666666666666, 10, 0.6875 ] + - [ "IL", 1442044800000, 218, 823, 1442062800000, 1442091600000, 0, 2745, 3, 12, 0.7333333333333333, 11, 0.75 ] + - [ "IL", 1442091600000, 707, 1530, 1442077200000, 1442084400000, 0, 2745, 3, 13, 0.8, 12, 0.8125 ] + - [ "IL", 1442084400000, 1137, 2667, 1442044800000, 1442059200000, 0, 2745, 3, 14, 0.8666666666666667, 13, 0.875 ] + - [ "IL", 1442059200000, 1205, 3872, 1442091600000, 1442052000000, 0, 2745, 3, 15, 0.9333333333333333, 14, 0.9375 ] + - [ "IL", 1442052000000, 2745, 6617, 1442084400000, null, 0, 2745, 3, 16, 1.0, 15, 1.0 ] + - [ "IN", 1442023200000, -142, -142, null, 1442080800000, -142, 12091, 1, 1, 0.0, 1, 0.045454545454545456 ] + - [ "IN", 1442080800000, 0, -142, null, 1442016000000, -142, 12091, 1, 2, 0.047619047619047616, 2, 0.09090909090909091 ] + - [ "IN", 1442016000000, 1, -141, 1442023200000, 1442095200000, -142, 12091, 1, 3, 0.09523809523809523, 3, 0.13636363636363635 ] + - [ "IN", 1442095200000, 4, -137, 1442080800000, 1442019600000, -142, 12091, 1, 4, 0.14285714285714285, 4, 0.18181818181818182 ] + - [ "IN", 1442019600000, 38, -99, 1442016000000, 1442041200000, -142, 12091, 1, 5, 0.19047619047619047, 5, 0.22727272727272727 ] + - [ "IN", 1442041200000, 80, -19, 1442095200000, 1442066400000, -142, 12091, 1, 6, 0.23809523809523808, 6, 0.2727272727272727 ] + - [ "IN", 1442066400000, 116, 97, 1442019600000, 1442088000000, -142, 12091, 1, 7, 0.2857142857142857, 7, 0.3181818181818182 ] + - [ "IN", 1442088000000, 121, 218, 1442041200000, 1442037600000, -142, 12091, 1, 8, 0.3333333333333333, 8, 0.36363636363636365 ] + - [ "IN", 1442037600000, 135, 353, 1442066400000, 1442055600000, -142, 12091, 2, 9, 0.38095238095238093, 9, 0.4090909090909091 ] + - [ "IN", 1442055600000, 166, 519, 1442088000000, 1442084400000, -142, 12091, 2, 10, 0.42857142857142855, 10, 0.45454545454545453 ] + - [ "IN", 1442084400000, 187, 706, 1442037600000, 1442048400000, -142, 12091, 2, 11, 0.47619047619047616, 11, 0.5 ] + - [ "IN", 1442048400000, 262, 968, 1442055600000, 1442052000000, -142, 12091, 2, 12, 0.5238095238095238, 12, 0.5454545454545454 ] + - [ "IN", 1442052000000, 534, 1502, 1442084400000, 1442059200000, -142, 12091, 2, 13, 0.5714285714285714, 13, 0.5909090909090909 ] + - [ "IN", 1442059200000, 708, 2210, 1442048400000, 1442026800000, -142, 12091, 2, 14, 0.6190476190476191, 14, 0.6363636363636364 ] + - [ "IN", 1442026800000, 974, 3184, 1442052000000, 1442073600000, -142, 12091, 2, 15, 0.6666666666666666, 15, 0.6818181818181818 ] + - [ "IN", 1442073600000, 1170, 4354, 1442059200000, 1442034000000, -142, 12091, 3, 16, 0.7142857142857143, 16, 0.7272727272727273 ] + - [ "IN", 1442034000000, 1350, 5704, 1442026800000, 1442030400000, -142, 12091, 3, 17, 0.7619047619047619, 17, 0.7727272727272727 ] + - [ "IN", 1442030400000, 1448, 7152, 1442073600000, 1442062800000, -142, 12091, 3, 18, 0.8095238095238095, 18, 0.8181818181818182 ] + - [ "IN", 1442062800000, 1547, 8699, 1442034000000, 1442044800000, -142, 12091, 3, 19, 0.8571428571428571, 19, 0.8636363636363636 ] + - [ "IN", 1442044800000, 2677, 11376, 1442030400000, 1442077200000, -142, 12091, 3, 20, 0.9047619047619048, 20, 0.9090909090909091 ] + - [ "IN", 1442077200000, 5699, 17075, 1442062800000, 1442070000000, -142, 12091, 3, 21, 0.9523809523809523, 21, 0.9545454545454546 ] + - [ "IN", 1442070000000, 12091, 29166, 1442044800000, null, -142, 12091, 3, 22, 1.0, 22, 1.0 ] + - [ "IQ", 1442095200000, -2, -2, null, 1442041200000, -2, 6, 1, 1, 0.0, 1, 0.25 ] + - [ "IQ", 1442041200000, -1, -3, null, 1442052000000, -2, 6, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "IQ", 1442052000000, 0, -3, 1442095200000, 1442044800000, -2, 6, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "IQ", 1442044800000, 6, 3, 1442041200000, null, -2, 6, 3, 4, 1.0, 4, 1.0 ] + - [ "IR", 1442073600000, -193, -193, null, 1442055600000, -193, 1455, 1, 1, 0.0, 1, 0.07692307692307693 ] + - [ "IR", 1442055600000, -124, -317, null, 1442041200000, -193, 1455, 1, 2, 0.08333333333333333, 2, 0.15384615384615385 ] + - [ "IR", 1442041200000, -79, -396, 1442073600000, 1442077200000, -193, 1455, 1, 3, 0.16666666666666666, 3, 0.23076923076923078 ] + - [ "IR", 1442077200000, -34, -430, 1442055600000, 1442034000000, -193, 1455, 1, 4, 0.25, 4, 0.3076923076923077 ] + - [ "IR", 1442034000000, -8, -438, 1442041200000, 1442026800000, -193, 1455, 1, 5, 0.3333333333333333, 5, 0.38461538461538464 ] + - [ "IR", 1442026800000, 0, -438, 1442077200000, 1442091600000, -193, 1455, 2, 6, 0.4166666666666667, 6, 0.5384615384615384 ] + - [ "IR", 1442091600000, 0, -438, 1442034000000, 1442080800000, -193, 1455, 2, 6, 0.4166666666666667, 6, 0.5384615384615384 ] + - [ "IR", 1442080800000, 131, -307, 1442026800000, 1442052000000, -193, 1455, 2, 8, 0.5833333333333334, 7, 0.6153846153846154 ] + - [ "IR", 1442052000000, 155, -152, 1442091600000, 1442044800000, -193, 1455, 2, 9, 0.6666666666666666, 8, 0.6923076923076923 ] + - [ "IR", 1442044800000, 306, 154, 1442080800000, 1442030400000, -193, 1455, 3, 10, 0.75, 9, 0.7692307692307693 ] + - [ "IR", 1442030400000, 375, 529, 1442052000000, 1442088000000, -193, 1455, 3, 11, 0.8333333333333334, 10, 0.8461538461538461 ] + - [ "IR", 1442088000000, 714, 1243, 1442044800000, 1442059200000, -193, 1455, 3, 12, 0.9166666666666666, 11, 0.9230769230769231 ] + - [ "IR", 1442059200000, 1455, 2698, 1442030400000, null, -193, 1455, 3, 13, 1.0, 12, 1.0 ] + - [ "IT", 1442030400000, -17, -17, null, 1442037600000, -17, 6240, 1, 1, 0.0, 1, 0.041666666666666664 ] + - [ "IT", 1442037600000, -9, -26, null, 1442016000000, -17, 6240, 1, 2, 0.043478260869565216, 2, 0.08333333333333333 ] + - [ "IT", 1442016000000, 0, -26, 1442030400000, 1442041200000, -17, 6240, 1, 3, 0.08695652173913043, 3, 0.125 ] + - [ "IT", 1442041200000, 20, -6, 1442037600000, 1442070000000, -17, 6240, 1, 4, 0.13043478260869565, 4, 0.16666666666666666 ] + - [ "IT", 1442070000000, 81, 75, 1442016000000, 1442023200000, -17, 6240, 1, 5, 0.17391304347826086, 5, 0.20833333333333334 ] + - [ "IT", 1442023200000, 111, 186, 1442041200000, 1442019600000, -17, 6240, 1, 6, 0.21739130434782608, 6, 0.25 ] + - [ "IT", 1442019600000, 183, 369, 1442070000000, 1442026800000, -17, 6240, 1, 7, 0.2608695652173913, 7, 0.2916666666666667 ] + - [ "IT", 1442026800000, 222, 591, 1442023200000, 1442091600000, -17, 6240, 1, 8, 0.30434782608695654, 8, 0.3333333333333333 ] + - [ "IT", 1442091600000, 351, 942, 1442019600000, 1442059200000, -17, 6240, 2, 9, 0.34782608695652173, 9, 0.375 ] + - [ "IT", 1442059200000, 542, 1484, 1442026800000, 1442098800000, -17, 6240, 2, 10, 0.391304347826087, 10, 0.4166666666666667 ] + - [ "IT", 1442098800000, 565, 2049, 1442091600000, 1442048400000, -17, 6240, 2, 11, 0.43478260869565216, 11, 0.4583333333333333 ] + - [ "IT", 1442048400000, 676, 2725, 1442059200000, 1442034000000, -17, 6240, 2, 12, 0.4782608695652174, 12, 0.5 ] + - [ "IT", 1442034000000, 1006, 3731, 1442098800000, 1442044800000, -17, 6240, 2, 13, 0.5217391304347826, 13, 0.5416666666666666 ] + - [ "IT", 1442044800000, 1483, 5214, 1442048400000, 1442052000000, -17, 6240, 2, 14, 0.5652173913043478, 14, 0.5833333333333334 ] + - [ "IT", 1442052000000, 1880, 7094, 1442034000000, 1442062800000, -17, 6240, 2, 15, 0.6086956521739131, 15, 0.625 ] + - [ "IT", 1442062800000, 1938, 9032, 1442044800000, 1442077200000, -17, 6240, 2, 16, 0.6521739130434783, 16, 0.6666666666666666 ] + - [ "IT", 1442077200000, 2188, 11220, 1442052000000, 1442073600000, -17, 6240, 3, 17, 0.6956521739130435, 17, 0.7083333333333334 ] + - [ "IT", 1442073600000, 2586, 13806, 1442062800000, 1442084400000, -17, 6240, 3, 18, 0.7391304347826086, 18, 0.75 ] + - [ "IT", 1442084400000, 2660, 16466, 1442077200000, 1442095200000, -17, 6240, 3, 19, 0.782608695652174, 19, 0.7916666666666666 ] + - [ "IT", 1442095200000, 2940, 19406, 1442073600000, 1442088000000, -17, 6240, 3, 20, 0.8260869565217391, 20, 0.8333333333333334 ] + - [ "IT", 1442088000000, 3746, 23152, 1442084400000, 1442066400000, -17, 6240, 3, 21, 0.8695652173913043, 21, 0.875 ] + - [ "IT", 1442066400000, 4155, 27307, 1442095200000, 1442080800000, -17, 6240, 3, 22, 0.9130434782608695, 22, 0.9166666666666666 ] + - [ "IT", 1442080800000, 5544, 32851, 1442088000000, 1442055600000, -17, 6240, 3, 23, 0.9565217391304348, 23, 0.9583333333333334 ] + - [ "IT", 1442055600000, 6240, 39091, 1442066400000, null, -17, 6240, 3, 24, 1.0, 24, 1.0 ] + - [ "JM", 1442070000000, 30, 30, null, null, 30, 30, 1, 1, 0.0, 1, 1.0 ] + - [ "JO", 1442055600000, -2, -2, null, 1442059200000, -2, 4, 1, 1, 0.0, 1, 0.3333333333333333 ] + - [ "JO", 1442059200000, 0, -2, null, 1442080800000, -2, 4, 2, 2, 0.5, 2, 0.6666666666666666 ] + - [ "JO", 1442080800000, 4, 2, 1442055600000, null, -2, 4, 3, 3, 1.0, 3, 1.0 ] + - [ "JP", 1442016000000, -113, -113, null, 1442059200000, -113, 2789, 1, 1, 0.0, 1, 0.041666666666666664 ] + - [ "JP", 1442059200000, -85, -198, null, 1442098800000, -113, 2789, 1, 2, 0.043478260869565216, 2, 0.08333333333333333 ] + - [ "JP", 1442098800000, -6, -204, 1442016000000, 1442095200000, -113, 2789, 1, 3, 0.08695652173913043, 3, 0.125 ] + - [ "JP", 1442095200000, 0, -204, 1442059200000, 1442084400000, -113, 2789, 1, 4, 0.13043478260869565, 4, 0.16666666666666666 ] + - [ "JP", 1442084400000, 13, -191, 1442098800000, 1442077200000, -113, 2789, 1, 5, 0.17391304347826086, 5, 0.20833333333333334 ] + - [ "JP", 1442077200000, 51, -140, 1442095200000, 1442088000000, -113, 2789, 1, 6, 0.21739130434782608, 6, 0.25 ] + - [ "JP", 1442088000000, 57, -83, 1442084400000, 1442070000000, -113, 2789, 1, 7, 0.2608695652173913, 7, 0.2916666666666667 ] + - [ "JP", 1442070000000, 79, -4, 1442077200000, 1442066400000, -113, 2789, 1, 8, 0.30434782608695654, 8, 0.3333333333333333 ] + - [ "JP", 1442066400000, 167, 163, 1442088000000, 1442091600000, -113, 2789, 2, 9, 0.34782608695652173, 9, 0.375 ] + - [ "JP", 1442091600000, 228, 391, 1442070000000, 1442080800000, -113, 2789, 2, 10, 0.391304347826087, 10, 0.4166666666666667 ] + - [ "JP", 1442080800000, 420, 811, 1442066400000, 1442062800000, -113, 2789, 2, 11, 0.43478260869565216, 11, 0.4583333333333333 ] + - [ "JP", 1442062800000, 803, 1614, 1442091600000, 1442030400000, -113, 2789, 2, 12, 0.4782608695652174, 12, 0.5 ] + - [ "JP", 1442030400000, 805, 2419, 1442080800000, 1442034000000, -113, 2789, 2, 13, 0.5217391304347826, 13, 0.5416666666666666 ] + - [ "JP", 1442034000000, 910, 3329, 1442062800000, 1442055600000, -113, 2789, 2, 14, 0.5652173913043478, 14, 0.5833333333333334 ] + - [ "JP", 1442055600000, 998, 4327, 1442030400000, 1442026800000, -113, 2789, 2, 15, 0.6086956521739131, 15, 0.625 ] + - [ "JP", 1442026800000, 1035, 5362, 1442034000000, 1442073600000, -113, 2789, 2, 16, 0.6521739130434783, 16, 0.6666666666666666 ] + - [ "JP", 1442073600000, 1162, 6524, 1442055600000, 1442041200000, -113, 2789, 3, 17, 0.6956521739130435, 17, 0.7083333333333334 ] + - [ "JP", 1442041200000, 1373, 7897, 1442026800000, 1442044800000, -113, 2789, 3, 18, 0.7391304347826086, 18, 0.75 ] + - [ "JP", 1442044800000, 1569, 9466, 1442073600000, 1442023200000, -113, 2789, 3, 19, 0.782608695652174, 19, 0.7916666666666666 ] + - [ "JP", 1442023200000, 1959, 11425, 1442041200000, 1442048400000, -113, 2789, 3, 20, 0.8260869565217391, 20, 0.8333333333333334 ] + - [ "JP", 1442048400000, 1981, 13406, 1442044800000, 1442019600000, -113, 2789, 3, 21, 0.8695652173913043, 21, 0.875 ] + - [ "JP", 1442019600000, 2002, 15408, 1442023200000, 1442037600000, -113, 2789, 3, 22, 0.9130434782608695, 22, 0.9166666666666666 ] + - [ "JP", 1442037600000, 2181, 17589, 1442048400000, 1442052000000, -113, 2789, 3, 23, 0.9565217391304348, 23, 0.9583333333333334 ] + - [ "JP", 1442052000000, 2789, 20378, 1442019600000, null, -113, 2789, 3, 24, 1.0, 24, 1.0 ] + - [ "KE", 1442044800000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0 ] + - [ "KG", 1442073600000, 6, 6, null, null, 6, 6, 1, 1, 0.0, 1, 1.0 ] + - [ "KR", 1442048400000, -374, -374, null, 1442026800000, -374, 3640, 1, 1, 0.0, 1, 0.045454545454545456 ] + - [ "KR", 1442026800000, -179, -553, null, 1442077200000, -374, 3640, 1, 2, 0.047619047619047616, 2, 0.09090909090909091 ] + - [ "KR", 1442077200000, -40, -593, 1442048400000, 1442098800000, -374, 3640, 1, 3, 0.09523809523809523, 3, 0.13636363636363635 ] + - [ "KR", 1442098800000, -36, -629, 1442026800000, 1442080800000, -374, 3640, 1, 4, 0.14285714285714285, 4, 0.18181818181818182 ] + - [ "KR", 1442080800000, -33, -662, 1442077200000, 1442052000000, -374, 3640, 1, 5, 0.19047619047619047, 5, 0.22727272727272727 ] + - [ "KR", 1442052000000, -3, -665, 1442098800000, 1442041200000, -374, 3640, 1, 6, 0.23809523809523808, 6, 0.2727272727272727 ] + - [ "KR", 1442041200000, 20, -645, 1442080800000, 1442037600000, -374, 3640, 1, 7, 0.2857142857142857, 7, 0.3181818181818182 ] + - [ "KR", 1442037600000, 26, -619, 1442052000000, 1442059200000, -374, 3640, 1, 8, 0.3333333333333333, 8, 0.36363636363636365 ] + - [ "KR", 1442059200000, 208, -411, 1442041200000, 1442070000000, -374, 3640, 2, 9, 0.38095238095238093, 9, 0.4090909090909091 ] + - [ "KR", 1442070000000, 222, -189, 1442037600000, 1442084400000, -374, 3640, 2, 10, 0.42857142857142855, 10, 0.45454545454545453 ] + - [ "KR", 1442084400000, 314, 125, 1442059200000, 1442023200000, -374, 3640, 2, 11, 0.47619047619047616, 11, 0.5 ] + - [ "KR", 1442023200000, 319, 444, 1442070000000, 1442034000000, -374, 3640, 2, 12, 0.5238095238095238, 12, 0.5454545454545454 ] + - [ "KR", 1442034000000, 434, 878, 1442084400000, 1442019600000, -374, 3640, 2, 13, 0.5714285714285714, 13, 0.5909090909090909 ] + - [ "KR", 1442019600000, 445, 1323, 1442023200000, 1442088000000, -374, 3640, 2, 14, 0.6190476190476191, 14, 0.6363636363636364 ] + - [ "KR", 1442088000000, 524, 1847, 1442034000000, 1442095200000, -374, 3640, 2, 15, 0.6666666666666666, 15, 0.6818181818181818 ] + - [ "KR", 1442095200000, 827, 2674, 1442019600000, 1442044800000, -374, 3640, 3, 16, 0.7142857142857143, 16, 0.7272727272727273 ] + - [ "KR", 1442044800000, 829, 3503, 1442088000000, 1442016000000, -374, 3640, 3, 17, 0.7619047619047619, 17, 0.7727272727272727 ] + - [ "KR", 1442016000000, 1024, 4527, 1442095200000, 1442030400000, -374, 3640, 3, 18, 0.8095238095238095, 18, 0.8181818181818182 ] + - [ "KR", 1442030400000, 1035, 5562, 1442044800000, 1442062800000, -374, 3640, 3, 19, 0.8571428571428571, 19, 0.8636363636363636 ] + - [ "KR", 1442062800000, 1096, 6658, 1442016000000, 1442066400000, -374, 3640, 3, 20, 0.9047619047619048, 20, 0.9090909090909091 ] + - [ "KR", 1442066400000, 3299, 9957, 1442030400000, 1442055600000, -374, 3640, 3, 21, 0.9523809523809523, 21, 0.9545454545454546 ] + - [ "KR", 1442055600000, 3640, 13597, 1442062800000, null, -374, 3640, 3, 22, 1.0, 22, 1.0 ] + - [ "KW", 1442080800000, -33, -33, null, 1442055600000, -33, 1815, 1, 1, 0.0, 1, 0.25 ] + - [ "KW", 1442055600000, -2, -35, null, 1442077200000, -33, 1815, 1, 2, 0.3333333333333333, 2, 0.75 ] + - [ "KW", 1442077200000, -2, -37, 1442080800000, 1442070000000, -33, 1815, 2, 2, 0.3333333333333333, 2, 0.75 ] + - [ "KW", 1442070000000, 1815, 1778, 1442055600000, null, -33, 1815, 3, 4, 1.0, 3, 1.0 ] + - [ "KZ", 1442077200000, -317, -317, null, 1442084400000, -317, 439, 1, 1, 0.0, 1, 0.09090909090909091 ] + - [ "KZ", 1442084400000, -22, -339, null, 1442062800000, -317, 439, 1, 2, 0.1, 2, 0.18181818181818182 ] + - [ "KZ", 1442062800000, 0, -339, 1442077200000, 1442066400000, -317, 439, 1, 3, 0.2, 3, 0.36363636363636365 ] + - [ "KZ", 1442066400000, 0, -339, 1442084400000, 1442059200000, -317, 439, 1, 3, 0.2, 3, 0.36363636363636365 ] + - [ "KZ", 1442059200000, 33, -306, 1442062800000, 1442055600000, -317, 439, 2, 5, 0.4, 4, 0.45454545454545453 ] + - [ "KZ", 1442055600000, 63, -243, 1442066400000, 1442095200000, -317, 439, 2, 6, 0.5, 5, 0.5454545454545454 ] + - [ "KZ", 1442095200000, 91, -152, 1442059200000, 1442034000000, -317, 439, 2, 7, 0.6, 6, 0.6363636363636364 ] + - [ "KZ", 1442034000000, 161, 9, 1442055600000, 1442044800000, -317, 439, 2, 8, 0.7, 7, 0.7272727272727273 ] + - [ "KZ", 1442044800000, 401, 410, 1442095200000, 1442052000000, -317, 439, 3, 9, 0.8, 8, 0.8181818181818182 ] + - [ "KZ", 1442052000000, 412, 822, 1442034000000, 1442048400000, -317, 439, 3, 10, 0.9, 9, 0.9090909090909091 ] + - [ "KZ", 1442048400000, 439, 1261, 1442044800000, null, -317, 439, 3, 11, 1.0, 10, 1.0 ] + - [ "LB", 1442055600000, -67, -67, null, null, -67, -67, 1, 1, 0.0, 1, 1.0 ] + - [ "LK", 1442084400000, -3, -3, null, 1442048400000, -3, 79, 1, 1, 0.0, 1, 0.25 ] + - [ "LK", 1442048400000, 8, 5, null, 1442052000000, -3, 79, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "LK", 1442052000000, 47, 52, 1442084400000, 1442026800000, -3, 79, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "LK", 1442026800000, 79, 131, 1442048400000, null, -3, 79, 3, 4, 1.0, 4, 1.0 ] + - [ "LT", 1442098800000, -24, -24, null, 1442080800000, -24, 12, 1, 1, 0.0, 1, 0.5 ] + - [ "LT", 1442080800000, 12, -12, null, null, -24, 12, 2, 2, 1.0, 2, 1.0 ] + - [ "LU", 1442066400000, 0, 0, null, 1442095200000, 0, 525, 1, 1, 0.0, 1, 0.25 ] + - [ "LU", 1442095200000, 2, 2, null, 1442059200000, 0, 525, 1, 2, 0.3333333333333333, 2, 0.5 ] + - [ "LU", 1442059200000, 79, 81, 1442066400000, 1442077200000, 0, 525, 2, 3, 0.6666666666666666, 3, 0.75 ] + - [ "LU", 1442077200000, 525, 606, 1442095200000, null, 0, 525, 3, 4, 1.0, 4, 1.0 ] + - [ "LV", 1442095200000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0 ] + - [ "MA", 1442059200000, -56, -56, null, 1442019600000, -56, 250, 1, 1, 0.0, 1, 0.14285714285714285 ] + - [ "MA", 1442019600000, -1, -57, null, 1442062800000, -56, 250, 1, 2, 0.16666666666666666, 2, 0.2857142857142857 ] + - [ "MA", 1442062800000, 0, -57, 1442059200000, 1442080800000, -56, 250, 1, 3, 0.3333333333333333, 3, 0.42857142857142855 ] + - [ "MA", 1442080800000, 5, -52, 1442019600000, 1442098800000, -56, 250, 2, 4, 0.5, 4, 0.5714285714285714 ] + - [ "MA", 1442098800000, 8, -44, 1442062800000, 1442055600000, -56, 250, 2, 5, 0.6666666666666666, 5, 0.7142857142857143 ] + - [ "MA", 1442055600000, 23, -21, 1442080800000, 1442077200000, -56, 250, 3, 6, 0.8333333333333334, 6, 0.8571428571428571 ] + - [ "MA", 1442077200000, 250, 229, 1442098800000, null, -56, 250, 3, 7, 1.0, 7, 1.0 ] + - [ "MD", 1442077200000, 6916, 6916, null, null, 6916, 6916, 1, 1, 0.0, 1, 1.0 ] + - [ "ME", 1442073600000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0 ] + - [ "MH", 1442052000000, 40, 40, null, null, 40, 40, 1, 1, 0.0, 1, 1.0 ] + - [ "MK", 1442077200000, -72, -72, null, null, -72, -72, 1, 1, 0.0, 1, 1.0 ] + - [ "MM", 1442070000000, 3, 3, null, 1442073600000, 3, 25, 1, 1, 0.0, 1, 0.5 ] + - [ "MM", 1442073600000, 25, 28, null, null, 3, 25, 2, 2, 1.0, 2, 1.0 ] + - [ "MO", 1442070000000, 18, 18, null, 1442034000000, 18, 30, 1, 1, 0.0, 1, 0.5 ] + - [ "MO", 1442034000000, 30, 48, null, null, 18, 30, 2, 2, 1.0, 2, 1.0 ] + - [ "MR", 1442080800000, 10, 10, null, null, 10, 10, 1, 1, 0.0, 1, 1.0 ] + - [ "MT", 1442048400000, -1, -1, null, null, -1, -1, 1, 1, 0.0, 1, 1.0 ] + - [ "MV", 1442073600000, -3, -3, null, null, -3, -3, 1, 1, 0.0, 1, 1.0 ] + - [ "MX", 1442095200000, -456, -456, null, 1442080800000, -456, 3874, 1, 1, 0.0, 1, 0.058823529411764705 ] + - [ "MX", 1442080800000, -376, -832, null, 1442041200000, -456, 3874, 1, 2, 0.0625, 2, 0.11764705882352941 ] + - [ "MX", 1442041200000, -294, -1126, 1442095200000, 1442016000000, -456, 3874, 1, 3, 0.125, 3, 0.17647058823529413 ] + - [ "MX", 1442016000000, -67, -1193, 1442080800000, 1442073600000, -456, 3874, 1, 4, 0.1875, 4, 0.23529411764705882 ] + - [ "MX", 1442073600000, -21, -1214, 1442041200000, 1442066400000, -456, 3874, 1, 5, 0.25, 5, 0.29411764705882354 ] + - [ "MX", 1442066400000, -1, -1215, 1442016000000, 1442070000000, -456, 3874, 1, 6, 0.3125, 6, 0.4117647058823529 ] + - [ "MX", 1442070000000, -1, -1216, 1442073600000, 1442037600000, -456, 3874, 2, 6, 0.3125, 6, 0.4117647058823529 ] + - [ "MX", 1442037600000, 4, -1212, 1442066400000, 1442098800000, -456, 3874, 2, 8, 0.4375, 7, 0.47058823529411764 ] + - [ "MX", 1442098800000, 28, -1184, 1442070000000, 1442030400000, -456, 3874, 2, 9, 0.5, 8, 0.5294117647058824 ] + - [ "MX", 1442030400000, 373, -811, 1442037600000, 1442088000000, -456, 3874, 2, 10, 0.5625, 9, 0.5882352941176471 ] + - [ "MX", 1442088000000, 494, -317, 1442098800000, 1442023200000, -456, 3874, 2, 11, 0.625, 10, 0.6470588235294118 ] + - [ "MX", 1442023200000, 549, 232, 1442030400000, 1442091600000, -456, 3874, 2, 12, 0.6875, 11, 0.7058823529411765 ] + - [ "MX", 1442091600000, 799, 1031, 1442088000000, 1442034000000, -456, 3874, 3, 13, 0.75, 12, 0.7647058823529411 ] + - [ "MX", 1442034000000, 944, 1975, 1442023200000, 1442084400000, -456, 3874, 3, 14, 0.8125, 13, 0.8235294117647058 ] + - [ "MX", 1442084400000, 981, 2956, 1442091600000, 1442026800000, -456, 3874, 3, 15, 0.875, 14, 0.8823529411764706 ] + - [ "MX", 1442026800000, 3642, 6598, 1442034000000, 1442077200000, -456, 3874, 3, 16, 0.9375, 15, 0.9411764705882353 ] + - [ "MX", 1442077200000, 3874, 10472, 1442084400000, null, -456, 3874, 3, 17, 1.0, 16, 1.0 ] + - [ "MY", 1442044800000, -127, -127, null, 1442077200000, -127, 1028, 1, 1, 0.0, 1, 0.08333333333333333 ] + - [ "MY", 1442077200000, -10, -137, null, 1442019600000, -127, 1028, 1, 2, 0.09090909090909091, 2, 0.16666666666666666 ] + - [ "MY", 1442019600000, -7, -144, 1442044800000, 1442030400000, -127, 1028, 1, 3, 0.18181818181818182, 3, 0.25 ] + - [ "MY", 1442030400000, -3, -147, 1442077200000, 1442059200000, -127, 1028, 1, 4, 0.2727272727272727, 4, 0.3333333333333333 ] + - [ "MY", 1442059200000, 0, -147, 1442019600000, 1442055600000, -127, 1028, 2, 5, 0.36363636363636365, 5, 0.4166666666666667 ] + - [ "MY", 1442055600000, 1, -146, 1442030400000, 1442066400000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666 ] + - [ "MY", 1442066400000, 1, -145, 1442059200000, 1442073600000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666 ] + - [ "MY", 1442073600000, 1, -144, 1442055600000, 1442048400000, -127, 1028, 2, 6, 0.45454545454545453, 6, 0.6666666666666666 ] + - [ "MY", 1442048400000, 649, 505, 1442066400000, 1442098800000, -127, 1028, 3, 9, 0.7272727272727273, 7, 0.75 ] + - [ "MY", 1442098800000, 739, 1244, 1442073600000, 1442041200000, -127, 1028, 3, 10, 0.8181818181818182, 8, 0.8333333333333334 ] + - [ "MY", 1442041200000, 935, 2179, 1442048400000, 1442034000000, -127, 1028, 3, 11, 0.9090909090909091, 9, 0.9166666666666666 ] + - [ "MY", 1442034000000, 1028, 3207, 1442098800000, null, -127, 1028, 3, 12, 1.0, 10, 1.0 ] + - [ "NG", 1442070000000, 6, 6, null, 1442052000000, 6, 208, 1, 1, 0.0, 1, 0.5 ] + - [ "NG", 1442052000000, 208, 214, null, null, 6, 208, 2, 2, 1.0, 2, 1.0 ] + - [ "NL", 1442070000000, -84, -84, null, 1442062800000, -84, 8947, 1, 1, 0.0, 1, 0.058823529411764705 ] + - [ "NL", 1442062800000, -30, -114, null, 1442034000000, -84, 8947, 1, 2, 0.0625, 2, 0.11764705882352941 ] + - [ "NL", 1442034000000, 0, -114, 1442070000000, 1442098800000, -84, 8947, 1, 3, 0.125, 3, 0.17647058823529413 ] + - [ "NL", 1442098800000, 4, -110, 1442062800000, 1442088000000, -84, 8947, 1, 4, 0.1875, 4, 0.23529411764705882 ] + - [ "NL", 1442088000000, 12, -98, 1442034000000, 1442044800000, -84, 8947, 1, 5, 0.25, 5, 0.29411764705882354 ] + - [ "NL", 1442044800000, 16, -82, 1442098800000, 1442091600000, -84, 8947, 1, 6, 0.3125, 6, 0.35294117647058826 ] + - [ "NL", 1442091600000, 19, -63, 1442088000000, 1442052000000, -84, 8947, 2, 7, 0.375, 7, 0.4117647058823529 ] + - [ "NL", 1442052000000, 53, -10, 1442044800000, 1442066400000, -84, 8947, 2, 8, 0.4375, 8, 0.47058823529411764 ] + - [ "NL", 1442066400000, 61, 51, 1442091600000, 1442095200000, -84, 8947, 2, 9, 0.5, 9, 0.5294117647058824 ] + - [ "NL", 1442095200000, 70, 121, 1442052000000, 1442055600000, -84, 8947, 2, 10, 0.5625, 10, 0.5882352941176471 ] + - [ "NL", 1442055600000, 105, 226, 1442066400000, 1442073600000, -84, 8947, 2, 11, 0.625, 11, 0.6470588235294118 ] + - [ "NL", 1442073600000, 166, 392, 1442095200000, 1442059200000, -84, 8947, 2, 12, 0.6875, 12, 0.7058823529411765 ] + - [ "NL", 1442059200000, 206, 598, 1442055600000, 1442084400000, -84, 8947, 3, 13, 0.75, 13, 0.7647058823529411 ] + - [ "NL", 1442084400000, 436, 1034, 1442073600000, 1442077200000, -84, 8947, 3, 14, 0.8125, 14, 0.8235294117647058 ] + - [ "NL", 1442077200000, 878, 1912, 1442059200000, 1442048400000, -84, 8947, 3, 15, 0.875, 15, 0.8823529411764706 ] + - [ "NL", 1442048400000, 1303, 3215, 1442084400000, 1442080800000, -84, 8947, 3, 16, 0.9375, 16, 0.9411764705882353 ] + - [ "NL", 1442080800000, 8947, 12162, 1442077200000, null, -84, 8947, 3, 17, 1.0, 17, 1.0 ] + - [ "NO", 1442048400000, -447, -447, null, 1442095200000, -447, 447, 1, 1, 0.0, 1, 0.09090909090909091 ] + - [ "NO", 1442095200000, -1, -448, null, 1442098800000, -447, 447, 1, 2, 0.1, 2, 0.18181818181818182 ] + - [ "NO", 1442098800000, 2, -446, 1442048400000, 1442088000000, -447, 447, 1, 3, 0.2, 3, 0.2727272727272727 ] + - [ "NO", 1442088000000, 15, -431, 1442095200000, 1442091600000, -447, 447, 1, 4, 0.3, 4, 0.45454545454545453 ] + - [ "NO", 1442091600000, 15, -416, 1442098800000, 1442055600000, -447, 447, 2, 4, 0.3, 4, 0.45454545454545453 ] + - [ "NO", 1442055600000, 29, -387, 1442088000000, 1442080800000, -447, 447, 2, 6, 0.5, 5, 0.5454545454545454 ] + - [ "NO", 1442080800000, 31, -356, 1442091600000, 1442019600000, -447, 447, 2, 7, 0.6, 6, 0.6363636363636364 ] + - [ "NO", 1442019600000, 48, -308, 1442055600000, 1442066400000, -447, 447, 2, 8, 0.7, 7, 0.7272727272727273 ] + - [ "NO", 1442066400000, 71, -237, 1442080800000, 1442073600000, -447, 447, 3, 9, 0.8, 8, 0.8181818181818182 ] + - [ "NO", 1442073600000, 222, -15, 1442019600000, 1442052000000, -447, 447, 3, 10, 0.9, 9, 0.9090909090909091 ] + - [ "NO", 1442052000000, 447, 432, 1442066400000, null, -447, 447, 3, 11, 1.0, 10, 1.0 ] + - [ "NP", 1442048400000, 61, 61, null, null, 61, 61, 1, 1, 0.0, 1, 1.0 ] + - [ "NZ", 1442084400000, -52, -52, null, 1442095200000, -52, 635, 1, 1, 0.0, 1, 0.1111111111111111 ] + - [ "NZ", 1442095200000, -4, -56, null, 1442098800000, -52, 635, 1, 2, 0.125, 2, 0.2222222222222222 ] + - [ "NZ", 1442098800000, -2, -58, 1442084400000, 1442019600000, -52, 635, 1, 3, 0.25, 3, 0.3333333333333333 ] + - [ "NZ", 1442019600000, 28, -30, 1442095200000, 1442037600000, -52, 635, 2, 4, 0.375, 4, 0.4444444444444444 ] + - [ "NZ", 1442037600000, 66, 36, 1442098800000, 1442048400000, -52, 635, 2, 5, 0.5, 5, 0.5555555555555556 ] + - [ "NZ", 1442048400000, 189, 225, 1442019600000, 1442088000000, -52, 635, 2, 6, 0.625, 6, 0.6666666666666666 ] + - [ "NZ", 1442088000000, 405, 630, 1442037600000, 1442059200000, -52, 635, 3, 7, 0.75, 7, 0.7777777777777778 ] + - [ "NZ", 1442059200000, 428, 1058, 1442048400000, 1442026800000, -52, 635, 3, 8, 0.875, 8, 0.8888888888888888 ] + - [ "NZ", 1442026800000, 635, 1693, 1442088000000, null, -52, 635, 3, 9, 1.0, 9, 1.0 ] + - [ "OM", 1442052000000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0 ] + - [ "PA", 1442026800000, 0, 0, null, null, 0, 0, 1, 1, 0.0, 1, 1.0 ] + - [ "PE", 1442077200000, -163, -163, null, 1442084400000, -163, 1861, 1, 1, 0.0, 1, 0.1111111111111111 ] + - [ "PE", 1442084400000, -68, -231, null, 1442095200000, -163, 1861, 1, 2, 0.125, 2, 0.2222222222222222 ] + - [ "PE", 1442095200000, -19, -250, 1442077200000, 1442026800000, -163, 1861, 1, 3, 0.25, 3, 0.3333333333333333 ] + - [ "PE", 1442026800000, -12, -262, 1442084400000, 1442062800000, -163, 1861, 2, 4, 0.375, 4, 0.5555555555555556 ] + - [ "PE", 1442062800000, -12, -274, 1442095200000, 1442080800000, -163, 1861, 2, 4, 0.375, 4, 0.5555555555555556 ] + - [ "PE", 1442080800000, -2, -276, 1442026800000, 1442023200000, -163, 1861, 2, 6, 0.625, 5, 0.6666666666666666 ] + - [ "PE", 1442023200000, 26, -250, 1442062800000, 1442019600000, -163, 1861, 3, 7, 0.75, 6, 0.7777777777777778 ] + - [ "PE", 1442019600000, 523, 273, 1442080800000, 1442098800000, -163, 1861, 3, 8, 0.875, 7, 0.8888888888888888 ] + - [ "PE", 1442098800000, 1861, 2134, 1442023200000, null, -163, 1861, 3, 9, 1.0, 8, 1.0 ] + - [ "PH", 1442073600000, -227, -227, null, 1442041200000, -227, 1969, 1, 1, 0.0, 1, 0.047619047619047616 ] + - [ "PH", 1442041200000, 0, -227, null, 1442077200000, -227, 1969, 1, 2, 0.05, 2, 0.09523809523809523 ] + - [ "PH", 1442077200000, 2, -225, 1442073600000, 1442019600000, -227, 1969, 1, 3, 0.1, 3, 0.14285714285714285 ] + - [ "PH", 1442019600000, 6, -219, 1442041200000, 1442098800000, -227, 1969, 1, 4, 0.15, 4, 0.19047619047619047 ] + - [ "PH", 1442098800000, 8, -211, 1442077200000, 1442037600000, -227, 1969, 1, 5, 0.2, 5, 0.23809523809523808 ] + - [ "PH", 1442037600000, 17, -194, 1442019600000, 1442052000000, -227, 1969, 1, 6, 0.25, 6, 0.2857142857142857 ] + - [ "PH", 1442052000000, 22, -172, 1442098800000, 1442030400000, -227, 1969, 1, 7, 0.3, 7, 0.3333333333333333 ] + - [ "PH", 1442030400000, 26, -146, 1442037600000, 1442080800000, -227, 1969, 2, 8, 0.35, 8, 0.38095238095238093 ] + - [ "PH", 1442080800000, 32, -114, 1442052000000, 1442070000000, -227, 1969, 2, 9, 0.4, 9, 0.42857142857142855 ] + - [ "PH", 1442070000000, 34, -80, 1442030400000, 1442084400000, -227, 1969, 2, 10, 0.45, 10, 0.47619047619047616 ] + - [ "PH", 1442084400000, 39, -41, 1442080800000, 1442044800000, -227, 1969, 2, 11, 0.5, 11, 0.5238095238095238 ] + - [ "PH", 1442044800000, 55, 14, 1442070000000, 1442034000000, -227, 1969, 2, 12, 0.55, 12, 0.5714285714285714 ] + - [ "PH", 1442034000000, 59, 73, 1442084400000, 1442048400000, -227, 1969, 2, 13, 0.6, 13, 0.6190476190476191 ] + - [ "PH", 1442048400000, 62, 135, 1442044800000, 1442062800000, -227, 1969, 2, 14, 0.65, 14, 0.6666666666666666 ] + - [ "PH", 1442062800000, 171, 306, 1442034000000, 1442059200000, -227, 1969, 3, 15, 0.7, 15, 0.7142857142857143 ] + - [ "PH", 1442059200000, 273, 579, 1442048400000, 1442023200000, -227, 1969, 3, 16, 0.75, 16, 0.7619047619047619 ] + - [ "PH", 1442023200000, 459, 1038, 1442062800000, 1442091600000, -227, 1969, 3, 17, 0.8, 17, 0.8095238095238095 ] + - [ "PH", 1442091600000, 816, 1854, 1442059200000, 1442026800000, -227, 1969, 3, 18, 0.85, 18, 0.8571428571428571 ] + - [ "PH", 1442026800000, 910, 2764, 1442023200000, 1442066400000, -227, 1969, 3, 19, 0.9, 19, 0.9047619047619048 ] + - [ "PH", 1442066400000, 1880, 4644, 1442091600000, 1442055600000, -227, 1969, 3, 20, 0.95, 20, 0.9523809523809523 ] + - [ "PH", 1442055600000, 1969, 6613, 1442026800000, null, -227, 1969, 3, 21, 1.0, 21, 1.0 ] + - [ "PK", 1442048400000, 15, 15, null, 1442062800000, 15, 335, 1, 1, 0.0, 1, 0.14285714285714285 ] + - [ "PK", 1442062800000, 23, 38, null, 1442041200000, 15, 335, 1, 2, 0.16666666666666666, 2, 0.2857142857142857 ] + - [ "PK", 1442041200000, 24, 62, 1442048400000, 1442070000000, 15, 335, 1, 3, 0.3333333333333333, 3, 0.42857142857142855 ] + - [ "PK", 1442070000000, 43, 105, 1442062800000, 1442037600000, 15, 335, 2, 4, 0.5, 4, 0.5714285714285714 ] + - [ "PK", 1442037600000, 100, 205, 1442041200000, 1442026800000, 15, 335, 2, 5, 0.6666666666666666, 5, 0.7142857142857143 ] + - [ "PK", 1442026800000, 101, 306, 1442070000000, 1442019600000, 15, 335, 3, 6, 0.8333333333333334, 6, 0.8571428571428571 ] + - [ "PK", 1442019600000, 335, 641, 1442037600000, null, 15, 335, 3, 7, 1.0, 7, 1.0 ] + - [ "PL", 1442098800000, -9, -9, null, 1442080800000, -9, 4171, 1, 1, 0.0, 1, 0.05555555555555555 ] + - [ "PL", 1442080800000, 7, -2, null, 1442084400000, -9, 4171, 1, 2, 0.058823529411764705, 2, 0.1111111111111111 ] + - [ "PL", 1442084400000, 13, 11, 1442098800000, 1442073600000, -9, 4171, 1, 3, 0.11764705882352941, 3, 0.16666666666666666 ] + - [ "PL", 1442073600000, 30, 41, 1442080800000, 1442066400000, -9, 4171, 1, 4, 0.17647058823529413, 4, 0.2222222222222222 ] + - [ "PL", 1442066400000, 34, 75, 1442084400000, 1442037600000, -9, 4171, 1, 5, 0.23529411764705882, 5, 0.2777777777777778 ] + - [ "PL", 1442037600000, 95, 170, 1442073600000, 1442070000000, -9, 4171, 1, 6, 0.29411764705882354, 6, 0.3333333333333333 ] + - [ "PL", 1442070000000, 146, 316, 1442066400000, 1442059200000, -9, 4171, 2, 7, 0.35294117647058826, 7, 0.3888888888888889 ] + - [ "PL", 1442059200000, 199, 515, 1442037600000, 1442041200000, -9, 4171, 2, 8, 0.4117647058823529, 8, 0.4444444444444444 ] + - [ "PL", 1442041200000, 281, 796, 1442070000000, 1442044800000, -9, 4171, 2, 9, 0.47058823529411764, 9, 0.5 ] + - [ "PL", 1442044800000, 319, 1115, 1442059200000, 1442077200000, -9, 4171, 2, 10, 0.5294117647058824, 10, 0.5555555555555556 ] + - [ "PL", 1442077200000, 324, 1439, 1442041200000, 1442052000000, -9, 4171, 2, 11, 0.5882352941176471, 11, 0.6111111111111112 ] + - [ "PL", 1442052000000, 330, 1769, 1442044800000, 1442088000000, -9, 4171, 2, 12, 0.6470588235294118, 12, 0.6666666666666666 ] + - [ "PL", 1442088000000, 346, 2115, 1442077200000, 1442048400000, -9, 4171, 3, 13, 0.7058823529411765, 13, 0.7222222222222222 ] + - [ "PL", 1442048400000, 366, 2481, 1442052000000, 1442055600000, -9, 4171, 3, 14, 0.7647058823529411, 14, 0.7777777777777778 ] + - [ "PL", 1442055600000, 410, 2891, 1442088000000, 1442091600000, -9, 4171, 3, 15, 0.8235294117647058, 15, 0.8333333333333334 ] + - [ "PL", 1442091600000, 902, 3793, 1442048400000, 1442095200000, -9, 4171, 3, 16, 0.8823529411764706, 16, 0.8888888888888888 ] + - [ "PL", 1442095200000, 1851, 5644, 1442055600000, 1442062800000, -9, 4171, 3, 17, 0.9411764705882353, 17, 0.9444444444444444 ] + - [ "PL", 1442062800000, 4171, 9815, 1442091600000, null, -9, 4171, 3, 18, 1.0, 18, 1.0 ] + - [ "PR", 1442059200000, -35, -35, null, 1442030400000, -35, 29, 1, 1, 0.0, 1, 0.2 ] + - [ "PR", 1442030400000, 2, -33, null, 1442077200000, -35, 29, 1, 2, 0.25, 2, 0.4 ] + - [ "PR", 1442077200000, 5, -28, 1442059200000, 1442026800000, -35, 29, 2, 3, 0.5, 3, 0.6 ] + - [ "PR", 1442026800000, 22, -6, 1442030400000, 1442095200000, -35, 29, 2, 4, 0.75, 4, 0.8 ] + - [ "PR", 1442095200000, 29, 23, 1442077200000, null, -35, 29, 3, 5, 1.0, 5, 1.0 ] + - [ "PT", 1442080800000, -79, -79, null, 1442077200000, -79, 3470, 1, 1, 0.0, 1, 0.1 ] + - [ "PT", 1442077200000, -75, -154, null, 1442098800000, -79, 3470, 1, 2, 0.1111111111111111, 2, 0.2 ] + - [ "PT", 1442098800000, 2, -152, 1442080800000, 1442044800000, -79, 3470, 1, 3, 0.2222222222222222, 3, 0.3 ] + - [ "PT", 1442044800000, 11, -141, 1442077200000, 1442066400000, -79, 3470, 1, 4, 0.3333333333333333, 4, 0.4 ] + - [ "PT", 1442066400000, 12, -129, 1442098800000, 1442095200000, -79, 3470, 2, 5, 0.4444444444444444, 5, 0.5 ] + - [ "PT", 1442095200000, 19, -110, 1442044800000, 1442052000000, -79, 3470, 2, 6, 0.5555555555555556, 6, 0.6 ] + - [ "PT", 1442052000000, 102, -8, 1442066400000, 1442019600000, -79, 3470, 2, 7, 0.6666666666666666, 7, 0.7 ] + - [ "PT", 1442019600000, 172, 164, 1442095200000, 1442088000000, -79, 3470, 3, 8, 0.7777777777777778, 8, 0.8 ] + - [ "PT", 1442088000000, 403, 567, 1442052000000, 1442070000000, -79, 3470, 3, 9, 0.8888888888888888, 9, 0.9 ] + - [ "PT", 1442070000000, 3470, 4037, 1442019600000, null, -79, 3470, 3, 10, 1.0, 10, 1.0 ] + - [ "PY", 1442019600000, 1, 1, null, 1442080800000, 1, 628, 1, 1, 0.0, 1, 0.3333333333333333 ] + - [ "PY", 1442080800000, 5, 6, null, 1442084400000, 1, 628, 2, 2, 0.5, 2, 0.6666666666666666 ] + - [ "PY", 1442084400000, 628, 634, 1442019600000, null, 1, 628, 3, 3, 1.0, 3, 1.0 ] + - [ "QA", 1442041200000, 13, 13, null, null, 13, 13, 1, 1, 0.0, 1, 1.0 ] + - [ "RO", 1442070000000, -29, -29, null, 1442091600000, -29, 845, 1, 1, 0.0, 1, 0.1 ] + - [ "RO", 1442091600000, 0, -29, null, 1442073600000, -29, 845, 1, 2, 0.1111111111111111, 2, 0.2 ] + - [ "RO", 1442073600000, 15, -14, 1442070000000, 1442055600000, -29, 845, 1, 3, 0.2222222222222222, 3, 0.3 ] + - [ "RO", 1442055600000, 26, 12, 1442091600000, 1442034000000, -29, 845, 1, 4, 0.3333333333333333, 4, 0.4 ] + - [ "RO", 1442034000000, 68, 80, 1442073600000, 1442044800000, -29, 845, 2, 5, 0.4444444444444444, 5, 0.5 ] + - [ "RO", 1442044800000, 284, 364, 1442055600000, 1442052000000, -29, 845, 2, 6, 0.5555555555555556, 6, 0.6 ] + - [ "RO", 1442052000000, 319, 683, 1442034000000, 1442062800000, -29, 845, 2, 7, 0.6666666666666666, 7, 0.7 ] + - [ "RO", 1442062800000, 541, 1224, 1442044800000, 1442095200000, -29, 845, 3, 8, 0.7777777777777778, 8, 0.8 ] + - [ "RO", 1442095200000, 824, 2048, 1442052000000, 1442041200000, -29, 845, 3, 9, 0.8888888888888888, 9, 0.9 ] + - [ "RO", 1442041200000, 845, 2893, 1442062800000, null, -29, 845, 3, 10, 1.0, 10, 1.0 ] + - [ "RS", 1442091600000, -15, -15, null, 1442066400000, -15, 813, 1, 1, 0.0, 1, 0.14285714285714285 ] + - [ "RS", 1442066400000, 0, -15, null, 1442080800000, -15, 813, 1, 2, 0.16666666666666666, 2, 0.42857142857142855 ] + - [ "RS", 1442080800000, 0, -15, 1442091600000, 1442019600000, -15, 813, 1, 2, 0.16666666666666666, 2, 0.42857142857142855 ] + - [ "RS", 1442019600000, 6, -9, 1442066400000, 1442062800000, -15, 813, 2, 4, 0.5, 3, 0.5714285714285714 ] + - [ "RS", 1442062800000, 13, 4, 1442080800000, 1442084400000, -15, 813, 2, 5, 0.6666666666666666, 4, 0.7142857142857143 ] + - [ "RS", 1442084400000, 89, 93, 1442019600000, 1442073600000, -15, 813, 3, 6, 0.8333333333333334, 5, 0.8571428571428571 ] + - [ "RS", 1442073600000, 813, 906, 1442062800000, null, -15, 813, 3, 7, 1.0, 6, 1.0 ] + - [ "RU", 1442037600000, -324, -324, null, 1442026800000, -324, 12098, 1, 1, 0.0, 1, 0.043478260869565216 ] + - [ "RU", 1442026800000, 0, -324, null, 1442030400000, -324, 12098, 1, 2, 0.045454545454545456, 2, 0.08695652173913043 ] + - [ "RU", 1442030400000, 76, -248, 1442037600000, 1442062800000, -324, 12098, 1, 3, 0.09090909090909091, 3, 0.13043478260869565 ] + - [ "RU", 1442062800000, 168, -80, 1442026800000, 1442023200000, -324, 12098, 1, 4, 0.13636363636363635, 4, 0.17391304347826086 ] + - [ "RU", 1442023200000, 299, 219, 1442030400000, 1442095200000, -324, 12098, 1, 5, 0.18181818181818182, 5, 0.21739130434782608 ] + - [ "RU", 1442095200000, 435, 654, 1442062800000, 1442055600000, -324, 12098, 1, 6, 0.22727272727272727, 6, 0.2608695652173913 ] + - [ "RU", 1442055600000, 499, 1153, 1442023200000, 1442041200000, -324, 12098, 1, 7, 0.2727272727272727, 7, 0.30434782608695654 ] + - [ "RU", 1442041200000, 580, 1733, 1442095200000, 1442080800000, -324, 12098, 1, 8, 0.3181818181818182, 8, 0.34782608695652173 ] + - [ "RU", 1442080800000, 655, 2388, 1442055600000, 1442034000000, -324, 12098, 2, 9, 0.36363636363636365, 9, 0.391304347826087 ] + - [ "RU", 1442034000000, 658, 3046, 1442041200000, 1442048400000, -324, 12098, 2, 10, 0.4090909090909091, 10, 0.43478260869565216 ] + - [ "RU", 1442048400000, 1027, 4073, 1442080800000, 1442077200000, -324, 12098, 2, 11, 0.45454545454545453, 11, 0.4782608695652174 ] + - [ "RU", 1442077200000, 1162, 5235, 1442034000000, 1442052000000, -324, 12098, 2, 12, 0.5, 12, 0.5217391304347826 ] + - [ "RU", 1442052000000, 1214, 6449, 1442048400000, 1442073600000, -324, 12098, 2, 13, 0.5454545454545454, 13, 0.5652173913043478 ] + - [ "RU", 1442073600000, 1618, 8067, 1442077200000, 1442066400000, -324, 12098, 2, 14, 0.5909090909090909, 14, 0.6086956521739131 ] + - [ "RU", 1442066400000, 2047, 10114, 1442052000000, 1442019600000, -324, 12098, 2, 15, 0.6363636363636364, 15, 0.6521739130434783 ] + - [ "RU", 1442019600000, 2214, 12328, 1442073600000, 1442044800000, -324, 12098, 2, 16, 0.6818181818181818, 16, 0.6956521739130435 ] + - [ "RU", 1442044800000, 2564, 14892, 1442066400000, 1442088000000, -324, 12098, 3, 17, 0.7272727272727273, 17, 0.7391304347826086 ] + - [ "RU", 1442088000000, 2596, 17488, 1442019600000, 1442091600000, -324, 12098, 3, 18, 0.7727272727272727, 18, 0.782608695652174 ] + - [ "RU", 1442091600000, 3449, 20937, 1442044800000, 1442059200000, -324, 12098, 3, 19, 0.8181818181818182, 19, 0.8260869565217391 ] + - [ "RU", 1442059200000, 3902, 24839, 1442088000000, 1442070000000, -324, 12098, 3, 20, 0.8636363636363636, 20, 0.8695652173913043 ] + - [ "RU", 1442070000000, 4706, 29545, 1442091600000, 1442084400000, -324, 12098, 3, 21, 0.9090909090909091, 21, 0.9130434782608695 ] + - [ "RU", 1442084400000, 6461, 36006, 1442059200000, 1442098800000, -324, 12098, 3, 22, 0.9545454545454546, 22, 0.9565217391304348 ] + - [ "RU", 1442098800000, 12098, 48104, 1442070000000, null, -324, 12098, 3, 23, 1.0, 23, 1.0 ] + - [ "SA", 1442037600000, -97, -97, null, 1442077200000, -97, 1276, 1, 1, 0.0, 1, 0.125 ] + - [ "SA", 1442077200000, -50, -147, null, 1442059200000, -97, 1276, 1, 2, 0.14285714285714285, 2, 0.25 ] + - [ "SA", 1442059200000, 0, -147, 1442037600000, 1442073600000, -97, 1276, 1, 3, 0.2857142857142857, 3, 0.375 ] + - [ "SA", 1442073600000, 2, -145, 1442077200000, 1442055600000, -97, 1276, 2, 4, 0.42857142857142855, 4, 0.5 ] + - [ "SA", 1442055600000, 11, -134, 1442059200000, 1442048400000, -97, 1276, 2, 5, 0.5714285714285714, 5, 0.625 ] + - [ "SA", 1442048400000, 14, -120, 1442073600000, 1442084400000, -97, 1276, 2, 6, 0.7142857142857143, 6, 0.75 ] + - [ "SA", 1442084400000, 458, 338, 1442055600000, 1442066400000, -97, 1276, 3, 7, 0.8571428571428571, 7, 0.875 ] + - [ "SA", 1442066400000, 1276, 1614, 1442048400000, null, -97, 1276, 3, 8, 1.0, 8, 1.0 ] + - [ "SE", 1442048400000, -145, -145, null, 1442055600000, -145, 1476, 1, 1, 0.0, 1, 0.06666666666666667 ] + - [ "SE", 1442055600000, -5, -150, null, 1442091600000, -145, 1476, 1, 2, 0.07142857142857142, 2, 0.13333333333333333 ] + - [ "SE", 1442091600000, -1, -151, 1442048400000, 1442098800000, -145, 1476, 1, 3, 0.14285714285714285, 3, 0.2 ] + - [ "SE", 1442098800000, 0, -151, 1442055600000, 1442052000000, -145, 1476, 1, 4, 0.21428571428571427, 4, 0.26666666666666666 ] + - [ "SE", 1442052000000, 1, -150, 1442091600000, 1442023200000, -145, 1476, 1, 5, 0.2857142857142857, 5, 0.3333333333333333 ] + - [ "SE", 1442023200000, 3, -147, 1442098800000, 1442066400000, -145, 1476, 2, 6, 0.35714285714285715, 6, 0.4 ] + - [ "SE", 1442066400000, 14, -133, 1442052000000, 1442030400000, -145, 1476, 2, 7, 0.42857142857142855, 7, 0.4666666666666667 ] + - [ "SE", 1442030400000, 30, -103, 1442023200000, 1442084400000, -145, 1476, 2, 8, 0.5, 8, 0.5333333333333333 ] + - [ "SE", 1442084400000, 37, -66, 1442066400000, 1442095200000, -145, 1476, 2, 9, 0.5714285714285714, 9, 0.6 ] + - [ "SE", 1442095200000, 61, -5, 1442030400000, 1442070000000, -145, 1476, 2, 10, 0.6428571428571429, 10, 0.6666666666666666 ] + - [ "SE", 1442070000000, 78, 73, 1442084400000, 1442080800000, -145, 1476, 3, 11, 0.7142857142857143, 11, 0.7333333333333333 ] + - [ "SE", 1442080800000, 89, 162, 1442095200000, 1442041200000, -145, 1476, 3, 12, 0.7857142857142857, 12, 0.8 ] + - [ "SE", 1442041200000, 91, 253, 1442070000000, 1442019600000, -145, 1476, 3, 13, 0.8571428571428571, 13, 0.8666666666666667 ] + - [ "SE", 1442019600000, 109, 362, 1442080800000, 1442059200000, -145, 1476, 3, 14, 0.9285714285714286, 14, 0.9333333333333333 ] + - [ "SE", 1442059200000, 1476, 1838, 1442041200000, null, -145, 1476, 3, 15, 1.0, 15, 1.0 ] + - [ "SG", 1442066400000, 0, 0, null, 1442030400000, 0, 2758, 1, 1, 0.0, 1, 0.125 ] + - [ "SG", 1442030400000, 1, 1, null, 1442037600000, 0, 2758, 1, 2, 0.14285714285714285, 2, 0.25 ] + - [ "SG", 1442037600000, 3, 4, 1442066400000, 1442048400000, 0, 2758, 1, 3, 0.2857142857142857, 3, 0.375 ] + - [ "SG", 1442048400000, 52, 56, 1442030400000, 1442041200000, 0, 2758, 2, 4, 0.42857142857142855, 4, 0.5 ] + - [ "SG", 1442041200000, 59, 115, 1442037600000, 1442044800000, 0, 2758, 2, 5, 0.5714285714285714, 5, 0.625 ] + - [ "SG", 1442044800000, 77, 192, 1442048400000, 1442062800000, 0, 2758, 2, 6, 0.7142857142857143, 6, 0.75 ] + - [ "SG", 1442062800000, 388, 580, 1442041200000, 1442026800000, 0, 2758, 3, 7, 0.8571428571428571, 7, 0.875 ] + - [ "SG", 1442026800000, 2758, 3338, 1442044800000, null, 0, 2758, 3, 8, 1.0, 8, 1.0 ] + - [ "SI", 1442080800000, -45, -45, null, 1442091600000, -45, 9, 1, 1, 0.0, 1, 0.5 ] + - [ "SI", 1442091600000, 9, -36, null, null, -45, 9, 2, 2, 1.0, 2, 1.0 ] + - [ "SK", 1442084400000, -92, -92, null, 1442037600000, -92, 446, 1, 1, 0.0, 1, 0.16666666666666666 ] + - [ "SK", 1442037600000, -1, -93, null, 1442062800000, -92, 446, 1, 2, 0.2, 2, 0.3333333333333333 ] + - [ "SK", 1442062800000, 6, -87, 1442084400000, 1442098800000, -92, 446, 2, 3, 0.4, 3, 0.5 ] + - [ "SK", 1442098800000, 7, -80, 1442037600000, 1442052000000, -92, 446, 2, 4, 0.6, 4, 0.6666666666666666 ] + - [ "SK", 1442052000000, 13, -67, 1442062800000, 1442073600000, -92, 446, 3, 5, 0.8, 5, 0.8333333333333334 ] + - [ "SK", 1442073600000, 446, 379, 1442098800000, null, -92, 446, 3, 6, 1.0, 6, 1.0 ] + - [ "SV", 1442019600000, -1, -1, null, 1442088000000, -1, 106, 1, 1, 0.0, 1, 0.3333333333333333 ] + - [ "SV", 1442088000000, 9, 8, null, 1442084400000, -1, 106, 2, 2, 0.5, 2, 0.6666666666666666 ] + - [ "SV", 1442084400000, 106, 114, 1442019600000, null, -1, 106, 3, 3, 1.0, 3, 1.0 ] + - [ "TH", 1442062800000, -46, -46, null, 1442066400000, -46, 110, 1, 1, 0.0, 1, 0.1111111111111111 ] + - [ "TH", 1442066400000, -34, -80, null, 1442052000000, -46, 110, 1, 2, 0.125, 2, 0.2222222222222222 ] + - [ "TH", 1442052000000, -22, -102, 1442062800000, 1442034000000, -46, 110, 1, 3, 0.25, 3, 0.3333333333333333 ] + - [ "TH", 1442034000000, 0, -102, 1442066400000, 1442055600000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666 ] + - [ "TH", 1442055600000, 0, -102, 1442052000000, 1442070000000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666 ] + - [ "TH", 1442070000000, 0, -102, 1442034000000, 1442041200000, -46, 110, 2, 4, 0.375, 4, 0.6666666666666666 ] + - [ "TH", 1442041200000, 3, -99, 1442055600000, 1442084400000, -46, 110, 3, 7, 0.75, 5, 0.7777777777777778 ] + - [ "TH", 1442084400000, 13, -86, 1442070000000, 1442044800000, -46, 110, 3, 8, 0.875, 6, 0.8888888888888888 ] + - [ "TH", 1442044800000, 110, 24, 1442041200000, null, -46, 110, 3, 9, 1.0, 7, 1.0 ] + - [ "TJ", 1442048400000, 1471, 1471, null, null, 1471, 1471, 1, 1, 0.0, 1, 1.0 ] + - [ "TN", 1442098800000, -9, -9, null, null, -9, -9, 1, 1, 0.0, 1, 1.0 ] + - [ "TR", 1442095200000, -29, -29, null, 1442080800000, -29, 3048, 1, 1, 0.0, 1, 0.06666666666666667 ] + - [ "TR", 1442080800000, -1, -30, null, 1442041200000, -29, 3048, 1, 2, 0.07142857142857142, 2, 0.13333333333333333 ] + - [ "TR", 1442041200000, 1, -29, 1442095200000, 1442044800000, -29, 3048, 1, 3, 0.14285714285714285, 3, 0.2 ] + - [ "TR", 1442044800000, 41, 12, 1442080800000, 1442052000000, -29, 3048, 1, 4, 0.21428571428571427, 4, 0.3333333333333333 ] + - [ "TR", 1442052000000, 41, 53, 1442041200000, 1442066400000, -29, 3048, 1, 4, 0.21428571428571427, 4, 0.3333333333333333 ] + - [ "TR", 1442066400000, 85, 138, 1442044800000, 1442048400000, -29, 3048, 2, 6, 0.35714285714285715, 5, 0.4 ] + - [ "TR", 1442048400000, 88, 226, 1442052000000, 1442077200000, -29, 3048, 2, 7, 0.42857142857142855, 6, 0.4666666666666667 ] + - [ "TR", 1442077200000, 89, 315, 1442066400000, 1442084400000, -29, 3048, 2, 8, 0.5, 7, 0.5333333333333333 ] + - [ "TR", 1442084400000, 170, 485, 1442048400000, 1442070000000, -29, 3048, 2, 9, 0.5714285714285714, 8, 0.6 ] + - [ "TR", 1442070000000, 236, 721, 1442077200000, 1442055600000, -29, 3048, 2, 10, 0.6428571428571429, 9, 0.6666666666666666 ] + - [ "TR", 1442055600000, 299, 1020, 1442084400000, 1442023200000, -29, 3048, 3, 11, 0.7142857142857143, 10, 0.7333333333333333 ] + - [ "TR", 1442023200000, 306, 1326, 1442070000000, 1442062800000, -29, 3048, 3, 12, 0.7857142857142857, 11, 0.8 ] + - [ "TR", 1442062800000, 315, 1641, 1442055600000, 1442088000000, -29, 3048, 3, 13, 0.8571428571428571, 12, 0.8666666666666667 ] + - [ "TR", 1442088000000, 2389, 4030, 1442023200000, 1442091600000, -29, 3048, 3, 14, 0.9285714285714286, 13, 0.9333333333333333 ] + - [ "TR", 1442091600000, 3048, 7078, 1442062800000, null, -29, 3048, 3, 15, 1.0, 14, 1.0 ] + - [ "TT", 1442088000000, 9, 9, null, null, 9, 9, 1, 1, 0.0, 1, 1.0 ] + - [ "TW", 1442062800000, -272, -272, null, 1442059200000, -272, 772, 1, 1, 0.0, 1, 0.045454545454545456 ] + - [ "TW", 1442059200000, -157, -429, null, 1442095200000, -272, 772, 1, 2, 0.047619047619047616, 2, 0.09090909090909091 ] + - [ "TW", 1442095200000, -77, -506, 1442062800000, 1442098800000, -272, 772, 1, 3, 0.09523809523809523, 3, 0.13636363636363635 ] + - [ "TW", 1442098800000, -60, -566, 1442059200000, 1442019600000, -272, 772, 1, 4, 0.14285714285714285, 4, 0.18181818181818182 ] + - [ "TW", 1442019600000, 0, -566, 1442095200000, 1442030400000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182 ] + - [ "TW", 1442030400000, 0, -566, 1442098800000, 1442084400000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182 ] + - [ "TW", 1442084400000, 0, -566, 1442019600000, 1442044800000, -272, 772, 1, 5, 0.19047619047619047, 5, 0.3181818181818182 ] + - [ "TW", 1442044800000, 24, -542, 1442030400000, 1442052000000, -272, 772, 1, 8, 0.3333333333333333, 6, 0.45454545454545453 ] + - [ "TW", 1442052000000, 24, -518, 1442084400000, 1442080800000, -272, 772, 2, 8, 0.3333333333333333, 6, 0.45454545454545453 ] + - [ "TW", 1442080800000, 24, -494, 1442044800000, 1442055600000, -272, 772, 2, 8, 0.3333333333333333, 6, 0.45454545454545453 ] + - [ "TW", 1442055600000, 48, -446, 1442052000000, 1442048400000, -272, 772, 2, 11, 0.47619047619047616, 7, 0.5 ] + - [ "TW", 1442048400000, 75, -371, 1442080800000, 1442016000000, -272, 772, 2, 12, 0.5238095238095238, 8, 0.5454545454545454 ] + - [ "TW", 1442016000000, 92, -279, 1442055600000, 1442023200000, -272, 772, 2, 13, 0.5714285714285714, 9, 0.5909090909090909 ] + - [ "TW", 1442023200000, 97, -182, 1442048400000, 1442034000000, -272, 772, 2, 14, 0.6190476190476191, 10, 0.6363636363636364 ] + - [ "TW", 1442034000000, 143, -39, 1442016000000, 1442037600000, -272, 772, 2, 15, 0.6666666666666666, 11, 0.6818181818181818 ] + - [ "TW", 1442037600000, 266, 227, 1442023200000, 1442041200000, -272, 772, 3, 16, 0.7142857142857143, 12, 0.7272727272727273 ] + - [ "TW", 1442041200000, 366, 593, 1442034000000, 1442070000000, -272, 772, 3, 17, 0.7619047619047619, 13, 0.7727272727272727 ] + - [ "TW", 1442070000000, 485, 1078, 1442037600000, 1442077200000, -272, 772, 3, 18, 0.8095238095238095, 14, 0.8181818181818182 ] + - [ "TW", 1442077200000, 502, 1580, 1442041200000, 1442066400000, -272, 772, 3, 19, 0.8571428571428571, 15, 0.8636363636363636 ] + - [ "TW", 1442066400000, 624, 2204, 1442070000000, 1442026800000, -272, 772, 3, 20, 0.9047619047619048, 16, 0.9090909090909091 ] + - [ "TW", 1442026800000, 680, 2884, 1442077200000, 1442073600000, -272, 772, 3, 21, 0.9523809523809523, 17, 0.9545454545454546 ] + - [ "TW", 1442073600000, 772, 3656, 1442066400000, null, -272, 772, 3, 22, 1.0, 18, 1.0 ] + - [ "UA", 1442091600000, -388, -388, null, 1442077200000, -388, 14202, 1, 1, 0.0, 1, 0.05263157894736842 ] + - [ "UA", 1442077200000, -181, -569, null, 1442095200000, -388, 14202, 1, 2, 0.05555555555555555, 2, 0.10526315789473684 ] + - [ "UA", 1442095200000, -30, -599, 1442091600000, 1442088000000, -388, 14202, 1, 3, 0.1111111111111111, 3, 0.15789473684210525 ] + - [ "UA", 1442088000000, -21, -620, 1442077200000, 1442059200000, -388, 14202, 1, 4, 0.16666666666666666, 4, 0.21052631578947367 ] + - [ "UA", 1442059200000, -2, -622, 1442095200000, 1442037600000, -388, 14202, 1, 5, 0.2222222222222222, 5, 0.2631578947368421 ] + - [ "UA", 1442037600000, -1, -623, 1442088000000, 1442080800000, -388, 14202, 1, 6, 0.2777777777777778, 6, 0.3684210526315789 ] + - [ "UA", 1442080800000, -1, -624, 1442059200000, 1442048400000, -388, 14202, 1, 6, 0.2777777777777778, 6, 0.3684210526315789 ] + - [ "UA", 1442048400000, 2, -622, 1442037600000, 1442084400000, -388, 14202, 2, 8, 0.3888888888888889, 7, 0.42105263157894735 ] + - [ "UA", 1442084400000, 5, -617, 1442080800000, 1442098800000, -388, 14202, 2, 9, 0.4444444444444444, 8, 0.47368421052631576 ] + - [ "UA", 1442098800000, 38, -579, 1442048400000, 1442041200000, -388, 14202, 2, 10, 0.5, 9, 0.5263157894736842 ] + - [ "UA", 1442041200000, 74, -505, 1442084400000, 1442044800000, -388, 14202, 2, 11, 0.5555555555555556, 10, 0.5789473684210527 ] + - [ "UA", 1442044800000, 280, -225, 1442098800000, 1442066400000, -388, 14202, 2, 12, 0.6111111111111112, 11, 0.631578947368421 ] + - [ "UA", 1442066400000, 296, 71, 1442041200000, 1442052000000, -388, 14202, 2, 13, 0.6666666666666666, 12, 0.6842105263157895 ] + - [ "UA", 1442052000000, 410, 481, 1442044800000, 1442062800000, -388, 14202, 3, 14, 0.7222222222222222, 13, 0.7368421052631579 ] + - [ "UA", 1442062800000, 773, 1254, 1442066400000, 1442070000000, -388, 14202, 3, 15, 0.7777777777777778, 14, 0.7894736842105263 ] + - [ "UA", 1442070000000, 1733, 2987, 1442052000000, 1442034000000, -388, 14202, 3, 16, 0.8333333333333334, 15, 0.8421052631578947 ] + - [ "UA", 1442034000000, 3468, 6455, 1442062800000, 1442073600000, -388, 14202, 3, 17, 0.8888888888888888, 16, 0.8947368421052632 ] + - [ "UA", 1442073600000, 4241, 10696, 1442070000000, 1442055600000, -388, 14202, 3, 18, 0.9444444444444444, 17, 0.9473684210526315 ] + - [ "UA", 1442055600000, 14202, 24898, 1442034000000, null, -388, 14202, 3, 19, 1.0, 18, 1.0 ] + - [ "UG", 1442070000000, 1, 1, null, null, 1, 1, 1, 1, 0.0, 1, 1.0 ] + - [ "US", 1442048400000, -466, -466, null, 1442052000000, -466, 4001, 1, 1, 0.0, 1, 0.041666666666666664 ] + - [ "US", 1442052000000, -2, -468, null, 1442016000000, -466, 4001, 1, 2, 0.043478260869565216, 2, 0.08333333333333333 ] + - [ "US", 1442016000000, 0, -468, 1442048400000, 1442059200000, -466, 4001, 1, 3, 0.08695652173913043, 3, 0.125 ] + - [ "US", 1442059200000, 11, -457, 1442052000000, 1442062800000, -466, 4001, 1, 4, 0.13043478260869565, 4, 0.16666666666666666 ] + - [ "US", 1442062800000, 47, -410, 1442016000000, 1442044800000, -466, 4001, 1, 5, 0.17391304347826086, 5, 0.20833333333333334 ] + - [ "US", 1442044800000, 139, -271, 1442059200000, 1442055600000, -466, 4001, 1, 6, 0.21739130434782608, 6, 0.25 ] + - [ "US", 1442055600000, 156, -115, 1442062800000, 1442095200000, -466, 4001, 1, 7, 0.2608695652173913, 7, 0.2916666666666667 ] + - [ "US", 1442095200000, 416, 301, 1442044800000, 1442066400000, -466, 4001, 1, 8, 0.30434782608695654, 8, 0.3333333333333333 ] + - [ "US", 1442066400000, 772, 1073, 1442055600000, 1442019600000, -466, 4001, 2, 9, 0.34782608695652173, 9, 0.375 ] + - [ "US", 1442019600000, 1043, 2116, 1442095200000, 1442073600000, -466, 4001, 2, 10, 0.391304347826087, 10, 0.4166666666666667 ] + - [ "US", 1442073600000, 1100, 3216, 1442066400000, 1442026800000, -466, 4001, 2, 11, 0.43478260869565216, 11, 0.4583333333333333 ] + - [ "US", 1442026800000, 1512, 4728, 1442019600000, 1442088000000, -466, 4001, 2, 12, 0.4782608695652174, 12, 0.5 ] + - [ "US", 1442088000000, 1691, 6419, 1442073600000, 1442041200000, -466, 4001, 2, 13, 0.5217391304347826, 13, 0.5416666666666666 ] + - [ "US", 1442041200000, 1999, 8418, 1442026800000, 1442030400000, -466, 4001, 2, 14, 0.5652173913043478, 14, 0.5833333333333334 ] + - [ "US", 1442030400000, 2023, 10441, 1442088000000, 1442077200000, -466, 4001, 2, 15, 0.6086956521739131, 15, 0.625 ] + - [ "US", 1442077200000, 2168, 12609, 1442041200000, 1442091600000, -466, 4001, 2, 16, 0.6521739130434783, 16, 0.6666666666666666 ] + - [ "US", 1442091600000, 2502, 15111, 1442030400000, 1442084400000, -466, 4001, 3, 17, 0.6956521739130435, 17, 0.7083333333333334 ] + - [ "US", 1442084400000, 2523, 17634, 1442077200000, 1442023200000, -466, 4001, 3, 18, 0.7391304347826086, 18, 0.75 ] + - [ "US", 1442023200000, 2844, 20478, 1442091600000, 1442070000000, -466, 4001, 3, 19, 0.782608695652174, 19, 0.7916666666666666 ] + - [ "US", 1442070000000, 3505, 23983, 1442084400000, 1442098800000, -466, 4001, 3, 20, 0.8260869565217391, 20, 0.8333333333333334 ] + - [ "US", 1442098800000, 3575, 27558, 1442023200000, 1442034000000, -466, 4001, 3, 21, 0.8695652173913043, 21, 0.875 ] + - [ "US", 1442034000000, 3648, 31206, 1442070000000, 1442037600000, -466, 4001, 3, 22, 0.9130434782608695, 22, 0.9166666666666666 ] + - [ "US", 1442037600000, 3675, 34881, 1442098800000, 1442080800000, -466, 4001, 3, 23, 0.9565217391304348, 23, 0.9583333333333334 ] + - [ "US", 1442080800000, 4001, 38882, 1442034000000, null, -466, 4001, 3, 24, 1.0, 24, 1.0 ] + - [ "UY", 1442073600000, -42, -42, null, 1442037600000, -42, 517, 1, 1, 0.0, 1, 0.14285714285714285 ] + - [ "UY", 1442037600000, 1, -41, null, 1442077200000, -42, 517, 1, 2, 0.16666666666666666, 2, 0.2857142857142857 ] + - [ "UY", 1442077200000, 23, -18, 1442073600000, 1442026800000, -42, 517, 1, 3, 0.3333333333333333, 3, 0.42857142857142855 ] + - [ "UY", 1442026800000, 76, 58, 1442037600000, 1442019600000, -42, 517, 2, 4, 0.5, 4, 0.5714285714285714 ] + - [ "UY", 1442019600000, 77, 135, 1442077200000, 1442070000000, -42, 517, 2, 5, 0.6666666666666666, 5, 0.7142857142857143 ] + - [ "UY", 1442070000000, 284, 419, 1442026800000, 1442023200000, -42, 517, 3, 6, 0.8333333333333334, 6, 0.8571428571428571 ] + - [ "UY", 1442023200000, 517, 936, 1442019600000, null, -42, 517, 3, 7, 1.0, 7, 1.0 ] + - [ "UZ", 1442044800000, 1369, 1369, null, null, 1369, 1369, 1, 1, 0.0, 1, 1.0 ] + - [ "VE", 1442026800000, -17, -17, null, 1442034000000, -17, 420, 1, 1, 0.0, 1, 0.1 ] + - [ "VE", 1442034000000, -2, -19, null, 1442098800000, -17, 420, 1, 2, 0.1111111111111111, 2, 0.2 ] + - [ "VE", 1442098800000, 9, -10, 1442026800000, 1442066400000, -17, 420, 1, 3, 0.2222222222222222, 3, 0.3 ] + - [ "VE", 1442066400000, 18, 8, 1442034000000, 1442095200000, -17, 420, 1, 4, 0.3333333333333333, 4, 0.4 ] + - [ "VE", 1442095200000, 35, 43, 1442098800000, 1442030400000, -17, 420, 2, 5, 0.4444444444444444, 5, 0.5 ] + - [ "VE", 1442030400000, 51, 94, 1442066400000, 1442084400000, -17, 420, 2, 6, 0.5555555555555556, 6, 0.6 ] + - [ "VE", 1442084400000, 60, 154, 1442095200000, 1442023200000, -17, 420, 2, 7, 0.6666666666666666, 7, 0.7 ] + - [ "VE", 1442023200000, 115, 269, 1442030400000, 1442077200000, -17, 420, 3, 8, 0.7777777777777778, 8, 0.8 ] + - [ "VE", 1442077200000, 412, 681, 1442084400000, 1442070000000, -17, 420, 3, 9, 0.8888888888888888, 9, 0.9 ] + - [ "VE", 1442070000000, 420, 1101, 1442023200000, null, -17, 420, 3, 10, 1.0, 10, 1.0 ] + - [ "VG", 1442062800000, -238, -238, null, null, -238, -238, 1, 1, 0.0, 1, 1.0 ] + - [ "VN", 1442034000000, -29, -29, null, 1442048400000, -29, 811, 1, 1, 0.0, 1, 0.07692307692307693 ] + - [ "VN", 1442048400000, -15, -44, null, 1442037600000, -29, 811, 1, 2, 0.08333333333333333, 2, 0.15384615384615385 ] + - [ "VN", 1442037600000, -11, -55, 1442034000000, 1442084400000, -29, 811, 1, 3, 0.16666666666666666, 3, 0.23076923076923078 ] + - [ "VN", 1442084400000, -10, -65, 1442048400000, 1442023200000, -29, 811, 1, 4, 0.25, 4, 0.3076923076923077 ] + - [ "VN", 1442023200000, -9, -74, 1442037600000, 1442041200000, -29, 811, 1, 5, 0.3333333333333333, 5, 0.38461538461538464 ] + - [ "VN", 1442041200000, 0, -74, 1442084400000, 1442059200000, -29, 811, 2, 6, 0.4166666666666667, 6, 0.46153846153846156 ] + - [ "VN", 1442059200000, 8, -66, 1442023200000, 1442055600000, -29, 811, 2, 7, 0.5, 7, 0.5384615384615384 ] + - [ "VN", 1442055600000, 37, -29, 1442041200000, 1442026800000, -29, 811, 2, 8, 0.5833333333333334, 8, 0.6153846153846154 ] + - [ "VN", 1442026800000, 63, 34, 1442059200000, 1442052000000, -29, 811, 2, 9, 0.6666666666666666, 9, 0.6923076923076923 ] + - [ "VN", 1442052000000, 90, 124, 1442055600000, 1442062800000, -29, 811, 3, 10, 0.75, 10, 0.7692307692307693 ] + - [ "VN", 1442062800000, 146, 270, 1442026800000, 1442070000000, -29, 811, 3, 11, 0.8333333333333334, 11, 0.8461538461538461 ] + - [ "VN", 1442070000000, 479, 749, 1442052000000, 1442066400000, -29, 811, 3, 12, 0.9166666666666666, 12, 0.9230769230769231 ] + - [ "VN", 1442066400000, 811, 1560, 1442062800000, null, -29, 811, 3, 13, 1.0, 13, 1.0 ] + - [ "ZA", 1442034000000, -3, -3, null, 1442070000000, -3, 79, 1, 1, 0.0, 1, 0.2 ] + - [ "ZA", 1442070000000, 0, -3, null, 1442091600000, -3, 79, 1, 2, 0.25, 2, 0.4 ] + - [ "ZA", 1442091600000, 1, -2, 1442034000000, 1442059200000, -3, 79, 2, 3, 0.5, 3, 0.6 ] + - [ "ZA", 1442059200000, 50, 48, 1442070000000, 1442048400000, -3, 79, 2, 4, 0.75, 4, 0.8 ] + - [ "ZA", 1442048400000, 79, 127, 1442091600000, null, -3, 79, 3, 5, 1.0, 5, 1.0 ] + - [ "ZM", 1442041200000, 133, 133, null, null, 133, 133, 1, 1, 0.0, 1, 1.0 ] + - [ "ZW", 1442044800000, 0, 0, null, 1442048400000, 0, 254, 1, 1, 0.0, 1, 0.5 ] + - [ "ZW", 1442048400000, 254, 254, null, null, 0, 254, 2, 2, 1.0, 2, 1.0 ] \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest new file mode 100644 index 00000000000..b234094fea1 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest @@ -0,0 +1,1008 @@ +type: "operatorValidation" + +sql: " + SELECT + countryIsoCode, + FLOOR(__time TO HOUR) t, + SUM(delta) delta, + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode ORDER BY SUM(delta) RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) windowedDelta + FROM wikipedia + GROUP BY 1, 2" + +expectedOperators: + - { type: "naivePartition", partitionColumns: [ "d0" ] } + - type: "window" + processor: + type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: false, lowOffset: 3, uppUnbounded: false, uppOffset: 2 } + aggregations: + - { type: "longSum", name: "w0", fieldName: "a0" } + +expectedResults: + - [ "",1442016000000,29873,603401 ] + - [ "",1442019600000,173892,856027 ] + - [ "",1442023200000,399636,1022699 ] + - [ "",1442026800000,252626,1353656 ] + - [ "",1442030400000,166672,1524388 ] + - [ "",1442034000000,330957,1893946 ] + - [ "",1442037600000,200605,1810312 ] + - [ "",1442041200000,543450,1866002 ] + - [ "",1442044800000,316002,2486700 ] + - [ "",1442048400000,308316,2439701 ] + - [ "",1442052000000,787370,2698393 ] + - [ "",1442055600000,283958,2544408 ] + - [ "",1442059200000,459297,2579990 ] + - [ "",1442062800000,389465,2630189 ] + - [ "",1442066400000,351584,2218213 ] + - [ "",1442070000000,358515,2326738 ] + - [ "",1442073600000,375394,2320518 ] + - [ "",1442077200000,392483,2303622 ] + - [ "",1442080800000,453077,2255910 ] + - [ "",1442084400000,372569,2411822 ] + - [ "",1442088000000,303872,2410929 ] + - [ "",1442091600000,514427,2294605 ] + - [ "",1442095200000,374501,1841528 ] + - [ "",1442098800000,276159,1468959 ] + - [ "AE",1442030400000,118,150 ] + - [ "AE",1442044800000,-7,147 ] + - [ "AE",1442048400000,39,136 ] + - [ "AE",1442052000000,-3,182 ] + - [ "AE",1442059200000,-11,6387 ] + - [ "AE",1442070000000,46,6436 ] + - [ "AE",1442077200000,6323,6397 ] + - [ "AE",1442080800000,42,6400 ] + - [ "AL",1442077200000,26,80 ] + - [ "AL",1442091600000,54,80 ] + - [ "AO",1442041200000,-26,18 ] + - [ "AO",1442052000000,-18,740 ] + - [ "AO",1442088000000,62,740 ] + - [ "AO",1442098800000,722,740 ] + - [ "AR",1442019600000,1,3159 ] + - [ "AR",1442023200000,2514,3156 ] + - [ "AR",1442026800000,644,3368 ] + - [ "AR",1442030400000,-3,3449 ] + - [ "AR",1442034000000,212,3394 ] + - [ "AR",1442037600000,81,1090 ] + - [ "AR",1442055600000,-54,475 ] + - [ "AR",1442059200000,210,478 ] + - [ "AR",1442062800000,29,643 ] + - [ "AR",1442066400000,0,-29 ] + - [ "AR",1442070000000,377,26 ] + - [ "AR",1442077200000,-591,-189 ] + - [ "AR",1442080800000,1,122 ] + - [ "AR",1442084400000,-5,752 ] + - [ "AR",1442091600000,340,439 ] + - [ "AR",1442095200000,630,1030 ] + - [ "AR",1442098800000,64,1029 ] + - [ "AT",1442052000000,4793,4638 ] + - [ "AT",1442062800000,-155,4910 ] + - [ "AT",1442066400000,0,4908 ] + - [ "AT",1442070000000,272,11958 ] + - [ "AT",1442084400000,-2,7254 ] + - [ "AT",1442088000000,7050,7409 ] + - [ "AT",1442091600000,89,7409 ] + - [ "AU",1442016000000,0,305 ] + - [ "AU",1442019600000,253,493 ] + - [ "AU",1442023200000,52,116 ] + - [ "AU",1442026800000,188,399 ] + - [ "AU",1442030400000,-377,402 ] + - [ "AU",1442034000000,283,343 ] + - [ "AU",1442037600000,3,664 ] + - [ "AU",1442041200000,194,611 ] + - [ "AU",1442044800000,373,345 ] + - [ "AU",1442048400000,135,244 ] + - [ "AU",1442052000000,-643,279 ] + - [ "AU",1442055600000,182,64 ] + - [ "AU",1442059200000,38,-321 ] + - [ "AU",1442066400000,-21,-455 ] + - [ "AU",1442070000000,-12,1326 ] + - [ "AU",1442077200000,1,1539 ] + - [ "AU",1442091600000,1138,2019 ] + - [ "AU",1442095200000,395,2040 ] + - [ "AU",1442098800000,518,2052 ] + - [ "BA",1442048400000,-13,-177 ] + - [ "BA",1442052000000,38,-178 ] + - [ "BA",1442055600000,-202,-178 ] + - [ "BA",1442084400000,-1,-178 ] + - [ "BD",1442019600000,0,930 ] + - [ "BD",1442041200000,854,1033 ] + - [ "BD",1442066400000,76,1108 ] + - [ "BD",1442073600000,103,1106 ] + - [ "BD",1442077200000,75,1106 ] + - [ "BD",1442091600000,-2,252 ] + - [ "BE",1442030400000,-103,-45 ] + - [ "BE",1442048400000,59,188 ] + - [ "BE",1442052000000,-1,279 ] + - [ "BE",1442055600000,233,415 ] + - [ "BE",1442062800000,91,537 ] + - [ "BE",1442066400000,136,479 ] + - [ "BE",1442073600000,19,663 ] + - [ "BE",1442080800000,1,497 ] + - [ "BE",1442084400000,183,507 ] + - [ "BE",1442088000000,67,380 ] + - [ "BE",1442091600000,101,361 ] + - [ "BE",1442098800000,9,360 ] + - [ "BG",1442041200000,9,19136 ] + - [ "BG",1442052000000,18936,19191 ] + - [ "BG",1442059200000,191,19592 ] + - [ "BG",1442070000000,55,19592 ] + - [ "BG",1442084400000,401,19583 ] + - [ "BH",1442052000000,44,44 ] + - [ "BO",1442080800000,4,4 ] + - [ "BO",1442088000000,4,4 ] + - [ "BO",1442095200000,-4,4 ] + - [ "BR",1442016000000,-248,1003 ] + - [ "BR",1442019600000,372,1054 ] + - [ "BR",1442023200000,879,1084 ] + - [ "BR",1442026800000,51,1105 ] + - [ "BR",1442030400000,30,1620 ] + - [ "BR",1442034000000,21,1251 ] + - [ "BR",1442037600000,267,443 ] + - [ "BR",1442041200000,3,624 ] + - [ "BR",1442044800000,71,836 ] + - [ "BR",1442052000000,232,888 ] + - [ "BR",1442055600000,242,714 ] + - [ "BR",1442059200000,73,1745 ] + - [ "BR",1442062800000,93,2210 ] + - [ "BR",1442066400000,1034,4065 ] + - [ "BR",1442070000000,536,6076 ] + - [ "BR",1442073600000,2087,5736 ] + - [ "BR",1442077200000,2253,6135 ] + - [ "BR",1442080800000,-267,5316 ] + - [ "BR",1442084400000,492,4791 ] + - [ "BR",1442088000000,215,3452 ] + - [ "BR",1442091600000,11,554 ] + - [ "BR",1442095200000,748,821 ] + - [ "BR",1442098800000,-645,329 ] + - [ "BY",1442055600000,1,2061 ] + - [ "BY",1442059200000,1464,2091 ] + - [ "BY",1442073600000,596,2119 ] + - [ "BY",1442077200000,30,2120 ] + - [ "BY",1442080800000,28,2152 ] + - [ "BY",1442084400000,1,688 ] + - [ "BY",1442088000000,33,92 ] + - [ "CA",1442016000000,-371,2099 ] + - [ "CA",1442019600000,2184,4315 ] + - [ "CA",1442023200000,286,4268 ] + - [ "CA",1442026800000,2216,4446 ] + - [ "CA",1442030400000,-47,4685 ] + - [ "CA",1442034000000,178,2506 ] + - [ "CA",1442037600000,-132,2221 ] + - [ "CA",1442041200000,5,43 ] + - [ "CA",1442044800000,1,1126 ] + - [ "CA",1442052000000,38,581 ] + - [ "CA",1442059200000,1036,1020 ] + - [ "CA",1442062800000,-367,1200 ] + - [ "CA",1442066400000,307,1285 ] + - [ "CA",1442070000000,185,965 ] + - [ "CA",1442073600000,86,410 ] + - [ "CA",1442077200000,-282,821 ] + - [ "CA",1442080800000,481,549 ] + - [ "CA",1442084400000,44,3222 ] + - [ "CA",1442088000000,35,2775 ] + - [ "CA",1442091600000,2858,3221 ] + - [ "CA",1442095200000,-361,2740 ] + - [ "CA",1442098800000,164,2696 ] + - [ "CH",1442037600000,59,203 ] + - [ "CH",1442041200000,198,227 ] + - [ "CH",1442044800000,-54,274 ] + - [ "CH",1442048400000,24,274 ] + - [ "CH",1442052000000,47,237 ] + - [ "CH",1442055600000,0,50 ] + - [ "CH",1442062800000,22,464 ] + - [ "CH",1442070000000,11,446 ] + - [ "CH",1442073600000,360,412 ] + - [ "CH",1442077200000,6,479 ] + - [ "CH",1442084400000,13,457 ] + - [ "CH",1442091600000,67,446 ] + - [ "CL",1442016000000,161,-194 ] + - [ "CL",1442019600000,-370,-154 ] + - [ "CL",1442023200000,15,-155 ] + - [ "CL",1442030400000,40,-153 ] + - [ "CL",1442034000000,-1,-315 ] + - [ "CL",1442037600000,2,445 ] + - [ "CL",1442041200000,-1,418 ] + - [ "CL",1442052000000,390,395 ] + - [ "CL",1442059200000,-12,355 ] + - [ "CL",1442062800000,17,366 ] + - [ "CL",1442066400000,-41,520 ] + - [ "CL",1442070000000,13,115 ] + - [ "CL",1442073600000,153,144 ] + - [ "CL",1442077200000,-15,253 ] + - [ "CL",1442080800000,17,580 ] + - [ "CL",1442084400000,126,587 ] + - [ "CL",1442088000000,286,158 ] + - [ "CL",1442091600000,20,182 ] + - [ "CL",1442095200000,-276,165 ] + - [ "CL",1442098800000,9,39 ] + - [ "CN",1442023200000,-13,239 ] + - [ "CN",1442026800000,154,532 ] + - [ "CN",1442037600000,98,532 ] + - [ "CN",1442048400000,293,601 ] + - [ "CN",1442052000000,0,622 ] + - [ "CN",1442055600000,69,453 ] + - [ "CN",1442059200000,8,345 ] + - [ "CN",1442066400000,-15,51 ] + - [ "CN",1442080800000,-10,51 ] + - [ "CN",1442084400000,-1,-18 ] + - [ "CO",1442016000000,16,37 ] + - [ "CO",1442019600000,12,478 ] + - [ "CO",1442023200000,9,951 ] + - [ "CO",1442030400000,441,1239 ] + - [ "CO",1442059200000,473,1178 ] + - [ "CO",1442066400000,288,41026 ] + - [ "CO",1442070000000,-45,41598 ] + - [ "CO",1442073600000,39860,41182 ] + - [ "CO",1442077200000,581,40760 ] + - [ "CO",1442080800000,25,57622 ] + - [ "CO",1442084400000,51,58044 ] + - [ "CO",1442088000000,17150,18474 ] + - [ "CO",1442091600000,377,17976 ] + - [ "CO",1442095200000,290,17951 ] + - [ "CO",1442098800000,83,17900 ] + - [ "CR",1442019600000,62,264 ] + - [ "CR",1442023200000,62,2761 ] + - [ "CR",1442026800000,140,2812 ] + - [ "CR",1442030400000,2497,3006 ] + - [ "CR",1442041200000,51,3107 ] + - [ "CR",1442044800000,194,3117 ] + - [ "CR",1442048400000,163,2977 ] + - [ "CR",1442088000000,72,480 ] + - [ "CZ",1442026800000,-19,77 ] + - [ "CZ",1442034000000,78,1150 ] + - [ "CZ",1442037600000,18,1171 ] + - [ "CZ",1442055600000,1073,1171 ] + - [ "CZ",1442059200000,21,1358 ] + - [ "CZ",1442062800000,0,3331 ] + - [ "CZ",1442070000000,168,3428 ] + - [ "CZ",1442073600000,2051,2327 ] + - [ "CZ",1442077200000,115,2308 ] + - [ "CZ",1442080800000,-28,2308 ] + - [ "CZ",1442098800000,2,2140 ] + - [ "DE",1442016000000,167,231 ] + - [ "DE",1442019600000,0,604 ] + - [ "DE",1442023200000,64,962 ] + - [ "DE",1442030400000,373,1506 ] + - [ "DE",1442034000000,358,1536 ] + - [ "DE",1442037600000,544,2515 ] + - [ "DE",1442041200000,197,3262 ] + - [ "DE",1442044800000,979,4489 ] + - [ "DE",1442048400000,811,5654 ] + - [ "DE",1442052000000,1600,5399 ] + - [ "DE",1442055600000,1523,5485 ] + - [ "DE",1442059200000,289,6083 ] + - [ "DE",1442062800000,283,6938 ] + - [ "DE",1442066400000,1577,11413 ] + - [ "DE",1442070000000,1666,12078 ] + - [ "DE",1442073600000,6075,12922 ] + - [ "DE",1442077200000,2188,12514 ] + - [ "DE",1442080800000,1133,11127 ] + - [ "DE",1442084400000,-125,13816 ] + - [ "DE",1442088000000,190,8748 ] + - [ "DE",1442091600000,4355,6889 ] + - [ "DE",1442095200000,1007,5756 ] + - [ "DE",1442098800000,329,5881 ] + - [ "DK",1442037600000,10,41 ] + - [ "DK",1442044800000,36,83 ] + - [ "DK",1442048400000,-5,83 ] + - [ "DK",1442055600000,42,84 ] + - [ "DK",1442059200000,0,490 ] + - [ "DK",1442062800000,1,445 ] + - [ "DK",1442066400000,416,511 ] + - [ "DK",1442077200000,-9,372 ] + - [ "DK",1442080800000,61,511 ] + - [ "DK",1442084400000,-97,510 ] + - [ "DK",1442091600000,139,94 ] + - [ "DK",1442095200000,0,103 ] + - [ "DO",1442023200000,8,243 ] + - [ "DO",1442066400000,35,251 ] + - [ "DO",1442073600000,200,264 ] + - [ "DO",1442084400000,8,264 ] + - [ "DO",1442095200000,13,256 ] + - [ "DZ",1442077200000,-1,-1 ] + - [ "EC",1442019600000,29,20 ] + - [ "EC",1442023200000,-9,-346 ] + - [ "EC",1442030400000,0,222 ] + - [ "EC",1442077200000,-366,232 ] + - [ "EC",1442084400000,568,203 ] + - [ "EC",1442095200000,10,212 ] + - [ "EE",1442041200000,37,18 ] + - [ "EE",1442044800000,-19,18 ] + - [ "EG",1442026800000,16,142 ] + - [ "EG",1442055600000,14,143 ] + - [ "EG",1442062800000,112,170 ] + - [ "EG",1442073600000,1,170 ] + - [ "EG",1442091600000,27,154 ] + - [ "ES",1442019600000,103,46 ] + - [ "ES",1442023200000,-5,49 ] + - [ "ES",1442034000000,-52,167 ] + - [ "ES",1442037600000,3,-2 ] + - [ "ES",1442041200000,118,53 ] + - [ "ES",1442044800000,-169,54 ] + - [ "ES",1442048400000,158,601 ] + - [ "ES",1442052000000,-4,1684 ] + - [ "ES",1442055600000,495,1495 ] + - [ "ES",1442059200000,1086,2125 ] + - [ "ES",1442062800000,-71,2028 ] + - [ "ES",1442066400000,461,2186 ] + - [ "ES",1442070000000,61,2931 ] + - [ "ES",1442073600000,154,2182 ] + - [ "ES",1442077200000,1240,2123 ] + - [ "ES",1442084400000,337,2362 ] + - [ "ES",1442088000000,-130,4807 ] + - [ "ES",1442091600000,700,5111 ] + - [ "ES",1442095200000,2506,3871 ] + - [ "ES",1442098800000,458,3534 ] + - [ "FI",1442030400000,1491,1517 ] + - [ "FI",1442037600000,14,1703 ] + - [ "FI",1442048400000,12,2110 ] + - [ "FI",1442052000000,186,2129 ] + - [ "FI",1442059200000,407,821 ] + - [ "FI",1442062800000,19,806 ] + - [ "FI",1442066400000,183,994 ] + - [ "FI",1442073600000,-1,912 ] + - [ "FI",1442077200000,200,1400 ] + - [ "FI",1442080800000,104,1450 ] + - [ "FI",1442084400000,895,1267 ] + - [ "FI",1442095200000,69,1268 ] + - [ "FR",1442016000000,-1,1212 ] + - [ "FR",1442019600000,585,1298 ] + - [ "FR",1442023200000,628,1774 ] + - [ "FR",1442026800000,86,5948 ] + - [ "FR",1442034000000,476,6553 ] + - [ "FR",1442037600000,4174,6140 ] + - [ "FR",1442041200000,604,8539 ] + - [ "FR",1442044800000,172,9090 ] + - [ "FR",1442048400000,3027,9077 ] + - [ "FR",1442052000000,637,9553 ] + - [ "FR",1442055600000,463,14625 ] + - [ "FR",1442059200000,4650,16969 ] + - [ "FR",1442062800000,5676,14416 ] + - [ "FR",1442066400000,2516,17301 ] + - [ "FR",1442070000000,474,16394 ] + - [ "FR",1442073600000,3522,12301 ] + - [ "FR",1442077200000,-444,13268 ] + - [ "FR",1442080800000,557,12624 ] + - [ "FR",1442084400000,6643,12891 ] + - [ "FR",1442088000000,1872,9456 ] + - [ "FR",1442091600000,741,10036 ] + - [ "FR",1442095200000,87,9479 ] + - [ "FR",1442098800000,136,2836 ] + - [ "GB",1442016000000,-44,1826 ] + - [ "GB",1442019600000,54,2165 ] + - [ "GB",1442023200000,1816,4689 ] + - [ "GB",1442026800000,339,4677 ] + - [ "GB",1442030400000,2524,5265 ] + - [ "GB",1442034000000,-12,5253 ] + - [ "GB",1442037600000,544,3469 ] + - [ "GB",1442041200000,42,3870 ] + - [ "GB",1442044800000,32,1514 ] + - [ "GB",1442048400000,740,1979 ] + - [ "GB",1442052000000,168,17546 ] + - [ "GB",1442055600000,453,23247 ] + - [ "GB",1442059200000,16111,23886 ] + - [ "GB",1442062800000,5743,23520 ] + - [ "GB",1442066400000,671,24000 ] + - [ "GB",1442070000000,374,24682 ] + - [ "GB",1442073600000,648,10015 ] + - [ "GB",1442077200000,1135,4656 ] + - [ "GB",1442080800000,1444,5578 ] + - [ "GB",1442084400000,384,6015 ] + - [ "GB",1442088000000,1593,5605 ] + - [ "GB",1442091600000,811,4519 ] + - [ "GB",1442095200000,238,3075 ] + - [ "GB",1442098800000,49,2691 ] + - [ "GE",1442044800000,-21,-113 ] + - [ "GE",1442052000000,-108,-140 ] + - [ "GE",1442062800000,16,-140 ] + - [ "GE",1442080800000,-27,-140 ] + - [ "GH",1442088000000,0,0 ] + - [ "GR",1442019600000,82,89 ] + - [ "GR",1442034000000,0,63 ] + - [ "GR",1442041200000,7,71 ] + - [ "GR",1442048400000,-26,73 ] + - [ "GR",1442062800000,8,-323 ] + - [ "GR",1442070000000,2,-235 ] + - [ "GR",1442073600000,-314,-63 ] + - [ "GR",1442080800000,88,86 ] + - [ "GR",1442084400000,179,78 ] + - [ "GR",1442091600000,123,76 ] + - [ "GT",1442023200000,-167,7 ] + - [ "GT",1442026800000,173,7 ] + - [ "GT",1442098800000,1,7 ] + - [ "HK",1442019600000,-113,2090 ] + - [ "HK",1442023200000,2414,2247 ] + - [ "HK",1442026800000,-211,3384 ] + - [ "HK",1442030400000,157,4020 ] + - [ "HK",1442034000000,1137,4118 ] + - [ "HK",1442037600000,636,1725 ] + - [ "HK",1442041200000,-15,1937 ] + - [ "HK",1442044800000,21,1795 ] + - [ "HK",1442048400000,1,1462 ] + - [ "HK",1442052000000,15,828 ] + - [ "HK",1442055600000,804,844 ] + - [ "HK",1442059200000,2,862 ] + - [ "HK",1442062800000,1,1175 ] + - [ "HK",1442066400000,39,6705 ] + - [ "HK",1442070000000,314,5901 ] + - [ "HK",1442073600000,5545,5896 ] + - [ "HK",1442080800000,0,5894 ] + - [ "HK",1442091600000,-3,5855 ] + - [ "HK",1442095200000,-1,5541 ] + - [ "HN",1442026800000,-1,-1 ] + - [ "HR",1442070000000,32,90 ] + - [ "HR",1442073600000,0,310 ] + - [ "HR",1442077200000,58,300 ] + - [ "HR",1442080800000,220,382 ] + - [ "HR",1442084400000,-10,350 ] + - [ "HR",1442088000000,82,350 ] + - [ "HU",1442019600000,46,334 ] + - [ "HU",1442037600000,197,881 ] + - [ "HU",1442041200000,91,1380 ] + - [ "HU",1442044800000,547,1378 ] + - [ "HU",1442048400000,499,1382 ] + - [ "HU",1442055600000,-2,1427 ] + - [ "HU",1442062800000,50,1477 ] + - [ "HU",1442080800000,242,859 ] + - [ "HU",1442084400000,141,355 ] + - [ "HU",1442088000000,-71,628 ] + - [ "HU",1442091600000,-5,688 ] + - [ "HU",1442095200000,271,446 ] + - [ "HU",1442098800000,110,305 ] + - [ "ID",1442023200000,106,-31 ] + - [ "ID",1442026800000,-416,-12 ] + - [ "ID",1442030400000,279,2 ] + - [ "ID",1442034000000,19,4 ] + - [ "ID",1442037600000,14,-490 ] + - [ "ID",1442041200000,2,-58 ] + - [ "ID",1442044800000,-388,-320 ] + - [ "ID",1442055600000,16,-297 ] + - [ "ID",1442059200000,17,-290 ] + - [ "ID",1442070000000,42,-272 ] + - [ "ID",1442091600000,21,129 ] + - [ "ID",1442095200000,20,113 ] + - [ "ID",1442098800000,13,96 ] + - [ "IE",1442026800000,1,29 ] + - [ "IE",1442030400000,1,1091 ] + - [ "IE",1442048400000,27,991 ] + - [ "IE",1442066400000,1062,1394 ] + - [ "IE",1442070000000,-100,2212 ] + - [ "IE",1442077200000,403,2140 ] + - [ "IE",1442084400000,819,2113 ] + - [ "IE",1442091600000,-71,1051 ] + - [ "IL",1442041200000,35,278 ] + - [ "IL",1442044800000,218,3023 ] + - [ "IL",1442048400000,25,3027 ] + - [ "IL",1442052000000,2745,4232 ] + - [ "IL",1442055600000,4,4377 ] + - [ "IL",1442059200000,1205,4162 ] + - [ "IL",1442062800000,180,4186 ] + - [ "IL",1442066400000,3,1472 ] + - [ "IL",1442070000000,49,1655 ] + - [ "IL",1442073600000,31,538 ] + - [ "IL",1442077200000,187,1495 ] + - [ "IL",1442080800000,88,2199 ] + - [ "IL",1442084400000,1137,2150 ] + - [ "IL",1442091600000,707,2122 ] + - [ "IL",1442095200000,0,1935 ] + - [ "IL",1442098800000,3,1847 ] + - [ "IN",1442016000000,1,-103 ] + - [ "IN",1442019600000,38,871 ] + - [ "IN",1442023200000,-142,2319 ] + - [ "IN",1442026800000,974,3669 ] + - [ "IN",1442030400000,1448,3803 ] + - [ "IN",1442034000000,1350,3845 ] + - [ "IN",1442037600000,135,6664 ] + - [ "IN",1442041200000,80,5952 ] + - [ "IN",1442044800000,2677,5038 ] + - [ "IN",1442048400000,262,3854 ] + - [ "IN",1442052000000,534,4427 ] + - [ "IN",1442055600000,166,5894 ] + - [ "IN",1442059200000,708,3333 ] + - [ "IN",1442062800000,1547,15162 ] + - [ "IN",1442066400000,116,15798 ] + - [ "IN",1442070000000,12091,21331 ] + - [ "IN",1442073600000,1170,20623 ] + - [ "IN",1442077200000,5699,19263 ] + - [ "IN",1442080800000,0,19268 ] + - [ "IN",1442084400000,187,7181 ] + - [ "IN",1442088000000,121,6011 ] + - [ "IN",1442095200000,4,312 ] + - [ "IQ",1442041200000,-1,5 ] + - [ "IQ",1442044800000,6,3 ] + - [ "IQ",1442052000000,0,3 ] + - [ "IQ",1442095200000,-2,3 ] + - [ "IR",1442026800000,0,367 ] + - [ "IR",1442030400000,375,288 ] + - [ "IR",1442034000000,-8,594 ] + - [ "IR",1442041200000,-79,749 ] + - [ "IR",1442044800000,306,625 ] + - [ "IR",1442052000000,155,1705 ] + - [ "IR",1442055600000,-124,1520 ] + - [ "IR",1442059200000,1455,1565 ] + - [ "IR",1442073600000,-193,1390 ] + - [ "IR",1442077200000,-34,1949 ] + - [ "IR",1442080800000,131,2073 ] + - [ "IR",1442088000000,714,618 ] + - [ "IR",1442091600000,0,811 ] + - [ "IT",1442016000000,0,294 ] + - [ "IT",1442019600000,183,516 ] + - [ "IT",1442023200000,111,499 ] + - [ "IT",1442026800000,222,1505 ] + - [ "IT",1442030400000,-17,1496 ] + - [ "IT",1442034000000,1006,1333 ] + - [ "IT",1442037600000,-9,2705 ] + - [ "IT",1442041200000,20,3159 ] + - [ "IT",1442044800000,1483,5056 ] + - [ "IT",1442048400000,676,10290 ] + - [ "IT",1442052000000,1880,10841 ] + - [ "IT",1442055600000,6240,12759 ] + - [ "IT",1442059200000,542,15431 ] + - [ "IT",1442062800000,1938,14836 ] + - [ "IT",1442066400000,4155,15542 ] + - [ "IT",1442070000000,81,11490 ] + - [ "IT",1442073600000,2586,16492 ] + - [ "IT",1442077200000,2188,17214 ] + - [ "IT",1442080800000,5544,16805 ] + - [ "IT",1442084400000,2660,17075 ] + - [ "IT",1442088000000,3746,17429 ] + - [ "IT",1442091600000,351,15806 ] + - [ "IT",1442095200000,2940,10262 ] + - [ "IT",1442098800000,565,7602 ] + - [ "JM",1442070000000,30,30 ] + - [ "JO",1442055600000,-2,2 ] + - [ "JO",1442059200000,0,2 ] + - [ "JO",1442080800000,4,2 ] + - [ "JP",1442016000000,-113,3848 ] + - [ "JP",1442019600000,2002,4883 ] + - [ "JP",1442023200000,1959,5688 ] + - [ "JP",1442026800000,1035,6598 ] + - [ "JP",1442030400000,805,8892 ] + - [ "JP",1442034000000,910,8263 ] + - [ "JP",1442037600000,2181,7873 ] + - [ "JP",1442041200000,1373,8819 ] + - [ "JP",1442044800000,1569,10803 ] + - [ "JP",1442048400000,1981,10891 ] + - [ "JP",1442052000000,2789,8625 ] + - [ "JP",1442055600000,998,8055 ] + - [ "JP",1442059200000,-85,6653 ] + - [ "JP",1442062800000,803,4751 ] + - [ "JP",1442066400000,167,3124 ] + - [ "JP",1442070000000,79,2177 ] + - [ "JP",1442073600000,1162,2682 ] + - [ "JP",1442077200000,51,1892 ] + - [ "JP",1442080800000,420,1782 ] + - [ "JP",1442084400000,13,1931 ] + - [ "JP",1442088000000,57,769 ] + - [ "JP",1442091600000,228,712 ] + - [ "JP",1442095200000,0,292 ] + - [ "JP",1442098800000,-6,279 ] + - [ "KE",1442044800000,-1,-1 ] + - [ "KG",1442073600000,6,6 ] + - [ "KR",1442016000000,1024,1788 ] + - [ "KR",1442019600000,445,1609 ] + - [ "KR",1442023200000,319,2644 ] + - [ "KR",1442026800000,-179,3078 ] + - [ "KR",1442030400000,1035,2080 ] + - [ "KR",1442034000000,434,1655 ] + - [ "KR",1442037600000,26,2165 ] + - [ "KR",1442041200000,20,1970 ] + - [ "KR",1442044800000,829,932 ] + - [ "KR",1442048400000,-374,4138 ] + - [ "KR",1442052000000,-3,4320 ] + - [ "KR",1442055600000,3640,5396 ] + - [ "KR",1442059200000,208,7866 ] + - [ "KR",1442062800000,1096,8462 ] + - [ "KR",1442066400000,3299,8425 ] + - [ "KR",1442070000000,222,4752 ] + - [ "KR",1442077200000,-40,4858 ] + - [ "KR",1442080800000,-33,4286 ] + - [ "KR",1442084400000,314,1814 ] + - [ "KR",1442088000000,524,1556 ] + - [ "KR",1442095200000,827,1596 ] + - [ "KR",1442098800000,-36,1629 ] + - [ "KW",1442055600000,-2,1811 ] + - [ "KW",1442070000000,1815,1778 ] + - [ "KW",1442077200000,-2,1778 ] + - [ "KW",1442080800000,-33,1778 ] + - [ "KZ",1442034000000,161,1001 ] + - [ "KZ",1442044800000,401,1413 ] + - [ "KZ",1442048400000,439,1476 ] + - [ "KZ",1442052000000,412,1509 ] + - [ "KZ",1442055600000,63,1348 ] + - [ "KZ",1442059200000,33,947 ] + - [ "KZ",1442062800000,0,191 ] + - [ "KZ",1442066400000,0,-243 ] + - [ "KZ",1442077200000,-317,-215 ] + - [ "KZ",1442084400000,-22,-248 ] + - [ "KZ",1442095200000,91,-248 ] + - [ "LB",1442055600000,-67,-67 ] + - [ "LK",1442026800000,79,134 ] + - [ "LK",1442048400000,8,131 ] + - [ "LK",1442052000000,47,131 ] + - [ "LK",1442084400000,-3,131 ] + - [ "LT",1442080800000,12,-12 ] + - [ "LT",1442098800000,-24,-12 ] + - [ "LU",1442059200000,79,604 ] + - [ "LU",1442066400000,0,606 ] + - [ "LU",1442077200000,525,606 ] + - [ "LU",1442095200000,2,606 ] + - [ "LV",1442095200000,0,0 ] + - [ "MA",1442019600000,-1,-34 ] + - [ "MA",1442055600000,23,-34 ] + - [ "MA",1442059200000,-56,216 ] + - [ "MA",1442062800000,0,221 ] + - [ "MA",1442077200000,250,230 ] + - [ "MA",1442080800000,5,207 ] + - [ "MA",1442098800000,8,263 ] + - [ "MD",1442077200000,6916,6916 ] + - [ "ME",1442073600000,0,0 ] + - [ "MH",1442052000000,40,40 ] + - [ "MK",1442077200000,-72,-72 ] + - [ "MM",1442070000000,3,28 ] + - [ "MM",1442073600000,25,28 ] + - [ "MO",1442034000000,30,48 ] + - [ "MO",1442070000000,18,48 ] + - [ "MR",1442080800000,10,10 ] + - [ "MT",1442048400000,-1,-1 ] + - [ "MV",1442073600000,-3,-3 ] + - [ "MX",1442016000000,-67,4124 ] + - [ "MX",1442023200000,549,4497 ] + - [ "MX",1442026800000,3642,5441 ] + - [ "MX",1442030400000,373,5445 ] + - [ "MX",1442034000000,944,5218 ] + - [ "MX",1442037600000,4,4668 ] + - [ "MX",1442041200000,-294,1025 ] + - [ "MX",1442066400000,-1,631 ] + - [ "MX",1442070000000,-1,3561 ] + - [ "MX",1442073600000,-21,3181 ] + - [ "MX",1442077200000,3874,4456 ] + - [ "MX",1442080800000,-376,4951 ] + - [ "MX",1442084400000,981,5751 ] + - [ "MX",1442088000000,494,5316 ] + - [ "MX",1442091600000,799,1470 ] + - [ "MX",1442095200000,-456,1846 ] + - [ "MX",1442098800000,28,865 ] + - [ "MY",1442019600000,-7,1018 ] + - [ "MY",1442030400000,-3,1953 ] + - [ "MY",1442034000000,1028,1826 ] + - [ "MY",1442041200000,935,2475 ] + - [ "MY",1442044800000,-127,2483 ] + - [ "MY",1442048400000,649,2486 ] + - [ "MY",1442055600000,1,1459 ] + - [ "MY",1442059200000,0,525 ] + - [ "MY",1442066400000,1,642 ] + - [ "MY",1442073600000,1,732 ] + - [ "MY",1442077200000,-10,731 ] + - [ "MY",1442098800000,739,731 ] + - [ "NG",1442052000000,208,214 ] + - [ "NG",1442070000000,6,214 ] + - [ "NL",1442034000000,0,1319 ] + - [ "NL",1442044800000,16,1372 ] + - [ "NL",1442048400000,1303,1477 ] + - [ "NL",1442052000000,53,1683 ] + - [ "NL",1442055600000,105,1653 ] + - [ "NL",1442059200000,206,1698 ] + - [ "NL",1442062800000,-30,311 ] + - [ "NL",1442066400000,61,424 ] + - [ "NL",1442070000000,-84,1197 ] + - [ "NL",1442073600000,166,9938 ] + - [ "NL",1442077200000,878,10404 ] + - [ "NL",1442080800000,8947,10355 ] + - [ "NL",1442084400000,436,10458 ] + - [ "NL",1442088000000,12,10362 ] + - [ "NL",1442091600000,19,9488 ] + - [ "NL",1442095200000,70,541 ] + - [ "NL",1442098800000,4,105 ] + - [ "NO",1442019600000,48,48 ] + - [ "NO",1442048400000,-447,77 ] + - [ "NO",1442052000000,447,148 ] + - [ "NO",1442055600000,29,370 ] + - [ "NO",1442066400000,71,353 ] + - [ "NO",1442073600000,222,815 ] + - [ "NO",1442080800000,31,383 ] + - [ "NO",1442088000000,15,353 ] + - [ "NO",1442091600000,15,284 ] + - [ "NO",1442095200000,-1,62 ] + - [ "NO",1442098800000,2,31 ] + - [ "NP",1442048400000,61,61 ] + - [ "NZ",1442019600000,28,729 ] + - [ "NZ",1442026800000,635,918 ] + - [ "NZ",1442037600000,66,1346 ] + - [ "NZ",1442048400000,189,1294 ] + - [ "NZ",1442059200000,428,1671 ] + - [ "NZ",1442084400000,-52,1032 ] + - [ "NZ",1442088000000,405,964 ] + - [ "NZ",1442095200000,-4,775 ] + - [ "NZ",1442098800000,-2,347 ] + - [ "OM",1442052000000,0,0 ] + - [ "PA",1442026800000,0,0 ] + - [ "PE",1442019600000,523,537 ] + - [ "PE",1442023200000,26,525 ] + - [ "PE",1442026800000,-12,362 ] + - [ "PE",1442062800000,-12,360 ] + - [ "PE",1442077200000,-163,-231 ] + - [ "PE",1442080800000,-2,-276 ] + - [ "PE",1442084400000,-68,1597 ] + - [ "PE",1442095200000,-19,1609 ] + - [ "PE",1442098800000,1861,1772 ] + - [ "PH",1442019600000,6,1375 ] + - [ "PH",1442023200000,459,1401 ] + - [ "PH",1442026800000,910,1460 ] + - [ "PH",1442030400000,26,1477 ] + - [ "PH",1442034000000,59,1471 ] + - [ "PH",1442037600000,17,1067 ] + - [ "PH",1442041200000,0,219 ] + - [ "PH",1442044800000,55,215 ] + - [ "PH",1442048400000,62,2125 ] + - [ "PH",1442052000000,22,2381 ] + - [ "PH",1442055600000,1969,2552 ] + - [ "PH",1442059200000,273,4377 ] + - [ "PH",1442062800000,171,4349 ] + - [ "PH",1442066400000,1880,4100 ] + - [ "PH",1442070000000,34,2133 ] + - [ "PH",1442073600000,-227,1892 ] + - [ "PH",1442077200000,2,1760 ] + - [ "PH",1442080800000,32,696 ] + - [ "PH",1442084400000,39,670 ] + - [ "PH",1442091600000,816,897 ] + - [ "PH",1442098800000,8,895 ] + - [ "PK",1442019600000,335,536 ] + - [ "PK",1442026800000,101,560 ] + - [ "PK",1442037600000,100,575 ] + - [ "PK",1442041200000,24,598 ] + - [ "PK",1442048400000,15,306 ] + - [ "PK",1442062800000,23,205 ] + - [ "PK",1442070000000,43,105 ] + - [ "PL",1442037600000,95,695 ] + - [ "PL",1442041200000,281,1061 ] + - [ "PL",1442044800000,319,1391 ] + - [ "PL",1442048400000,366,1801 ] + - [ "PL",1442052000000,330,1905 ] + - [ "PL",1442055600000,410,5795 ] + - [ "PL",1442059200000,199,5510 ] + - [ "PL",1442062800000,4171,5290 ] + - [ "PL",1442066400000,34,4990 ] + - [ "PL",1442070000000,146,4904 ] + - [ "PL",1442073600000,30,4712 ] + - [ "PL",1442077200000,324,554 ] + - [ "PL",1442080800000,7,866 ] + - [ "PL",1442084400000,13,1622 ] + - [ "PL",1442088000000,346,3443 ] + - [ "PL",1442091600000,902,3110 ] + - [ "PL",1442095200000,1851,3103 ] + - [ "PL",1442098800000,-9,3090 ] + - [ "PR",1442026800000,22,-11 ] + - [ "PR",1442030400000,2,-6 ] + - [ "PR",1442059200000,-35,23 ] + - [ "PR",1442077200000,5,23 ] + - [ "PR",1442095200000,29,1 ] + - [ "PT",1442019600000,172,285 ] + - [ "PT",1442044800000,11,297 ] + - [ "PT",1442052000000,102,3767 ] + - [ "PT",1442066400000,12,3692 ] + - [ "PT",1442070000000,3470,3441 ] + - [ "PT",1442077200000,-75,3833 ] + - [ "PT",1442080800000,-79,3750 ] + - [ "PT",1442088000000,403,3740 ] + - [ "PT",1442095200000,19,270 ] + - [ "PT",1442098800000,2,345 ] + - [ "PY",1442019600000,1,634 ] + - [ "PY",1442080800000,5,634 ] + - [ "PY",1442084400000,628,634 ] + - [ "QA",1442041200000,13,13 ] + - [ "RO",1442034000000,68,1197 ] + - [ "RO",1442041200000,845,1516 ] + - [ "RO",1442044800000,284,1542 ] + - [ "RO",1442052000000,319,2083 ] + - [ "RO",1442055600000,26,1986 ] + - [ "RO",1442062800000,541,1156 ] + - [ "RO",1442070000000,-29,872 ] + - [ "RO",1442073600000,15,1377 ] + - [ "RO",1442091600000,0,1351 ] + - [ "RO",1442095200000,824,810 ] + - [ "RS",1442019600000,6,19 ] + - [ "RS",1442062800000,13,832 ] + - [ "RS",1442066400000,0,832 ] + - [ "RS",1442073600000,813,921 ] + - [ "RS",1442080800000,0,900 ] + - [ "RS",1442084400000,89,887 ] + - [ "RS",1442091600000,-15,887 ] + - [ "RU",1442019600000,2214,2513 ] + - [ "RU",1442023200000,299,2589 ] + - [ "RU",1442026800000,0,3247 ] + - [ "RU",1442030400000,76,2923 ] + - [ "RU",1442034000000,658,1289 ] + - [ "RU",1442037600000,-324,3554 ] + - [ "RU",1442041200000,580,4581 ] + - [ "RU",1442044800000,2564,5719 ] + - [ "RU",1442048400000,1027,5560 ] + - [ "RU",1442052000000,1214,9786 ] + - [ "RU",1442055600000,499,9374 ] + - [ "RU",1442059200000,3902,8857 ] + - [ "RU",1442062800000,168,12536 ] + - [ "RU",1442066400000,2047,12940 ] + - [ "RU",1442070000000,4706,13603 ] + - [ "RU",1442073600000,1618,10356 ] + - [ "RU",1442077200000,1162,16649 ] + - [ "RU",1442080800000,655,17198 ] + - [ "RU",1442084400000,6461,15941 ] + - [ "RU",1442088000000,2596,14758 ] + - [ "RU",1442091600000,3449,25694 ] + - [ "RU",1442095200000,435,25039 ] + - [ "RU",1442098800000,12098,18578 ] + - [ "SA",1442037600000,-97,-72 ] + - [ "SA",1442048400000,14,-72 ] + - [ "SA",1442055600000,11,1204 ] + - [ "SA",1442059200000,0,1206 ] + - [ "SA",1442066400000,1276,1253 ] + - [ "SA",1442073600000,2,1697 ] + - [ "SA",1442077200000,-50,1686 ] + - [ "SA",1442084400000,458,1686 ] + - [ "SE",1442019600000,109,142 ] + - [ "SE",1442023200000,3,233 ] + - [ "SE",1442030400000,30,88 ] + - [ "SE",1442041200000,91,89 ] + - [ "SE",1442048400000,-145,-25 ] + - [ "SE",1442052000000,1,1448 ] + - [ "SE",1442055600000,-5,1432 ] + - [ "SE",1442059200000,1476,1419 ] + - [ "SE",1442066400000,14,1653 ] + - [ "SE",1442070000000,78,1689 ] + - [ "SE",1442080800000,89,1693 ] + - [ "SE",1442084400000,37,278 ] + - [ "SE",1442091600000,-1,264 ] + - [ "SE",1442095200000,61,186 ] + - [ "SE",1442098800000,0,97 ] + - [ "SG",1442026800000,2758,2762 ] + - [ "SG",1442030400000,1,2821 ] + - [ "SG",1442037600000,3,2898 ] + - [ "SG",1442041200000,59,2950 ] + - [ "SG",1442044800000,77,580 ] + - [ "SG",1442048400000,52,579 ] + - [ "SG",1442062800000,388,576 ] + - [ "SG",1442066400000,0,517 ] + - [ "SI",1442080800000,-45,-36 ] + - [ "SI",1442091600000,9,-36 ] + - [ "SK",1442037600000,-1,18 ] + - [ "SK",1442052000000,13,464 ] + - [ "SK",1442062800000,6,372 ] + - [ "SK",1442073600000,446,379 ] + - [ "SK",1442084400000,-92,380 ] + - [ "SK",1442098800000,7,367 ] + - [ "SV",1442019600000,-1,114 ] + - [ "SV",1442084400000,106,114 ] + - [ "SV",1442088000000,9,114 ] + - [ "TH",1442034000000,0,113 ] + - [ "TH",1442041200000,3,91 ] + - [ "TH",1442044800000,110,91 ] + - [ "TH",1442052000000,-22,45 ] + - [ "TH",1442055600000,0,11 ] + - [ "TH",1442062800000,-46,8 ] + - [ "TH",1442066400000,-34,-89 ] + - [ "TH",1442070000000,0,-67 ] + - [ "TH",1442084400000,13,-67 ] + - [ "TJ",1442048400000,1471,1471 ] + - [ "TN",1442098800000,-9,-9 ] + - [ "TR",1442023200000,306,348 ] + - [ "TR",1442041200000,1,436 ] + - [ "TR",1442044800000,41,477 ] + - [ "TR",1442048400000,88,776 ] + - [ "TR",1442052000000,41,785 ] + - [ "TR",1442055600000,299,869 ] + - [ "TR",1442062800000,315,1064 ] + - [ "TR",1442066400000,85,1065 ] + - [ "TR",1442070000000,236,1023 ] + - [ "TR",1442077200000,89,894 ] + - [ "TR",1442080800000,-1,2968 ] + - [ "TR",1442084400000,170,5931 ] + - [ "TR",1442088000000,2389,5666 ] + - [ "TR",1442091600000,3048,5577 ] + - [ "TR",1442095200000,-29,5578 ] + - [ "TT",1442088000000,9,9 ] + - [ "TW",1442016000000,92,189 ] + - [ "TW",1442019600000,0,869 ] + - [ "TW",1442023200000,97,869 ] + - [ "TW",1442026800000,680,1012 ] + - [ "TW",1442030400000,0,1186 ] + - [ "TW",1442034000000,143,1552 ] + - [ "TW",1442037600000,266,1479 ] + - [ "TW",1442041200000,366,874 ] + - [ "TW",1442044800000,24,898 ] + - [ "TW",1442048400000,75,803 ] + - [ "TW",1442052000000,24,380 ] + - [ "TW",1442055600000,48,-258 ] + - [ "TW",1442059200000,-157,342 ] + - [ "TW",1442062800000,-272,752 ] + - [ "TW",1442066400000,624,1500 ] + - [ "TW",1442070000000,485,1954 ] + - [ "TW",1442073600000,772,2135 ] + - [ "TW",1442077200000,502,2407 ] + - [ "TW",1442080800000,24,1706 ] + - [ "TW",1442084400000,0,1161 ] + - [ "TW",1442095200000,-77,389 ] + - [ "TW",1442098800000,-60,-113 ] + - [ "UA",1442034000000,3468,3541 ] + - [ "UA",1442037600000,-1,3821 ] + - [ "UA",1442041200000,74,3823 ] + - [ "UA",1442044800000,280,4233 ] + - [ "UA",1442048400000,2,14967 ] + - [ "UA",1442052000000,410,14966 ] + - [ "UA",1442055600000,14202,15665 ] + - [ "UA",1442059200000,-2,15681 ] + - [ "UA",1442062800000,773,17412 ] + - [ "UA",1442066400000,296,21243 ] + - [ "UA",1442070000000,1733,6860 ] + - [ "UA",1442073600000,4241,6861 ] + - [ "UA",1442077200000,-181,6093 ] + - [ "UA",1442080800000,-1,5776 ] + - [ "UA",1442084400000,5,3655 ] + - [ "UA",1442088000000,-21,-616 ] + - [ "UA",1442091600000,-388,-397 ] + - [ "UA",1442095200000,-30,-396 ] + - [ "UA",1442098800000,38,-401 ] + - [ "UG",1442070000000,1,1 ] + - [ "US",1442016000000,0,3887 ] + - [ "US",1442019600000,1043,5399 ] + - [ "US",1442023200000,2844,7422 ] + - [ "US",1442026800000,1512,11070 ] + - [ "US",1442030400000,2023,14745 ] + - [ "US",1442034000000,3648,15701 ] + - [ "US",1442037600000,3675,12996 ] + - [ "US",1442041200000,1999,11018 ] + - [ "US",1442044800000,139,8993 ] + - [ "US",1442048400000,-466,5501 ] + - [ "US",1442052000000,-2,1837 ] + - [ "US",1442055600000,156,-115 ] + - [ "US",1442059200000,11,518 ] + - [ "US",1442062800000,47,4489 ] + - [ "US",1442066400000,772,5591 ] + - [ "US",1442070000000,3505,7603 ] + - [ "US",1442073600000,1100,11593 ] + - [ "US",1442077200000,2168,14069 ] + - [ "US",1442080800000,4001,14988 ] + - [ "US",1442084400000,2523,13985 ] + - [ "US",1442088000000,1691,13301 ] + - [ "US",1442091600000,2502,14708 ] + - [ "US",1442095200000,416,10707 ] + - [ "US",1442098800000,3575,8184 ] + - [ "UY",1442019600000,77,670 ] + - [ "UY",1442023200000,517,671 ] + - [ "UY",1442026800000,76,955 ] + - [ "UY",1442037600000,1,913 ] + - [ "UY",1442070000000,284,859 ] + - [ "UY",1442073600000,-42,342 ] + - [ "UY",1442077200000,23,266 ] + - [ "UZ",1442044800000,1369,1369 ] + - [ "VE",1442023200000,115,149 ] + - [ "VE",1442026800000,-17,147 ] + - [ "VE",1442030400000,51,165 ] + - [ "VE",1442034000000,-2,585 ] + - [ "VE",1442066400000,18,882 ] + - [ "VE",1442070000000,420,959 ] + - [ "VE",1442077200000,412,943 ] + - [ "VE",1442084400000,60,954 ] + - [ "VE",1442095200000,35,936 ] + - [ "VE",1442098800000,9,516 ] + - [ "VG",1442062800000,-238,-238 ] + - [ "VN",1442023200000,-9,25 ] + - [ "VN",1442026800000,63,14 ] + - [ "VN",1442034000000,-29,14 ] + - [ "VN",1442037600000,-11,-1 ] + - [ "VN",1442041200000,0,98 ] + - [ "VN",1442048400000,-15,72 ] + - [ "VN",1442052000000,90,109 ] + - [ "VN",1442055600000,37,266 ] + - [ "VN",1442059200000,8,1077 ] + - [ "VN",1442062800000,146,1571 ] + - [ "VN",1442066400000,811,1471 ] + - [ "VN",1442070000000,479,1434 ] + - [ "VN",1442084400000,-10,1426 ] + - [ "ZA",1442034000000,-3,126 ] + - [ "ZA",1442048400000,79,126 ] + - [ "ZA",1442059200000,50,127 ] + - [ "ZA",1442070000000,0,127 ] + - [ "ZA",1442091600000,1,130 ] + - [ "ZM",1442041200000,133,133 ] + - [ "ZW",1442044800000,0,254 ] + - [ "ZW",1442048400000,254,254 ] \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest index 41895946953..0dd2ddc35d0 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest @@ -1,1019 +1,1020 @@ type: "operatorValidation" -sql: " - SELECT - countryIsoCode, - FLOOR(__time TO HOUR) t, - SUM(delta) delta, - SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode) totalDelta, - LAG(FLOOR(__time TO HOUR), 2) OVER (PARTITION BY countryIsoCode) laggardTime, - LEAD(FLOOR(__time TO HOUR), 1) OVER (PARTITION BY countryIsoCode) leadTime, - FIRST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode) AS firstDelay, - LAST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode) AS lastDelay, - NTILE(3) OVER (PARTITION BY countryIsoCode) AS delayNTile - FROM wikipedia - GROUP BY 1, 2" +sql: | + SELECT + countryIsoCode, + FLOOR(__time TO HOUR) t, + SUM(delta) delta, + SUM(SUM(delta)) OVER (PARTITION BY countryIsoCode) totalDelta, + LAG(FLOOR(__time TO HOUR), 2) OVER (PARTITION BY countryIsoCode) laggardTime, + LEAD(FLOOR(__time TO HOUR), 1) OVER (PARTITION BY countryIsoCode) leadTime, + FIRST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode) AS firstDelay, + LAST_VALUE(SUM(delta)) OVER (PARTITION BY countryIsoCode) AS lastDelay, + NTILE(3) OVER (PARTITION BY countryIsoCode) AS delayNTile + FROM wikipedia + GROUP BY 1, 2 expectedOperators: - - { type: "naivePartition", partitionColumns: ["d0"] } + - { type: "naivePartition", partitionColumns: [ "d0" ] } - type: "window" processor: type: "composing" processors: - - { "type":"offset", "inputColumn":"d1", "outputColumn":"w1", "offset":-2 } - - { "type":"offset", "inputColumn":"d1", "outputColumn":"w2", "offset":1 } - - { "type":"first", "inputColumn":"a0", "outputColumn":"w3" } - - { "type":"last", "inputColumn":"a0", "outputColumn":"w4" } - - { "type": "percentile", "outputColumn": "w5", "numBuckets": 3 } - - type: "aggregate" - aggregations: - - { "type":"longSum", "name":"w0", "fieldName":"a0" } + - { "type": "offset", "inputColumn": "d1", "outputColumn": "w1", "offset": -2 } + - { "type": "offset", "inputColumn": "d1", "outputColumn": "w2", "offset": 1 } + - { "type": "first", "inputColumn": "a0", "outputColumn": "w3" } + - { "type": "last", "inputColumn": "a0", "outputColumn": "w4" } + - { "type": "percentile", "outputColumn": "w5", "numBuckets": 3 } + - type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + aggregations: + - { "type": "longSum", "name": "w0", "fieldName": "a0" } expectedResults: - - ["", 1442016000000, 29873, 8414700, null, 1442019600000, 29873, 276159, 1] - - ["", 1442019600000, 173892, 8414700, null, 1442023200000, 29873, 276159, 1] - - ["", 1442023200000, 399636, 8414700, 1442016000000, 1442026800000, 29873, 276159, 1] - - ["", 1442026800000, 252626, 8414700, 1442019600000, 1442030400000, 29873, 276159, 1] - - ["", 1442030400000, 166672, 8414700, 1442023200000, 1442034000000, 29873, 276159, 1] - - ["", 1442034000000, 330957, 8414700, 1442026800000, 1442037600000, 29873, 276159, 1] - - ["", 1442037600000, 200605, 8414700, 1442030400000, 1442041200000, 29873, 276159, 1] - - ["", 1442041200000, 543450, 8414700, 1442034000000, 1442044800000, 29873, 276159, 1] - - ["", 1442044800000, 316002, 8414700, 1442037600000, 1442048400000, 29873, 276159, 2] - - ["", 1442048400000, 308316, 8414700, 1442041200000, 1442052000000, 29873, 276159, 2] - - ["", 1442052000000, 787370, 8414700, 1442044800000, 1442055600000, 29873, 276159, 2] - - ["", 1442055600000, 283958, 8414700, 1442048400000, 1442059200000, 29873, 276159, 2] - - ["", 1442059200000, 459297, 8414700, 1442052000000, 1442062800000, 29873, 276159, 2] - - ["", 1442062800000, 389465, 8414700, 1442055600000, 1442066400000, 29873, 276159, 2] - - ["", 1442066400000, 351584, 8414700, 1442059200000, 1442070000000, 29873, 276159, 2] - - ["", 1442070000000, 358515, 8414700, 1442062800000, 1442073600000, 29873, 276159, 2] - - ["", 1442073600000, 375394, 8414700, 1442066400000, 1442077200000, 29873, 276159, 3] - - ["", 1442077200000, 392483, 8414700, 1442070000000, 1442080800000, 29873, 276159, 3] - - ["", 1442080800000, 453077, 8414700, 1442073600000, 1442084400000, 29873, 276159, 3] - - ["", 1442084400000, 372569, 8414700, 1442077200000, 1442088000000, 29873, 276159, 3] - - ["", 1442088000000, 303872, 8414700, 1442080800000, 1442091600000, 29873, 276159, 3] - - ["", 1442091600000, 514427, 8414700, 1442084400000, 1442095200000, 29873, 276159, 3] - - ["", 1442095200000, 374501, 8414700, 1442088000000, 1442098800000, 29873, 276159, 3] - - ["", 1442098800000, 276159, 8414700, 1442091600000, null, 29873, 276159, 3] - - ["AE", 1442030400000, 118, 6547, null, 1442044800000, 118, 42, 1] - - ["AE", 1442044800000, -7, 6547, null, 1442048400000, 118, 42, 1] - - ["AE", 1442048400000, 39, 6547, 1442030400000, 1442052000000, 118, 42, 1] - - ["AE", 1442052000000, -3, 6547, 1442044800000, 1442059200000, 118, 42, 2] - - ["AE", 1442059200000, -11, 6547, 1442048400000, 1442070000000, 118, 42, 2] - - ["AE", 1442070000000, 46, 6547, 1442052000000, 1442077200000, 118, 42, 2] - - ["AE", 1442077200000, 6323, 6547, 1442059200000, 1442080800000, 118, 42, 3] - - ["AE", 1442080800000, 42, 6547, 1442070000000, null, 118, 42, 3] - - ["AL", 1442077200000, 26, 80, null, 1442091600000, 26, 54, 1] - - ["AL", 1442091600000, 54, 80, null, null, 26, 54, 2] - - ["AO", 1442041200000, -26, 740, null, 1442052000000, -26, 722, 1] - - ["AO", 1442052000000, -18, 740, null, 1442088000000, -26, 722, 1] - - ["AO", 1442088000000, 62, 740, 1442041200000, 1442098800000, -26, 722, 2] - - ["AO", 1442098800000, 722, 740, 1442052000000, null, -26, 722, 3] - - ["AR", 1442019600000, 1, 4450, null, 1442023200000, 1, 64, 1] - - ["AR", 1442023200000, 2514, 4450, null, 1442026800000, 1, 64, 1] - - ["AR", 1442026800000, 644, 4450, 1442019600000, 1442030400000, 1, 64, 1] - - ["AR", 1442030400000, -3, 4450, 1442023200000, 1442034000000, 1, 64, 1] - - ["AR", 1442034000000, 212, 4450, 1442026800000, 1442037600000, 1, 64, 1] - - ["AR", 1442037600000, 81, 4450, 1442030400000, 1442055600000, 1, 64, 1] - - ["AR", 1442055600000, -54, 4450, 1442034000000, 1442059200000, 1, 64, 2] - - ["AR", 1442059200000, 210, 4450, 1442037600000, 1442062800000, 1, 64, 2] - - ["AR", 1442062800000, 29, 4450, 1442055600000, 1442066400000, 1, 64, 2] - - ["AR", 1442066400000, 0, 4450, 1442059200000, 1442070000000, 1, 64, 2] - - ["AR", 1442070000000, 377, 4450, 1442062800000, 1442077200000, 1, 64, 2] - - ["AR", 1442077200000, -591, 4450, 1442066400000, 1442080800000, 1, 64, 2] - - ["AR", 1442080800000, 1, 4450, 1442070000000, 1442084400000, 1, 64, 3] - - ["AR", 1442084400000, -5, 4450, 1442077200000, 1442091600000, 1, 64, 3] - - ["AR", 1442091600000, 340, 4450, 1442080800000, 1442095200000, 1, 64, 3] - - ["AR", 1442095200000, 630, 4450, 1442084400000, 1442098800000, 1, 64, 3] - - ["AR", 1442098800000, 64, 4450, 1442091600000, null, 1, 64, 3] - - ["AT", 1442052000000, 4793, 12047, null, 1442062800000, 4793, 89, 1] - - ["AT", 1442062800000, -155, 12047, null, 1442066400000, 4793, 89, 1] - - ["AT", 1442066400000, 0, 12047, 1442052000000, 1442070000000, 4793, 89, 1] - - ["AT", 1442070000000, 272, 12047, 1442062800000, 1442084400000, 4793, 89, 2] - - ["AT", 1442084400000, -2, 12047, 1442066400000, 1442088000000, 4793, 89, 2] - - ["AT", 1442088000000, 7050, 12047, 1442070000000, 1442091600000, 4793, 89, 3] - - ["AT", 1442091600000, 89, 12047, 1442084400000, null, 4793, 89, 3] - - ["AU", 1442016000000, 0, 2700, null, 1442019600000, 0, 518, 1] - - ["AU", 1442019600000, 253, 2700, null, 1442023200000, 0, 518, 1] - - ["AU", 1442023200000, 52, 2700, 1442016000000, 1442026800000, 0, 518, 1] - - ["AU", 1442026800000, 188, 2700, 1442019600000, 1442030400000, 0, 518, 1] - - ["AU", 1442030400000, -377, 2700, 1442023200000, 1442034000000, 0, 518, 1] - - ["AU", 1442034000000, 283, 2700, 1442026800000, 1442037600000, 0, 518, 1] - - ["AU", 1442037600000, 3, 2700, 1442030400000, 1442041200000, 0, 518, 1] - - ["AU", 1442041200000, 194, 2700, 1442034000000, 1442044800000, 0, 518, 2] - - ["AU", 1442044800000, 373, 2700, 1442037600000, 1442048400000, 0, 518, 2] - - ["AU", 1442048400000, 135, 2700, 1442041200000, 1442052000000, 0, 518, 2] - - ["AU", 1442052000000, -643, 2700, 1442044800000, 1442055600000, 0, 518, 2] - - ["AU", 1442055600000, 182, 2700, 1442048400000, 1442059200000, 0, 518, 2] - - ["AU", 1442059200000, 38, 2700, 1442052000000, 1442066400000, 0, 518, 2] - - ["AU", 1442066400000, -21, 2700, 1442055600000, 1442070000000, 0, 518, 3] - - ["AU", 1442070000000, -12, 2700, 1442059200000, 1442077200000, 0, 518, 3] - - ["AU", 1442077200000, 1, 2700, 1442066400000, 1442091600000, 0, 518, 3] - - ["AU", 1442091600000, 1138, 2700, 1442070000000, 1442095200000, 0, 518, 3] - - ["AU", 1442095200000, 395, 2700, 1442077200000, 1442098800000, 0, 518, 3] - - ["AU", 1442098800000, 518, 2700, 1442091600000, null, 0, 518, 3] - - ["BA", 1442048400000, -13, -178, null, 1442052000000, -13, -1, 1] - - ["BA", 1442052000000, 38, -178, null, 1442055600000, -13, -1, 1] - - ["BA", 1442055600000, -202, -178, 1442048400000, 1442084400000, -13, -1, 2] - - ["BA", 1442084400000, -1, -178, 1442052000000, null, -13, -1, 3] - - ["BD", 1442019600000, 0, 1106, null, 1442041200000, 0, -2, 1] - - ["BD", 1442041200000, 854, 1106, null, 1442066400000, 0, -2, 1] - - ["BD", 1442066400000, 76, 1106, 1442019600000, 1442073600000, 0, -2, 2] - - ["BD", 1442073600000, 103, 1106, 1442041200000, 1442077200000, 0, -2, 2] - - ["BD", 1442077200000, 75, 1106, 1442066400000, 1442091600000, 0, -2, 3] - - ["BD", 1442091600000, -2, 1106, 1442073600000, null, 0, -2, 3] - - ["BE", 1442030400000, -103, 795, null, 1442048400000, -103, 9, 1] - - ["BE", 1442048400000, 59, 795, null, 1442052000000, -103, 9, 1] - - ["BE", 1442052000000, -1, 795, 1442030400000, 1442055600000, -103, 9, 1] - - ["BE", 1442055600000, 233, 795, 1442048400000, 1442062800000, -103, 9, 1] - - ["BE", 1442062800000, 91, 795, 1442052000000, 1442066400000, -103, 9, 2] - - ["BE", 1442066400000, 136, 795, 1442055600000, 1442073600000, -103, 9, 2] - - ["BE", 1442073600000, 19, 795, 1442062800000, 1442080800000, -103, 9, 2] - - ["BE", 1442080800000, 1, 795, 1442066400000, 1442084400000, -103, 9, 2] - - ["BE", 1442084400000, 183, 795, 1442073600000, 1442088000000, -103, 9, 3] - - ["BE", 1442088000000, 67, 795, 1442080800000, 1442091600000, -103, 9, 3] - - ["BE", 1442091600000, 101, 795, 1442084400000, 1442098800000, -103, 9, 3] - - ["BE", 1442098800000, 9, 795, 1442088000000, null, -103, 9, 3] - - ["BG", 1442041200000, 9, 19592, null, 1442052000000, 9, 401, 1] - - ["BG", 1442052000000, 18936, 19592, null, 1442059200000, 9, 401, 1] - - ["BG", 1442059200000, 191, 19592, 1442041200000, 1442070000000, 9, 401, 2] - - ["BG", 1442070000000, 55, 19592, 1442052000000, 1442084400000, 9, 401, 2] - - ["BG", 1442084400000, 401, 19592, 1442059200000, null, 9, 401, 3] - - ["BH", 1442052000000, 44, 44, null, null, 44, 44, 1] - - ["BO", 1442080800000, 4, 4, null, 1442088000000, 4, -4, 1] - - ["BO", 1442088000000, 4, 4, null, 1442095200000, 4, -4, 2] - - ["BO", 1442095200000, -4, 4, 1442080800000, null, 4, -4, 3] - - ["BR", 1442016000000, -248, 8550, null, 1442019600000, -248, -645, 1] - - ["BR", 1442019600000, 372, 8550, null, 1442023200000, -248, -645, 1] - - ["BR", 1442023200000, 879, 8550, 1442016000000, 1442026800000, -248, -645, 1] - - ["BR", 1442026800000, 51, 8550, 1442019600000, 1442030400000, -248, -645, 1] - - ["BR", 1442030400000, 30, 8550, 1442023200000, 1442034000000, -248, -645, 1] - - ["BR", 1442034000000, 21, 8550, 1442026800000, 1442037600000, -248, -645, 1] - - ["BR", 1442037600000, 267, 8550, 1442030400000, 1442041200000, -248, -645, 1] - - ["BR", 1442041200000, 3, 8550, 1442034000000, 1442044800000, -248, -645, 1] - - ["BR", 1442044800000, 71, 8550, 1442037600000, 1442052000000, -248, -645, 2] - - ["BR", 1442052000000, 232, 8550, 1442041200000, 1442055600000, -248, -645, 2] - - ["BR", 1442055600000, 242, 8550, 1442044800000, 1442059200000, -248, -645, 2] - - ["BR", 1442059200000, 73, 8550, 1442052000000, 1442062800000, -248, -645, 2] - - ["BR", 1442062800000, 93, 8550, 1442055600000, 1442066400000, -248, -645, 2] - - ["BR", 1442066400000, 1034, 8550, 1442059200000, 1442070000000, -248, -645, 2] - - ["BR", 1442070000000, 536, 8550, 1442062800000, 1442073600000, -248, -645, 2] - - ["BR", 1442073600000, 2087, 8550, 1442066400000, 1442077200000, -248, -645, 2] - - ["BR", 1442077200000, 2253, 8550, 1442070000000, 1442080800000, -248, -645, 3] - - ["BR", 1442080800000, -267, 8550, 1442073600000, 1442084400000, -248, -645, 3] - - ["BR", 1442084400000, 492, 8550, 1442077200000, 1442088000000, -248, -645, 3] - - ["BR", 1442088000000, 215, 8550, 1442080800000, 1442091600000, -248, -645, 3] - - ["BR", 1442091600000, 11, 8550, 1442084400000, 1442095200000, -248, -645, 3] - - ["BR", 1442095200000, 748, 8550, 1442088000000, 1442098800000, -248, -645, 3] - - ["BR", 1442098800000, -645, 8550, 1442091600000, null, -248, -645, 3] - - ["BY", 1442055600000, 1, 2153, null, 1442059200000, 1, 33, 1] - - ["BY", 1442059200000, 1464, 2153, null, 1442073600000, 1, 33, 1] - - ["BY", 1442073600000, 596, 2153, 1442055600000, 1442077200000, 1, 33, 1] - - ["BY", 1442077200000, 30, 2153, 1442059200000, 1442080800000, 1, 33, 2] - - ["BY", 1442080800000, 28, 2153, 1442073600000, 1442084400000, 1, 33, 2] - - ["BY", 1442084400000, 1, 2153, 1442077200000, 1442088000000, 1, 33, 3] - - ["BY", 1442088000000, 33, 2153, 1442080800000, null, 1, 33, 3] - - ["CA", 1442016000000, -371, 8544, null, 1442019600000, -371, 164, 1] - - ["CA", 1442019600000, 2184, 8544, null, 1442023200000, -371, 164, 1] - - ["CA", 1442023200000, 286, 8544, 1442016000000, 1442026800000, -371, 164, 1] - - ["CA", 1442026800000, 2216, 8544, 1442019600000, 1442030400000, -371, 164, 1] - - ["CA", 1442030400000, -47, 8544, 1442023200000, 1442034000000, -371, 164, 1] - - ["CA", 1442034000000, 178, 8544, 1442026800000, 1442037600000, -371, 164, 1] - - ["CA", 1442037600000, -132, 8544, 1442030400000, 1442041200000, -371, 164, 1] - - ["CA", 1442041200000, 5, 8544, 1442034000000, 1442044800000, -371, 164, 1] - - ["CA", 1442044800000, 1, 8544, 1442037600000, 1442052000000, -371, 164, 2] - - ["CA", 1442052000000, 38, 8544, 1442041200000, 1442059200000, -371, 164, 2] - - ["CA", 1442059200000, 1036, 8544, 1442044800000, 1442062800000, -371, 164, 2] - - ["CA", 1442062800000, -367, 8544, 1442052000000, 1442066400000, -371, 164, 2] - - ["CA", 1442066400000, 307, 8544, 1442059200000, 1442070000000, -371, 164, 2] - - ["CA", 1442070000000, 185, 8544, 1442062800000, 1442073600000, -371, 164, 2] - - ["CA", 1442073600000, 86, 8544, 1442066400000, 1442077200000, -371, 164, 2] - - ["CA", 1442077200000, -282, 8544, 1442070000000, 1442080800000, -371, 164, 3] - - ["CA", 1442080800000, 481, 8544, 1442073600000, 1442084400000, -371, 164, 3] - - ["CA", 1442084400000, 44, 8544, 1442077200000, 1442088000000, -371, 164, 3] - - ["CA", 1442088000000, 35, 8544, 1442080800000, 1442091600000, -371, 164, 3] - - ["CA", 1442091600000, 2858, 8544, 1442084400000, 1442095200000, -371, 164, 3] - - ["CA", 1442095200000, -361, 8544, 1442088000000, 1442098800000, -371, 164, 3] - - ["CA", 1442098800000, 164, 8544, 1442091600000, null, -371, 164, 3] - - ["CH", 1442037600000, 59, 753, null, 1442041200000, 59, 67, 1] - - ["CH", 1442041200000, 198, 753, null, 1442044800000, 59, 67, 1] - - ["CH", 1442044800000, -54, 753, 1442037600000, 1442048400000, 59, 67, 1] - - ["CH", 1442048400000, 24, 753, 1442041200000, 1442052000000, 59, 67, 1] - - ["CH", 1442052000000, 47, 753, 1442044800000, 1442055600000, 59, 67, 2] - - ["CH", 1442055600000, 0, 753, 1442048400000, 1442062800000, 59, 67, 2] - - ["CH", 1442062800000, 22, 753, 1442052000000, 1442070000000, 59, 67, 2] - - ["CH", 1442070000000, 11, 753, 1442055600000, 1442073600000, 59, 67, 2] - - ["CH", 1442073600000, 360, 753, 1442062800000, 1442077200000, 59, 67, 3] - - ["CH", 1442077200000, 6, 753, 1442070000000, 1442084400000, 59, 67, 3] - - ["CH", 1442084400000, 13, 753, 1442073600000, 1442091600000, 59, 67, 3] - - ["CH", 1442091600000, 67, 753, 1442077200000, null, 59, 67, 3] - - ["CL", 1442016000000, 161, 533, null, 1442019600000, 161, 9, 1] - - ["CL", 1442019600000, -370, 533, null, 1442023200000, 161, 9, 1] - - ["CL", 1442023200000, 15, 533, 1442016000000, 1442030400000, 161, 9, 1] - - ["CL", 1442030400000, 40, 533, 1442019600000, 1442034000000, 161, 9, 1] - - ["CL", 1442034000000, -1, 533, 1442023200000, 1442037600000, 161, 9, 1] - - ["CL", 1442037600000, 2, 533, 1442030400000, 1442041200000, 161, 9, 1] - - ["CL", 1442041200000, -1, 533, 1442034000000, 1442052000000, 161, 9, 1] - - ["CL", 1442052000000, 390, 533, 1442037600000, 1442059200000, 161, 9, 2] - - ["CL", 1442059200000, -12, 533, 1442041200000, 1442062800000, 161, 9, 2] - - ["CL", 1442062800000, 17, 533, 1442052000000, 1442066400000, 161, 9, 2] - - ["CL", 1442066400000, -41, 533, 1442059200000, 1442070000000, 161, 9, 2] - - ["CL", 1442070000000, 13, 533, 1442062800000, 1442073600000, 161, 9, 2] - - ["CL", 1442073600000, 153, 533, 1442066400000, 1442077200000, 161, 9, 2] - - ["CL", 1442077200000, -15, 533, 1442070000000, 1442080800000, 161, 9, 2] - - ["CL", 1442080800000, 17, 533, 1442073600000, 1442084400000, 161, 9, 3] - - ["CL", 1442084400000, 126, 533, 1442077200000, 1442088000000, 161, 9, 3] - - ["CL", 1442088000000, 286, 533, 1442080800000, 1442091600000, 161, 9, 3] - - ["CL", 1442091600000, 20, 533, 1442084400000, 1442095200000, 161, 9, 3] - - ["CL", 1442095200000, -276, 533, 1442088000000, 1442098800000, 161, 9, 3] - - ["CL", 1442098800000, 9, 533, 1442091600000, null, 161, 9, 3] - - ["CN", 1442023200000, -13, 583, null, 1442026800000, -13, -1, 1] - - ["CN", 1442026800000, 154, 583, null, 1442037600000, -13, -1, 1] - - ["CN", 1442037600000, 98, 583, 1442023200000, 1442048400000, -13, -1, 1] - - ["CN", 1442048400000, 293, 583, 1442026800000, 1442052000000, -13, -1, 1] - - ["CN", 1442052000000, 0, 583, 1442037600000, 1442055600000, -13, -1, 2] - - ["CN", 1442055600000, 69, 583, 1442048400000, 1442059200000, -13, -1, 2] - - ["CN", 1442059200000, 8, 583, 1442052000000, 1442066400000, -13, -1, 2] - - ["CN", 1442066400000, -15, 583, 1442055600000, 1442080800000, -13, -1, 3] - - ["CN", 1442080800000, -10, 583, 1442059200000, 1442084400000, -13, -1, 3] - - ["CN", 1442084400000, -1, 583, 1442066400000, null, -13, -1, 3] - - ["CO", 1442016000000, 16, 59611, null, 1442019600000, 16, 83, 1] - - ["CO", 1442019600000, 12, 59611, null, 1442023200000, 16, 83, 1] - - ["CO", 1442023200000, 9, 59611, 1442016000000, 1442030400000, 16, 83, 1] - - ["CO", 1442030400000, 441, 59611, 1442019600000, 1442059200000, 16, 83, 1] - - ["CO", 1442059200000, 473, 59611, 1442023200000, 1442066400000, 16, 83, 1] - - ["CO", 1442066400000, 288, 59611, 1442030400000, 1442070000000, 16, 83, 2] - - ["CO", 1442070000000, -45, 59611, 1442059200000, 1442073600000, 16, 83, 2] - - ["CO", 1442073600000, 39860, 59611, 1442066400000, 1442077200000, 16, 83, 2] - - ["CO", 1442077200000, 581, 59611, 1442070000000, 1442080800000, 16, 83, 2] - - ["CO", 1442080800000, 25, 59611, 1442073600000, 1442084400000, 16, 83, 2] - - ["CO", 1442084400000, 51, 59611, 1442077200000, 1442088000000, 16, 83, 3] - - ["CO", 1442088000000, 17150, 59611, 1442080800000, 1442091600000, 16, 83, 3] - - ["CO", 1442091600000, 377, 59611, 1442084400000, 1442095200000, 16, 83, 3] - - ["CO", 1442095200000, 290, 59611, 1442088000000, 1442098800000, 16, 83, 3] - - ["CO", 1442098800000, 83, 59611, 1442091600000, null, 16, 83, 3] - - ["CR", 1442019600000, 62, 3241, null, 1442023200000, 62, 72, 1] - - ["CR", 1442023200000, 62, 3241, null, 1442026800000, 62, 72, 1] - - ["CR", 1442026800000, 140, 3241, 1442019600000, 1442030400000, 62, 72, 1] - - ["CR", 1442030400000, 2497, 3241, 1442023200000, 1442041200000, 62, 72, 2] - - ["CR", 1442041200000, 51, 3241, 1442026800000, 1442044800000, 62, 72, 2] - - ["CR", 1442044800000, 194, 3241, 1442030400000, 1442048400000, 62, 72, 2] - - ["CR", 1442048400000, 163, 3241, 1442041200000, 1442088000000, 62, 72, 3] - - ["CR", 1442088000000, 72, 3241, 1442044800000, null, 62, 72, 3] - - ["CZ", 1442026800000, -19, 3479, null, 1442034000000, -19, 2, 1] - - ["CZ", 1442034000000, 78, 3479, null, 1442037600000, -19, 2, 1] - - ["CZ", 1442037600000, 18, 3479, 1442026800000, 1442055600000, -19, 2, 1] - - ["CZ", 1442055600000, 1073, 3479, 1442034000000, 1442059200000, -19, 2, 1] - - ["CZ", 1442059200000, 21, 3479, 1442037600000, 1442062800000, -19, 2, 2] - - ["CZ", 1442062800000, 0, 3479, 1442055600000, 1442070000000, -19, 2, 2] - - ["CZ", 1442070000000, 168, 3479, 1442059200000, 1442073600000, -19, 2, 2] - - ["CZ", 1442073600000, 2051, 3479, 1442062800000, 1442077200000, -19, 2, 2] - - ["CZ", 1442077200000, 115, 3479, 1442070000000, 1442080800000, -19, 2, 3] - - ["CZ", 1442080800000, -28, 3479, 1442073600000, 1442098800000, -19, 2, 3] - - ["CZ", 1442098800000, 2, 3479, 1442077200000, null, -19, 2, 3] - - ["DE", 1442016000000, 167, 25583, null, 1442019600000, 167, 329, 1] - - ["DE", 1442019600000, 0, 25583, null, 1442023200000, 167, 329, 1] - - ["DE", 1442023200000, 64, 25583, 1442016000000, 1442030400000, 167, 329, 1] - - ["DE", 1442030400000, 373, 25583, 1442019600000, 1442034000000, 167, 329, 1] - - ["DE", 1442034000000, 358, 25583, 1442023200000, 1442037600000, 167, 329, 1] - - ["DE", 1442037600000, 544, 25583, 1442030400000, 1442041200000, 167, 329, 1] - - ["DE", 1442041200000, 197, 25583, 1442034000000, 1442044800000, 167, 329, 1] - - ["DE", 1442044800000, 979, 25583, 1442037600000, 1442048400000, 167, 329, 1] - - ["DE", 1442048400000, 811, 25583, 1442041200000, 1442052000000, 167, 329, 2] - - ["DE", 1442052000000, 1600, 25583, 1442044800000, 1442055600000, 167, 329, 2] - - ["DE", 1442055600000, 1523, 25583, 1442048400000, 1442059200000, 167, 329, 2] - - ["DE", 1442059200000, 289, 25583, 1442052000000, 1442062800000, 167, 329, 2] - - ["DE", 1442062800000, 283, 25583, 1442055600000, 1442066400000, 167, 329, 2] - - ["DE", 1442066400000, 1577, 25583, 1442059200000, 1442070000000, 167, 329, 2] - - ["DE", 1442070000000, 1666, 25583, 1442062800000, 1442073600000, 167, 329, 2] - - ["DE", 1442073600000, 6075, 25583, 1442066400000, 1442077200000, 167, 329, 2] - - ["DE", 1442077200000, 2188, 25583, 1442070000000, 1442080800000, 167, 329, 3] - - ["DE", 1442080800000, 1133, 25583, 1442073600000, 1442084400000, 167, 329, 3] - - ["DE", 1442084400000, -125, 25583, 1442077200000, 1442088000000, 167, 329, 3] - - ["DE", 1442088000000, 190, 25583, 1442080800000, 1442091600000, 167, 329, 3] - - ["DE", 1442091600000, 4355, 25583, 1442084400000, 1442095200000, 167, 329, 3] - - ["DE", 1442095200000, 1007, 25583, 1442088000000, 1442098800000, 167, 329, 3] - - ["DE", 1442098800000, 329, 25583, 1442091600000, null, 167, 329, 3] - - ["DK", 1442037600000, 10, 594, null, 1442044800000, 10, 0, 1] - - ["DK", 1442044800000, 36, 594, null, 1442048400000, 10, 0, 1] - - ["DK", 1442048400000, -5, 594, 1442037600000, 1442055600000, 10, 0, 1] - - ["DK", 1442055600000, 42, 594, 1442044800000, 1442059200000, 10, 0, 1] - - ["DK", 1442059200000, 0, 594, 1442048400000, 1442062800000, 10, 0, 2] - - ["DK", 1442062800000, 1, 594, 1442055600000, 1442066400000, 10, 0, 2] - - ["DK", 1442066400000, 416, 594, 1442059200000, 1442077200000, 10, 0, 2] - - ["DK", 1442077200000, -9, 594, 1442062800000, 1442080800000, 10, 0, 2] - - ["DK", 1442080800000, 61, 594, 1442066400000, 1442084400000, 10, 0, 3] - - ["DK", 1442084400000, -97, 594, 1442077200000, 1442091600000, 10, 0, 3] - - ["DK", 1442091600000, 139, 594, 1442080800000, 1442095200000, 10, 0, 3] - - ["DK", 1442095200000, 0, 594, 1442084400000, null, 10, 0, 3] - - ["DO", 1442023200000, 8, 264, null, 1442066400000, 8, 13, 1] - - ["DO", 1442066400000, 35, 264, null, 1442073600000, 8, 13, 1] - - ["DO", 1442073600000, 200, 264, 1442023200000, 1442084400000, 8, 13, 2] - - ["DO", 1442084400000, 8, 264, 1442066400000, 1442095200000, 8, 13, 2] - - ["DO", 1442095200000, 13, 264, 1442073600000, null, 8, 13, 3] - - ["DZ", 1442077200000, -1, -1, null, null, -1, -1, 1] - - ["EC", 1442019600000, 29, 232, null, 1442023200000, 29, 10, 1] - - ["EC", 1442023200000, -9, 232, null, 1442030400000, 29, 10, 1] - - ["EC", 1442030400000, 0, 232, 1442019600000, 1442077200000, 29, 10, 2] - - ["EC", 1442077200000, -366, 232, 1442023200000, 1442084400000, 29, 10, 2] - - ["EC", 1442084400000, 568, 232, 1442030400000, 1442095200000, 29, 10, 3] - - ["EC", 1442095200000, 10, 232, 1442077200000, null, 29, 10, 3] - - ["EE", 1442041200000, 37, 18, null, 1442044800000, 37, -19, 1] - - ["EE", 1442044800000, -19, 18, null, null, 37, -19, 2] - - ["EG", 1442026800000, 16, 170, null, 1442055600000, 16, 27, 1] - - ["EG", 1442055600000, 14, 170, null, 1442062800000, 16, 27, 1] - - ["EG", 1442062800000, 112, 170, 1442026800000, 1442073600000, 16, 27, 2] - - ["EG", 1442073600000, 1, 170, 1442055600000, 1442091600000, 16, 27, 2] - - ["EG", 1442091600000, 27, 170, 1442062800000, null, 16, 27, 3] - - ["ES", 1442019600000, 103, 7449, null, 1442023200000, 103, 458, 1] - - ["ES", 1442023200000, -5, 7449, null, 1442034000000, 103, 458, 1] - - ["ES", 1442034000000, -52, 7449, 1442019600000, 1442037600000, 103, 458, 1] - - ["ES", 1442037600000, 3, 7449, 1442023200000, 1442041200000, 103, 458, 1] - - ["ES", 1442041200000, 118, 7449, 1442034000000, 1442044800000, 103, 458, 1] - - ["ES", 1442044800000, -169, 7449, 1442037600000, 1442048400000, 103, 458, 1] - - ["ES", 1442048400000, 158, 7449, 1442041200000, 1442052000000, 103, 458, 1] - - ["ES", 1442052000000, -4, 7449, 1442044800000, 1442055600000, 103, 458, 2] - - ["ES", 1442055600000, 495, 7449, 1442048400000, 1442059200000, 103, 458, 2] - - ["ES", 1442059200000, 1086, 7449, 1442052000000, 1442062800000, 103, 458, 2] - - ["ES", 1442062800000, -71, 7449, 1442055600000, 1442066400000, 103, 458, 2] - - ["ES", 1442066400000, 461, 7449, 1442059200000, 1442070000000, 103, 458, 2] - - ["ES", 1442070000000, 61, 7449, 1442062800000, 1442073600000, 103, 458, 2] - - ["ES", 1442073600000, 154, 7449, 1442066400000, 1442077200000, 103, 458, 2] - - ["ES", 1442077200000, 1240, 7449, 1442070000000, 1442084400000, 103, 458, 3] - - ["ES", 1442084400000, 337, 7449, 1442073600000, 1442088000000, 103, 458, 3] - - ["ES", 1442088000000, -130, 7449, 1442077200000, 1442091600000, 103, 458, 3] - - ["ES", 1442091600000, 700, 7449, 1442084400000, 1442095200000, 103, 458, 3] - - ["ES", 1442095200000, 2506, 7449, 1442088000000, 1442098800000, 103, 458, 3] - - ["ES", 1442098800000, 458, 7449, 1442091600000, null, 103, 458, 3] - - ["FI", 1442030400000, 1491, 3579, null, 1442037600000, 1491, 69, 1] - - ["FI", 1442037600000, 14, 3579, null, 1442048400000, 1491, 69, 1] - - ["FI", 1442048400000, 12, 3579, 1442030400000, 1442052000000, 1491, 69, 1] - - ["FI", 1442052000000, 186, 3579, 1442037600000, 1442059200000, 1491, 69, 1] - - ["FI", 1442059200000, 407, 3579, 1442048400000, 1442062800000, 1491, 69, 2] - - ["FI", 1442062800000, 19, 3579, 1442052000000, 1442066400000, 1491, 69, 2] - - ["FI", 1442066400000, 183, 3579, 1442059200000, 1442073600000, 1491, 69, 2] - - ["FI", 1442073600000, -1, 3579, 1442062800000, 1442077200000, 1491, 69, 2] - - ["FI", 1442077200000, 200, 3579, 1442066400000, 1442080800000, 1491, 69, 3] - - ["FI", 1442080800000, 104, 3579, 1442073600000, 1442084400000, 1491, 69, 3] - - ["FI", 1442084400000, 895, 3579, 1442077200000, 1442095200000, 1491, 69, 3] - - ["FI", 1442095200000, 69, 3579, 1442080800000, null, 1491, 69, 3] - - ["FR", 1442016000000, -1, 37281, null, 1442019600000, -1, 136, 1] - - ["FR", 1442019600000, 585, 37281, null, 1442023200000, -1, 136, 1] - - ["FR", 1442023200000, 628, 37281, 1442016000000, 1442026800000, -1, 136, 1] - - ["FR", 1442026800000, 86, 37281, 1442019600000, 1442034000000, -1, 136, 1] - - ["FR", 1442034000000, 476, 37281, 1442023200000, 1442037600000, -1, 136, 1] - - ["FR", 1442037600000, 4174, 37281, 1442026800000, 1442041200000, -1, 136, 1] - - ["FR", 1442041200000, 604, 37281, 1442034000000, 1442044800000, -1, 136, 1] - - ["FR", 1442044800000, 172, 37281, 1442037600000, 1442048400000, -1, 136, 1] - - ["FR", 1442048400000, 3027, 37281, 1442041200000, 1442052000000, -1, 136, 2] - - ["FR", 1442052000000, 637, 37281, 1442044800000, 1442055600000, -1, 136, 2] - - ["FR", 1442055600000, 463, 37281, 1442048400000, 1442059200000, -1, 136, 2] - - ["FR", 1442059200000, 4650, 37281, 1442052000000, 1442062800000, -1, 136, 2] - - ["FR", 1442062800000, 5676, 37281, 1442055600000, 1442066400000, -1, 136, 2] - - ["FR", 1442066400000, 2516, 37281, 1442059200000, 1442070000000, -1, 136, 2] - - ["FR", 1442070000000, 474, 37281, 1442062800000, 1442073600000, -1, 136, 2] - - ["FR", 1442073600000, 3522, 37281, 1442066400000, 1442077200000, -1, 136, 2] - - ["FR", 1442077200000, -444, 37281, 1442070000000, 1442080800000, -1, 136, 3] - - ["FR", 1442080800000, 557, 37281, 1442073600000, 1442084400000, -1, 136, 3] - - ["FR", 1442084400000, 6643, 37281, 1442077200000, 1442088000000, -1, 136, 3] - - ["FR", 1442088000000, 1872, 37281, 1442080800000, 1442091600000, -1, 136, 3] - - ["FR", 1442091600000, 741, 37281, 1442084400000, 1442095200000, -1, 136, 3] - - ["FR", 1442095200000, 87, 37281, 1442088000000, 1442098800000, -1, 136, 3] - - ["FR", 1442098800000, 136, 37281, 1442091600000, null, -1, 136, 3] - - ["GB", 1442016000000, -44, 35857, null, 1442019600000, -44, 49, 1] - - ["GB", 1442019600000, 54, 35857, null, 1442023200000, -44, 49, 1] - - ["GB", 1442023200000, 1816, 35857, 1442016000000, 1442026800000, -44, 49, 1] - - ["GB", 1442026800000, 339, 35857, 1442019600000, 1442030400000, -44, 49, 1] - - ["GB", 1442030400000, 2524, 35857, 1442023200000, 1442034000000, -44, 49, 1] - - ["GB", 1442034000000, -12, 35857, 1442026800000, 1442037600000, -44, 49, 1] - - ["GB", 1442037600000, 544, 35857, 1442030400000, 1442041200000, -44, 49, 1] - - ["GB", 1442041200000, 42, 35857, 1442034000000, 1442044800000, -44, 49, 1] - - ["GB", 1442044800000, 32, 35857, 1442037600000, 1442048400000, -44, 49, 2] - - ["GB", 1442048400000, 740, 35857, 1442041200000, 1442052000000, -44, 49, 2] - - ["GB", 1442052000000, 168, 35857, 1442044800000, 1442055600000, -44, 49, 2] - - ["GB", 1442055600000, 453, 35857, 1442048400000, 1442059200000, -44, 49, 2] - - ["GB", 1442059200000, 16111, 35857, 1442052000000, 1442062800000, -44, 49, 2] - - ["GB", 1442062800000, 5743, 35857, 1442055600000, 1442066400000, -44, 49, 2] - - ["GB", 1442066400000, 671, 35857, 1442059200000, 1442070000000, -44, 49, 2] - - ["GB", 1442070000000, 374, 35857, 1442062800000, 1442073600000, -44, 49, 2] - - ["GB", 1442073600000, 648, 35857, 1442066400000, 1442077200000, -44, 49, 3] - - ["GB", 1442077200000, 1135, 35857, 1442070000000, 1442080800000, -44, 49, 3] - - ["GB", 1442080800000, 1444, 35857, 1442073600000, 1442084400000, -44, 49, 3] - - ["GB", 1442084400000, 384, 35857, 1442077200000, 1442088000000, -44, 49, 3] - - ["GB", 1442088000000, 1593, 35857, 1442080800000, 1442091600000, -44, 49, 3] - - ["GB", 1442091600000, 811, 35857, 1442084400000, 1442095200000, -44, 49, 3] - - ["GB", 1442095200000, 238, 35857, 1442088000000, 1442098800000, -44, 49, 3] - - ["GB", 1442098800000, 49, 35857, 1442091600000, null, -44, 49, 3] - - ["GE", 1442044800000, -21, -140, null, 1442052000000, -21, -27, 1] - - ["GE", 1442052000000, -108, -140, null, 1442062800000, -21, -27, 1] - - ["GE", 1442062800000, 16, -140, 1442044800000, 1442080800000, -21, -27, 2] - - ["GE", 1442080800000, -27, -140, 1442052000000, null, -21, -27, 3] - - ["GH", 1442088000000, 0, 0, null, null, 0, 0, 1] - - ["GR", 1442019600000, 82, 149, null, 1442034000000, 82, 123, 1] - - ["GR", 1442034000000, 0, 149, null, 1442041200000, 82, 123, 1] - - ["GR", 1442041200000, 7, 149, 1442019600000, 1442048400000, 82, 123, 1] - - ["GR", 1442048400000, -26, 149, 1442034000000, 1442062800000, 82, 123, 1] - - ["GR", 1442062800000, 8, 149, 1442041200000, 1442070000000, 82, 123, 2] - - ["GR", 1442070000000, 2, 149, 1442048400000, 1442073600000, 82, 123, 2] - - ["GR", 1442073600000, -314, 149, 1442062800000, 1442080800000, 82, 123, 2] - - ["GR", 1442080800000, 88, 149, 1442070000000, 1442084400000, 82, 123, 3] - - ["GR", 1442084400000, 179, 149, 1442073600000, 1442091600000, 82, 123, 3] - - ["GR", 1442091600000, 123, 149, 1442080800000, null, 82, 123, 3] - - ["GT", 1442023200000, -167, 7, null, 1442026800000, -167, 1, 1] - - ["GT", 1442026800000, 173, 7, null, 1442098800000, -167, 1, 2] - - ["GT", 1442098800000, 1, 7, 1442023200000, null, -167, 1, 3] - - ["HK", 1442019600000, -113, 10743, null, 1442023200000, -113, -1, 1] - - ["HK", 1442023200000, 2414, 10743, null, 1442026800000, -113, -1, 1] - - ["HK", 1442026800000, -211, 10743, 1442019600000, 1442030400000, -113, -1, 1] - - ["HK", 1442030400000, 157, 10743, 1442023200000, 1442034000000, -113, -1, 1] - - ["HK", 1442034000000, 1137, 10743, 1442026800000, 1442037600000, -113, -1, 1] - - ["HK", 1442037600000, 636, 10743, 1442030400000, 1442041200000, -113, -1, 1] - - ["HK", 1442041200000, -15, 10743, 1442034000000, 1442044800000, -113, -1, 1] - - ["HK", 1442044800000, 21, 10743, 1442037600000, 1442048400000, -113, -1, 2] - - ["HK", 1442048400000, 1, 10743, 1442041200000, 1442052000000, -113, -1, 2] - - ["HK", 1442052000000, 15, 10743, 1442044800000, 1442055600000, -113, -1, 2] - - ["HK", 1442055600000, 804, 10743, 1442048400000, 1442059200000, -113, -1, 2] - - ["HK", 1442059200000, 2, 10743, 1442052000000, 1442062800000, -113, -1, 2] - - ["HK", 1442062800000, 1, 10743, 1442055600000, 1442066400000, -113, -1, 2] - - ["HK", 1442066400000, 39, 10743, 1442059200000, 1442070000000, -113, -1, 3] - - ["HK", 1442070000000, 314, 10743, 1442062800000, 1442073600000, -113, -1, 3] - - ["HK", 1442073600000, 5545, 10743, 1442066400000, 1442080800000, -113, -1, 3] - - ["HK", 1442080800000, 0, 10743, 1442070000000, 1442091600000, -113, -1, 3] - - ["HK", 1442091600000, -3, 10743, 1442073600000, 1442095200000, -113, -1, 3] - - ["HK", 1442095200000, -1, 10743, 1442080800000, null, -113, -1, 3] - - ["HN", 1442026800000, -1, -1, null, null, -1, -1, 1] - - ["HR", 1442070000000, 32, 382, null, 1442073600000, 32, 82, 1] - - ["HR", 1442073600000, 0, 382, null, 1442077200000, 32, 82, 1] - - ["HR", 1442077200000, 58, 382, 1442070000000, 1442080800000, 32, 82, 2] - - ["HR", 1442080800000, 220, 382, 1442073600000, 1442084400000, 32, 82, 2] - - ["HR", 1442084400000, -10, 382, 1442077200000, 1442088000000, 32, 82, 3] - - ["HR", 1442088000000, 82, 382, 1442080800000, null, 32, 82, 3] - - ["HU", 1442019600000, 46, 2116, null, 1442037600000, 46, 110, 1] - - ["HU", 1442037600000, 197, 2116, null, 1442041200000, 46, 110, 1] - - ["HU", 1442041200000, 91, 2116, 1442019600000, 1442044800000, 46, 110, 1] - - ["HU", 1442044800000, 547, 2116, 1442037600000, 1442048400000, 46, 110, 1] - - ["HU", 1442048400000, 499, 2116, 1442041200000, 1442055600000, 46, 110, 1] - - ["HU", 1442055600000, -2, 2116, 1442044800000, 1442062800000, 46, 110, 2] - - ["HU", 1442062800000, 50, 2116, 1442048400000, 1442080800000, 46, 110, 2] - - ["HU", 1442080800000, 242, 2116, 1442055600000, 1442084400000, 46, 110, 2] - - ["HU", 1442084400000, 141, 2116, 1442062800000, 1442088000000, 46, 110, 2] - - ["HU", 1442088000000, -71, 2116, 1442080800000, 1442091600000, 46, 110, 3] - - ["HU", 1442091600000, -5, 2116, 1442084400000, 1442095200000, 46, 110, 3] - - ["HU", 1442095200000, 271, 2116, 1442088000000, 1442098800000, 46, 110, 3] - - ["HU", 1442098800000, 110, 2116, 1442091600000, null, 46, 110, 3] - - ["ID", 1442023200000, 106, -255, null, 1442026800000, 106, 13, 1] - - ["ID", 1442026800000, -416, -255, null, 1442030400000, 106, 13, 1] - - ["ID", 1442030400000, 279, -255, 1442023200000, 1442034000000, 106, 13, 1] - - ["ID", 1442034000000, 19, -255, 1442026800000, 1442037600000, 106, 13, 1] - - ["ID", 1442037600000, 14, -255, 1442030400000, 1442041200000, 106, 13, 1] - - ["ID", 1442041200000, 2, -255, 1442034000000, 1442044800000, 106, 13, 2] - - ["ID", 1442044800000, -388, -255, 1442037600000, 1442055600000, 106, 13, 2] - - ["ID", 1442055600000, 16, -255, 1442041200000, 1442059200000, 106, 13, 2] - - ["ID", 1442059200000, 17, -255, 1442044800000, 1442070000000, 106, 13, 2] - - ["ID", 1442070000000, 42, -255, 1442055600000, 1442091600000, 106, 13, 3] - - ["ID", 1442091600000, 21, -255, 1442059200000, 1442095200000, 106, 13, 3] - - ["ID", 1442095200000, 20, -255, 1442070000000, 1442098800000, 106, 13, 3] - - ["ID", 1442098800000, 13, -255, 1442091600000, null, 106, 13, 3] - - ["IE", 1442026800000, 1, 2142, null, 1442030400000, 1, -71, 1] - - ["IE", 1442030400000, 1, 2142, null, 1442048400000, 1, -71, 1] - - ["IE", 1442048400000, 27, 2142, 1442026800000, 1442066400000, 1, -71, 1] - - ["IE", 1442066400000, 1062, 2142, 1442030400000, 1442070000000, 1, -71, 2] - - ["IE", 1442070000000, -100, 2142, 1442048400000, 1442077200000, 1, -71, 2] - - ["IE", 1442077200000, 403, 2142, 1442066400000, 1442084400000, 1, -71, 2] - - ["IE", 1442084400000, 819, 2142, 1442070000000, 1442091600000, 1, -71, 3] - - ["IE", 1442091600000, -71, 2142, 1442077200000, null, 1, -71, 3] - - ["IL", 1442041200000, 35, 6617, null, 1442044800000, 35, 3, 1] - - ["IL", 1442044800000, 218, 6617, null, 1442048400000, 35, 3, 1] - - ["IL", 1442048400000, 25, 6617, 1442041200000, 1442052000000, 35, 3, 1] - - ["IL", 1442052000000, 2745, 6617, 1442044800000, 1442055600000, 35, 3, 1] - - ["IL", 1442055600000, 4, 6617, 1442048400000, 1442059200000, 35, 3, 1] - - ["IL", 1442059200000, 1205, 6617, 1442052000000, 1442062800000, 35, 3, 1] - - ["IL", 1442062800000, 180, 6617, 1442055600000, 1442066400000, 35, 3, 2] - - ["IL", 1442066400000, 3, 6617, 1442059200000, 1442070000000, 35, 3, 2] - - ["IL", 1442070000000, 49, 6617, 1442062800000, 1442073600000, 35, 3, 2] - - ["IL", 1442073600000, 31, 6617, 1442066400000, 1442077200000, 35, 3, 2] - - ["IL", 1442077200000, 187, 6617, 1442070000000, 1442080800000, 35, 3, 2] - - ["IL", 1442080800000, 88, 6617, 1442073600000, 1442084400000, 35, 3, 3] - - ["IL", 1442084400000, 1137, 6617, 1442077200000, 1442091600000, 35, 3, 3] - - ["IL", 1442091600000, 707, 6617, 1442080800000, 1442095200000, 35, 3, 3] - - ["IL", 1442095200000, 0, 6617, 1442084400000, 1442098800000, 35, 3, 3] - - ["IL", 1442098800000, 3, 6617, 1442091600000, null, 35, 3, 3] - - ["IN", 1442016000000, 1, 29166, null, 1442019600000, 1, 4, 1] - - ["IN", 1442019600000, 38, 29166, null, 1442023200000, 1, 4, 1] - - ["IN", 1442023200000, -142, 29166, 1442016000000, 1442026800000, 1, 4, 1] - - ["IN", 1442026800000, 974, 29166, 1442019600000, 1442030400000, 1, 4, 1] - - ["IN", 1442030400000, 1448, 29166, 1442023200000, 1442034000000, 1, 4, 1] - - ["IN", 1442034000000, 1350, 29166, 1442026800000, 1442037600000, 1, 4, 1] - - ["IN", 1442037600000, 135, 29166, 1442030400000, 1442041200000, 1, 4, 1] - - ["IN", 1442041200000, 80, 29166, 1442034000000, 1442044800000, 1, 4, 1] - - ["IN", 1442044800000, 2677, 29166, 1442037600000, 1442048400000, 1, 4, 2] - - ["IN", 1442048400000, 262, 29166, 1442041200000, 1442052000000, 1, 4, 2] - - ["IN", 1442052000000, 534, 29166, 1442044800000, 1442055600000, 1, 4, 2] - - ["IN", 1442055600000, 166, 29166, 1442048400000, 1442059200000, 1, 4, 2] - - ["IN", 1442059200000, 708, 29166, 1442052000000, 1442062800000, 1, 4, 2] - - ["IN", 1442062800000, 1547, 29166, 1442055600000, 1442066400000, 1, 4, 2] - - ["IN", 1442066400000, 116, 29166, 1442059200000, 1442070000000, 1, 4, 2] - - ["IN", 1442070000000, 12091, 29166, 1442062800000, 1442073600000, 1, 4, 3] - - ["IN", 1442073600000, 1170, 29166, 1442066400000, 1442077200000, 1, 4, 3] - - ["IN", 1442077200000, 5699, 29166, 1442070000000, 1442080800000, 1, 4, 3] - - ["IN", 1442080800000, 0, 29166, 1442073600000, 1442084400000, 1, 4, 3] - - ["IN", 1442084400000, 187, 29166, 1442077200000, 1442088000000, 1, 4, 3] - - ["IN", 1442088000000, 121, 29166, 1442080800000, 1442095200000, 1, 4, 3] - - ["IN", 1442095200000, 4, 29166, 1442084400000, null, 1, 4, 3] - - ["IQ", 1442041200000, -1, 3, null, 1442044800000, -1, -2, 1] - - ["IQ", 1442044800000, 6, 3, null, 1442052000000, -1, -2, 1] - - ["IQ", 1442052000000, 0, 3, 1442041200000, 1442095200000, -1, -2, 2] - - ["IQ", 1442095200000, -2, 3, 1442044800000, null, -1, -2, 3] - - ["IR", 1442026800000, 0, 2698, null, 1442030400000, 0, 0, 1] - - ["IR", 1442030400000, 375, 2698, null, 1442034000000, 0, 0, 1] - - ["IR", 1442034000000, -8, 2698, 1442026800000, 1442041200000, 0, 0, 1] - - ["IR", 1442041200000, -79, 2698, 1442030400000, 1442044800000, 0, 0, 1] - - ["IR", 1442044800000, 306, 2698, 1442034000000, 1442052000000, 0, 0, 1] - - ["IR", 1442052000000, 155, 2698, 1442041200000, 1442055600000, 0, 0, 2] - - ["IR", 1442055600000, -124, 2698, 1442044800000, 1442059200000, 0, 0, 2] - - ["IR", 1442059200000, 1455, 2698, 1442052000000, 1442073600000, 0, 0, 2] - - ["IR", 1442073600000, -193, 2698, 1442055600000, 1442077200000, 0, 0, 2] - - ["IR", 1442077200000, -34, 2698, 1442059200000, 1442080800000, 0, 0, 3] - - ["IR", 1442080800000, 131, 2698, 1442073600000, 1442088000000, 0, 0, 3] - - ["IR", 1442088000000, 714, 2698, 1442077200000, 1442091600000, 0, 0, 3] - - ["IR", 1442091600000, 0, 2698, 1442080800000, null, 0, 0, 3] - - ["IT", 1442016000000, 0, 39091, null, 1442019600000, 0, 565, 1] - - ["IT", 1442019600000, 183, 39091, null, 1442023200000, 0, 565, 1] - - ["IT", 1442023200000, 111, 39091, 1442016000000, 1442026800000, 0, 565, 1] - - ["IT", 1442026800000, 222, 39091, 1442019600000, 1442030400000, 0, 565, 1] - - ["IT", 1442030400000, -17, 39091, 1442023200000, 1442034000000, 0, 565, 1] - - ["IT", 1442034000000, 1006, 39091, 1442026800000, 1442037600000, 0, 565, 1] - - ["IT", 1442037600000, -9, 39091, 1442030400000, 1442041200000, 0, 565, 1] - - ["IT", 1442041200000, 20, 39091, 1442034000000, 1442044800000, 0, 565, 1] - - ["IT", 1442044800000, 1483, 39091, 1442037600000, 1442048400000, 0, 565, 2] - - ["IT", 1442048400000, 676, 39091, 1442041200000, 1442052000000, 0, 565, 2] - - ["IT", 1442052000000, 1880, 39091, 1442044800000, 1442055600000, 0, 565, 2] - - ["IT", 1442055600000, 6240, 39091, 1442048400000, 1442059200000, 0, 565, 2] - - ["IT", 1442059200000, 542, 39091, 1442052000000, 1442062800000, 0, 565, 2] - - ["IT", 1442062800000, 1938, 39091, 1442055600000, 1442066400000, 0, 565, 2] - - ["IT", 1442066400000, 4155, 39091, 1442059200000, 1442070000000, 0, 565, 2] - - ["IT", 1442070000000, 81, 39091, 1442062800000, 1442073600000, 0, 565, 2] - - ["IT", 1442073600000, 2586, 39091, 1442066400000, 1442077200000, 0, 565, 3] - - ["IT", 1442077200000, 2188, 39091, 1442070000000, 1442080800000, 0, 565, 3] - - ["IT", 1442080800000, 5544, 39091, 1442073600000, 1442084400000, 0, 565, 3] - - ["IT", 1442084400000, 2660, 39091, 1442077200000, 1442088000000, 0, 565, 3] - - ["IT", 1442088000000, 3746, 39091, 1442080800000, 1442091600000, 0, 565, 3] - - ["IT", 1442091600000, 351, 39091, 1442084400000, 1442095200000, 0, 565, 3] - - ["IT", 1442095200000, 2940, 39091, 1442088000000, 1442098800000, 0, 565, 3] - - ["IT", 1442098800000, 565, 39091, 1442091600000, null, 0, 565, 3] - - ["JM", 1442070000000, 30, 30, null, null, 30, 30, 1] - - ["JO", 1442055600000, -2, 2, null, 1442059200000, -2, 4, 1] - - ["JO", 1442059200000, 0, 2, null, 1442080800000, -2, 4, 2] - - ["JO", 1442080800000, 4, 2, 1442055600000, null, -2, 4, 3] - - ["JP", 1442016000000, -113, 20378, null, 1442019600000, -113, -6, 1] - - ["JP", 1442019600000, 2002, 20378, null, 1442023200000, -113, -6, 1] - - ["JP", 1442023200000, 1959, 20378, 1442016000000, 1442026800000, -113, -6, 1] - - ["JP", 1442026800000, 1035, 20378, 1442019600000, 1442030400000, -113, -6, 1] - - ["JP", 1442030400000, 805, 20378, 1442023200000, 1442034000000, -113, -6, 1] - - ["JP", 1442034000000, 910, 20378, 1442026800000, 1442037600000, -113, -6, 1] - - ["JP", 1442037600000, 2181, 20378, 1442030400000, 1442041200000, -113, -6, 1] - - ["JP", 1442041200000, 1373, 20378, 1442034000000, 1442044800000, -113, -6, 1] - - ["JP", 1442044800000, 1569, 20378, 1442037600000, 1442048400000, -113, -6, 2] - - ["JP", 1442048400000, 1981, 20378, 1442041200000, 1442052000000, -113, -6, 2] - - ["JP", 1442052000000, 2789, 20378, 1442044800000, 1442055600000, -113, -6, 2] - - ["JP", 1442055600000, 998, 20378, 1442048400000, 1442059200000, -113, -6, 2] - - ["JP", 1442059200000, -85, 20378, 1442052000000, 1442062800000, -113, -6, 2] - - ["JP", 1442062800000, 803, 20378, 1442055600000, 1442066400000, -113, -6, 2] - - ["JP", 1442066400000, 167, 20378, 1442059200000, 1442070000000, -113, -6, 2] - - ["JP", 1442070000000, 79, 20378, 1442062800000, 1442073600000, -113, -6, 2] - - ["JP", 1442073600000, 1162, 20378, 1442066400000, 1442077200000, -113, -6, 3] - - ["JP", 1442077200000, 51, 20378, 1442070000000, 1442080800000, -113, -6, 3] - - ["JP", 1442080800000, 420, 20378, 1442073600000, 1442084400000, -113, -6, 3] - - ["JP", 1442084400000, 13, 20378, 1442077200000, 1442088000000, -113, -6, 3] - - ["JP", 1442088000000, 57, 20378, 1442080800000, 1442091600000, -113, -6, 3] - - ["JP", 1442091600000, 228, 20378, 1442084400000, 1442095200000, -113, -6, 3] - - ["JP", 1442095200000, 0, 20378, 1442088000000, 1442098800000, -113, -6, 3] - - ["JP", 1442098800000, -6, 20378, 1442091600000, null, -113, -6, 3] - - ["KE", 1442044800000, -1, -1, null, null, -1, -1, 1] - - ["KG", 1442073600000, 6, 6, null, null, 6, 6, 1] - - ["KR", 1442016000000, 1024, 13597, null, 1442019600000, 1024, -36, 1] - - ["KR", 1442019600000, 445, 13597, null, 1442023200000, 1024, -36, 1] - - ["KR", 1442023200000, 319, 13597, 1442016000000, 1442026800000, 1024, -36, 1] - - ["KR", 1442026800000, -179, 13597, 1442019600000, 1442030400000, 1024, -36, 1] - - ["KR", 1442030400000, 1035, 13597, 1442023200000, 1442034000000, 1024, -36, 1] - - ["KR", 1442034000000, 434, 13597, 1442026800000, 1442037600000, 1024, -36, 1] - - ["KR", 1442037600000, 26, 13597, 1442030400000, 1442041200000, 1024, -36, 1] - - ["KR", 1442041200000, 20, 13597, 1442034000000, 1442044800000, 1024, -36, 1] - - ["KR", 1442044800000, 829, 13597, 1442037600000, 1442048400000, 1024, -36, 2] - - ["KR", 1442048400000, -374, 13597, 1442041200000, 1442052000000, 1024, -36, 2] - - ["KR", 1442052000000, -3, 13597, 1442044800000, 1442055600000, 1024, -36, 2] - - ["KR", 1442055600000, 3640, 13597, 1442048400000, 1442059200000, 1024, -36, 2] - - ["KR", 1442059200000, 208, 13597, 1442052000000, 1442062800000, 1024, -36, 2] - - ["KR", 1442062800000, 1096, 13597, 1442055600000, 1442066400000, 1024, -36, 2] - - ["KR", 1442066400000, 3299, 13597, 1442059200000, 1442070000000, 1024, -36, 2] - - ["KR", 1442070000000, 222, 13597, 1442062800000, 1442077200000, 1024, -36, 3] - - ["KR", 1442077200000, -40, 13597, 1442066400000, 1442080800000, 1024, -36, 3] - - ["KR", 1442080800000, -33, 13597, 1442070000000, 1442084400000, 1024, -36, 3] - - ["KR", 1442084400000, 314, 13597, 1442077200000, 1442088000000, 1024, -36, 3] - - ["KR", 1442088000000, 524, 13597, 1442080800000, 1442095200000, 1024, -36, 3] - - ["KR", 1442095200000, 827, 13597, 1442084400000, 1442098800000, 1024, -36, 3] - - ["KR", 1442098800000, -36, 13597, 1442088000000, null, 1024, -36, 3] - - ["KW", 1442055600000, -2, 1778, null, 1442070000000, -2, -33, 1] - - ["KW", 1442070000000, 1815, 1778, null, 1442077200000, -2, -33, 1] - - ["KW", 1442077200000, -2, 1778, 1442055600000, 1442080800000, -2, -33, 2] - - ["KW", 1442080800000, -33, 1778, 1442070000000, null, -2, -33, 3] - - ["KZ", 1442034000000, 161, 1261, null, 1442044800000, 161, 91, 1] - - ["KZ", 1442044800000, 401, 1261, null, 1442048400000, 161, 91, 1] - - ["KZ", 1442048400000, 439, 1261, 1442034000000, 1442052000000, 161, 91, 1] - - ["KZ", 1442052000000, 412, 1261, 1442044800000, 1442055600000, 161, 91, 1] - - ["KZ", 1442055600000, 63, 1261, 1442048400000, 1442059200000, 161, 91, 2] - - ["KZ", 1442059200000, 33, 1261, 1442052000000, 1442062800000, 161, 91, 2] - - ["KZ", 1442062800000, 0, 1261, 1442055600000, 1442066400000, 161, 91, 2] - - ["KZ", 1442066400000, 0, 1261, 1442059200000, 1442077200000, 161, 91, 2] - - ["KZ", 1442077200000, -317, 1261, 1442062800000, 1442084400000, 161, 91, 3] - - ["KZ", 1442084400000, -22, 1261, 1442066400000, 1442095200000, 161, 91, 3] - - ["KZ", 1442095200000, 91, 1261, 1442077200000, null, 161, 91, 3] - - ["LB", 1442055600000, -67, -67, null, null, -67, -67, 1] - - ["LK", 1442026800000, 79, 131, null, 1442048400000, 79, -3, 1] - - ["LK", 1442048400000, 8, 131, null, 1442052000000, 79, -3, 1] - - ["LK", 1442052000000, 47, 131, 1442026800000, 1442084400000, 79, -3, 2] - - ["LK", 1442084400000, -3, 131, 1442048400000, null, 79, -3, 3] - - ["LT", 1442080800000, 12, -12, null, 1442098800000, 12, -24, 1] - - ["LT", 1442098800000, -24, -12, null, null, 12, -24, 2] - - ["LU", 1442059200000, 79, 606, null, 1442066400000, 79, 2, 1] - - ["LU", 1442066400000, 0, 606, null, 1442077200000, 79, 2, 1] - - ["LU", 1442077200000, 525, 606, 1442059200000, 1442095200000, 79, 2, 2] - - ["LU", 1442095200000, 2, 606, 1442066400000, null, 79, 2, 3] - - ["LV", 1442095200000, 0, 0, null, null, 0, 0, 1] - - ["MA", 1442019600000, -1, 229, null, 1442055600000, -1, 8, 1] - - ["MA", 1442055600000, 23, 229, null, 1442059200000, -1, 8, 1] - - ["MA", 1442059200000, -56, 229, 1442019600000, 1442062800000, -1, 8, 1] - - ["MA", 1442062800000, 0, 229, 1442055600000, 1442077200000, -1, 8, 2] - - ["MA", 1442077200000, 250, 229, 1442059200000, 1442080800000, -1, 8, 2] - - ["MA", 1442080800000, 5, 229, 1442062800000, 1442098800000, -1, 8, 3] - - ["MA", 1442098800000, 8, 229, 1442077200000, null, -1, 8, 3] - - ["MD", 1442077200000, 6916, 6916, null, null, 6916, 6916, 1] - - ["ME", 1442073600000, 0, 0, null, null, 0, 0, 1] - - ["MH", 1442052000000, 40, 40, null, null, 40, 40, 1] - - ["MK", 1442077200000, -72, -72, null, null, -72, -72, 1] - - ["MM", 1442070000000, 3, 28, null, 1442073600000, 3, 25, 1] - - ["MM", 1442073600000, 25, 28, null, null, 3, 25, 2] - - ["MO", 1442034000000, 30, 48, null, 1442070000000, 30, 18, 1] - - ["MO", 1442070000000, 18, 48, null, null, 30, 18, 2] - - ["MR", 1442080800000, 10, 10, null, null, 10, 10, 1] - - ["MT", 1442048400000, -1, -1, null, null, -1, -1, 1] - - ["MV", 1442073600000, -3, -3, null, null, -3, -3, 1] - - ["MX", 1442016000000, -67, 10472, null, 1442023200000, -67, 28, 1] - - ["MX", 1442023200000, 549, 10472, null, 1442026800000, -67, 28, 1] - - ["MX", 1442026800000, 3642, 10472, 1442016000000, 1442030400000, -67, 28, 1] - - ["MX", 1442030400000, 373, 10472, 1442023200000, 1442034000000, -67, 28, 1] - - ["MX", 1442034000000, 944, 10472, 1442026800000, 1442037600000, -67, 28, 1] - - ["MX", 1442037600000, 4, 10472, 1442030400000, 1442041200000, -67, 28, 1] - - ["MX", 1442041200000, -294, 10472, 1442034000000, 1442066400000, -67, 28, 2] - - ["MX", 1442066400000, -1, 10472, 1442037600000, 1442070000000, -67, 28, 2] - - ["MX", 1442070000000, -1, 10472, 1442041200000, 1442073600000, -67, 28, 2] - - ["MX", 1442073600000, -21, 10472, 1442066400000, 1442077200000, -67, 28, 2] - - ["MX", 1442077200000, 3874, 10472, 1442070000000, 1442080800000, -67, 28, 2] - - ["MX", 1442080800000, -376, 10472, 1442073600000, 1442084400000, -67, 28, 2] - - ["MX", 1442084400000, 981, 10472, 1442077200000, 1442088000000, -67, 28, 3] - - ["MX", 1442088000000, 494, 10472, 1442080800000, 1442091600000, -67, 28, 3] - - ["MX", 1442091600000, 799, 10472, 1442084400000, 1442095200000, -67, 28, 3] - - ["MX", 1442095200000, -456, 10472, 1442088000000, 1442098800000, -67, 28, 3] - - ["MX", 1442098800000, 28, 10472, 1442091600000, null, -67, 28, 3] - - ["MY", 1442019600000, -7, 3207, null, 1442030400000, -7, 739, 1] - - ["MY", 1442030400000, -3, 3207, null, 1442034000000, -7, 739, 1] - - ["MY", 1442034000000, 1028, 3207, 1442019600000, 1442041200000, -7, 739, 1] - - ["MY", 1442041200000, 935, 3207, 1442030400000, 1442044800000, -7, 739, 1] - - ["MY", 1442044800000, -127, 3207, 1442034000000, 1442048400000, -7, 739, 2] - - ["MY", 1442048400000, 649, 3207, 1442041200000, 1442055600000, -7, 739, 2] - - ["MY", 1442055600000, 1, 3207, 1442044800000, 1442059200000, -7, 739, 2] - - ["MY", 1442059200000, 0, 3207, 1442048400000, 1442066400000, -7, 739, 2] - - ["MY", 1442066400000, 1, 3207, 1442055600000, 1442073600000, -7, 739, 3] - - ["MY", 1442073600000, 1, 3207, 1442059200000, 1442077200000, -7, 739, 3] - - ["MY", 1442077200000, -10, 3207, 1442066400000, 1442098800000, -7, 739, 3] - - ["MY", 1442098800000, 739, 3207, 1442073600000, null, -7, 739, 3] - - ["NG", 1442052000000, 208, 214, null, 1442070000000, 208, 6, 1] - - ["NG", 1442070000000, 6, 214, null, null, 208, 6, 2] - - ["NL", 1442034000000, 0, 12162, null, 1442044800000, 0, 4, 1] - - ["NL", 1442044800000, 16, 12162, null, 1442048400000, 0, 4, 1] - - ["NL", 1442048400000, 1303, 12162, 1442034000000, 1442052000000, 0, 4, 1] - - ["NL", 1442052000000, 53, 12162, 1442044800000, 1442055600000, 0, 4, 1] - - ["NL", 1442055600000, 105, 12162, 1442048400000, 1442059200000, 0, 4, 1] - - ["NL", 1442059200000, 206, 12162, 1442052000000, 1442062800000, 0, 4, 1] - - ["NL", 1442062800000, -30, 12162, 1442055600000, 1442066400000, 0, 4, 2] - - ["NL", 1442066400000, 61, 12162, 1442059200000, 1442070000000, 0, 4, 2] - - ["NL", 1442070000000, -84, 12162, 1442062800000, 1442073600000, 0, 4, 2] - - ["NL", 1442073600000, 166, 12162, 1442066400000, 1442077200000, 0, 4, 2] - - ["NL", 1442077200000, 878, 12162, 1442070000000, 1442080800000, 0, 4, 2] - - ["NL", 1442080800000, 8947, 12162, 1442073600000, 1442084400000, 0, 4, 2] - - ["NL", 1442084400000, 436, 12162, 1442077200000, 1442088000000, 0, 4, 3] - - ["NL", 1442088000000, 12, 12162, 1442080800000, 1442091600000, 0, 4, 3] - - ["NL", 1442091600000, 19, 12162, 1442084400000, 1442095200000, 0, 4, 3] - - ["NL", 1442095200000, 70, 12162, 1442088000000, 1442098800000, 0, 4, 3] - - ["NL", 1442098800000, 4, 12162, 1442091600000, null, 0, 4, 3] - - ["NO", 1442019600000, 48, 432, null, 1442048400000, 48, 2, 1] - - ["NO", 1442048400000, -447, 432, null, 1442052000000, 48, 2, 1] - - ["NO", 1442052000000, 447, 432, 1442019600000, 1442055600000, 48, 2, 1] - - ["NO", 1442055600000, 29, 432, 1442048400000, 1442066400000, 48, 2, 1] - - ["NO", 1442066400000, 71, 432, 1442052000000, 1442073600000, 48, 2, 2] - - ["NO", 1442073600000, 222, 432, 1442055600000, 1442080800000, 48, 2, 2] - - ["NO", 1442080800000, 31, 432, 1442066400000, 1442088000000, 48, 2, 2] - - ["NO", 1442088000000, 15, 432, 1442073600000, 1442091600000, 48, 2, 2] - - ["NO", 1442091600000, 15, 432, 1442080800000, 1442095200000, 48, 2, 3] - - ["NO", 1442095200000, -1, 432, 1442088000000, 1442098800000, 48, 2, 3] - - ["NO", 1442098800000, 2, 432, 1442091600000, null, 48, 2, 3] - - ["NP", 1442048400000, 61, 61, null, null, 61, 61, 1] - - ["NZ", 1442019600000, 28, 1693, null, 1442026800000, 28, -2, 1] - - ["NZ", 1442026800000, 635, 1693, null, 1442037600000, 28, -2, 1] - - ["NZ", 1442037600000, 66, 1693, 1442019600000, 1442048400000, 28, -2, 1] - - ["NZ", 1442048400000, 189, 1693, 1442026800000, 1442059200000, 28, -2, 2] - - ["NZ", 1442059200000, 428, 1693, 1442037600000, 1442084400000, 28, -2, 2] - - ["NZ", 1442084400000, -52, 1693, 1442048400000, 1442088000000, 28, -2, 2] - - ["NZ", 1442088000000, 405, 1693, 1442059200000, 1442095200000, 28, -2, 3] - - ["NZ", 1442095200000, -4, 1693, 1442084400000, 1442098800000, 28, -2, 3] - - ["NZ", 1442098800000, -2, 1693, 1442088000000, null, 28, -2, 3] - - ["OM", 1442052000000, 0, 0, null, null, 0, 0, 1] - - ["PA", 1442026800000, 0, 0, null, null, 0, 0, 1] - - ["PE", 1442019600000, 523, 2134, null, 1442023200000, 523, 1861, 1] - - ["PE", 1442023200000, 26, 2134, null, 1442026800000, 523, 1861, 1] - - ["PE", 1442026800000, -12, 2134, 1442019600000, 1442062800000, 523, 1861, 1] - - ["PE", 1442062800000, -12, 2134, 1442023200000, 1442077200000, 523, 1861, 2] - - ["PE", 1442077200000, -163, 2134, 1442026800000, 1442080800000, 523, 1861, 2] - - ["PE", 1442080800000, -2, 2134, 1442062800000, 1442084400000, 523, 1861, 2] - - ["PE", 1442084400000, -68, 2134, 1442077200000, 1442095200000, 523, 1861, 3] - - ["PE", 1442095200000, -19, 2134, 1442080800000, 1442098800000, 523, 1861, 3] - - ["PE", 1442098800000, 1861, 2134, 1442084400000, null, 523, 1861, 3] - - ["PH", 1442019600000, 6, 6613, null, 1442023200000, 6, 8, 1] - - ["PH", 1442023200000, 459, 6613, null, 1442026800000, 6, 8, 1] - - ["PH", 1442026800000, 910, 6613, 1442019600000, 1442030400000, 6, 8, 1] - - ["PH", 1442030400000, 26, 6613, 1442023200000, 1442034000000, 6, 8, 1] - - ["PH", 1442034000000, 59, 6613, 1442026800000, 1442037600000, 6, 8, 1] - - ["PH", 1442037600000, 17, 6613, 1442030400000, 1442041200000, 6, 8, 1] - - ["PH", 1442041200000, 0, 6613, 1442034000000, 1442044800000, 6, 8, 1] - - ["PH", 1442044800000, 55, 6613, 1442037600000, 1442048400000, 6, 8, 2] - - ["PH", 1442048400000, 62, 6613, 1442041200000, 1442052000000, 6, 8, 2] - - ["PH", 1442052000000, 22, 6613, 1442044800000, 1442055600000, 6, 8, 2] - - ["PH", 1442055600000, 1969, 6613, 1442048400000, 1442059200000, 6, 8, 2] - - ["PH", 1442059200000, 273, 6613, 1442052000000, 1442062800000, 6, 8, 2] - - ["PH", 1442062800000, 171, 6613, 1442055600000, 1442066400000, 6, 8, 2] - - ["PH", 1442066400000, 1880, 6613, 1442059200000, 1442070000000, 6, 8, 2] - - ["PH", 1442070000000, 34, 6613, 1442062800000, 1442073600000, 6, 8, 3] - - ["PH", 1442073600000, -227, 6613, 1442066400000, 1442077200000, 6, 8, 3] - - ["PH", 1442077200000, 2, 6613, 1442070000000, 1442080800000, 6, 8, 3] - - ["PH", 1442080800000, 32, 6613, 1442073600000, 1442084400000, 6, 8, 3] - - ["PH", 1442084400000, 39, 6613, 1442077200000, 1442091600000, 6, 8, 3] - - ["PH", 1442091600000, 816, 6613, 1442080800000, 1442098800000, 6, 8, 3] - - ["PH", 1442098800000, 8, 6613, 1442084400000, null, 6, 8, 3] - - ["PK", 1442019600000, 335, 641, null, 1442026800000, 335, 43, 1] - - ["PK", 1442026800000, 101, 641, null, 1442037600000, 335, 43, 1] - - ["PK", 1442037600000, 100, 641, 1442019600000, 1442041200000, 335, 43, 1] - - ["PK", 1442041200000, 24, 641, 1442026800000, 1442048400000, 335, 43, 2] - - ["PK", 1442048400000, 15, 641, 1442037600000, 1442062800000, 335, 43, 2] - - ["PK", 1442062800000, 23, 641, 1442041200000, 1442070000000, 335, 43, 3] - - ["PK", 1442070000000, 43, 641, 1442048400000, null, 335, 43, 3] - - ["PL", 1442037600000, 95, 9815, null, 1442041200000, 95, -9, 1] - - ["PL", 1442041200000, 281, 9815, null, 1442044800000, 95, -9, 1] - - ["PL", 1442044800000, 319, 9815, 1442037600000, 1442048400000, 95, -9, 1] - - ["PL", 1442048400000, 366, 9815, 1442041200000, 1442052000000, 95, -9, 1] - - ["PL", 1442052000000, 330, 9815, 1442044800000, 1442055600000, 95, -9, 1] - - ["PL", 1442055600000, 410, 9815, 1442048400000, 1442059200000, 95, -9, 1] - - ["PL", 1442059200000, 199, 9815, 1442052000000, 1442062800000, 95, -9, 2] - - ["PL", 1442062800000, 4171, 9815, 1442055600000, 1442066400000, 95, -9, 2] - - ["PL", 1442066400000, 34, 9815, 1442059200000, 1442070000000, 95, -9, 2] - - ["PL", 1442070000000, 146, 9815, 1442062800000, 1442073600000, 95, -9, 2] - - ["PL", 1442073600000, 30, 9815, 1442066400000, 1442077200000, 95, -9, 2] - - ["PL", 1442077200000, 324, 9815, 1442070000000, 1442080800000, 95, -9, 2] - - ["PL", 1442080800000, 7, 9815, 1442073600000, 1442084400000, 95, -9, 3] - - ["PL", 1442084400000, 13, 9815, 1442077200000, 1442088000000, 95, -9, 3] - - ["PL", 1442088000000, 346, 9815, 1442080800000, 1442091600000, 95, -9, 3] - - ["PL", 1442091600000, 902, 9815, 1442084400000, 1442095200000, 95, -9, 3] - - ["PL", 1442095200000, 1851, 9815, 1442088000000, 1442098800000, 95, -9, 3] - - ["PL", 1442098800000, -9, 9815, 1442091600000, null, 95, -9, 3] - - ["PR", 1442026800000, 22, 23, null, 1442030400000, 22, 29, 1] - - ["PR", 1442030400000, 2, 23, null, 1442059200000, 22, 29, 1] - - ["PR", 1442059200000, -35, 23, 1442026800000, 1442077200000, 22, 29, 2] - - ["PR", 1442077200000, 5, 23, 1442030400000, 1442095200000, 22, 29, 2] - - ["PR", 1442095200000, 29, 23, 1442059200000, null, 22, 29, 3] - - ["PT", 1442019600000, 172, 4037, null, 1442044800000, 172, 2, 1] - - ["PT", 1442044800000, 11, 4037, null, 1442052000000, 172, 2, 1] - - ["PT", 1442052000000, 102, 4037, 1442019600000, 1442066400000, 172, 2, 1] - - ["PT", 1442066400000, 12, 4037, 1442044800000, 1442070000000, 172, 2, 1] - - ["PT", 1442070000000, 3470, 4037, 1442052000000, 1442077200000, 172, 2, 2] - - ["PT", 1442077200000, -75, 4037, 1442066400000, 1442080800000, 172, 2, 2] - - ["PT", 1442080800000, -79, 4037, 1442070000000, 1442088000000, 172, 2, 2] - - ["PT", 1442088000000, 403, 4037, 1442077200000, 1442095200000, 172, 2, 3] - - ["PT", 1442095200000, 19, 4037, 1442080800000, 1442098800000, 172, 2, 3] - - ["PT", 1442098800000, 2, 4037, 1442088000000, null, 172, 2, 3] - - ["PY", 1442019600000, 1, 634, null, 1442080800000, 1, 628, 1] - - ["PY", 1442080800000, 5, 634, null, 1442084400000, 1, 628, 2] - - ["PY", 1442084400000, 628, 634, 1442019600000, null, 1, 628, 3] - - ["QA", 1442041200000, 13, 13, null, null, 13, 13, 1] - - ["RO", 1442034000000, 68, 2893, null, 1442041200000, 68, 824, 1] - - ["RO", 1442041200000, 845, 2893, null, 1442044800000, 68, 824, 1] - - ["RO", 1442044800000, 284, 2893, 1442034000000, 1442052000000, 68, 824, 1] - - ["RO", 1442052000000, 319, 2893, 1442041200000, 1442055600000, 68, 824, 1] - - ["RO", 1442055600000, 26, 2893, 1442044800000, 1442062800000, 68, 824, 2] - - ["RO", 1442062800000, 541, 2893, 1442052000000, 1442070000000, 68, 824, 2] - - ["RO", 1442070000000, -29, 2893, 1442055600000, 1442073600000, 68, 824, 2] - - ["RO", 1442073600000, 15, 2893, 1442062800000, 1442091600000, 68, 824, 3] - - ["RO", 1442091600000, 0, 2893, 1442070000000, 1442095200000, 68, 824, 3] - - ["RO", 1442095200000, 824, 2893, 1442073600000, null, 68, 824, 3] - - ["RS", 1442019600000, 6, 906, null, 1442062800000, 6, -15, 1] - - ["RS", 1442062800000, 13, 906, null, 1442066400000, 6, -15, 1] - - ["RS", 1442066400000, 0, 906, 1442019600000, 1442073600000, 6, -15, 1] - - ["RS", 1442073600000, 813, 906, 1442062800000, 1442080800000, 6, -15, 2] - - ["RS", 1442080800000, 0, 906, 1442066400000, 1442084400000, 6, -15, 2] - - ["RS", 1442084400000, 89, 906, 1442073600000, 1442091600000, 6, -15, 3] - - ["RS", 1442091600000, -15, 906, 1442080800000, null, 6, -15, 3] - - ["RU", 1442019600000, 2214, 48104, null, 1442023200000, 2214, 12098, 1] - - ["RU", 1442023200000, 299, 48104, null, 1442026800000, 2214, 12098, 1] - - ["RU", 1442026800000, 0, 48104, 1442019600000, 1442030400000, 2214, 12098, 1] - - ["RU", 1442030400000, 76, 48104, 1442023200000, 1442034000000, 2214, 12098, 1] - - ["RU", 1442034000000, 658, 48104, 1442026800000, 1442037600000, 2214, 12098, 1] - - ["RU", 1442037600000, -324, 48104, 1442030400000, 1442041200000, 2214, 12098, 1] - - ["RU", 1442041200000, 580, 48104, 1442034000000, 1442044800000, 2214, 12098, 1] - - ["RU", 1442044800000, 2564, 48104, 1442037600000, 1442048400000, 2214, 12098, 1] - - ["RU", 1442048400000, 1027, 48104, 1442041200000, 1442052000000, 2214, 12098, 2] - - ["RU", 1442052000000, 1214, 48104, 1442044800000, 1442055600000, 2214, 12098, 2] - - ["RU", 1442055600000, 499, 48104, 1442048400000, 1442059200000, 2214, 12098, 2] - - ["RU", 1442059200000, 3902, 48104, 1442052000000, 1442062800000, 2214, 12098, 2] - - ["RU", 1442062800000, 168, 48104, 1442055600000, 1442066400000, 2214, 12098, 2] - - ["RU", 1442066400000, 2047, 48104, 1442059200000, 1442070000000, 2214, 12098, 2] - - ["RU", 1442070000000, 4706, 48104, 1442062800000, 1442073600000, 2214, 12098, 2] - - ["RU", 1442073600000, 1618, 48104, 1442066400000, 1442077200000, 2214, 12098, 2] - - ["RU", 1442077200000, 1162, 48104, 1442070000000, 1442080800000, 2214, 12098, 3] - - ["RU", 1442080800000, 655, 48104, 1442073600000, 1442084400000, 2214, 12098, 3] - - ["RU", 1442084400000, 6461, 48104, 1442077200000, 1442088000000, 2214, 12098, 3] - - ["RU", 1442088000000, 2596, 48104, 1442080800000, 1442091600000, 2214, 12098, 3] - - ["RU", 1442091600000, 3449, 48104, 1442084400000, 1442095200000, 2214, 12098, 3] - - ["RU", 1442095200000, 435, 48104, 1442088000000, 1442098800000, 2214, 12098, 3] - - ["RU", 1442098800000, 12098, 48104, 1442091600000, null, 2214, 12098, 3] - - ["SA", 1442037600000, -97, 1614, null, 1442048400000, -97, 458, 1] - - ["SA", 1442048400000, 14, 1614, null, 1442055600000, -97, 458, 1] - - ["SA", 1442055600000, 11, 1614, 1442037600000, 1442059200000, -97, 458, 1] - - ["SA", 1442059200000, 0, 1614, 1442048400000, 1442066400000, -97, 458, 2] - - ["SA", 1442066400000, 1276, 1614, 1442055600000, 1442073600000, -97, 458, 2] - - ["SA", 1442073600000, 2, 1614, 1442059200000, 1442077200000, -97, 458, 2] - - ["SA", 1442077200000, -50, 1614, 1442066400000, 1442084400000, -97, 458, 3] - - ["SA", 1442084400000, 458, 1614, 1442073600000, null, -97, 458, 3] - - ["SE", 1442019600000, 109, 1838, null, 1442023200000, 109, 0, 1] - - ["SE", 1442023200000, 3, 1838, null, 1442030400000, 109, 0, 1] - - ["SE", 1442030400000, 30, 1838, 1442019600000, 1442041200000, 109, 0, 1] - - ["SE", 1442041200000, 91, 1838, 1442023200000, 1442048400000, 109, 0, 1] - - ["SE", 1442048400000, -145, 1838, 1442030400000, 1442052000000, 109, 0, 1] - - ["SE", 1442052000000, 1, 1838, 1442041200000, 1442055600000, 109, 0, 2] - - ["SE", 1442055600000, -5, 1838, 1442048400000, 1442059200000, 109, 0, 2] - - ["SE", 1442059200000, 1476, 1838, 1442052000000, 1442066400000, 109, 0, 2] - - ["SE", 1442066400000, 14, 1838, 1442055600000, 1442070000000, 109, 0, 2] - - ["SE", 1442070000000, 78, 1838, 1442059200000, 1442080800000, 109, 0, 2] - - ["SE", 1442080800000, 89, 1838, 1442066400000, 1442084400000, 109, 0, 3] - - ["SE", 1442084400000, 37, 1838, 1442070000000, 1442091600000, 109, 0, 3] - - ["SE", 1442091600000, -1, 1838, 1442080800000, 1442095200000, 109, 0, 3] - - ["SE", 1442095200000, 61, 1838, 1442084400000, 1442098800000, 109, 0, 3] - - ["SE", 1442098800000, 0, 1838, 1442091600000, null, 109, 0, 3] - - ["SG", 1442026800000, 2758, 3338, null, 1442030400000, 2758, 0, 1] - - ["SG", 1442030400000, 1, 3338, null, 1442037600000, 2758, 0, 1] - - ["SG", 1442037600000, 3, 3338, 1442026800000, 1442041200000, 2758, 0, 1] - - ["SG", 1442041200000, 59, 3338, 1442030400000, 1442044800000, 2758, 0, 2] - - ["SG", 1442044800000, 77, 3338, 1442037600000, 1442048400000, 2758, 0, 2] - - ["SG", 1442048400000, 52, 3338, 1442041200000, 1442062800000, 2758, 0, 2] - - ["SG", 1442062800000, 388, 3338, 1442044800000, 1442066400000, 2758, 0, 3] - - ["SG", 1442066400000, 0, 3338, 1442048400000, null, 2758, 0, 3] - - ["SI", 1442080800000, -45, -36, null, 1442091600000, -45, 9, 1] - - ["SI", 1442091600000, 9, -36, null, null, -45, 9, 2] - - ["SK", 1442037600000, -1, 379, null, 1442052000000, -1, 7, 1] - - ["SK", 1442052000000, 13, 379, null, 1442062800000, -1, 7, 1] - - ["SK", 1442062800000, 6, 379, 1442037600000, 1442073600000, -1, 7, 2] - - ["SK", 1442073600000, 446, 379, 1442052000000, 1442084400000, -1, 7, 2] - - ["SK", 1442084400000, -92, 379, 1442062800000, 1442098800000, -1, 7, 3] - - ["SK", 1442098800000, 7, 379, 1442073600000, null, -1, 7, 3] - - ["SV", 1442019600000, -1, 114, null, 1442084400000, -1, 9, 1] - - ["SV", 1442084400000, 106, 114, null, 1442088000000, -1, 9, 2] - - ["SV", 1442088000000, 9, 114, 1442019600000, null, -1, 9, 3] - - ["TH", 1442034000000, 0, 24, null, 1442041200000, 0, 13, 1] - - ["TH", 1442041200000, 3, 24, null, 1442044800000, 0, 13, 1] - - ["TH", 1442044800000, 110, 24, 1442034000000, 1442052000000, 0, 13, 1] - - ["TH", 1442052000000, -22, 24, 1442041200000, 1442055600000, 0, 13, 2] - - ["TH", 1442055600000, 0, 24, 1442044800000, 1442062800000, 0, 13, 2] - - ["TH", 1442062800000, -46, 24, 1442052000000, 1442066400000, 0, 13, 2] - - ["TH", 1442066400000, -34, 24, 1442055600000, 1442070000000, 0, 13, 3] - - ["TH", 1442070000000, 0, 24, 1442062800000, 1442084400000, 0, 13, 3] - - ["TH", 1442084400000, 13, 24, 1442066400000, null, 0, 13, 3] - - ["TJ", 1442048400000, 1471, 1471, null, null, 1471, 1471, 1] - - ["TN", 1442098800000, -9, -9, null, null, -9, -9, 1] - - ["TR", 1442023200000, 306, 7078, null, 1442041200000, 306, -29, 1] - - ["TR", 1442041200000, 1, 7078, null, 1442044800000, 306, -29, 1] - - ["TR", 1442044800000, 41, 7078, 1442023200000, 1442048400000, 306, -29, 1] - - ["TR", 1442048400000, 88, 7078, 1442041200000, 1442052000000, 306, -29, 1] - - ["TR", 1442052000000, 41, 7078, 1442044800000, 1442055600000, 306, -29, 1] - - ["TR", 1442055600000, 299, 7078, 1442048400000, 1442062800000, 306, -29, 2] - - ["TR", 1442062800000, 315, 7078, 1442052000000, 1442066400000, 306, -29, 2] - - ["TR", 1442066400000, 85, 7078, 1442055600000, 1442070000000, 306, -29, 2] - - ["TR", 1442070000000, 236, 7078, 1442062800000, 1442077200000, 306, -29, 2] - - ["TR", 1442077200000, 89, 7078, 1442066400000, 1442080800000, 306, -29, 2] - - ["TR", 1442080800000, -1, 7078, 1442070000000, 1442084400000, 306, -29, 3] - - ["TR", 1442084400000, 170, 7078, 1442077200000, 1442088000000, 306, -29, 3] - - ["TR", 1442088000000, 2389, 7078, 1442080800000, 1442091600000, 306, -29, 3] - - ["TR", 1442091600000, 3048, 7078, 1442084400000, 1442095200000, 306, -29, 3] - - ["TR", 1442095200000, -29, 7078, 1442088000000, null, 306, -29, 3] - - ["TT", 1442088000000, 9, 9, null, null, 9, 9, 1] - - ["TW", 1442016000000, 92, 3656, null, 1442019600000, 92, -60, 1] - - ["TW", 1442019600000, 0, 3656, null, 1442023200000, 92, -60, 1] - - ["TW", 1442023200000, 97, 3656, 1442016000000, 1442026800000, 92, -60, 1] - - ["TW", 1442026800000, 680, 3656, 1442019600000, 1442030400000, 92, -60, 1] - - ["TW", 1442030400000, 0, 3656, 1442023200000, 1442034000000, 92, -60, 1] - - ["TW", 1442034000000, 143, 3656, 1442026800000, 1442037600000, 92, -60, 1] - - ["TW", 1442037600000, 266, 3656, 1442030400000, 1442041200000, 92, -60, 1] - - ["TW", 1442041200000, 366, 3656, 1442034000000, 1442044800000, 92, -60, 1] - - ["TW", 1442044800000, 24, 3656, 1442037600000, 1442048400000, 92, -60, 2] - - ["TW", 1442048400000, 75, 3656, 1442041200000, 1442052000000, 92, -60, 2] - - ["TW", 1442052000000, 24, 3656, 1442044800000, 1442055600000, 92, -60, 2] - - ["TW", 1442055600000, 48, 3656, 1442048400000, 1442059200000, 92, -60, 2] - - ["TW", 1442059200000, -157, 3656, 1442052000000, 1442062800000, 92, -60, 2] - - ["TW", 1442062800000, -272, 3656, 1442055600000, 1442066400000, 92, -60, 2] - - ["TW", 1442066400000, 624, 3656, 1442059200000, 1442070000000, 92, -60, 2] - - ["TW", 1442070000000, 485, 3656, 1442062800000, 1442073600000, 92, -60, 3] - - ["TW", 1442073600000, 772, 3656, 1442066400000, 1442077200000, 92, -60, 3] - - ["TW", 1442077200000, 502, 3656, 1442070000000, 1442080800000, 92, -60, 3] - - ["TW", 1442080800000, 24, 3656, 1442073600000, 1442084400000, 92, -60, 3] - - ["TW", 1442084400000, 0, 3656, 1442077200000, 1442095200000, 92, -60, 3] - - ["TW", 1442095200000, -77, 3656, 1442080800000, 1442098800000, 92, -60, 3] - - ["TW", 1442098800000, -60, 3656, 1442084400000, null, 92, -60, 3] - - ["UA", 1442034000000, 3468, 24898, null, 1442037600000, 3468, 38, 1] - - ["UA", 1442037600000, -1, 24898, null, 1442041200000, 3468, 38, 1] - - ["UA", 1442041200000, 74, 24898, 1442034000000, 1442044800000, 3468, 38, 1] - - ["UA", 1442044800000, 280, 24898, 1442037600000, 1442048400000, 3468, 38, 1] - - ["UA", 1442048400000, 2, 24898, 1442041200000, 1442052000000, 3468, 38, 1] - - ["UA", 1442052000000, 410, 24898, 1442044800000, 1442055600000, 3468, 38, 1] - - ["UA", 1442055600000, 14202, 24898, 1442048400000, 1442059200000, 3468, 38, 1] - - ["UA", 1442059200000, -2, 24898, 1442052000000, 1442062800000, 3468, 38, 2] - - ["UA", 1442062800000, 773, 24898, 1442055600000, 1442066400000, 3468, 38, 2] - - ["UA", 1442066400000, 296, 24898, 1442059200000, 1442070000000, 3468, 38, 2] - - ["UA", 1442070000000, 1733, 24898, 1442062800000, 1442073600000, 3468, 38, 2] - - ["UA", 1442073600000, 4241, 24898, 1442066400000, 1442077200000, 3468, 38, 2] - - ["UA", 1442077200000, -181, 24898, 1442070000000, 1442080800000, 3468, 38, 2] - - ["UA", 1442080800000, -1, 24898, 1442073600000, 1442084400000, 3468, 38, 3] - - ["UA", 1442084400000, 5, 24898, 1442077200000, 1442088000000, 3468, 38, 3] - - ["UA", 1442088000000, -21, 24898, 1442080800000, 1442091600000, 3468, 38, 3] - - ["UA", 1442091600000, -388, 24898, 1442084400000, 1442095200000, 3468, 38, 3] - - ["UA", 1442095200000, -30, 24898, 1442088000000, 1442098800000, 3468, 38, 3] - - ["UA", 1442098800000, 38, 24898, 1442091600000, null, 3468, 38, 3] - - ["UG", 1442070000000, 1, 1, null, null, 1, 1, 1] - - ["US", 1442016000000, 0, 38882, null, 1442019600000, 0, 3575, 1] - - ["US", 1442019600000, 1043, 38882, null, 1442023200000, 0, 3575, 1] - - ["US", 1442023200000, 2844, 38882, 1442016000000, 1442026800000, 0, 3575, 1] - - ["US", 1442026800000, 1512, 38882, 1442019600000, 1442030400000, 0, 3575, 1] - - ["US", 1442030400000, 2023, 38882, 1442023200000, 1442034000000, 0, 3575, 1] - - ["US", 1442034000000, 3648, 38882, 1442026800000, 1442037600000, 0, 3575, 1] - - ["US", 1442037600000, 3675, 38882, 1442030400000, 1442041200000, 0, 3575, 1] - - ["US", 1442041200000, 1999, 38882, 1442034000000, 1442044800000, 0, 3575, 1] - - ["US", 1442044800000, 139, 38882, 1442037600000, 1442048400000, 0, 3575, 2] - - ["US", 1442048400000, -466, 38882, 1442041200000, 1442052000000, 0, 3575, 2] - - ["US", 1442052000000, -2, 38882, 1442044800000, 1442055600000, 0, 3575, 2] - - ["US", 1442055600000, 156, 38882, 1442048400000, 1442059200000, 0, 3575, 2] - - ["US", 1442059200000, 11, 38882, 1442052000000, 1442062800000, 0, 3575, 2] - - ["US", 1442062800000, 47, 38882, 1442055600000, 1442066400000, 0, 3575, 2] - - ["US", 1442066400000, 772, 38882, 1442059200000, 1442070000000, 0, 3575, 2] - - ["US", 1442070000000, 3505, 38882, 1442062800000, 1442073600000, 0, 3575, 2] - - ["US", 1442073600000, 1100, 38882, 1442066400000, 1442077200000, 0, 3575, 3] - - ["US", 1442077200000, 2168, 38882, 1442070000000, 1442080800000, 0, 3575, 3] - - ["US", 1442080800000, 4001, 38882, 1442073600000, 1442084400000, 0, 3575, 3] - - ["US", 1442084400000, 2523, 38882, 1442077200000, 1442088000000, 0, 3575, 3] - - ["US", 1442088000000, 1691, 38882, 1442080800000, 1442091600000, 0, 3575, 3] - - ["US", 1442091600000, 2502, 38882, 1442084400000, 1442095200000, 0, 3575, 3] - - ["US", 1442095200000, 416, 38882, 1442088000000, 1442098800000, 0, 3575, 3] - - ["US", 1442098800000, 3575, 38882, 1442091600000, null, 0, 3575, 3] - - ["UY", 1442019600000, 77, 936, null, 1442023200000, 77, 23, 1] - - ["UY", 1442023200000, 517, 936, null, 1442026800000, 77, 23, 1] - - ["UY", 1442026800000, 76, 936, 1442019600000, 1442037600000, 77, 23, 1] - - ["UY", 1442037600000, 1, 936, 1442023200000, 1442070000000, 77, 23, 2] - - ["UY", 1442070000000, 284, 936, 1442026800000, 1442073600000, 77, 23, 2] - - ["UY", 1442073600000, -42, 936, 1442037600000, 1442077200000, 77, 23, 3] - - ["UY", 1442077200000, 23, 936, 1442070000000, null, 77, 23, 3] - - ["UZ", 1442044800000, 1369, 1369, null, null, 1369, 1369, 1] - - ["VE", 1442023200000, 115, 1101, null, 1442026800000, 115, 9, 1] - - ["VE", 1442026800000, -17, 1101, null, 1442030400000, 115, 9, 1] - - ["VE", 1442030400000, 51, 1101, 1442023200000, 1442034000000, 115, 9, 1] - - ["VE", 1442034000000, -2, 1101, 1442026800000, 1442066400000, 115, 9, 1] - - ["VE", 1442066400000, 18, 1101, 1442030400000, 1442070000000, 115, 9, 2] - - ["VE", 1442070000000, 420, 1101, 1442034000000, 1442077200000, 115, 9, 2] - - ["VE", 1442077200000, 412, 1101, 1442066400000, 1442084400000, 115, 9, 2] - - ["VE", 1442084400000, 60, 1101, 1442070000000, 1442095200000, 115, 9, 3] - - ["VE", 1442095200000, 35, 1101, 1442077200000, 1442098800000, 115, 9, 3] - - ["VE", 1442098800000, 9, 1101, 1442084400000, null, 115, 9, 3] - - ["VG", 1442062800000, -238, -238, null, null, -238, -238, 1] - - ["VN", 1442023200000, -9, 1560, null, 1442026800000, -9, -10, 1] - - ["VN", 1442026800000, 63, 1560, null, 1442034000000, -9, -10, 1] - - ["VN", 1442034000000, -29, 1560, 1442023200000, 1442037600000, -9, -10, 1] - - ["VN", 1442037600000, -11, 1560, 1442026800000, 1442041200000, -9, -10, 1] - - ["VN", 1442041200000, 0, 1560, 1442034000000, 1442048400000, -9, -10, 1] - - ["VN", 1442048400000, -15, 1560, 1442037600000, 1442052000000, -9, -10, 2] - - ["VN", 1442052000000, 90, 1560, 1442041200000, 1442055600000, -9, -10, 2] - - ["VN", 1442055600000, 37, 1560, 1442048400000, 1442059200000, -9, -10, 2] - - ["VN", 1442059200000, 8, 1560, 1442052000000, 1442062800000, -9, -10, 2] - - ["VN", 1442062800000, 146, 1560, 1442055600000, 1442066400000, -9, -10, 3] - - ["VN", 1442066400000, 811, 1560, 1442059200000, 1442070000000, -9, -10, 3] - - ["VN", 1442070000000, 479, 1560, 1442062800000, 1442084400000, -9, -10, 3] - - ["VN", 1442084400000, -10, 1560, 1442066400000, null, -9, -10, 3] - - ["ZA", 1442034000000, -3, 127, null, 1442048400000, -3, 1, 1] - - ["ZA", 1442048400000, 79, 127, null, 1442059200000, -3, 1, 1] - - ["ZA", 1442059200000, 50, 127, 1442034000000, 1442070000000, -3, 1, 2] - - ["ZA", 1442070000000, 0, 127, 1442048400000, 1442091600000, -3, 1, 2] - - ["ZA", 1442091600000, 1, 127, 1442059200000, null, -3, 1, 3] - - ["ZM", 1442041200000, 133, 133, null, null, 133, 133, 1] - - ["ZW", 1442044800000, 0, 254, null, 1442048400000, 0, 254, 1] - - ["ZW", 1442048400000, 254, 254, null, null, 0, 254, 2] \ No newline at end of file + - [ "", 1442016000000, 29873, 8414700, null, 1442019600000, 29873, 276159, 1 ] + - [ "", 1442019600000, 173892, 8414700, null, 1442023200000, 29873, 276159, 1 ] + - [ "", 1442023200000, 399636, 8414700, 1442016000000, 1442026800000, 29873, 276159, 1 ] + - [ "", 1442026800000, 252626, 8414700, 1442019600000, 1442030400000, 29873, 276159, 1 ] + - [ "", 1442030400000, 166672, 8414700, 1442023200000, 1442034000000, 29873, 276159, 1 ] + - [ "", 1442034000000, 330957, 8414700, 1442026800000, 1442037600000, 29873, 276159, 1 ] + - [ "", 1442037600000, 200605, 8414700, 1442030400000, 1442041200000, 29873, 276159, 1 ] + - [ "", 1442041200000, 543450, 8414700, 1442034000000, 1442044800000, 29873, 276159, 1 ] + - [ "", 1442044800000, 316002, 8414700, 1442037600000, 1442048400000, 29873, 276159, 2 ] + - [ "", 1442048400000, 308316, 8414700, 1442041200000, 1442052000000, 29873, 276159, 2 ] + - [ "", 1442052000000, 787370, 8414700, 1442044800000, 1442055600000, 29873, 276159, 2 ] + - [ "", 1442055600000, 283958, 8414700, 1442048400000, 1442059200000, 29873, 276159, 2 ] + - [ "", 1442059200000, 459297, 8414700, 1442052000000, 1442062800000, 29873, 276159, 2 ] + - [ "", 1442062800000, 389465, 8414700, 1442055600000, 1442066400000, 29873, 276159, 2 ] + - [ "", 1442066400000, 351584, 8414700, 1442059200000, 1442070000000, 29873, 276159, 2 ] + - [ "", 1442070000000, 358515, 8414700, 1442062800000, 1442073600000, 29873, 276159, 2 ] + - [ "", 1442073600000, 375394, 8414700, 1442066400000, 1442077200000, 29873, 276159, 3 ] + - [ "", 1442077200000, 392483, 8414700, 1442070000000, 1442080800000, 29873, 276159, 3 ] + - [ "", 1442080800000, 453077, 8414700, 1442073600000, 1442084400000, 29873, 276159, 3 ] + - [ "", 1442084400000, 372569, 8414700, 1442077200000, 1442088000000, 29873, 276159, 3 ] + - [ "", 1442088000000, 303872, 8414700, 1442080800000, 1442091600000, 29873, 276159, 3 ] + - [ "", 1442091600000, 514427, 8414700, 1442084400000, 1442095200000, 29873, 276159, 3 ] + - [ "", 1442095200000, 374501, 8414700, 1442088000000, 1442098800000, 29873, 276159, 3 ] + - [ "", 1442098800000, 276159, 8414700, 1442091600000, null, 29873, 276159, 3 ] + - [ "AE", 1442030400000, 118, 6547, null, 1442044800000, 118, 42, 1 ] + - [ "AE", 1442044800000, -7, 6547, null, 1442048400000, 118, 42, 1 ] + - [ "AE", 1442048400000, 39, 6547, 1442030400000, 1442052000000, 118, 42, 1 ] + - [ "AE", 1442052000000, -3, 6547, 1442044800000, 1442059200000, 118, 42, 2 ] + - [ "AE", 1442059200000, -11, 6547, 1442048400000, 1442070000000, 118, 42, 2 ] + - [ "AE", 1442070000000, 46, 6547, 1442052000000, 1442077200000, 118, 42, 2 ] + - [ "AE", 1442077200000, 6323, 6547, 1442059200000, 1442080800000, 118, 42, 3 ] + - [ "AE", 1442080800000, 42, 6547, 1442070000000, null, 118, 42, 3 ] + - [ "AL", 1442077200000, 26, 80, null, 1442091600000, 26, 54, 1 ] + - [ "AL", 1442091600000, 54, 80, null, null, 26, 54, 2 ] + - [ "AO", 1442041200000, -26, 740, null, 1442052000000, -26, 722, 1 ] + - [ "AO", 1442052000000, -18, 740, null, 1442088000000, -26, 722, 1 ] + - [ "AO", 1442088000000, 62, 740, 1442041200000, 1442098800000, -26, 722, 2 ] + - [ "AO", 1442098800000, 722, 740, 1442052000000, null, -26, 722, 3 ] + - [ "AR", 1442019600000, 1, 4450, null, 1442023200000, 1, 64, 1 ] + - [ "AR", 1442023200000, 2514, 4450, null, 1442026800000, 1, 64, 1 ] + - [ "AR", 1442026800000, 644, 4450, 1442019600000, 1442030400000, 1, 64, 1 ] + - [ "AR", 1442030400000, -3, 4450, 1442023200000, 1442034000000, 1, 64, 1 ] + - [ "AR", 1442034000000, 212, 4450, 1442026800000, 1442037600000, 1, 64, 1 ] + - [ "AR", 1442037600000, 81, 4450, 1442030400000, 1442055600000, 1, 64, 1 ] + - [ "AR", 1442055600000, -54, 4450, 1442034000000, 1442059200000, 1, 64, 2 ] + - [ "AR", 1442059200000, 210, 4450, 1442037600000, 1442062800000, 1, 64, 2 ] + - [ "AR", 1442062800000, 29, 4450, 1442055600000, 1442066400000, 1, 64, 2 ] + - [ "AR", 1442066400000, 0, 4450, 1442059200000, 1442070000000, 1, 64, 2 ] + - [ "AR", 1442070000000, 377, 4450, 1442062800000, 1442077200000, 1, 64, 2 ] + - [ "AR", 1442077200000, -591, 4450, 1442066400000, 1442080800000, 1, 64, 2 ] + - [ "AR", 1442080800000, 1, 4450, 1442070000000, 1442084400000, 1, 64, 3 ] + - [ "AR", 1442084400000, -5, 4450, 1442077200000, 1442091600000, 1, 64, 3 ] + - [ "AR", 1442091600000, 340, 4450, 1442080800000, 1442095200000, 1, 64, 3 ] + - [ "AR", 1442095200000, 630, 4450, 1442084400000, 1442098800000, 1, 64, 3 ] + - [ "AR", 1442098800000, 64, 4450, 1442091600000, null, 1, 64, 3 ] + - [ "AT", 1442052000000, 4793, 12047, null, 1442062800000, 4793, 89, 1 ] + - [ "AT", 1442062800000, -155, 12047, null, 1442066400000, 4793, 89, 1 ] + - [ "AT", 1442066400000, 0, 12047, 1442052000000, 1442070000000, 4793, 89, 1 ] + - [ "AT", 1442070000000, 272, 12047, 1442062800000, 1442084400000, 4793, 89, 2 ] + - [ "AT", 1442084400000, -2, 12047, 1442066400000, 1442088000000, 4793, 89, 2 ] + - [ "AT", 1442088000000, 7050, 12047, 1442070000000, 1442091600000, 4793, 89, 3 ] + - [ "AT", 1442091600000, 89, 12047, 1442084400000, null, 4793, 89, 3 ] + - [ "AU", 1442016000000, 0, 2700, null, 1442019600000, 0, 518, 1 ] + - [ "AU", 1442019600000, 253, 2700, null, 1442023200000, 0, 518, 1 ] + - [ "AU", 1442023200000, 52, 2700, 1442016000000, 1442026800000, 0, 518, 1 ] + - [ "AU", 1442026800000, 188, 2700, 1442019600000, 1442030400000, 0, 518, 1 ] + - [ "AU", 1442030400000, -377, 2700, 1442023200000, 1442034000000, 0, 518, 1 ] + - [ "AU", 1442034000000, 283, 2700, 1442026800000, 1442037600000, 0, 518, 1 ] + - [ "AU", 1442037600000, 3, 2700, 1442030400000, 1442041200000, 0, 518, 1 ] + - [ "AU", 1442041200000, 194, 2700, 1442034000000, 1442044800000, 0, 518, 2 ] + - [ "AU", 1442044800000, 373, 2700, 1442037600000, 1442048400000, 0, 518, 2 ] + - [ "AU", 1442048400000, 135, 2700, 1442041200000, 1442052000000, 0, 518, 2 ] + - [ "AU", 1442052000000, -643, 2700, 1442044800000, 1442055600000, 0, 518, 2 ] + - [ "AU", 1442055600000, 182, 2700, 1442048400000, 1442059200000, 0, 518, 2 ] + - [ "AU", 1442059200000, 38, 2700, 1442052000000, 1442066400000, 0, 518, 2 ] + - [ "AU", 1442066400000, -21, 2700, 1442055600000, 1442070000000, 0, 518, 3 ] + - [ "AU", 1442070000000, -12, 2700, 1442059200000, 1442077200000, 0, 518, 3 ] + - [ "AU", 1442077200000, 1, 2700, 1442066400000, 1442091600000, 0, 518, 3 ] + - [ "AU", 1442091600000, 1138, 2700, 1442070000000, 1442095200000, 0, 518, 3 ] + - [ "AU", 1442095200000, 395, 2700, 1442077200000, 1442098800000, 0, 518, 3 ] + - [ "AU", 1442098800000, 518, 2700, 1442091600000, null, 0, 518, 3 ] + - [ "BA", 1442048400000, -13, -178, null, 1442052000000, -13, -1, 1 ] + - [ "BA", 1442052000000, 38, -178, null, 1442055600000, -13, -1, 1 ] + - [ "BA", 1442055600000, -202, -178, 1442048400000, 1442084400000, -13, -1, 2 ] + - [ "BA", 1442084400000, -1, -178, 1442052000000, null, -13, -1, 3 ] + - [ "BD", 1442019600000, 0, 1106, null, 1442041200000, 0, -2, 1 ] + - [ "BD", 1442041200000, 854, 1106, null, 1442066400000, 0, -2, 1 ] + - [ "BD", 1442066400000, 76, 1106, 1442019600000, 1442073600000, 0, -2, 2 ] + - [ "BD", 1442073600000, 103, 1106, 1442041200000, 1442077200000, 0, -2, 2 ] + - [ "BD", 1442077200000, 75, 1106, 1442066400000, 1442091600000, 0, -2, 3 ] + - [ "BD", 1442091600000, -2, 1106, 1442073600000, null, 0, -2, 3 ] + - [ "BE", 1442030400000, -103, 795, null, 1442048400000, -103, 9, 1 ] + - [ "BE", 1442048400000, 59, 795, null, 1442052000000, -103, 9, 1 ] + - [ "BE", 1442052000000, -1, 795, 1442030400000, 1442055600000, -103, 9, 1 ] + - [ "BE", 1442055600000, 233, 795, 1442048400000, 1442062800000, -103, 9, 1 ] + - [ "BE", 1442062800000, 91, 795, 1442052000000, 1442066400000, -103, 9, 2 ] + - [ "BE", 1442066400000, 136, 795, 1442055600000, 1442073600000, -103, 9, 2 ] + - [ "BE", 1442073600000, 19, 795, 1442062800000, 1442080800000, -103, 9, 2 ] + - [ "BE", 1442080800000, 1, 795, 1442066400000, 1442084400000, -103, 9, 2 ] + - [ "BE", 1442084400000, 183, 795, 1442073600000, 1442088000000, -103, 9, 3 ] + - [ "BE", 1442088000000, 67, 795, 1442080800000, 1442091600000, -103, 9, 3 ] + - [ "BE", 1442091600000, 101, 795, 1442084400000, 1442098800000, -103, 9, 3 ] + - [ "BE", 1442098800000, 9, 795, 1442088000000, null, -103, 9, 3 ] + - [ "BG", 1442041200000, 9, 19592, null, 1442052000000, 9, 401, 1 ] + - [ "BG", 1442052000000, 18936, 19592, null, 1442059200000, 9, 401, 1 ] + - [ "BG", 1442059200000, 191, 19592, 1442041200000, 1442070000000, 9, 401, 2 ] + - [ "BG", 1442070000000, 55, 19592, 1442052000000, 1442084400000, 9, 401, 2 ] + - [ "BG", 1442084400000, 401, 19592, 1442059200000, null, 9, 401, 3 ] + - [ "BH", 1442052000000, 44, 44, null, null, 44, 44, 1 ] + - [ "BO", 1442080800000, 4, 4, null, 1442088000000, 4, -4, 1 ] + - [ "BO", 1442088000000, 4, 4, null, 1442095200000, 4, -4, 2 ] + - [ "BO", 1442095200000, -4, 4, 1442080800000, null, 4, -4, 3 ] + - [ "BR", 1442016000000, -248, 8550, null, 1442019600000, -248, -645, 1 ] + - [ "BR", 1442019600000, 372, 8550, null, 1442023200000, -248, -645, 1 ] + - [ "BR", 1442023200000, 879, 8550, 1442016000000, 1442026800000, -248, -645, 1 ] + - [ "BR", 1442026800000, 51, 8550, 1442019600000, 1442030400000, -248, -645, 1 ] + - [ "BR", 1442030400000, 30, 8550, 1442023200000, 1442034000000, -248, -645, 1 ] + - [ "BR", 1442034000000, 21, 8550, 1442026800000, 1442037600000, -248, -645, 1 ] + - [ "BR", 1442037600000, 267, 8550, 1442030400000, 1442041200000, -248, -645, 1 ] + - [ "BR", 1442041200000, 3, 8550, 1442034000000, 1442044800000, -248, -645, 1 ] + - [ "BR", 1442044800000, 71, 8550, 1442037600000, 1442052000000, -248, -645, 2 ] + - [ "BR", 1442052000000, 232, 8550, 1442041200000, 1442055600000, -248, -645, 2 ] + - [ "BR", 1442055600000, 242, 8550, 1442044800000, 1442059200000, -248, -645, 2 ] + - [ "BR", 1442059200000, 73, 8550, 1442052000000, 1442062800000, -248, -645, 2 ] + - [ "BR", 1442062800000, 93, 8550, 1442055600000, 1442066400000, -248, -645, 2 ] + - [ "BR", 1442066400000, 1034, 8550, 1442059200000, 1442070000000, -248, -645, 2 ] + - [ "BR", 1442070000000, 536, 8550, 1442062800000, 1442073600000, -248, -645, 2 ] + - [ "BR", 1442073600000, 2087, 8550, 1442066400000, 1442077200000, -248, -645, 2 ] + - [ "BR", 1442077200000, 2253, 8550, 1442070000000, 1442080800000, -248, -645, 3 ] + - [ "BR", 1442080800000, -267, 8550, 1442073600000, 1442084400000, -248, -645, 3 ] + - [ "BR", 1442084400000, 492, 8550, 1442077200000, 1442088000000, -248, -645, 3 ] + - [ "BR", 1442088000000, 215, 8550, 1442080800000, 1442091600000, -248, -645, 3 ] + - [ "BR", 1442091600000, 11, 8550, 1442084400000, 1442095200000, -248, -645, 3 ] + - [ "BR", 1442095200000, 748, 8550, 1442088000000, 1442098800000, -248, -645, 3 ] + - [ "BR", 1442098800000, -645, 8550, 1442091600000, null, -248, -645, 3 ] + - [ "BY", 1442055600000, 1, 2153, null, 1442059200000, 1, 33, 1 ] + - [ "BY", 1442059200000, 1464, 2153, null, 1442073600000, 1, 33, 1 ] + - [ "BY", 1442073600000, 596, 2153, 1442055600000, 1442077200000, 1, 33, 1 ] + - [ "BY", 1442077200000, 30, 2153, 1442059200000, 1442080800000, 1, 33, 2 ] + - [ "BY", 1442080800000, 28, 2153, 1442073600000, 1442084400000, 1, 33, 2 ] + - [ "BY", 1442084400000, 1, 2153, 1442077200000, 1442088000000, 1, 33, 3 ] + - [ "BY", 1442088000000, 33, 2153, 1442080800000, null, 1, 33, 3 ] + - [ "CA", 1442016000000, -371, 8544, null, 1442019600000, -371, 164, 1 ] + - [ "CA", 1442019600000, 2184, 8544, null, 1442023200000, -371, 164, 1 ] + - [ "CA", 1442023200000, 286, 8544, 1442016000000, 1442026800000, -371, 164, 1 ] + - [ "CA", 1442026800000, 2216, 8544, 1442019600000, 1442030400000, -371, 164, 1 ] + - [ "CA", 1442030400000, -47, 8544, 1442023200000, 1442034000000, -371, 164, 1 ] + - [ "CA", 1442034000000, 178, 8544, 1442026800000, 1442037600000, -371, 164, 1 ] + - [ "CA", 1442037600000, -132, 8544, 1442030400000, 1442041200000, -371, 164, 1 ] + - [ "CA", 1442041200000, 5, 8544, 1442034000000, 1442044800000, -371, 164, 1 ] + - [ "CA", 1442044800000, 1, 8544, 1442037600000, 1442052000000, -371, 164, 2 ] + - [ "CA", 1442052000000, 38, 8544, 1442041200000, 1442059200000, -371, 164, 2 ] + - [ "CA", 1442059200000, 1036, 8544, 1442044800000, 1442062800000, -371, 164, 2 ] + - [ "CA", 1442062800000, -367, 8544, 1442052000000, 1442066400000, -371, 164, 2 ] + - [ "CA", 1442066400000, 307, 8544, 1442059200000, 1442070000000, -371, 164, 2 ] + - [ "CA", 1442070000000, 185, 8544, 1442062800000, 1442073600000, -371, 164, 2 ] + - [ "CA", 1442073600000, 86, 8544, 1442066400000, 1442077200000, -371, 164, 2 ] + - [ "CA", 1442077200000, -282, 8544, 1442070000000, 1442080800000, -371, 164, 3 ] + - [ "CA", 1442080800000, 481, 8544, 1442073600000, 1442084400000, -371, 164, 3 ] + - [ "CA", 1442084400000, 44, 8544, 1442077200000, 1442088000000, -371, 164, 3 ] + - [ "CA", 1442088000000, 35, 8544, 1442080800000, 1442091600000, -371, 164, 3 ] + - [ "CA", 1442091600000, 2858, 8544, 1442084400000, 1442095200000, -371, 164, 3 ] + - [ "CA", 1442095200000, -361, 8544, 1442088000000, 1442098800000, -371, 164, 3 ] + - [ "CA", 1442098800000, 164, 8544, 1442091600000, null, -371, 164, 3 ] + - [ "CH", 1442037600000, 59, 753, null, 1442041200000, 59, 67, 1 ] + - [ "CH", 1442041200000, 198, 753, null, 1442044800000, 59, 67, 1 ] + - [ "CH", 1442044800000, -54, 753, 1442037600000, 1442048400000, 59, 67, 1 ] + - [ "CH", 1442048400000, 24, 753, 1442041200000, 1442052000000, 59, 67, 1 ] + - [ "CH", 1442052000000, 47, 753, 1442044800000, 1442055600000, 59, 67, 2 ] + - [ "CH", 1442055600000, 0, 753, 1442048400000, 1442062800000, 59, 67, 2 ] + - [ "CH", 1442062800000, 22, 753, 1442052000000, 1442070000000, 59, 67, 2 ] + - [ "CH", 1442070000000, 11, 753, 1442055600000, 1442073600000, 59, 67, 2 ] + - [ "CH", 1442073600000, 360, 753, 1442062800000, 1442077200000, 59, 67, 3 ] + - [ "CH", 1442077200000, 6, 753, 1442070000000, 1442084400000, 59, 67, 3 ] + - [ "CH", 1442084400000, 13, 753, 1442073600000, 1442091600000, 59, 67, 3 ] + - [ "CH", 1442091600000, 67, 753, 1442077200000, null, 59, 67, 3 ] + - [ "CL", 1442016000000, 161, 533, null, 1442019600000, 161, 9, 1 ] + - [ "CL", 1442019600000, -370, 533, null, 1442023200000, 161, 9, 1 ] + - [ "CL", 1442023200000, 15, 533, 1442016000000, 1442030400000, 161, 9, 1 ] + - [ "CL", 1442030400000, 40, 533, 1442019600000, 1442034000000, 161, 9, 1 ] + - [ "CL", 1442034000000, -1, 533, 1442023200000, 1442037600000, 161, 9, 1 ] + - [ "CL", 1442037600000, 2, 533, 1442030400000, 1442041200000, 161, 9, 1 ] + - [ "CL", 1442041200000, -1, 533, 1442034000000, 1442052000000, 161, 9, 1 ] + - [ "CL", 1442052000000, 390, 533, 1442037600000, 1442059200000, 161, 9, 2 ] + - [ "CL", 1442059200000, -12, 533, 1442041200000, 1442062800000, 161, 9, 2 ] + - [ "CL", 1442062800000, 17, 533, 1442052000000, 1442066400000, 161, 9, 2 ] + - [ "CL", 1442066400000, -41, 533, 1442059200000, 1442070000000, 161, 9, 2 ] + - [ "CL", 1442070000000, 13, 533, 1442062800000, 1442073600000, 161, 9, 2 ] + - [ "CL", 1442073600000, 153, 533, 1442066400000, 1442077200000, 161, 9, 2 ] + - [ "CL", 1442077200000, -15, 533, 1442070000000, 1442080800000, 161, 9, 2 ] + - [ "CL", 1442080800000, 17, 533, 1442073600000, 1442084400000, 161, 9, 3 ] + - [ "CL", 1442084400000, 126, 533, 1442077200000, 1442088000000, 161, 9, 3 ] + - [ "CL", 1442088000000, 286, 533, 1442080800000, 1442091600000, 161, 9, 3 ] + - [ "CL", 1442091600000, 20, 533, 1442084400000, 1442095200000, 161, 9, 3 ] + - [ "CL", 1442095200000, -276, 533, 1442088000000, 1442098800000, 161, 9, 3 ] + - [ "CL", 1442098800000, 9, 533, 1442091600000, null, 161, 9, 3 ] + - [ "CN", 1442023200000, -13, 583, null, 1442026800000, -13, -1, 1 ] + - [ "CN", 1442026800000, 154, 583, null, 1442037600000, -13, -1, 1 ] + - [ "CN", 1442037600000, 98, 583, 1442023200000, 1442048400000, -13, -1, 1 ] + - [ "CN", 1442048400000, 293, 583, 1442026800000, 1442052000000, -13, -1, 1 ] + - [ "CN", 1442052000000, 0, 583, 1442037600000, 1442055600000, -13, -1, 2 ] + - [ "CN", 1442055600000, 69, 583, 1442048400000, 1442059200000, -13, -1, 2 ] + - [ "CN", 1442059200000, 8, 583, 1442052000000, 1442066400000, -13, -1, 2 ] + - [ "CN", 1442066400000, -15, 583, 1442055600000, 1442080800000, -13, -1, 3 ] + - [ "CN", 1442080800000, -10, 583, 1442059200000, 1442084400000, -13, -1, 3 ] + - [ "CN", 1442084400000, -1, 583, 1442066400000, null, -13, -1, 3 ] + - [ "CO", 1442016000000, 16, 59611, null, 1442019600000, 16, 83, 1 ] + - [ "CO", 1442019600000, 12, 59611, null, 1442023200000, 16, 83, 1 ] + - [ "CO", 1442023200000, 9, 59611, 1442016000000, 1442030400000, 16, 83, 1 ] + - [ "CO", 1442030400000, 441, 59611, 1442019600000, 1442059200000, 16, 83, 1 ] + - [ "CO", 1442059200000, 473, 59611, 1442023200000, 1442066400000, 16, 83, 1 ] + - [ "CO", 1442066400000, 288, 59611, 1442030400000, 1442070000000, 16, 83, 2 ] + - [ "CO", 1442070000000, -45, 59611, 1442059200000, 1442073600000, 16, 83, 2 ] + - [ "CO", 1442073600000, 39860, 59611, 1442066400000, 1442077200000, 16, 83, 2 ] + - [ "CO", 1442077200000, 581, 59611, 1442070000000, 1442080800000, 16, 83, 2 ] + - [ "CO", 1442080800000, 25, 59611, 1442073600000, 1442084400000, 16, 83, 2 ] + - [ "CO", 1442084400000, 51, 59611, 1442077200000, 1442088000000, 16, 83, 3 ] + - [ "CO", 1442088000000, 17150, 59611, 1442080800000, 1442091600000, 16, 83, 3 ] + - [ "CO", 1442091600000, 377, 59611, 1442084400000, 1442095200000, 16, 83, 3 ] + - [ "CO", 1442095200000, 290, 59611, 1442088000000, 1442098800000, 16, 83, 3 ] + - [ "CO", 1442098800000, 83, 59611, 1442091600000, null, 16, 83, 3 ] + - [ "CR", 1442019600000, 62, 3241, null, 1442023200000, 62, 72, 1 ] + - [ "CR", 1442023200000, 62, 3241, null, 1442026800000, 62, 72, 1 ] + - [ "CR", 1442026800000, 140, 3241, 1442019600000, 1442030400000, 62, 72, 1 ] + - [ "CR", 1442030400000, 2497, 3241, 1442023200000, 1442041200000, 62, 72, 2 ] + - [ "CR", 1442041200000, 51, 3241, 1442026800000, 1442044800000, 62, 72, 2 ] + - [ "CR", 1442044800000, 194, 3241, 1442030400000, 1442048400000, 62, 72, 2 ] + - [ "CR", 1442048400000, 163, 3241, 1442041200000, 1442088000000, 62, 72, 3 ] + - [ "CR", 1442088000000, 72, 3241, 1442044800000, null, 62, 72, 3 ] + - [ "CZ", 1442026800000, -19, 3479, null, 1442034000000, -19, 2, 1 ] + - [ "CZ", 1442034000000, 78, 3479, null, 1442037600000, -19, 2, 1 ] + - [ "CZ", 1442037600000, 18, 3479, 1442026800000, 1442055600000, -19, 2, 1 ] + - [ "CZ", 1442055600000, 1073, 3479, 1442034000000, 1442059200000, -19, 2, 1 ] + - [ "CZ", 1442059200000, 21, 3479, 1442037600000, 1442062800000, -19, 2, 2 ] + - [ "CZ", 1442062800000, 0, 3479, 1442055600000, 1442070000000, -19, 2, 2 ] + - [ "CZ", 1442070000000, 168, 3479, 1442059200000, 1442073600000, -19, 2, 2 ] + - [ "CZ", 1442073600000, 2051, 3479, 1442062800000, 1442077200000, -19, 2, 2 ] + - [ "CZ", 1442077200000, 115, 3479, 1442070000000, 1442080800000, -19, 2, 3 ] + - [ "CZ", 1442080800000, -28, 3479, 1442073600000, 1442098800000, -19, 2, 3 ] + - [ "CZ", 1442098800000, 2, 3479, 1442077200000, null, -19, 2, 3 ] + - [ "DE", 1442016000000, 167, 25583, null, 1442019600000, 167, 329, 1 ] + - [ "DE", 1442019600000, 0, 25583, null, 1442023200000, 167, 329, 1 ] + - [ "DE", 1442023200000, 64, 25583, 1442016000000, 1442030400000, 167, 329, 1 ] + - [ "DE", 1442030400000, 373, 25583, 1442019600000, 1442034000000, 167, 329, 1 ] + - [ "DE", 1442034000000, 358, 25583, 1442023200000, 1442037600000, 167, 329, 1 ] + - [ "DE", 1442037600000, 544, 25583, 1442030400000, 1442041200000, 167, 329, 1 ] + - [ "DE", 1442041200000, 197, 25583, 1442034000000, 1442044800000, 167, 329, 1 ] + - [ "DE", 1442044800000, 979, 25583, 1442037600000, 1442048400000, 167, 329, 1 ] + - [ "DE", 1442048400000, 811, 25583, 1442041200000, 1442052000000, 167, 329, 2 ] + - [ "DE", 1442052000000, 1600, 25583, 1442044800000, 1442055600000, 167, 329, 2 ] + - [ "DE", 1442055600000, 1523, 25583, 1442048400000, 1442059200000, 167, 329, 2 ] + - [ "DE", 1442059200000, 289, 25583, 1442052000000, 1442062800000, 167, 329, 2 ] + - [ "DE", 1442062800000, 283, 25583, 1442055600000, 1442066400000, 167, 329, 2 ] + - [ "DE", 1442066400000, 1577, 25583, 1442059200000, 1442070000000, 167, 329, 2 ] + - [ "DE", 1442070000000, 1666, 25583, 1442062800000, 1442073600000, 167, 329, 2 ] + - [ "DE", 1442073600000, 6075, 25583, 1442066400000, 1442077200000, 167, 329, 2 ] + - [ "DE", 1442077200000, 2188, 25583, 1442070000000, 1442080800000, 167, 329, 3 ] + - [ "DE", 1442080800000, 1133, 25583, 1442073600000, 1442084400000, 167, 329, 3 ] + - [ "DE", 1442084400000, -125, 25583, 1442077200000, 1442088000000, 167, 329, 3 ] + - [ "DE", 1442088000000, 190, 25583, 1442080800000, 1442091600000, 167, 329, 3 ] + - [ "DE", 1442091600000, 4355, 25583, 1442084400000, 1442095200000, 167, 329, 3 ] + - [ "DE", 1442095200000, 1007, 25583, 1442088000000, 1442098800000, 167, 329, 3 ] + - [ "DE", 1442098800000, 329, 25583, 1442091600000, null, 167, 329, 3 ] + - [ "DK", 1442037600000, 10, 594, null, 1442044800000, 10, 0, 1 ] + - [ "DK", 1442044800000, 36, 594, null, 1442048400000, 10, 0, 1 ] + - [ "DK", 1442048400000, -5, 594, 1442037600000, 1442055600000, 10, 0, 1 ] + - [ "DK", 1442055600000, 42, 594, 1442044800000, 1442059200000, 10, 0, 1 ] + - [ "DK", 1442059200000, 0, 594, 1442048400000, 1442062800000, 10, 0, 2 ] + - [ "DK", 1442062800000, 1, 594, 1442055600000, 1442066400000, 10, 0, 2 ] + - [ "DK", 1442066400000, 416, 594, 1442059200000, 1442077200000, 10, 0, 2 ] + - [ "DK", 1442077200000, -9, 594, 1442062800000, 1442080800000, 10, 0, 2 ] + - [ "DK", 1442080800000, 61, 594, 1442066400000, 1442084400000, 10, 0, 3 ] + - [ "DK", 1442084400000, -97, 594, 1442077200000, 1442091600000, 10, 0, 3 ] + - [ "DK", 1442091600000, 139, 594, 1442080800000, 1442095200000, 10, 0, 3 ] + - [ "DK", 1442095200000, 0, 594, 1442084400000, null, 10, 0, 3 ] + - [ "DO", 1442023200000, 8, 264, null, 1442066400000, 8, 13, 1 ] + - [ "DO", 1442066400000, 35, 264, null, 1442073600000, 8, 13, 1 ] + - [ "DO", 1442073600000, 200, 264, 1442023200000, 1442084400000, 8, 13, 2 ] + - [ "DO", 1442084400000, 8, 264, 1442066400000, 1442095200000, 8, 13, 2 ] + - [ "DO", 1442095200000, 13, 264, 1442073600000, null, 8, 13, 3 ] + - [ "DZ", 1442077200000, -1, -1, null, null, -1, -1, 1 ] + - [ "EC", 1442019600000, 29, 232, null, 1442023200000, 29, 10, 1 ] + - [ "EC", 1442023200000, -9, 232, null, 1442030400000, 29, 10, 1 ] + - [ "EC", 1442030400000, 0, 232, 1442019600000, 1442077200000, 29, 10, 2 ] + - [ "EC", 1442077200000, -366, 232, 1442023200000, 1442084400000, 29, 10, 2 ] + - [ "EC", 1442084400000, 568, 232, 1442030400000, 1442095200000, 29, 10, 3 ] + - [ "EC", 1442095200000, 10, 232, 1442077200000, null, 29, 10, 3 ] + - [ "EE", 1442041200000, 37, 18, null, 1442044800000, 37, -19, 1 ] + - [ "EE", 1442044800000, -19, 18, null, null, 37, -19, 2 ] + - [ "EG", 1442026800000, 16, 170, null, 1442055600000, 16, 27, 1 ] + - [ "EG", 1442055600000, 14, 170, null, 1442062800000, 16, 27, 1 ] + - [ "EG", 1442062800000, 112, 170, 1442026800000, 1442073600000, 16, 27, 2 ] + - [ "EG", 1442073600000, 1, 170, 1442055600000, 1442091600000, 16, 27, 2 ] + - [ "EG", 1442091600000, 27, 170, 1442062800000, null, 16, 27, 3 ] + - [ "ES", 1442019600000, 103, 7449, null, 1442023200000, 103, 458, 1 ] + - [ "ES", 1442023200000, -5, 7449, null, 1442034000000, 103, 458, 1 ] + - [ "ES", 1442034000000, -52, 7449, 1442019600000, 1442037600000, 103, 458, 1 ] + - [ "ES", 1442037600000, 3, 7449, 1442023200000, 1442041200000, 103, 458, 1 ] + - [ "ES", 1442041200000, 118, 7449, 1442034000000, 1442044800000, 103, 458, 1 ] + - [ "ES", 1442044800000, -169, 7449, 1442037600000, 1442048400000, 103, 458, 1 ] + - [ "ES", 1442048400000, 158, 7449, 1442041200000, 1442052000000, 103, 458, 1 ] + - [ "ES", 1442052000000, -4, 7449, 1442044800000, 1442055600000, 103, 458, 2 ] + - [ "ES", 1442055600000, 495, 7449, 1442048400000, 1442059200000, 103, 458, 2 ] + - [ "ES", 1442059200000, 1086, 7449, 1442052000000, 1442062800000, 103, 458, 2 ] + - [ "ES", 1442062800000, -71, 7449, 1442055600000, 1442066400000, 103, 458, 2 ] + - [ "ES", 1442066400000, 461, 7449, 1442059200000, 1442070000000, 103, 458, 2 ] + - [ "ES", 1442070000000, 61, 7449, 1442062800000, 1442073600000, 103, 458, 2 ] + - [ "ES", 1442073600000, 154, 7449, 1442066400000, 1442077200000, 103, 458, 2 ] + - [ "ES", 1442077200000, 1240, 7449, 1442070000000, 1442084400000, 103, 458, 3 ] + - [ "ES", 1442084400000, 337, 7449, 1442073600000, 1442088000000, 103, 458, 3 ] + - [ "ES", 1442088000000, -130, 7449, 1442077200000, 1442091600000, 103, 458, 3 ] + - [ "ES", 1442091600000, 700, 7449, 1442084400000, 1442095200000, 103, 458, 3 ] + - [ "ES", 1442095200000, 2506, 7449, 1442088000000, 1442098800000, 103, 458, 3 ] + - [ "ES", 1442098800000, 458, 7449, 1442091600000, null, 103, 458, 3 ] + - [ "FI", 1442030400000, 1491, 3579, null, 1442037600000, 1491, 69, 1 ] + - [ "FI", 1442037600000, 14, 3579, null, 1442048400000, 1491, 69, 1 ] + - [ "FI", 1442048400000, 12, 3579, 1442030400000, 1442052000000, 1491, 69, 1 ] + - [ "FI", 1442052000000, 186, 3579, 1442037600000, 1442059200000, 1491, 69, 1 ] + - [ "FI", 1442059200000, 407, 3579, 1442048400000, 1442062800000, 1491, 69, 2 ] + - [ "FI", 1442062800000, 19, 3579, 1442052000000, 1442066400000, 1491, 69, 2 ] + - [ "FI", 1442066400000, 183, 3579, 1442059200000, 1442073600000, 1491, 69, 2 ] + - [ "FI", 1442073600000, -1, 3579, 1442062800000, 1442077200000, 1491, 69, 2 ] + - [ "FI", 1442077200000, 200, 3579, 1442066400000, 1442080800000, 1491, 69, 3 ] + - [ "FI", 1442080800000, 104, 3579, 1442073600000, 1442084400000, 1491, 69, 3 ] + - [ "FI", 1442084400000, 895, 3579, 1442077200000, 1442095200000, 1491, 69, 3 ] + - [ "FI", 1442095200000, 69, 3579, 1442080800000, null, 1491, 69, 3 ] + - [ "FR", 1442016000000, -1, 37281, null, 1442019600000, -1, 136, 1 ] + - [ "FR", 1442019600000, 585, 37281, null, 1442023200000, -1, 136, 1 ] + - [ "FR", 1442023200000, 628, 37281, 1442016000000, 1442026800000, -1, 136, 1 ] + - [ "FR", 1442026800000, 86, 37281, 1442019600000, 1442034000000, -1, 136, 1 ] + - [ "FR", 1442034000000, 476, 37281, 1442023200000, 1442037600000, -1, 136, 1 ] + - [ "FR", 1442037600000, 4174, 37281, 1442026800000, 1442041200000, -1, 136, 1 ] + - [ "FR", 1442041200000, 604, 37281, 1442034000000, 1442044800000, -1, 136, 1 ] + - [ "FR", 1442044800000, 172, 37281, 1442037600000, 1442048400000, -1, 136, 1 ] + - [ "FR", 1442048400000, 3027, 37281, 1442041200000, 1442052000000, -1, 136, 2 ] + - [ "FR", 1442052000000, 637, 37281, 1442044800000, 1442055600000, -1, 136, 2 ] + - [ "FR", 1442055600000, 463, 37281, 1442048400000, 1442059200000, -1, 136, 2 ] + - [ "FR", 1442059200000, 4650, 37281, 1442052000000, 1442062800000, -1, 136, 2 ] + - [ "FR", 1442062800000, 5676, 37281, 1442055600000, 1442066400000, -1, 136, 2 ] + - [ "FR", 1442066400000, 2516, 37281, 1442059200000, 1442070000000, -1, 136, 2 ] + - [ "FR", 1442070000000, 474, 37281, 1442062800000, 1442073600000, -1, 136, 2 ] + - [ "FR", 1442073600000, 3522, 37281, 1442066400000, 1442077200000, -1, 136, 2 ] + - [ "FR", 1442077200000, -444, 37281, 1442070000000, 1442080800000, -1, 136, 3 ] + - [ "FR", 1442080800000, 557, 37281, 1442073600000, 1442084400000, -1, 136, 3 ] + - [ "FR", 1442084400000, 6643, 37281, 1442077200000, 1442088000000, -1, 136, 3 ] + - [ "FR", 1442088000000, 1872, 37281, 1442080800000, 1442091600000, -1, 136, 3 ] + - [ "FR", 1442091600000, 741, 37281, 1442084400000, 1442095200000, -1, 136, 3 ] + - [ "FR", 1442095200000, 87, 37281, 1442088000000, 1442098800000, -1, 136, 3 ] + - [ "FR", 1442098800000, 136, 37281, 1442091600000, null, -1, 136, 3 ] + - [ "GB", 1442016000000, -44, 35857, null, 1442019600000, -44, 49, 1 ] + - [ "GB", 1442019600000, 54, 35857, null, 1442023200000, -44, 49, 1 ] + - [ "GB", 1442023200000, 1816, 35857, 1442016000000, 1442026800000, -44, 49, 1 ] + - [ "GB", 1442026800000, 339, 35857, 1442019600000, 1442030400000, -44, 49, 1 ] + - [ "GB", 1442030400000, 2524, 35857, 1442023200000, 1442034000000, -44, 49, 1 ] + - [ "GB", 1442034000000, -12, 35857, 1442026800000, 1442037600000, -44, 49, 1 ] + - [ "GB", 1442037600000, 544, 35857, 1442030400000, 1442041200000, -44, 49, 1 ] + - [ "GB", 1442041200000, 42, 35857, 1442034000000, 1442044800000, -44, 49, 1 ] + - [ "GB", 1442044800000, 32, 35857, 1442037600000, 1442048400000, -44, 49, 2 ] + - [ "GB", 1442048400000, 740, 35857, 1442041200000, 1442052000000, -44, 49, 2 ] + - [ "GB", 1442052000000, 168, 35857, 1442044800000, 1442055600000, -44, 49, 2 ] + - [ "GB", 1442055600000, 453, 35857, 1442048400000, 1442059200000, -44, 49, 2 ] + - [ "GB", 1442059200000, 16111, 35857, 1442052000000, 1442062800000, -44, 49, 2 ] + - [ "GB", 1442062800000, 5743, 35857, 1442055600000, 1442066400000, -44, 49, 2 ] + - [ "GB", 1442066400000, 671, 35857, 1442059200000, 1442070000000, -44, 49, 2 ] + - [ "GB", 1442070000000, 374, 35857, 1442062800000, 1442073600000, -44, 49, 2 ] + - [ "GB", 1442073600000, 648, 35857, 1442066400000, 1442077200000, -44, 49, 3 ] + - [ "GB", 1442077200000, 1135, 35857, 1442070000000, 1442080800000, -44, 49, 3 ] + - [ "GB", 1442080800000, 1444, 35857, 1442073600000, 1442084400000, -44, 49, 3 ] + - [ "GB", 1442084400000, 384, 35857, 1442077200000, 1442088000000, -44, 49, 3 ] + - [ "GB", 1442088000000, 1593, 35857, 1442080800000, 1442091600000, -44, 49, 3 ] + - [ "GB", 1442091600000, 811, 35857, 1442084400000, 1442095200000, -44, 49, 3 ] + - [ "GB", 1442095200000, 238, 35857, 1442088000000, 1442098800000, -44, 49, 3 ] + - [ "GB", 1442098800000, 49, 35857, 1442091600000, null, -44, 49, 3 ] + - [ "GE", 1442044800000, -21, -140, null, 1442052000000, -21, -27, 1 ] + - [ "GE", 1442052000000, -108, -140, null, 1442062800000, -21, -27, 1 ] + - [ "GE", 1442062800000, 16, -140, 1442044800000, 1442080800000, -21, -27, 2 ] + - [ "GE", 1442080800000, -27, -140, 1442052000000, null, -21, -27, 3 ] + - [ "GH", 1442088000000, 0, 0, null, null, 0, 0, 1 ] + - [ "GR", 1442019600000, 82, 149, null, 1442034000000, 82, 123, 1 ] + - [ "GR", 1442034000000, 0, 149, null, 1442041200000, 82, 123, 1 ] + - [ "GR", 1442041200000, 7, 149, 1442019600000, 1442048400000, 82, 123, 1 ] + - [ "GR", 1442048400000, -26, 149, 1442034000000, 1442062800000, 82, 123, 1 ] + - [ "GR", 1442062800000, 8, 149, 1442041200000, 1442070000000, 82, 123, 2 ] + - [ "GR", 1442070000000, 2, 149, 1442048400000, 1442073600000, 82, 123, 2 ] + - [ "GR", 1442073600000, -314, 149, 1442062800000, 1442080800000, 82, 123, 2 ] + - [ "GR", 1442080800000, 88, 149, 1442070000000, 1442084400000, 82, 123, 3 ] + - [ "GR", 1442084400000, 179, 149, 1442073600000, 1442091600000, 82, 123, 3 ] + - [ "GR", 1442091600000, 123, 149, 1442080800000, null, 82, 123, 3 ] + - [ "GT", 1442023200000, -167, 7, null, 1442026800000, -167, 1, 1 ] + - [ "GT", 1442026800000, 173, 7, null, 1442098800000, -167, 1, 2 ] + - [ "GT", 1442098800000, 1, 7, 1442023200000, null, -167, 1, 3 ] + - [ "HK", 1442019600000, -113, 10743, null, 1442023200000, -113, -1, 1 ] + - [ "HK", 1442023200000, 2414, 10743, null, 1442026800000, -113, -1, 1 ] + - [ "HK", 1442026800000, -211, 10743, 1442019600000, 1442030400000, -113, -1, 1 ] + - [ "HK", 1442030400000, 157, 10743, 1442023200000, 1442034000000, -113, -1, 1 ] + - [ "HK", 1442034000000, 1137, 10743, 1442026800000, 1442037600000, -113, -1, 1 ] + - [ "HK", 1442037600000, 636, 10743, 1442030400000, 1442041200000, -113, -1, 1 ] + - [ "HK", 1442041200000, -15, 10743, 1442034000000, 1442044800000, -113, -1, 1 ] + - [ "HK", 1442044800000, 21, 10743, 1442037600000, 1442048400000, -113, -1, 2 ] + - [ "HK", 1442048400000, 1, 10743, 1442041200000, 1442052000000, -113, -1, 2 ] + - [ "HK", 1442052000000, 15, 10743, 1442044800000, 1442055600000, -113, -1, 2 ] + - [ "HK", 1442055600000, 804, 10743, 1442048400000, 1442059200000, -113, -1, 2 ] + - [ "HK", 1442059200000, 2, 10743, 1442052000000, 1442062800000, -113, -1, 2 ] + - [ "HK", 1442062800000, 1, 10743, 1442055600000, 1442066400000, -113, -1, 2 ] + - [ "HK", 1442066400000, 39, 10743, 1442059200000, 1442070000000, -113, -1, 3 ] + - [ "HK", 1442070000000, 314, 10743, 1442062800000, 1442073600000, -113, -1, 3 ] + - [ "HK", 1442073600000, 5545, 10743, 1442066400000, 1442080800000, -113, -1, 3 ] + - [ "HK", 1442080800000, 0, 10743, 1442070000000, 1442091600000, -113, -1, 3 ] + - [ "HK", 1442091600000, -3, 10743, 1442073600000, 1442095200000, -113, -1, 3 ] + - [ "HK", 1442095200000, -1, 10743, 1442080800000, null, -113, -1, 3 ] + - [ "HN", 1442026800000, -1, -1, null, null, -1, -1, 1 ] + - [ "HR", 1442070000000, 32, 382, null, 1442073600000, 32, 82, 1 ] + - [ "HR", 1442073600000, 0, 382, null, 1442077200000, 32, 82, 1 ] + - [ "HR", 1442077200000, 58, 382, 1442070000000, 1442080800000, 32, 82, 2 ] + - [ "HR", 1442080800000, 220, 382, 1442073600000, 1442084400000, 32, 82, 2 ] + - [ "HR", 1442084400000, -10, 382, 1442077200000, 1442088000000, 32, 82, 3 ] + - [ "HR", 1442088000000, 82, 382, 1442080800000, null, 32, 82, 3 ] + - [ "HU", 1442019600000, 46, 2116, null, 1442037600000, 46, 110, 1 ] + - [ "HU", 1442037600000, 197, 2116, null, 1442041200000, 46, 110, 1 ] + - [ "HU", 1442041200000, 91, 2116, 1442019600000, 1442044800000, 46, 110, 1 ] + - [ "HU", 1442044800000, 547, 2116, 1442037600000, 1442048400000, 46, 110, 1 ] + - [ "HU", 1442048400000, 499, 2116, 1442041200000, 1442055600000, 46, 110, 1 ] + - [ "HU", 1442055600000, -2, 2116, 1442044800000, 1442062800000, 46, 110, 2 ] + - [ "HU", 1442062800000, 50, 2116, 1442048400000, 1442080800000, 46, 110, 2 ] + - [ "HU", 1442080800000, 242, 2116, 1442055600000, 1442084400000, 46, 110, 2 ] + - [ "HU", 1442084400000, 141, 2116, 1442062800000, 1442088000000, 46, 110, 2 ] + - [ "HU", 1442088000000, -71, 2116, 1442080800000, 1442091600000, 46, 110, 3 ] + - [ "HU", 1442091600000, -5, 2116, 1442084400000, 1442095200000, 46, 110, 3 ] + - [ "HU", 1442095200000, 271, 2116, 1442088000000, 1442098800000, 46, 110, 3 ] + - [ "HU", 1442098800000, 110, 2116, 1442091600000, null, 46, 110, 3 ] + - [ "ID", 1442023200000, 106, -255, null, 1442026800000, 106, 13, 1 ] + - [ "ID", 1442026800000, -416, -255, null, 1442030400000, 106, 13, 1 ] + - [ "ID", 1442030400000, 279, -255, 1442023200000, 1442034000000, 106, 13, 1 ] + - [ "ID", 1442034000000, 19, -255, 1442026800000, 1442037600000, 106, 13, 1 ] + - [ "ID", 1442037600000, 14, -255, 1442030400000, 1442041200000, 106, 13, 1 ] + - [ "ID", 1442041200000, 2, -255, 1442034000000, 1442044800000, 106, 13, 2 ] + - [ "ID", 1442044800000, -388, -255, 1442037600000, 1442055600000, 106, 13, 2 ] + - [ "ID", 1442055600000, 16, -255, 1442041200000, 1442059200000, 106, 13, 2 ] + - [ "ID", 1442059200000, 17, -255, 1442044800000, 1442070000000, 106, 13, 2 ] + - [ "ID", 1442070000000, 42, -255, 1442055600000, 1442091600000, 106, 13, 3 ] + - [ "ID", 1442091600000, 21, -255, 1442059200000, 1442095200000, 106, 13, 3 ] + - [ "ID", 1442095200000, 20, -255, 1442070000000, 1442098800000, 106, 13, 3 ] + - [ "ID", 1442098800000, 13, -255, 1442091600000, null, 106, 13, 3 ] + - [ "IE", 1442026800000, 1, 2142, null, 1442030400000, 1, -71, 1 ] + - [ "IE", 1442030400000, 1, 2142, null, 1442048400000, 1, -71, 1 ] + - [ "IE", 1442048400000, 27, 2142, 1442026800000, 1442066400000, 1, -71, 1 ] + - [ "IE", 1442066400000, 1062, 2142, 1442030400000, 1442070000000, 1, -71, 2 ] + - [ "IE", 1442070000000, -100, 2142, 1442048400000, 1442077200000, 1, -71, 2 ] + - [ "IE", 1442077200000, 403, 2142, 1442066400000, 1442084400000, 1, -71, 2 ] + - [ "IE", 1442084400000, 819, 2142, 1442070000000, 1442091600000, 1, -71, 3 ] + - [ "IE", 1442091600000, -71, 2142, 1442077200000, null, 1, -71, 3 ] + - [ "IL", 1442041200000, 35, 6617, null, 1442044800000, 35, 3, 1 ] + - [ "IL", 1442044800000, 218, 6617, null, 1442048400000, 35, 3, 1 ] + - [ "IL", 1442048400000, 25, 6617, 1442041200000, 1442052000000, 35, 3, 1 ] + - [ "IL", 1442052000000, 2745, 6617, 1442044800000, 1442055600000, 35, 3, 1 ] + - [ "IL", 1442055600000, 4, 6617, 1442048400000, 1442059200000, 35, 3, 1 ] + - [ "IL", 1442059200000, 1205, 6617, 1442052000000, 1442062800000, 35, 3, 1 ] + - [ "IL", 1442062800000, 180, 6617, 1442055600000, 1442066400000, 35, 3, 2 ] + - [ "IL", 1442066400000, 3, 6617, 1442059200000, 1442070000000, 35, 3, 2 ] + - [ "IL", 1442070000000, 49, 6617, 1442062800000, 1442073600000, 35, 3, 2 ] + - [ "IL", 1442073600000, 31, 6617, 1442066400000, 1442077200000, 35, 3, 2 ] + - [ "IL", 1442077200000, 187, 6617, 1442070000000, 1442080800000, 35, 3, 2 ] + - [ "IL", 1442080800000, 88, 6617, 1442073600000, 1442084400000, 35, 3, 3 ] + - [ "IL", 1442084400000, 1137, 6617, 1442077200000, 1442091600000, 35, 3, 3 ] + - [ "IL", 1442091600000, 707, 6617, 1442080800000, 1442095200000, 35, 3, 3 ] + - [ "IL", 1442095200000, 0, 6617, 1442084400000, 1442098800000, 35, 3, 3 ] + - [ "IL", 1442098800000, 3, 6617, 1442091600000, null, 35, 3, 3 ] + - [ "IN", 1442016000000, 1, 29166, null, 1442019600000, 1, 4, 1 ] + - [ "IN", 1442019600000, 38, 29166, null, 1442023200000, 1, 4, 1 ] + - [ "IN", 1442023200000, -142, 29166, 1442016000000, 1442026800000, 1, 4, 1 ] + - [ "IN", 1442026800000, 974, 29166, 1442019600000, 1442030400000, 1, 4, 1 ] + - [ "IN", 1442030400000, 1448, 29166, 1442023200000, 1442034000000, 1, 4, 1 ] + - [ "IN", 1442034000000, 1350, 29166, 1442026800000, 1442037600000, 1, 4, 1 ] + - [ "IN", 1442037600000, 135, 29166, 1442030400000, 1442041200000, 1, 4, 1 ] + - [ "IN", 1442041200000, 80, 29166, 1442034000000, 1442044800000, 1, 4, 1 ] + - [ "IN", 1442044800000, 2677, 29166, 1442037600000, 1442048400000, 1, 4, 2 ] + - [ "IN", 1442048400000, 262, 29166, 1442041200000, 1442052000000, 1, 4, 2 ] + - [ "IN", 1442052000000, 534, 29166, 1442044800000, 1442055600000, 1, 4, 2 ] + - [ "IN", 1442055600000, 166, 29166, 1442048400000, 1442059200000, 1, 4, 2 ] + - [ "IN", 1442059200000, 708, 29166, 1442052000000, 1442062800000, 1, 4, 2 ] + - [ "IN", 1442062800000, 1547, 29166, 1442055600000, 1442066400000, 1, 4, 2 ] + - [ "IN", 1442066400000, 116, 29166, 1442059200000, 1442070000000, 1, 4, 2 ] + - [ "IN", 1442070000000, 12091, 29166, 1442062800000, 1442073600000, 1, 4, 3 ] + - [ "IN", 1442073600000, 1170, 29166, 1442066400000, 1442077200000, 1, 4, 3 ] + - [ "IN", 1442077200000, 5699, 29166, 1442070000000, 1442080800000, 1, 4, 3 ] + - [ "IN", 1442080800000, 0, 29166, 1442073600000, 1442084400000, 1, 4, 3 ] + - [ "IN", 1442084400000, 187, 29166, 1442077200000, 1442088000000, 1, 4, 3 ] + - [ "IN", 1442088000000, 121, 29166, 1442080800000, 1442095200000, 1, 4, 3 ] + - [ "IN", 1442095200000, 4, 29166, 1442084400000, null, 1, 4, 3 ] + - [ "IQ", 1442041200000, -1, 3, null, 1442044800000, -1, -2, 1 ] + - [ "IQ", 1442044800000, 6, 3, null, 1442052000000, -1, -2, 1 ] + - [ "IQ", 1442052000000, 0, 3, 1442041200000, 1442095200000, -1, -2, 2 ] + - [ "IQ", 1442095200000, -2, 3, 1442044800000, null, -1, -2, 3 ] + - [ "IR", 1442026800000, 0, 2698, null, 1442030400000, 0, 0, 1 ] + - [ "IR", 1442030400000, 375, 2698, null, 1442034000000, 0, 0, 1 ] + - [ "IR", 1442034000000, -8, 2698, 1442026800000, 1442041200000, 0, 0, 1 ] + - [ "IR", 1442041200000, -79, 2698, 1442030400000, 1442044800000, 0, 0, 1 ] + - [ "IR", 1442044800000, 306, 2698, 1442034000000, 1442052000000, 0, 0, 1 ] + - [ "IR", 1442052000000, 155, 2698, 1442041200000, 1442055600000, 0, 0, 2 ] + - [ "IR", 1442055600000, -124, 2698, 1442044800000, 1442059200000, 0, 0, 2 ] + - [ "IR", 1442059200000, 1455, 2698, 1442052000000, 1442073600000, 0, 0, 2 ] + - [ "IR", 1442073600000, -193, 2698, 1442055600000, 1442077200000, 0, 0, 2 ] + - [ "IR", 1442077200000, -34, 2698, 1442059200000, 1442080800000, 0, 0, 3 ] + - [ "IR", 1442080800000, 131, 2698, 1442073600000, 1442088000000, 0, 0, 3 ] + - [ "IR", 1442088000000, 714, 2698, 1442077200000, 1442091600000, 0, 0, 3 ] + - [ "IR", 1442091600000, 0, 2698, 1442080800000, null, 0, 0, 3 ] + - [ "IT", 1442016000000, 0, 39091, null, 1442019600000, 0, 565, 1 ] + - [ "IT", 1442019600000, 183, 39091, null, 1442023200000, 0, 565, 1 ] + - [ "IT", 1442023200000, 111, 39091, 1442016000000, 1442026800000, 0, 565, 1 ] + - [ "IT", 1442026800000, 222, 39091, 1442019600000, 1442030400000, 0, 565, 1 ] + - [ "IT", 1442030400000, -17, 39091, 1442023200000, 1442034000000, 0, 565, 1 ] + - [ "IT", 1442034000000, 1006, 39091, 1442026800000, 1442037600000, 0, 565, 1 ] + - [ "IT", 1442037600000, -9, 39091, 1442030400000, 1442041200000, 0, 565, 1 ] + - [ "IT", 1442041200000, 20, 39091, 1442034000000, 1442044800000, 0, 565, 1 ] + - [ "IT", 1442044800000, 1483, 39091, 1442037600000, 1442048400000, 0, 565, 2 ] + - [ "IT", 1442048400000, 676, 39091, 1442041200000, 1442052000000, 0, 565, 2 ] + - [ "IT", 1442052000000, 1880, 39091, 1442044800000, 1442055600000, 0, 565, 2 ] + - [ "IT", 1442055600000, 6240, 39091, 1442048400000, 1442059200000, 0, 565, 2 ] + - [ "IT", 1442059200000, 542, 39091, 1442052000000, 1442062800000, 0, 565, 2 ] + - [ "IT", 1442062800000, 1938, 39091, 1442055600000, 1442066400000, 0, 565, 2 ] + - [ "IT", 1442066400000, 4155, 39091, 1442059200000, 1442070000000, 0, 565, 2 ] + - [ "IT", 1442070000000, 81, 39091, 1442062800000, 1442073600000, 0, 565, 2 ] + - [ "IT", 1442073600000, 2586, 39091, 1442066400000, 1442077200000, 0, 565, 3 ] + - [ "IT", 1442077200000, 2188, 39091, 1442070000000, 1442080800000, 0, 565, 3 ] + - [ "IT", 1442080800000, 5544, 39091, 1442073600000, 1442084400000, 0, 565, 3 ] + - [ "IT", 1442084400000, 2660, 39091, 1442077200000, 1442088000000, 0, 565, 3 ] + - [ "IT", 1442088000000, 3746, 39091, 1442080800000, 1442091600000, 0, 565, 3 ] + - [ "IT", 1442091600000, 351, 39091, 1442084400000, 1442095200000, 0, 565, 3 ] + - [ "IT", 1442095200000, 2940, 39091, 1442088000000, 1442098800000, 0, 565, 3 ] + - [ "IT", 1442098800000, 565, 39091, 1442091600000, null, 0, 565, 3 ] + - [ "JM", 1442070000000, 30, 30, null, null, 30, 30, 1 ] + - [ "JO", 1442055600000, -2, 2, null, 1442059200000, -2, 4, 1 ] + - [ "JO", 1442059200000, 0, 2, null, 1442080800000, -2, 4, 2 ] + - [ "JO", 1442080800000, 4, 2, 1442055600000, null, -2, 4, 3 ] + - [ "JP", 1442016000000, -113, 20378, null, 1442019600000, -113, -6, 1 ] + - [ "JP", 1442019600000, 2002, 20378, null, 1442023200000, -113, -6, 1 ] + - [ "JP", 1442023200000, 1959, 20378, 1442016000000, 1442026800000, -113, -6, 1 ] + - [ "JP", 1442026800000, 1035, 20378, 1442019600000, 1442030400000, -113, -6, 1 ] + - [ "JP", 1442030400000, 805, 20378, 1442023200000, 1442034000000, -113, -6, 1 ] + - [ "JP", 1442034000000, 910, 20378, 1442026800000, 1442037600000, -113, -6, 1 ] + - [ "JP", 1442037600000, 2181, 20378, 1442030400000, 1442041200000, -113, -6, 1 ] + - [ "JP", 1442041200000, 1373, 20378, 1442034000000, 1442044800000, -113, -6, 1 ] + - [ "JP", 1442044800000, 1569, 20378, 1442037600000, 1442048400000, -113, -6, 2 ] + - [ "JP", 1442048400000, 1981, 20378, 1442041200000, 1442052000000, -113, -6, 2 ] + - [ "JP", 1442052000000, 2789, 20378, 1442044800000, 1442055600000, -113, -6, 2 ] + - [ "JP", 1442055600000, 998, 20378, 1442048400000, 1442059200000, -113, -6, 2 ] + - [ "JP", 1442059200000, -85, 20378, 1442052000000, 1442062800000, -113, -6, 2 ] + - [ "JP", 1442062800000, 803, 20378, 1442055600000, 1442066400000, -113, -6, 2 ] + - [ "JP", 1442066400000, 167, 20378, 1442059200000, 1442070000000, -113, -6, 2 ] + - [ "JP", 1442070000000, 79, 20378, 1442062800000, 1442073600000, -113, -6, 2 ] + - [ "JP", 1442073600000, 1162, 20378, 1442066400000, 1442077200000, -113, -6, 3 ] + - [ "JP", 1442077200000, 51, 20378, 1442070000000, 1442080800000, -113, -6, 3 ] + - [ "JP", 1442080800000, 420, 20378, 1442073600000, 1442084400000, -113, -6, 3 ] + - [ "JP", 1442084400000, 13, 20378, 1442077200000, 1442088000000, -113, -6, 3 ] + - [ "JP", 1442088000000, 57, 20378, 1442080800000, 1442091600000, -113, -6, 3 ] + - [ "JP", 1442091600000, 228, 20378, 1442084400000, 1442095200000, -113, -6, 3 ] + - [ "JP", 1442095200000, 0, 20378, 1442088000000, 1442098800000, -113, -6, 3 ] + - [ "JP", 1442098800000, -6, 20378, 1442091600000, null, -113, -6, 3 ] + - [ "KE", 1442044800000, -1, -1, null, null, -1, -1, 1 ] + - [ "KG", 1442073600000, 6, 6, null, null, 6, 6, 1 ] + - [ "KR", 1442016000000, 1024, 13597, null, 1442019600000, 1024, -36, 1 ] + - [ "KR", 1442019600000, 445, 13597, null, 1442023200000, 1024, -36, 1 ] + - [ "KR", 1442023200000, 319, 13597, 1442016000000, 1442026800000, 1024, -36, 1 ] + - [ "KR", 1442026800000, -179, 13597, 1442019600000, 1442030400000, 1024, -36, 1 ] + - [ "KR", 1442030400000, 1035, 13597, 1442023200000, 1442034000000, 1024, -36, 1 ] + - [ "KR", 1442034000000, 434, 13597, 1442026800000, 1442037600000, 1024, -36, 1 ] + - [ "KR", 1442037600000, 26, 13597, 1442030400000, 1442041200000, 1024, -36, 1 ] + - [ "KR", 1442041200000, 20, 13597, 1442034000000, 1442044800000, 1024, -36, 1 ] + - [ "KR", 1442044800000, 829, 13597, 1442037600000, 1442048400000, 1024, -36, 2 ] + - [ "KR", 1442048400000, -374, 13597, 1442041200000, 1442052000000, 1024, -36, 2 ] + - [ "KR", 1442052000000, -3, 13597, 1442044800000, 1442055600000, 1024, -36, 2 ] + - [ "KR", 1442055600000, 3640, 13597, 1442048400000, 1442059200000, 1024, -36, 2 ] + - [ "KR", 1442059200000, 208, 13597, 1442052000000, 1442062800000, 1024, -36, 2 ] + - [ "KR", 1442062800000, 1096, 13597, 1442055600000, 1442066400000, 1024, -36, 2 ] + - [ "KR", 1442066400000, 3299, 13597, 1442059200000, 1442070000000, 1024, -36, 2 ] + - [ "KR", 1442070000000, 222, 13597, 1442062800000, 1442077200000, 1024, -36, 3 ] + - [ "KR", 1442077200000, -40, 13597, 1442066400000, 1442080800000, 1024, -36, 3 ] + - [ "KR", 1442080800000, -33, 13597, 1442070000000, 1442084400000, 1024, -36, 3 ] + - [ "KR", 1442084400000, 314, 13597, 1442077200000, 1442088000000, 1024, -36, 3 ] + - [ "KR", 1442088000000, 524, 13597, 1442080800000, 1442095200000, 1024, -36, 3 ] + - [ "KR", 1442095200000, 827, 13597, 1442084400000, 1442098800000, 1024, -36, 3 ] + - [ "KR", 1442098800000, -36, 13597, 1442088000000, null, 1024, -36, 3 ] + - [ "KW", 1442055600000, -2, 1778, null, 1442070000000, -2, -33, 1 ] + - [ "KW", 1442070000000, 1815, 1778, null, 1442077200000, -2, -33, 1 ] + - [ "KW", 1442077200000, -2, 1778, 1442055600000, 1442080800000, -2, -33, 2 ] + - [ "KW", 1442080800000, -33, 1778, 1442070000000, null, -2, -33, 3 ] + - [ "KZ", 1442034000000, 161, 1261, null, 1442044800000, 161, 91, 1 ] + - [ "KZ", 1442044800000, 401, 1261, null, 1442048400000, 161, 91, 1 ] + - [ "KZ", 1442048400000, 439, 1261, 1442034000000, 1442052000000, 161, 91, 1 ] + - [ "KZ", 1442052000000, 412, 1261, 1442044800000, 1442055600000, 161, 91, 1 ] + - [ "KZ", 1442055600000, 63, 1261, 1442048400000, 1442059200000, 161, 91, 2 ] + - [ "KZ", 1442059200000, 33, 1261, 1442052000000, 1442062800000, 161, 91, 2 ] + - [ "KZ", 1442062800000, 0, 1261, 1442055600000, 1442066400000, 161, 91, 2 ] + - [ "KZ", 1442066400000, 0, 1261, 1442059200000, 1442077200000, 161, 91, 2 ] + - [ "KZ", 1442077200000, -317, 1261, 1442062800000, 1442084400000, 161, 91, 3 ] + - [ "KZ", 1442084400000, -22, 1261, 1442066400000, 1442095200000, 161, 91, 3 ] + - [ "KZ", 1442095200000, 91, 1261, 1442077200000, null, 161, 91, 3 ] + - [ "LB", 1442055600000, -67, -67, null, null, -67, -67, 1 ] + - [ "LK", 1442026800000, 79, 131, null, 1442048400000, 79, -3, 1 ] + - [ "LK", 1442048400000, 8, 131, null, 1442052000000, 79, -3, 1 ] + - [ "LK", 1442052000000, 47, 131, 1442026800000, 1442084400000, 79, -3, 2 ] + - [ "LK", 1442084400000, -3, 131, 1442048400000, null, 79, -3, 3 ] + - [ "LT", 1442080800000, 12, -12, null, 1442098800000, 12, -24, 1 ] + - [ "LT", 1442098800000, -24, -12, null, null, 12, -24, 2 ] + - [ "LU", 1442059200000, 79, 606, null, 1442066400000, 79, 2, 1 ] + - [ "LU", 1442066400000, 0, 606, null, 1442077200000, 79, 2, 1 ] + - [ "LU", 1442077200000, 525, 606, 1442059200000, 1442095200000, 79, 2, 2 ] + - [ "LU", 1442095200000, 2, 606, 1442066400000, null, 79, 2, 3 ] + - [ "LV", 1442095200000, 0, 0, null, null, 0, 0, 1 ] + - [ "MA", 1442019600000, -1, 229, null, 1442055600000, -1, 8, 1 ] + - [ "MA", 1442055600000, 23, 229, null, 1442059200000, -1, 8, 1 ] + - [ "MA", 1442059200000, -56, 229, 1442019600000, 1442062800000, -1, 8, 1 ] + - [ "MA", 1442062800000, 0, 229, 1442055600000, 1442077200000, -1, 8, 2 ] + - [ "MA", 1442077200000, 250, 229, 1442059200000, 1442080800000, -1, 8, 2 ] + - [ "MA", 1442080800000, 5, 229, 1442062800000, 1442098800000, -1, 8, 3 ] + - [ "MA", 1442098800000, 8, 229, 1442077200000, null, -1, 8, 3 ] + - [ "MD", 1442077200000, 6916, 6916, null, null, 6916, 6916, 1 ] + - [ "ME", 1442073600000, 0, 0, null, null, 0, 0, 1 ] + - [ "MH", 1442052000000, 40, 40, null, null, 40, 40, 1 ] + - [ "MK", 1442077200000, -72, -72, null, null, -72, -72, 1 ] + - [ "MM", 1442070000000, 3, 28, null, 1442073600000, 3, 25, 1 ] + - [ "MM", 1442073600000, 25, 28, null, null, 3, 25, 2 ] + - [ "MO", 1442034000000, 30, 48, null, 1442070000000, 30, 18, 1 ] + - [ "MO", 1442070000000, 18, 48, null, null, 30, 18, 2 ] + - [ "MR", 1442080800000, 10, 10, null, null, 10, 10, 1 ] + - [ "MT", 1442048400000, -1, -1, null, null, -1, -1, 1 ] + - [ "MV", 1442073600000, -3, -3, null, null, -3, -3, 1 ] + - [ "MX", 1442016000000, -67, 10472, null, 1442023200000, -67, 28, 1 ] + - [ "MX", 1442023200000, 549, 10472, null, 1442026800000, -67, 28, 1 ] + - [ "MX", 1442026800000, 3642, 10472, 1442016000000, 1442030400000, -67, 28, 1 ] + - [ "MX", 1442030400000, 373, 10472, 1442023200000, 1442034000000, -67, 28, 1 ] + - [ "MX", 1442034000000, 944, 10472, 1442026800000, 1442037600000, -67, 28, 1 ] + - [ "MX", 1442037600000, 4, 10472, 1442030400000, 1442041200000, -67, 28, 1 ] + - [ "MX", 1442041200000, -294, 10472, 1442034000000, 1442066400000, -67, 28, 2 ] + - [ "MX", 1442066400000, -1, 10472, 1442037600000, 1442070000000, -67, 28, 2 ] + - [ "MX", 1442070000000, -1, 10472, 1442041200000, 1442073600000, -67, 28, 2 ] + - [ "MX", 1442073600000, -21, 10472, 1442066400000, 1442077200000, -67, 28, 2 ] + - [ "MX", 1442077200000, 3874, 10472, 1442070000000, 1442080800000, -67, 28, 2 ] + - [ "MX", 1442080800000, -376, 10472, 1442073600000, 1442084400000, -67, 28, 2 ] + - [ "MX", 1442084400000, 981, 10472, 1442077200000, 1442088000000, -67, 28, 3 ] + - [ "MX", 1442088000000, 494, 10472, 1442080800000, 1442091600000, -67, 28, 3 ] + - [ "MX", 1442091600000, 799, 10472, 1442084400000, 1442095200000, -67, 28, 3 ] + - [ "MX", 1442095200000, -456, 10472, 1442088000000, 1442098800000, -67, 28, 3 ] + - [ "MX", 1442098800000, 28, 10472, 1442091600000, null, -67, 28, 3 ] + - [ "MY", 1442019600000, -7, 3207, null, 1442030400000, -7, 739, 1 ] + - [ "MY", 1442030400000, -3, 3207, null, 1442034000000, -7, 739, 1 ] + - [ "MY", 1442034000000, 1028, 3207, 1442019600000, 1442041200000, -7, 739, 1 ] + - [ "MY", 1442041200000, 935, 3207, 1442030400000, 1442044800000, -7, 739, 1 ] + - [ "MY", 1442044800000, -127, 3207, 1442034000000, 1442048400000, -7, 739, 2 ] + - [ "MY", 1442048400000, 649, 3207, 1442041200000, 1442055600000, -7, 739, 2 ] + - [ "MY", 1442055600000, 1, 3207, 1442044800000, 1442059200000, -7, 739, 2 ] + - [ "MY", 1442059200000, 0, 3207, 1442048400000, 1442066400000, -7, 739, 2 ] + - [ "MY", 1442066400000, 1, 3207, 1442055600000, 1442073600000, -7, 739, 3 ] + - [ "MY", 1442073600000, 1, 3207, 1442059200000, 1442077200000, -7, 739, 3 ] + - [ "MY", 1442077200000, -10, 3207, 1442066400000, 1442098800000, -7, 739, 3 ] + - [ "MY", 1442098800000, 739, 3207, 1442073600000, null, -7, 739, 3 ] + - [ "NG", 1442052000000, 208, 214, null, 1442070000000, 208, 6, 1 ] + - [ "NG", 1442070000000, 6, 214, null, null, 208, 6, 2 ] + - [ "NL", 1442034000000, 0, 12162, null, 1442044800000, 0, 4, 1 ] + - [ "NL", 1442044800000, 16, 12162, null, 1442048400000, 0, 4, 1 ] + - [ "NL", 1442048400000, 1303, 12162, 1442034000000, 1442052000000, 0, 4, 1 ] + - [ "NL", 1442052000000, 53, 12162, 1442044800000, 1442055600000, 0, 4, 1 ] + - [ "NL", 1442055600000, 105, 12162, 1442048400000, 1442059200000, 0, 4, 1 ] + - [ "NL", 1442059200000, 206, 12162, 1442052000000, 1442062800000, 0, 4, 1 ] + - [ "NL", 1442062800000, -30, 12162, 1442055600000, 1442066400000, 0, 4, 2 ] + - [ "NL", 1442066400000, 61, 12162, 1442059200000, 1442070000000, 0, 4, 2 ] + - [ "NL", 1442070000000, -84, 12162, 1442062800000, 1442073600000, 0, 4, 2 ] + - [ "NL", 1442073600000, 166, 12162, 1442066400000, 1442077200000, 0, 4, 2 ] + - [ "NL", 1442077200000, 878, 12162, 1442070000000, 1442080800000, 0, 4, 2 ] + - [ "NL", 1442080800000, 8947, 12162, 1442073600000, 1442084400000, 0, 4, 2 ] + - [ "NL", 1442084400000, 436, 12162, 1442077200000, 1442088000000, 0, 4, 3 ] + - [ "NL", 1442088000000, 12, 12162, 1442080800000, 1442091600000, 0, 4, 3 ] + - [ "NL", 1442091600000, 19, 12162, 1442084400000, 1442095200000, 0, 4, 3 ] + - [ "NL", 1442095200000, 70, 12162, 1442088000000, 1442098800000, 0, 4, 3 ] + - [ "NL", 1442098800000, 4, 12162, 1442091600000, null, 0, 4, 3 ] + - [ "NO", 1442019600000, 48, 432, null, 1442048400000, 48, 2, 1 ] + - [ "NO", 1442048400000, -447, 432, null, 1442052000000, 48, 2, 1 ] + - [ "NO", 1442052000000, 447, 432, 1442019600000, 1442055600000, 48, 2, 1 ] + - [ "NO", 1442055600000, 29, 432, 1442048400000, 1442066400000, 48, 2, 1 ] + - [ "NO", 1442066400000, 71, 432, 1442052000000, 1442073600000, 48, 2, 2 ] + - [ "NO", 1442073600000, 222, 432, 1442055600000, 1442080800000, 48, 2, 2 ] + - [ "NO", 1442080800000, 31, 432, 1442066400000, 1442088000000, 48, 2, 2 ] + - [ "NO", 1442088000000, 15, 432, 1442073600000, 1442091600000, 48, 2, 2 ] + - [ "NO", 1442091600000, 15, 432, 1442080800000, 1442095200000, 48, 2, 3 ] + - [ "NO", 1442095200000, -1, 432, 1442088000000, 1442098800000, 48, 2, 3 ] + - [ "NO", 1442098800000, 2, 432, 1442091600000, null, 48, 2, 3 ] + - [ "NP", 1442048400000, 61, 61, null, null, 61, 61, 1 ] + - [ "NZ", 1442019600000, 28, 1693, null, 1442026800000, 28, -2, 1 ] + - [ "NZ", 1442026800000, 635, 1693, null, 1442037600000, 28, -2, 1 ] + - [ "NZ", 1442037600000, 66, 1693, 1442019600000, 1442048400000, 28, -2, 1 ] + - [ "NZ", 1442048400000, 189, 1693, 1442026800000, 1442059200000, 28, -2, 2 ] + - [ "NZ", 1442059200000, 428, 1693, 1442037600000, 1442084400000, 28, -2, 2 ] + - [ "NZ", 1442084400000, -52, 1693, 1442048400000, 1442088000000, 28, -2, 2 ] + - [ "NZ", 1442088000000, 405, 1693, 1442059200000, 1442095200000, 28, -2, 3 ] + - [ "NZ", 1442095200000, -4, 1693, 1442084400000, 1442098800000, 28, -2, 3 ] + - [ "NZ", 1442098800000, -2, 1693, 1442088000000, null, 28, -2, 3 ] + - [ "OM", 1442052000000, 0, 0, null, null, 0, 0, 1 ] + - [ "PA", 1442026800000, 0, 0, null, null, 0, 0, 1 ] + - [ "PE", 1442019600000, 523, 2134, null, 1442023200000, 523, 1861, 1 ] + - [ "PE", 1442023200000, 26, 2134, null, 1442026800000, 523, 1861, 1 ] + - [ "PE", 1442026800000, -12, 2134, 1442019600000, 1442062800000, 523, 1861, 1 ] + - [ "PE", 1442062800000, -12, 2134, 1442023200000, 1442077200000, 523, 1861, 2 ] + - [ "PE", 1442077200000, -163, 2134, 1442026800000, 1442080800000, 523, 1861, 2 ] + - [ "PE", 1442080800000, -2, 2134, 1442062800000, 1442084400000, 523, 1861, 2 ] + - [ "PE", 1442084400000, -68, 2134, 1442077200000, 1442095200000, 523, 1861, 3 ] + - [ "PE", 1442095200000, -19, 2134, 1442080800000, 1442098800000, 523, 1861, 3 ] + - [ "PE", 1442098800000, 1861, 2134, 1442084400000, null, 523, 1861, 3 ] + - [ "PH", 1442019600000, 6, 6613, null, 1442023200000, 6, 8, 1 ] + - [ "PH", 1442023200000, 459, 6613, null, 1442026800000, 6, 8, 1 ] + - [ "PH", 1442026800000, 910, 6613, 1442019600000, 1442030400000, 6, 8, 1 ] + - [ "PH", 1442030400000, 26, 6613, 1442023200000, 1442034000000, 6, 8, 1 ] + - [ "PH", 1442034000000, 59, 6613, 1442026800000, 1442037600000, 6, 8, 1 ] + - [ "PH", 1442037600000, 17, 6613, 1442030400000, 1442041200000, 6, 8, 1 ] + - [ "PH", 1442041200000, 0, 6613, 1442034000000, 1442044800000, 6, 8, 1 ] + - [ "PH", 1442044800000, 55, 6613, 1442037600000, 1442048400000, 6, 8, 2 ] + - [ "PH", 1442048400000, 62, 6613, 1442041200000, 1442052000000, 6, 8, 2 ] + - [ "PH", 1442052000000, 22, 6613, 1442044800000, 1442055600000, 6, 8, 2 ] + - [ "PH", 1442055600000, 1969, 6613, 1442048400000, 1442059200000, 6, 8, 2 ] + - [ "PH", 1442059200000, 273, 6613, 1442052000000, 1442062800000, 6, 8, 2 ] + - [ "PH", 1442062800000, 171, 6613, 1442055600000, 1442066400000, 6, 8, 2 ] + - [ "PH", 1442066400000, 1880, 6613, 1442059200000, 1442070000000, 6, 8, 2 ] + - [ "PH", 1442070000000, 34, 6613, 1442062800000, 1442073600000, 6, 8, 3 ] + - [ "PH", 1442073600000, -227, 6613, 1442066400000, 1442077200000, 6, 8, 3 ] + - [ "PH", 1442077200000, 2, 6613, 1442070000000, 1442080800000, 6, 8, 3 ] + - [ "PH", 1442080800000, 32, 6613, 1442073600000, 1442084400000, 6, 8, 3 ] + - [ "PH", 1442084400000, 39, 6613, 1442077200000, 1442091600000, 6, 8, 3 ] + - [ "PH", 1442091600000, 816, 6613, 1442080800000, 1442098800000, 6, 8, 3 ] + - [ "PH", 1442098800000, 8, 6613, 1442084400000, null, 6, 8, 3 ] + - [ "PK", 1442019600000, 335, 641, null, 1442026800000, 335, 43, 1 ] + - [ "PK", 1442026800000, 101, 641, null, 1442037600000, 335, 43, 1 ] + - [ "PK", 1442037600000, 100, 641, 1442019600000, 1442041200000, 335, 43, 1 ] + - [ "PK", 1442041200000, 24, 641, 1442026800000, 1442048400000, 335, 43, 2 ] + - [ "PK", 1442048400000, 15, 641, 1442037600000, 1442062800000, 335, 43, 2 ] + - [ "PK", 1442062800000, 23, 641, 1442041200000, 1442070000000, 335, 43, 3 ] + - [ "PK", 1442070000000, 43, 641, 1442048400000, null, 335, 43, 3 ] + - [ "PL", 1442037600000, 95, 9815, null, 1442041200000, 95, -9, 1 ] + - [ "PL", 1442041200000, 281, 9815, null, 1442044800000, 95, -9, 1 ] + - [ "PL", 1442044800000, 319, 9815, 1442037600000, 1442048400000, 95, -9, 1 ] + - [ "PL", 1442048400000, 366, 9815, 1442041200000, 1442052000000, 95, -9, 1 ] + - [ "PL", 1442052000000, 330, 9815, 1442044800000, 1442055600000, 95, -9, 1 ] + - [ "PL", 1442055600000, 410, 9815, 1442048400000, 1442059200000, 95, -9, 1 ] + - [ "PL", 1442059200000, 199, 9815, 1442052000000, 1442062800000, 95, -9, 2 ] + - [ "PL", 1442062800000, 4171, 9815, 1442055600000, 1442066400000, 95, -9, 2 ] + - [ "PL", 1442066400000, 34, 9815, 1442059200000, 1442070000000, 95, -9, 2 ] + - [ "PL", 1442070000000, 146, 9815, 1442062800000, 1442073600000, 95, -9, 2 ] + - [ "PL", 1442073600000, 30, 9815, 1442066400000, 1442077200000, 95, -9, 2 ] + - [ "PL", 1442077200000, 324, 9815, 1442070000000, 1442080800000, 95, -9, 2 ] + - [ "PL", 1442080800000, 7, 9815, 1442073600000, 1442084400000, 95, -9, 3 ] + - [ "PL", 1442084400000, 13, 9815, 1442077200000, 1442088000000, 95, -9, 3 ] + - [ "PL", 1442088000000, 346, 9815, 1442080800000, 1442091600000, 95, -9, 3 ] + - [ "PL", 1442091600000, 902, 9815, 1442084400000, 1442095200000, 95, -9, 3 ] + - [ "PL", 1442095200000, 1851, 9815, 1442088000000, 1442098800000, 95, -9, 3 ] + - [ "PL", 1442098800000, -9, 9815, 1442091600000, null, 95, -9, 3 ] + - [ "PR", 1442026800000, 22, 23, null, 1442030400000, 22, 29, 1 ] + - [ "PR", 1442030400000, 2, 23, null, 1442059200000, 22, 29, 1 ] + - [ "PR", 1442059200000, -35, 23, 1442026800000, 1442077200000, 22, 29, 2 ] + - [ "PR", 1442077200000, 5, 23, 1442030400000, 1442095200000, 22, 29, 2 ] + - [ "PR", 1442095200000, 29, 23, 1442059200000, null, 22, 29, 3 ] + - [ "PT", 1442019600000, 172, 4037, null, 1442044800000, 172, 2, 1 ] + - [ "PT", 1442044800000, 11, 4037, null, 1442052000000, 172, 2, 1 ] + - [ "PT", 1442052000000, 102, 4037, 1442019600000, 1442066400000, 172, 2, 1 ] + - [ "PT", 1442066400000, 12, 4037, 1442044800000, 1442070000000, 172, 2, 1 ] + - [ "PT", 1442070000000, 3470, 4037, 1442052000000, 1442077200000, 172, 2, 2 ] + - [ "PT", 1442077200000, -75, 4037, 1442066400000, 1442080800000, 172, 2, 2 ] + - [ "PT", 1442080800000, -79, 4037, 1442070000000, 1442088000000, 172, 2, 2 ] + - [ "PT", 1442088000000, 403, 4037, 1442077200000, 1442095200000, 172, 2, 3 ] + - [ "PT", 1442095200000, 19, 4037, 1442080800000, 1442098800000, 172, 2, 3 ] + - [ "PT", 1442098800000, 2, 4037, 1442088000000, null, 172, 2, 3 ] + - [ "PY", 1442019600000, 1, 634, null, 1442080800000, 1, 628, 1 ] + - [ "PY", 1442080800000, 5, 634, null, 1442084400000, 1, 628, 2 ] + - [ "PY", 1442084400000, 628, 634, 1442019600000, null, 1, 628, 3 ] + - [ "QA", 1442041200000, 13, 13, null, null, 13, 13, 1 ] + - [ "RO", 1442034000000, 68, 2893, null, 1442041200000, 68, 824, 1 ] + - [ "RO", 1442041200000, 845, 2893, null, 1442044800000, 68, 824, 1 ] + - [ "RO", 1442044800000, 284, 2893, 1442034000000, 1442052000000, 68, 824, 1 ] + - [ "RO", 1442052000000, 319, 2893, 1442041200000, 1442055600000, 68, 824, 1 ] + - [ "RO", 1442055600000, 26, 2893, 1442044800000, 1442062800000, 68, 824, 2 ] + - [ "RO", 1442062800000, 541, 2893, 1442052000000, 1442070000000, 68, 824, 2 ] + - [ "RO", 1442070000000, -29, 2893, 1442055600000, 1442073600000, 68, 824, 2 ] + - [ "RO", 1442073600000, 15, 2893, 1442062800000, 1442091600000, 68, 824, 3 ] + - [ "RO", 1442091600000, 0, 2893, 1442070000000, 1442095200000, 68, 824, 3 ] + - [ "RO", 1442095200000, 824, 2893, 1442073600000, null, 68, 824, 3 ] + - [ "RS", 1442019600000, 6, 906, null, 1442062800000, 6, -15, 1 ] + - [ "RS", 1442062800000, 13, 906, null, 1442066400000, 6, -15, 1 ] + - [ "RS", 1442066400000, 0, 906, 1442019600000, 1442073600000, 6, -15, 1 ] + - [ "RS", 1442073600000, 813, 906, 1442062800000, 1442080800000, 6, -15, 2 ] + - [ "RS", 1442080800000, 0, 906, 1442066400000, 1442084400000, 6, -15, 2 ] + - [ "RS", 1442084400000, 89, 906, 1442073600000, 1442091600000, 6, -15, 3 ] + - [ "RS", 1442091600000, -15, 906, 1442080800000, null, 6, -15, 3 ] + - [ "RU", 1442019600000, 2214, 48104, null, 1442023200000, 2214, 12098, 1 ] + - [ "RU", 1442023200000, 299, 48104, null, 1442026800000, 2214, 12098, 1 ] + - [ "RU", 1442026800000, 0, 48104, 1442019600000, 1442030400000, 2214, 12098, 1 ] + - [ "RU", 1442030400000, 76, 48104, 1442023200000, 1442034000000, 2214, 12098, 1 ] + - [ "RU", 1442034000000, 658, 48104, 1442026800000, 1442037600000, 2214, 12098, 1 ] + - [ "RU", 1442037600000, -324, 48104, 1442030400000, 1442041200000, 2214, 12098, 1 ] + - [ "RU", 1442041200000, 580, 48104, 1442034000000, 1442044800000, 2214, 12098, 1 ] + - [ "RU", 1442044800000, 2564, 48104, 1442037600000, 1442048400000, 2214, 12098, 1 ] + - [ "RU", 1442048400000, 1027, 48104, 1442041200000, 1442052000000, 2214, 12098, 2 ] + - [ "RU", 1442052000000, 1214, 48104, 1442044800000, 1442055600000, 2214, 12098, 2 ] + - [ "RU", 1442055600000, 499, 48104, 1442048400000, 1442059200000, 2214, 12098, 2 ] + - [ "RU", 1442059200000, 3902, 48104, 1442052000000, 1442062800000, 2214, 12098, 2 ] + - [ "RU", 1442062800000, 168, 48104, 1442055600000, 1442066400000, 2214, 12098, 2 ] + - [ "RU", 1442066400000, 2047, 48104, 1442059200000, 1442070000000, 2214, 12098, 2 ] + - [ "RU", 1442070000000, 4706, 48104, 1442062800000, 1442073600000, 2214, 12098, 2 ] + - [ "RU", 1442073600000, 1618, 48104, 1442066400000, 1442077200000, 2214, 12098, 2 ] + - [ "RU", 1442077200000, 1162, 48104, 1442070000000, 1442080800000, 2214, 12098, 3 ] + - [ "RU", 1442080800000, 655, 48104, 1442073600000, 1442084400000, 2214, 12098, 3 ] + - [ "RU", 1442084400000, 6461, 48104, 1442077200000, 1442088000000, 2214, 12098, 3 ] + - [ "RU", 1442088000000, 2596, 48104, 1442080800000, 1442091600000, 2214, 12098, 3 ] + - [ "RU", 1442091600000, 3449, 48104, 1442084400000, 1442095200000, 2214, 12098, 3 ] + - [ "RU", 1442095200000, 435, 48104, 1442088000000, 1442098800000, 2214, 12098, 3 ] + - [ "RU", 1442098800000, 12098, 48104, 1442091600000, null, 2214, 12098, 3 ] + - [ "SA", 1442037600000, -97, 1614, null, 1442048400000, -97, 458, 1 ] + - [ "SA", 1442048400000, 14, 1614, null, 1442055600000, -97, 458, 1 ] + - [ "SA", 1442055600000, 11, 1614, 1442037600000, 1442059200000, -97, 458, 1 ] + - [ "SA", 1442059200000, 0, 1614, 1442048400000, 1442066400000, -97, 458, 2 ] + - [ "SA", 1442066400000, 1276, 1614, 1442055600000, 1442073600000, -97, 458, 2 ] + - [ "SA", 1442073600000, 2, 1614, 1442059200000, 1442077200000, -97, 458, 2 ] + - [ "SA", 1442077200000, -50, 1614, 1442066400000, 1442084400000, -97, 458, 3 ] + - [ "SA", 1442084400000, 458, 1614, 1442073600000, null, -97, 458, 3 ] + - [ "SE", 1442019600000, 109, 1838, null, 1442023200000, 109, 0, 1 ] + - [ "SE", 1442023200000, 3, 1838, null, 1442030400000, 109, 0, 1 ] + - [ "SE", 1442030400000, 30, 1838, 1442019600000, 1442041200000, 109, 0, 1 ] + - [ "SE", 1442041200000, 91, 1838, 1442023200000, 1442048400000, 109, 0, 1 ] + - [ "SE", 1442048400000, -145, 1838, 1442030400000, 1442052000000, 109, 0, 1 ] + - [ "SE", 1442052000000, 1, 1838, 1442041200000, 1442055600000, 109, 0, 2 ] + - [ "SE", 1442055600000, -5, 1838, 1442048400000, 1442059200000, 109, 0, 2 ] + - [ "SE", 1442059200000, 1476, 1838, 1442052000000, 1442066400000, 109, 0, 2 ] + - [ "SE", 1442066400000, 14, 1838, 1442055600000, 1442070000000, 109, 0, 2 ] + - [ "SE", 1442070000000, 78, 1838, 1442059200000, 1442080800000, 109, 0, 2 ] + - [ "SE", 1442080800000, 89, 1838, 1442066400000, 1442084400000, 109, 0, 3 ] + - [ "SE", 1442084400000, 37, 1838, 1442070000000, 1442091600000, 109, 0, 3 ] + - [ "SE", 1442091600000, -1, 1838, 1442080800000, 1442095200000, 109, 0, 3 ] + - [ "SE", 1442095200000, 61, 1838, 1442084400000, 1442098800000, 109, 0, 3 ] + - [ "SE", 1442098800000, 0, 1838, 1442091600000, null, 109, 0, 3 ] + - [ "SG", 1442026800000, 2758, 3338, null, 1442030400000, 2758, 0, 1 ] + - [ "SG", 1442030400000, 1, 3338, null, 1442037600000, 2758, 0, 1 ] + - [ "SG", 1442037600000, 3, 3338, 1442026800000, 1442041200000, 2758, 0, 1 ] + - [ "SG", 1442041200000, 59, 3338, 1442030400000, 1442044800000, 2758, 0, 2 ] + - [ "SG", 1442044800000, 77, 3338, 1442037600000, 1442048400000, 2758, 0, 2 ] + - [ "SG", 1442048400000, 52, 3338, 1442041200000, 1442062800000, 2758, 0, 2 ] + - [ "SG", 1442062800000, 388, 3338, 1442044800000, 1442066400000, 2758, 0, 3 ] + - [ "SG", 1442066400000, 0, 3338, 1442048400000, null, 2758, 0, 3 ] + - [ "SI", 1442080800000, -45, -36, null, 1442091600000, -45, 9, 1 ] + - [ "SI", 1442091600000, 9, -36, null, null, -45, 9, 2 ] + - [ "SK", 1442037600000, -1, 379, null, 1442052000000, -1, 7, 1 ] + - [ "SK", 1442052000000, 13, 379, null, 1442062800000, -1, 7, 1 ] + - [ "SK", 1442062800000, 6, 379, 1442037600000, 1442073600000, -1, 7, 2 ] + - [ "SK", 1442073600000, 446, 379, 1442052000000, 1442084400000, -1, 7, 2 ] + - [ "SK", 1442084400000, -92, 379, 1442062800000, 1442098800000, -1, 7, 3 ] + - [ "SK", 1442098800000, 7, 379, 1442073600000, null, -1, 7, 3 ] + - [ "SV", 1442019600000, -1, 114, null, 1442084400000, -1, 9, 1 ] + - [ "SV", 1442084400000, 106, 114, null, 1442088000000, -1, 9, 2 ] + - [ "SV", 1442088000000, 9, 114, 1442019600000, null, -1, 9, 3 ] + - [ "TH", 1442034000000, 0, 24, null, 1442041200000, 0, 13, 1 ] + - [ "TH", 1442041200000, 3, 24, null, 1442044800000, 0, 13, 1 ] + - [ "TH", 1442044800000, 110, 24, 1442034000000, 1442052000000, 0, 13, 1 ] + - [ "TH", 1442052000000, -22, 24, 1442041200000, 1442055600000, 0, 13, 2 ] + - [ "TH", 1442055600000, 0, 24, 1442044800000, 1442062800000, 0, 13, 2 ] + - [ "TH", 1442062800000, -46, 24, 1442052000000, 1442066400000, 0, 13, 2 ] + - [ "TH", 1442066400000, -34, 24, 1442055600000, 1442070000000, 0, 13, 3 ] + - [ "TH", 1442070000000, 0, 24, 1442062800000, 1442084400000, 0, 13, 3 ] + - [ "TH", 1442084400000, 13, 24, 1442066400000, null, 0, 13, 3 ] + - [ "TJ", 1442048400000, 1471, 1471, null, null, 1471, 1471, 1 ] + - [ "TN", 1442098800000, -9, -9, null, null, -9, -9, 1 ] + - [ "TR", 1442023200000, 306, 7078, null, 1442041200000, 306, -29, 1 ] + - [ "TR", 1442041200000, 1, 7078, null, 1442044800000, 306, -29, 1 ] + - [ "TR", 1442044800000, 41, 7078, 1442023200000, 1442048400000, 306, -29, 1 ] + - [ "TR", 1442048400000, 88, 7078, 1442041200000, 1442052000000, 306, -29, 1 ] + - [ "TR", 1442052000000, 41, 7078, 1442044800000, 1442055600000, 306, -29, 1 ] + - [ "TR", 1442055600000, 299, 7078, 1442048400000, 1442062800000, 306, -29, 2 ] + - [ "TR", 1442062800000, 315, 7078, 1442052000000, 1442066400000, 306, -29, 2 ] + - [ "TR", 1442066400000, 85, 7078, 1442055600000, 1442070000000, 306, -29, 2 ] + - [ "TR", 1442070000000, 236, 7078, 1442062800000, 1442077200000, 306, -29, 2 ] + - [ "TR", 1442077200000, 89, 7078, 1442066400000, 1442080800000, 306, -29, 2 ] + - [ "TR", 1442080800000, -1, 7078, 1442070000000, 1442084400000, 306, -29, 3 ] + - [ "TR", 1442084400000, 170, 7078, 1442077200000, 1442088000000, 306, -29, 3 ] + - [ "TR", 1442088000000, 2389, 7078, 1442080800000, 1442091600000, 306, -29, 3 ] + - [ "TR", 1442091600000, 3048, 7078, 1442084400000, 1442095200000, 306, -29, 3 ] + - [ "TR", 1442095200000, -29, 7078, 1442088000000, null, 306, -29, 3 ] + - [ "TT", 1442088000000, 9, 9, null, null, 9, 9, 1 ] + - [ "TW", 1442016000000, 92, 3656, null, 1442019600000, 92, -60, 1 ] + - [ "TW", 1442019600000, 0, 3656, null, 1442023200000, 92, -60, 1 ] + - [ "TW", 1442023200000, 97, 3656, 1442016000000, 1442026800000, 92, -60, 1 ] + - [ "TW", 1442026800000, 680, 3656, 1442019600000, 1442030400000, 92, -60, 1 ] + - [ "TW", 1442030400000, 0, 3656, 1442023200000, 1442034000000, 92, -60, 1 ] + - [ "TW", 1442034000000, 143, 3656, 1442026800000, 1442037600000, 92, -60, 1 ] + - [ "TW", 1442037600000, 266, 3656, 1442030400000, 1442041200000, 92, -60, 1 ] + - [ "TW", 1442041200000, 366, 3656, 1442034000000, 1442044800000, 92, -60, 1 ] + - [ "TW", 1442044800000, 24, 3656, 1442037600000, 1442048400000, 92, -60, 2 ] + - [ "TW", 1442048400000, 75, 3656, 1442041200000, 1442052000000, 92, -60, 2 ] + - [ "TW", 1442052000000, 24, 3656, 1442044800000, 1442055600000, 92, -60, 2 ] + - [ "TW", 1442055600000, 48, 3656, 1442048400000, 1442059200000, 92, -60, 2 ] + - [ "TW", 1442059200000, -157, 3656, 1442052000000, 1442062800000, 92, -60, 2 ] + - [ "TW", 1442062800000, -272, 3656, 1442055600000, 1442066400000, 92, -60, 2 ] + - [ "TW", 1442066400000, 624, 3656, 1442059200000, 1442070000000, 92, -60, 2 ] + - [ "TW", 1442070000000, 485, 3656, 1442062800000, 1442073600000, 92, -60, 3 ] + - [ "TW", 1442073600000, 772, 3656, 1442066400000, 1442077200000, 92, -60, 3 ] + - [ "TW", 1442077200000, 502, 3656, 1442070000000, 1442080800000, 92, -60, 3 ] + - [ "TW", 1442080800000, 24, 3656, 1442073600000, 1442084400000, 92, -60, 3 ] + - [ "TW", 1442084400000, 0, 3656, 1442077200000, 1442095200000, 92, -60, 3 ] + - [ "TW", 1442095200000, -77, 3656, 1442080800000, 1442098800000, 92, -60, 3 ] + - [ "TW", 1442098800000, -60, 3656, 1442084400000, null, 92, -60, 3 ] + - [ "UA", 1442034000000, 3468, 24898, null, 1442037600000, 3468, 38, 1 ] + - [ "UA", 1442037600000, -1, 24898, null, 1442041200000, 3468, 38, 1 ] + - [ "UA", 1442041200000, 74, 24898, 1442034000000, 1442044800000, 3468, 38, 1 ] + - [ "UA", 1442044800000, 280, 24898, 1442037600000, 1442048400000, 3468, 38, 1 ] + - [ "UA", 1442048400000, 2, 24898, 1442041200000, 1442052000000, 3468, 38, 1 ] + - [ "UA", 1442052000000, 410, 24898, 1442044800000, 1442055600000, 3468, 38, 1 ] + - [ "UA", 1442055600000, 14202, 24898, 1442048400000, 1442059200000, 3468, 38, 1 ] + - [ "UA", 1442059200000, -2, 24898, 1442052000000, 1442062800000, 3468, 38, 2 ] + - [ "UA", 1442062800000, 773, 24898, 1442055600000, 1442066400000, 3468, 38, 2 ] + - [ "UA", 1442066400000, 296, 24898, 1442059200000, 1442070000000, 3468, 38, 2 ] + - [ "UA", 1442070000000, 1733, 24898, 1442062800000, 1442073600000, 3468, 38, 2 ] + - [ "UA", 1442073600000, 4241, 24898, 1442066400000, 1442077200000, 3468, 38, 2 ] + - [ "UA", 1442077200000, -181, 24898, 1442070000000, 1442080800000, 3468, 38, 2 ] + - [ "UA", 1442080800000, -1, 24898, 1442073600000, 1442084400000, 3468, 38, 3 ] + - [ "UA", 1442084400000, 5, 24898, 1442077200000, 1442088000000, 3468, 38, 3 ] + - [ "UA", 1442088000000, -21, 24898, 1442080800000, 1442091600000, 3468, 38, 3 ] + - [ "UA", 1442091600000, -388, 24898, 1442084400000, 1442095200000, 3468, 38, 3 ] + - [ "UA", 1442095200000, -30, 24898, 1442088000000, 1442098800000, 3468, 38, 3 ] + - [ "UA", 1442098800000, 38, 24898, 1442091600000, null, 3468, 38, 3 ] + - [ "UG", 1442070000000, 1, 1, null, null, 1, 1, 1 ] + - [ "US", 1442016000000, 0, 38882, null, 1442019600000, 0, 3575, 1 ] + - [ "US", 1442019600000, 1043, 38882, null, 1442023200000, 0, 3575, 1 ] + - [ "US", 1442023200000, 2844, 38882, 1442016000000, 1442026800000, 0, 3575, 1 ] + - [ "US", 1442026800000, 1512, 38882, 1442019600000, 1442030400000, 0, 3575, 1 ] + - [ "US", 1442030400000, 2023, 38882, 1442023200000, 1442034000000, 0, 3575, 1 ] + - [ "US", 1442034000000, 3648, 38882, 1442026800000, 1442037600000, 0, 3575, 1 ] + - [ "US", 1442037600000, 3675, 38882, 1442030400000, 1442041200000, 0, 3575, 1 ] + - [ "US", 1442041200000, 1999, 38882, 1442034000000, 1442044800000, 0, 3575, 1 ] + - [ "US", 1442044800000, 139, 38882, 1442037600000, 1442048400000, 0, 3575, 2 ] + - [ "US", 1442048400000, -466, 38882, 1442041200000, 1442052000000, 0, 3575, 2 ] + - [ "US", 1442052000000, -2, 38882, 1442044800000, 1442055600000, 0, 3575, 2 ] + - [ "US", 1442055600000, 156, 38882, 1442048400000, 1442059200000, 0, 3575, 2 ] + - [ "US", 1442059200000, 11, 38882, 1442052000000, 1442062800000, 0, 3575, 2 ] + - [ "US", 1442062800000, 47, 38882, 1442055600000, 1442066400000, 0, 3575, 2 ] + - [ "US", 1442066400000, 772, 38882, 1442059200000, 1442070000000, 0, 3575, 2 ] + - [ "US", 1442070000000, 3505, 38882, 1442062800000, 1442073600000, 0, 3575, 2 ] + - [ "US", 1442073600000, 1100, 38882, 1442066400000, 1442077200000, 0, 3575, 3 ] + - [ "US", 1442077200000, 2168, 38882, 1442070000000, 1442080800000, 0, 3575, 3 ] + - [ "US", 1442080800000, 4001, 38882, 1442073600000, 1442084400000, 0, 3575, 3 ] + - [ "US", 1442084400000, 2523, 38882, 1442077200000, 1442088000000, 0, 3575, 3 ] + - [ "US", 1442088000000, 1691, 38882, 1442080800000, 1442091600000, 0, 3575, 3 ] + - [ "US", 1442091600000, 2502, 38882, 1442084400000, 1442095200000, 0, 3575, 3 ] + - [ "US", 1442095200000, 416, 38882, 1442088000000, 1442098800000, 0, 3575, 3 ] + - [ "US", 1442098800000, 3575, 38882, 1442091600000, null, 0, 3575, 3 ] + - [ "UY", 1442019600000, 77, 936, null, 1442023200000, 77, 23, 1 ] + - [ "UY", 1442023200000, 517, 936, null, 1442026800000, 77, 23, 1 ] + - [ "UY", 1442026800000, 76, 936, 1442019600000, 1442037600000, 77, 23, 1 ] + - [ "UY", 1442037600000, 1, 936, 1442023200000, 1442070000000, 77, 23, 2 ] + - [ "UY", 1442070000000, 284, 936, 1442026800000, 1442073600000, 77, 23, 2 ] + - [ "UY", 1442073600000, -42, 936, 1442037600000, 1442077200000, 77, 23, 3 ] + - [ "UY", 1442077200000, 23, 936, 1442070000000, null, 77, 23, 3 ] + - [ "UZ", 1442044800000, 1369, 1369, null, null, 1369, 1369, 1 ] + - [ "VE", 1442023200000, 115, 1101, null, 1442026800000, 115, 9, 1 ] + - [ "VE", 1442026800000, -17, 1101, null, 1442030400000, 115, 9, 1 ] + - [ "VE", 1442030400000, 51, 1101, 1442023200000, 1442034000000, 115, 9, 1 ] + - [ "VE", 1442034000000, -2, 1101, 1442026800000, 1442066400000, 115, 9, 1 ] + - [ "VE", 1442066400000, 18, 1101, 1442030400000, 1442070000000, 115, 9, 2 ] + - [ "VE", 1442070000000, 420, 1101, 1442034000000, 1442077200000, 115, 9, 2 ] + - [ "VE", 1442077200000, 412, 1101, 1442066400000, 1442084400000, 115, 9, 2 ] + - [ "VE", 1442084400000, 60, 1101, 1442070000000, 1442095200000, 115, 9, 3 ] + - [ "VE", 1442095200000, 35, 1101, 1442077200000, 1442098800000, 115, 9, 3 ] + - [ "VE", 1442098800000, 9, 1101, 1442084400000, null, 115, 9, 3 ] + - [ "VG", 1442062800000, -238, -238, null, null, -238, -238, 1 ] + - [ "VN", 1442023200000, -9, 1560, null, 1442026800000, -9, -10, 1 ] + - [ "VN", 1442026800000, 63, 1560, null, 1442034000000, -9, -10, 1 ] + - [ "VN", 1442034000000, -29, 1560, 1442023200000, 1442037600000, -9, -10, 1 ] + - [ "VN", 1442037600000, -11, 1560, 1442026800000, 1442041200000, -9, -10, 1 ] + - [ "VN", 1442041200000, 0, 1560, 1442034000000, 1442048400000, -9, -10, 1 ] + - [ "VN", 1442048400000, -15, 1560, 1442037600000, 1442052000000, -9, -10, 2 ] + - [ "VN", 1442052000000, 90, 1560, 1442041200000, 1442055600000, -9, -10, 2 ] + - [ "VN", 1442055600000, 37, 1560, 1442048400000, 1442059200000, -9, -10, 2 ] + - [ "VN", 1442059200000, 8, 1560, 1442052000000, 1442062800000, -9, -10, 2 ] + - [ "VN", 1442062800000, 146, 1560, 1442055600000, 1442066400000, -9, -10, 3 ] + - [ "VN", 1442066400000, 811, 1560, 1442059200000, 1442070000000, -9, -10, 3 ] + - [ "VN", 1442070000000, 479, 1560, 1442062800000, 1442084400000, -9, -10, 3 ] + - [ "VN", 1442084400000, -10, 1560, 1442066400000, null, -9, -10, 3 ] + - [ "ZA", 1442034000000, -3, 127, null, 1442048400000, -3, 1, 1 ] + - [ "ZA", 1442048400000, 79, 127, null, 1442059200000, -3, 1, 1 ] + - [ "ZA", 1442059200000, 50, 127, 1442034000000, 1442070000000, -3, 1, 2 ] + - [ "ZA", 1442070000000, 0, 127, 1442048400000, 1442091600000, -3, 1, 2 ] + - [ "ZA", 1442091600000, 1, 127, 1442059200000, null, -3, 1, 3 ] + - [ "ZM", 1442041200000, 133, 133, null, null, 133, 133, 1 ] + - [ "ZW", 1442044800000, 0, 254, null, 1442048400000, 0, 254, 1 ] + - [ "ZW", 1442048400000, 254, 254, null, null, 0, 254, 2 ] \ No newline at end of file