You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by gi...@apache.org on 2023/03/09 23:48:20 UTC

[druid] branch master updated: Window planning: use collation traits, improve subquery logic. (#13902)

This is an automated email from the ASF dual-hosted git repository.

gian pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/master by this push:
     new bf39b4d313 Window planning: use collation traits, improve subquery logic. (#13902)
bf39b4d313 is described below

commit bf39b4d313277be9e1dc91a6fe9d2749759ebb34
Author: Gian Merlino <gi...@gmail.com>
AuthorDate: Thu Mar 9 15:48:13 2023 -0800

    Window planning: use collation traits, improve subquery logic. (#13902)
    
    * Window planning: use collation traits, improve subquery logic.
    
    SQL changes:
    
    1) Attach RelCollation (sorting) trait to any PartialDruidQuery
       that ends in AGGREGATE or AGGREGATE_PROJECT. This allows planning to
       take advantage of the fact that Druid sorts by dimensions when
       doing aggregations.
    
    2) Windowing: inspect RelCollation trait from input, and insert naiveSort
       if, and only if, necessary.
    
    3) Windowing: add support for Project after Window, when the Project
       is a simple mapping. Helps eliminate subqueries.
    
    4) DruidRules: update logic for considering subqueries to reflect that
       subqueries are not required to be GroupBys, and that we have a bunch
       of new Stages now. With all of this evolution that has happened, the
       old logic didn't quite make sense.
    
    Native changes:
    
    1) Use merge sort (stable) rather than quicksort when sorting
       RowsAndColumns. Makes it easier to write test cases for plans that
       involve re-sorting the data.
    
    * Changes from review.
    
    * Mark the bad test as failing.
    
    * Additional update.
    
    * Fix failingTest.
    
    * Fix tests.
    
    * Mark a var final.
---
 .../query/rowsandcols/ArrayListRowsAndColumns.java |    3 +-
 .../semantic/DefaultNaiveSortMaker.java            |    4 +-
 .../sql/calcite/rel/DruidCorrelateUnnestRel.java   |    2 +-
 .../druid/sql/calcite/rel/DruidJoinQueryRel.java   |    4 +-
 .../druid/sql/calcite/rel/DruidOuterQueryRel.java  |    4 +-
 .../apache/druid/sql/calcite/rel/DruidQuery.java   |   38 +-
 .../druid/sql/calcite/rel/DruidQueryRel.java       |    3 +-
 .../sql/calcite/rel/DruidUnionDataSourceRel.java   |    2 +-
 .../druid/sql/calcite/rel/PartialDruidQuery.java   |  241 ++++-
 .../apache/druid/sql/calcite/rel/Windowing.java    |  159 ++-
 .../apache/druid/sql/calcite/rule/DruidRules.java  |   67 +-
 .../druid/sql/calcite/CalciteJoinQueryTest.java    |   50 +-
 .../apache/druid/sql/calcite/CalciteQueryTest.java |  126 +--
 .../org/apache/druid/sql/http/SqlResourceTest.java |    2 +-
 .../wikipediaAggregationsMultipleOrdering.sqlTest  |  142 +--
 .../tests/window/wikipediaScanWindow.sqlTest       |   23 +
 .../wikipediaSimplePartitionInitialSort.sqlTest    | 1024 ++++++++++++++++++++
 17 files changed, 1551 insertions(+), 343 deletions(-)

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 073fad16a4..92c10fb8e8 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
@@ -291,7 +291,8 @@ public class ArrayListRowsAndColumns<RowType> implements AppendableRowsAndColumn
       swappers.add(swapper);
     }
 
-    Arrays.quickSort(
+    // Use stable sort, so peer rows retain original order.
+    Arrays.mergeSort(
         0,
         rows.size(),
         (lhs, rhs) -> {
diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultNaiveSortMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultNaiveSortMaker.java
index a21651e0d2..45915dd9b5 100644
--- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultNaiveSortMaker.java
+++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultNaiveSortMaker.java
@@ -97,7 +97,9 @@ public class DefaultNaiveSortMaker implements NaiveSortMaker
       }
 
       final int numColsToCompare = index;
-      Arrays.quickSort(
+
+      // Use stable sort, so peer rows retain original order.
+      Arrays.mergeSort(
           0,
           rac.numRows(),
           (k1, k2) -> {
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java
index 847c41b63e..53421807f2 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidCorrelateUnnestRel.java
@@ -122,7 +122,7 @@ public class DruidCorrelateUnnestRel extends DruidRel<DruidCorrelateUnnestRel>
   {
     return new DruidCorrelateUnnestRel(
         getCluster(),
-        getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
+        newQueryBuilder.getTraitSet(getConvention()),
         correlateRel,
         newQueryBuilder,
         leftFilter,
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java
index 293dba817a..d0f6f7bef2 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java
@@ -124,7 +124,7 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
   {
     return new DruidJoinQueryRel(
         getCluster(),
-        getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
+        newQueryBuilder.getTraitSet(getConvention()),
         joinRel,
         leftFilter,
         newQueryBuilder,
@@ -136,7 +136,7 @@ public class DruidJoinQueryRel extends DruidRel<DruidJoinQueryRel>
   public DruidQuery toDruidQuery(final boolean finalizeAggregations)
   {
     final DruidRel<?> leftDruidRel = (DruidRel<?>) left;
-    final DruidQuery leftQuery = Preconditions.checkNotNull((leftDruidRel).toDruidQuery(false), "leftQuery");
+    final DruidQuery leftQuery = Preconditions.checkNotNull(leftDruidRel.toDruidQuery(false), "leftQuery");
     final RowSignature leftSignature = leftQuery.getOutputRowSignature();
     final DataSource leftDataSource;
 
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java
index d9bd16343e..e679f27553 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java
@@ -73,7 +73,7 @@ public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
   {
     return new DruidOuterQueryRel(
         sourceRel.getCluster(),
-        sourceRel.getTraitSet().plusAll(partialQuery.getRelTraits()),
+        partialQuery.getTraitSet(sourceRel.getConvention()),
         sourceRel,
         partialQuery,
         sourceRel.getPlannerContext()
@@ -91,7 +91,7 @@ public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
   {
     return new DruidOuterQueryRel(
         getCluster(),
-        getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
+        newQueryBuilder.getTraitSet(getConvention()),
         sourceRel,
         newQueryBuilder,
         getPlannerContext()
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
index 993ebe9371..3b02250966 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
@@ -473,6 +473,7 @@ public class DruidQuery
    * @param virtualColumnRegistry re-usable virtual column references
    * @param typeFactory           factory for SQL types
    * @return dimensions
+   *
    * @throws CannotBuildQueryException if dimensions cannot be computed
    */
   private static List<DimensionExpression> computeDimensions(
@@ -582,7 +583,9 @@ public class DruidQuery
    * @param finalizeAggregations  true if this query should include explicit finalization for all of its
    *                              aggregators, where required. Useful for subqueries where Druid's native query layer
    *                              does not do this automatically.
+   *
    * @return aggregations
+   *
    * @throws CannotBuildQueryException if dimensions cannot be computed
    */
   private static List<Aggregation> computeAggregations(
@@ -875,8 +878,6 @@ public class DruidQuery
    * <p>
    * Necessary because some combinations are unsafe, mainly because they would lead to the creation of too many
    * time-granular buckets during query processing.
-   *
-   * @see Granularity#getIterable(Interval) the problematic method call we are trying to avoid
    */
   private static boolean canUseQueryGranularity(
       final DataSource dataSource,
@@ -953,11 +954,6 @@ public class DruidQuery
    */
   private Query<?> computeQuery()
   {
-    if (windowing != null) {
-      // Windowing can only be handled by window queries.
-      return toWindowQuery();
-    }
-
     if (dataSource instanceof QueryDataSource) {
       // If there is a subquery, then we prefer the outer query to be a groupBy if possible, since this potentially
       // enables more efficient execution. (The groupBy query toolchest can handle some subqueries by itself, without
@@ -969,6 +965,11 @@ public class DruidQuery
       }
     }
 
+    final WindowOperatorQuery operatorQuery = toWindowQuery();
+    if (operatorQuery != null) {
+      return operatorQuery;
+    }
+
     final TimeBoundaryQuery timeBoundaryQuery = toTimeBoundaryQuery();
     if (timeBoundaryQuery != null) {
       return timeBoundaryQuery;
@@ -1009,7 +1010,8 @@ public class DruidQuery
         || grouping == null
         || grouping.getSubtotals().hasEffect(grouping.getDimensionSpecs())
         || grouping.getHavingFilter() != null
-        || selectProjection != null) {
+        || selectProjection != null
+        || windowing != null) {
       return null;
     }
 
@@ -1073,7 +1075,8 @@ public class DruidQuery
     if (!plannerContext.engineHasFeature(EngineFeature.TIMESERIES_QUERY)
         || grouping == null
         || grouping.getSubtotals().hasEffect(grouping.getDimensionSpecs())
-        || grouping.getHavingFilter() != null) {
+        || grouping.getHavingFilter() != null
+        || windowing != null) {
       return null;
     }
 
@@ -1193,7 +1196,7 @@ public class DruidQuery
     }
 
     // Must have GROUP BY one column, no GROUPING SETS, ORDER BY ≤ 1 column, LIMIT > 0 and ≤ maxTopNLimit,
-    // no OFFSET, no HAVING.
+    // no OFFSET, no HAVING, no windowing.
     final boolean topNOk = grouping != null
                            && grouping.getDimensions().size() == 1
                            && !grouping.getSubtotals().hasEffect(grouping.getDimensionSpecs())
@@ -1204,7 +1207,8 @@ public class DruidQuery
                                && sorting.getOffsetLimit().getLimit() <= plannerContext.getPlannerConfig()
                                                                                        .getMaxTopNLimit()
                                && !sorting.getOffsetLimit().hasOffset())
-                           && grouping.getHavingFilter() == null;
+                           && grouping.getHavingFilter() == null
+                           && windowing == null;
 
     if (!topNOk) {
       return null;
@@ -1283,7 +1287,7 @@ public class DruidQuery
   @Nullable
   private GroupByQuery toGroupByQuery()
   {
-    if (grouping == null) {
+    if (grouping == null || windowing != null) {
       return null;
     }
 
@@ -1428,8 +1432,8 @@ public class DruidQuery
   @Nullable
   private ScanQuery toScanQuery()
   {
-    if (grouping != null) {
-      // Scan cannot GROUP BY.
+    if (grouping != null || windowing != null) {
+      // Scan cannot GROUP BY or do windows.
       return null;
     }
 
@@ -1483,16 +1487,16 @@ public class DruidQuery
         // Cannot handle this ordering.
         // Scan cannot ORDER BY non-time columns.
         plannerContext.setPlanningError(
-            "SQL query requires order by non-time column %s that is not supported.",
+            "SQL query requires order by non-time column %s, which is not supported.",
             orderByColumns
         );
         return null;
       }
       if (!dataSource.isConcrete()) {
         // Cannot handle this ordering.
-        // Scan cannot ORDER BY non-time columns.
+        // Scan cannot ORDER BY non-concrete datasources on _any_ column.
         plannerContext.setPlanningError(
-            "SQL query is a scan and requires order by on a datasource[%s], which is not supported.",
+            "SQL query requires order by on non-concrete datasource [%s], which is not supported.",
             dataSource
         );
         return null;
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java
index 3e5c72a71f..61999f8ad6 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQueryRel.java
@@ -37,7 +37,6 @@ import org.apache.druid.sql.calcite.planner.PlannerContext;
 import org.apache.druid.sql.calcite.table.DruidTable;
 
 import javax.annotation.Nullable;
-
 import java.util.Set;
 
 /**
@@ -170,7 +169,7 @@ public class DruidQueryRel extends DruidRel<DruidQueryRel>
   {
     return new DruidQueryRel(
         getCluster(),
-        getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
+        newQueryBuilder.getTraitSet(getConvention()),
         table,
         druidTable,
         getPlannerContext(),
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionDataSourceRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionDataSourceRel.java
index 15e363af1e..5e213de711 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionDataSourceRel.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidUnionDataSourceRel.java
@@ -107,7 +107,7 @@ public class DruidUnionDataSourceRel extends DruidRel<DruidUnionDataSourceRel>
   {
     return new DruidUnionDataSourceRel(
         getCluster(),
-        getTraitSet().plusAll(newQueryBuilder.getRelTraits()),
+        newQueryBuilder.getTraitSet(getConvention()),
         unionRel,
         unionColumnNames,
         newQueryBuilder,
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java
index 0cd71af8e5..ae1a007b62 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/PartialDruidQuery.java
@@ -20,8 +20,15 @@
 package org.apache.druid.sql.calcite.rel;
 
 import com.google.common.base.Preconditions;
+import it.unimi.dsi.fastutil.ints.Int2IntMap;
+import it.unimi.dsi.fastutil.ints.Int2IntOpenHashMap;
+import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Filter;
@@ -31,6 +38,7 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlKind;
@@ -41,6 +49,7 @@ import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.sql.calcite.planner.PlannerContext;
 
 import javax.annotation.Nullable;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
 import java.util.function.Supplier;
@@ -61,6 +70,7 @@ public class PartialDruidQuery
   private final Sort sort;
   private final Project sortProject;
   private final Window window;
+  private final Project windowProject;
 
   public enum Stage
   {
@@ -71,19 +81,53 @@ public class PartialDruidQuery
     WHERE_FILTER,
     SELECT_PROJECT,
 
-    // AGGREGATE, HAVING_FILTER, AGGREGATE_PROJECT can only be present on non-WINDOW aggregating queries.
+    // AGGREGATE, HAVING_FILTER, AGGREGATE_PROJECT can be present on non-WINDOW aggregating queries.
     AGGREGATE,
-    HAVING_FILTER,
-    AGGREGATE_PROJECT,
+    HAVING_FILTER {
+      @Override
+      public boolean canFollow(Stage stage)
+      {
+        return stage == AGGREGATE;
+      }
+    },
+    AGGREGATE_PROJECT {
+      @Override
+      public boolean canFollow(Stage stage)
+      {
+        return stage == AGGREGATE || stage == HAVING_FILTER;
+      }
+    },
 
     // SORT, SORT_PROJECT may be present on any query, except ones with WINDOW.
     SORT,
-    SORT_PROJECT,
-
-    // WINDOW may be present only together with SCAN.
-    WINDOW,
+    SORT_PROJECT {
+      @Override
+      public boolean canFollow(Stage stage)
+      {
+        return stage == SORT;
+      }
+    },
+
+    // WINDOW, WINDOW_PROJECT may be present only together with SCAN.
+    WINDOW {
+      @Override
+      public boolean canFollow(Stage stage)
+      {
+        return stage == SCAN;
+      }
+    },
+    WINDOW_PROJECT {
+      @Override
+      public boolean canFollow(Stage stage)
+      {
+        return stage == WINDOW;
+      }
+    };
 
-    UNNEST_PROJECT
+    public boolean canFollow(final Stage stage)
+    {
+      return stage.compareTo(this) < 0;
+    }
   }
 
   private PartialDruidQuery(
@@ -97,6 +141,7 @@ public class PartialDruidQuery
       final Sort sort,
       final Project sortProject,
       final Window window,
+      final Project windowProject,
       final Project unnestProject
   )
   {
@@ -110,16 +155,28 @@ public class PartialDruidQuery
     this.sort = sort;
     this.sortProject = sortProject;
     this.window = window;
+    this.windowProject = windowProject;
     this.unnestProject = unnestProject;
   }
 
-  public static PartialDruidQuery create(final RelNode scanRel)
+  public static PartialDruidQuery create(final RelNode inputRel)
   {
     final Supplier<RelBuilder> builderSupplier = () -> RelFactories.LOGICAL_BUILDER.create(
-        scanRel.getCluster(),
-        scanRel.getTable() != null ? scanRel.getTable().getRelOptSchema() : null
+        inputRel.getCluster(),
+        inputRel.getTable() != null ? inputRel.getTable().getRelOptSchema() : null
+    );
+    return new PartialDruidQuery(builderSupplier, inputRel, null, null, null, null, null, null, null, null, null, null);
+  }
+
+  public static PartialDruidQuery createOuterQuery(final PartialDruidQuery inputQuery)
+  {
+    final RelNode inputRel = inputQuery.leafRel();
+    return create(
+        inputRel.copy(
+            inputQuery.getTraitSet(inputRel.getConvention()),
+            inputRel.getInputs()
+        )
     );
-    return new PartialDruidQuery(builderSupplier, scanRel, null, null, null, null, null, null, null, null, null);
   }
 
   public RelNode getScan()
@@ -172,6 +229,11 @@ public class PartialDruidQuery
     return window;
   }
 
+  public Project getWindowProject()
+  {
+    return windowProject;
+  }
+
   public PartialDruidQuery withWhereFilter(final Filter newWhereFilter)
   {
     validateStage(Stage.WHERE_FILTER);
@@ -186,6 +248,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -230,6 +293,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -248,6 +312,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -266,6 +331,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -284,6 +350,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -302,6 +369,7 @@ public class PartialDruidQuery
         newSort,
         sortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -320,6 +388,7 @@ public class PartialDruidQuery
         sort,
         newSortProject,
         window,
+        windowProject,
         unnestProject
     );
   }
@@ -338,6 +407,7 @@ public class PartialDruidQuery
         sort,
         sortProject,
         newWindow,
+        windowProject,
         unnestProject
     );
   }
@@ -355,18 +425,96 @@ public class PartialDruidQuery
         sort,
         sortProject,
         window,
+        windowProject,
         newUnnestProject
     );
   }
 
+  public PartialDruidQuery withWindowProject(final Project newWindowProject)
+  {
+    validateStage(Stage.WINDOW_PROJECT);
+    return new PartialDruidQuery(
+        builderSupplier,
+        scan,
+        whereFilter,
+        selectProject,
+        aggregate,
+        aggregateProject,
+        havingFilter,
+        sort,
+        sortProject,
+        window,
+        newWindowProject,
+        unnestProject
+    );
+  }
+
   public RelDataType getRowType()
   {
     return leafRel().getRowType();
   }
 
-  public RelTrait[] getRelTraits()
+  /**
+   * Get traits for this partial query.
+   *
+   * This is the traits from {@link #leafRel()}, plus {@link RelCollationTraitDef} if {@link #stage()} is
+   * {@link Stage#AGGREGATE} or {@link Stage#AGGREGATE_PROJECT} (to represent the fact that Druid sorts by grouping
+   * keys when grouping).
+   *
+   * @param convention convention to include in the returned array
+   */
+  public RelTraitSet getTraitSet(final Convention convention)
   {
-    return leafRel().getTraitSet().toArray(new RelTrait[0]);
+    final RelTraitSet leafRelTraits = leafRel().getTraitSet();
+
+    final Stage currentStage = stage();
+
+    switch (currentStage) {
+      case AGGREGATE:
+      case AGGREGATE_PROJECT:
+        final RelCollation collation = leafRelTraits.getTrait(RelCollationTraitDef.INSTANCE);
+        if ((collation == null || collation.getFieldCollations().isEmpty()) && aggregate.getGroupSets().size() == 1) {
+          // Druid sorts by grouping keys when grouping. Add the collation.
+          // Note: [aggregate.getGroupSets().size() == 1] above means that collation isn't added for GROUPING SETS.
+          final List<RelFieldCollation> sortFields = new ArrayList<>();
+
+          if (currentStage == Stage.AGGREGATE) {
+            for (int i = 0; i < aggregate.getGroupCount(); i++) {
+              sortFields.add(new RelFieldCollation(i));
+            }
+          } else {
+            // AGGREGATE_PROJECT
+            final List<RexNode> projectExprs = aggregateProject.getProjects();
+
+            // Build a map of all Project exprs that are input refs. Project expr index -> dimension index.
+            final Int2IntMap dimensionMapping = new Int2IntOpenHashMap();
+            dimensionMapping.defaultReturnValue(-1);
+            for (int i = 0; i < projectExprs.size(); i++) {
+              RexNode projectExpr = projectExprs.get(i);
+              if (projectExpr.isA(SqlKind.INPUT_REF)) {
+                dimensionMapping.put(((RexInputRef) projectExpr).getIndex(), i);
+              }
+            }
+
+            // Add collations for dimensions so long as they are all mappings.
+            for (int i = 0; i < aggregate.getGroupCount(); i++) {
+              final int mapping = dimensionMapping.applyAsInt(i);
+              if (mapping >= 0) {
+                sortFields.add(new RelFieldCollation(mapping));
+              } else {
+                // As soon as we see a non-mapping, stop adding.
+                break;
+              }
+            }
+          }
+
+          return leafRelTraits.plus(convention).plus(RelCollations.of(sortFields));
+        }
+        // Fall through.
+
+      default:
+        return leafRelTraits.plus(convention);
+    }
   }
 
   public DruidQuery build(
@@ -410,26 +558,7 @@ public class PartialDruidQuery
 
   public boolean canAccept(final Stage stage)
   {
-    final Stage currentStage = stage();
-
-    if (stage == Stage.WINDOW) {
-      // Special case: WINDOW can only be provided along with SCAN.
-      return currentStage == Stage.SCAN;
-    } else if (currentStage == Stage.SELECT_PROJECT && stage == Stage.SELECT_PROJECT) {
-      // Special case: allow layering SELECT_PROJECT on top of SELECT_PROJECT. Calcite's builtin rules cannot
-      // always collapse these, so we have to (one example: testSemiJoinWithOuterTimeExtract). See
-      // withSelectProject for the code here that handles this.
-      return true;
-    } else if (stage.compareTo(currentStage) <= 0) {
-      // Cannot go backwards.
-      return false;
-    } else if (stage.compareTo(Stage.AGGREGATE) > 0 && stage.compareTo(Stage.SORT) < 0 && aggregate == null) {
-      // Cannot do post-aggregation stages without an aggregation.
-      return false;
-    } else {
-      // If we are after the SORT phase, make sure we have a sort...
-      return stage.compareTo(Stage.SORT) <= 0 || sort != null;
-    }
+    return stage.canFollow(stage());
   }
 
   /**
@@ -441,7 +570,9 @@ public class PartialDruidQuery
   @SuppressWarnings("VariableNotUsedInsideIf")
   public Stage stage()
   {
-    if (window != null) {
+    if (windowProject != null) {
+      return Stage.WINDOW_PROJECT;
+    } else if (window != null) {
       return Stage.WINDOW;
     } else if (sortProject != null) {
       return Stage.SORT_PROJECT;
@@ -472,6 +603,8 @@ public class PartialDruidQuery
     final Stage currentStage = stage();
 
     switch (currentStage) {
+      case WINDOW_PROJECT:
+        return windowProject;
       case WINDOW:
         return window;
       case SORT_PROJECT:
@@ -504,7 +637,7 @@ public class PartialDruidQuery
 
     // Account for the cost of post-scan expressions.
     if (getSelectProject() != null) {
-      for (final RexNode rexNode : getSelectProject().getChildExps()) {
+      for (final RexNode rexNode : getSelectProject().getProjects()) {
         if (!rexNode.isA(SqlKind.INPUT_REF)) {
           cost += CostEstimates.COST_EXPRESSION;
         }
@@ -534,7 +667,7 @@ public class PartialDruidQuery
 
     // Account for the cost of post-aggregation expressions.
     if (getAggregateProject() != null) {
-      for (final RexNode rexNode : getAggregateProject().getChildExps()) {
+      for (final RexNode rexNode : getAggregateProject().getProjects()) {
         if (!rexNode.isA(SqlKind.INPUT_REF)) {
           cost += CostEstimates.COST_EXPRESSION;
         }
@@ -543,7 +676,7 @@ public class PartialDruidQuery
 
     // Account for the cost of post-sort expressions.
     if (getSortProject() != null) {
-      for (final RexNode rexNode : getSortProject().getChildExps()) {
+      for (final RexNode rexNode : getSortProject().getProjects()) {
         if (!rexNode.isA(SqlKind.INPUT_REF)) {
           cost += CostEstimates.COST_EXPRESSION;
         }
@@ -564,7 +697,7 @@ public class PartialDruidQuery
   }
 
   @Override
-  public boolean equals(final Object o)
+  public boolean equals(Object o)
   {
     if (this == o) {
       return true;
@@ -572,15 +705,18 @@ public class PartialDruidQuery
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
-    final PartialDruidQuery that = (PartialDruidQuery) o;
-    return Objects.equals(scan, that.scan) &&
-           Objects.equals(whereFilter, that.whereFilter) &&
-           Objects.equals(selectProject, that.selectProject) &&
-           Objects.equals(aggregate, that.aggregate) &&
-           Objects.equals(havingFilter, that.havingFilter) &&
-           Objects.equals(aggregateProject, that.aggregateProject) &&
-           Objects.equals(sort, that.sort) &&
-           Objects.equals(sortProject, that.sortProject);
+    PartialDruidQuery that = (PartialDruidQuery) o;
+    return Objects.equals(scan, that.scan)
+           && Objects.equals(whereFilter, that.whereFilter)
+           && Objects.equals(selectProject, that.selectProject)
+           && Objects.equals(aggregate, that.aggregate)
+           && Objects.equals(havingFilter, that.havingFilter)
+           && Objects.equals(aggregateProject, that.aggregateProject)
+           && Objects.equals(sort, that.sort)
+           && Objects.equals(sortProject, that.sortProject)
+           && Objects.equals(window, that.window)
+           && Objects.equals(windowProject, that.windowProject)
+           && Objects.equals(unnestProject, that.unnestProject);
   }
 
   @Override
@@ -594,7 +730,10 @@ public class PartialDruidQuery
         havingFilter,
         aggregateProject,
         sort,
-        sortProject
+        sortProject,
+        window,
+        windowProject,
+        unnestProject
     );
   }
 
@@ -610,6 +749,8 @@ public class PartialDruidQuery
            ", aggregateProject=" + aggregateProject +
            ", sort=" + sort +
            ", sortProject=" + sortProject +
+           ", window=" + window +
+           ", windowProject=" + windowProject +
            ", unnestProject=" + unnestProject +
            '}';
   }
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 6eb1c6b1e5..af5983d6ed 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
@@ -22,6 +22,8 @@ package org.apache.druid.sql.calcite.rel;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Project;
@@ -32,6 +34,7 @@ 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.calcite.util.mapping.Mappings;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.query.QueryException;
@@ -65,6 +68,7 @@ import org.apache.druid.sql.calcite.table.RowSignatures;
 import javax.annotation.Nonnull;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
 
@@ -111,7 +115,7 @@ public class Windowing
   public static Windowing fromCalciteStuff(
       final PartialDruidQuery partialQuery,
       final PlannerContext plannerContext,
-      final RowSignature rowSignature,
+      final RowSignature sourceRowSignature,
       final RexBuilder rexBuilder
   )
   {
@@ -119,31 +123,45 @@ public class Windowing
 
     ArrayList<OperatorFactory> ops = new ArrayList<>();
 
-    final List<String> expectedOutputColumns = new ArrayList<>(rowSignature.getColumnNames());
-    final String outputNamePrefix = Calcites.findUnusedPrefixForDigits("w", rowSignature.getColumnNames());
+    final List<String> windowOutputColumns = new ArrayList<>(sourceRowSignature.getColumnNames());
+    final String outputNamePrefix = Calcites.findUnusedPrefixForDigits("w", sourceRowSignature.getColumnNames());
     int outputNameCounter = 0;
+
+    // Track prior partition columns and sort columns group-to-group, so we only insert sorts and repartitions if
+    // we really need to.
+    List<String> priorPartitionColumns = null;
+    LinkedHashSet<ColumnWithDirection> priorSortColumns = new LinkedHashSet<>();
+
+    final RelCollation priorCollation = partialQuery.getScan().getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+    if (priorCollation != null) {
+      // Populate initial priorSortColumns using collation of the input to the window operation. Allows us to skip
+      // the initial sort operator if the rows were already in the desired order.
+      priorSortColumns = computeSortColumnsFromRelCollation(priorCollation, sourceRowSignature);
+    }
+
     for (int i = 0; i < window.groups.size(); ++i) {
-      final WindowGroup group = new WindowGroup(window, window.groups.get(i), rowSignature);
+      final WindowGroup group = new WindowGroup(window, window.groups.get(i), sourceRowSignature);
 
-      if (i > 0) {
-        LinkedHashSet<ColumnWithDirection> sortColumns = new LinkedHashSet<>();
-        for (String partitionColumn : group.getPartitionColumns()) {
-          sortColumns.add(ColumnWithDirection.ascending(partitionColumn));
-        }
-        sortColumns.addAll(group.getOrdering());
+      final LinkedHashSet<ColumnWithDirection> sortColumns = new LinkedHashSet<>();
+      for (String partitionColumn : group.getPartitionColumns()) {
+        sortColumns.add(ColumnWithDirection.ascending(partitionColumn));
+      }
+      sortColumns.addAll(group.getOrdering());
 
+      // Add sorting and partitioning if needed.
+      if (!sortMatches(priorSortColumns, sortColumns)) {
+        // Sort order needs to change. Resort and repartition.
         ops.add(new NaiveSortOperatorFactory(new ArrayList<>(sortColumns)));
+        ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
+        priorSortColumns = sortColumns;
+        priorPartitionColumns = group.getPartitionColumns();
+      } else if (!group.getPartitionColumns().equals(priorPartitionColumns)) {
+        // Sort order doesn't need to change, but partitioning does. Only repartition.
+        ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
+        priorPartitionColumns = group.getPartitionColumns();
       }
 
-      // 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.  Unfortunately, the way that the queries are re-written to DruidRels
-      // loses all the context of sub-queries, making it not possible to validate this without changing how the
-      // various Druid rules work (i.e. a very large blast radius change).  For now, it is easy enough to validate
-      // this when we build the native query, so we validate it there.
-
-      // Aggregations.
+      // Add aggregations.
       final List<AggregateCall> aggregateCalls = group.getAggregateCalls();
 
       final List<Processor> processors = new ArrayList<>();
@@ -151,13 +169,13 @@ public class Windowing
 
       for (AggregateCall aggregateCall : aggregateCalls) {
         final String aggName = outputNamePrefix + outputNameCounter++;
-        expectedOutputColumns.add(aggName);
+        windowOutputColumns.add(aggName);
 
         ProcessorMaker maker = KNOWN_WINDOW_FNS.get(aggregateCall.getAggregation().getName());
         if (maker == null) {
           final Aggregation aggregation = GroupByRules.translateAggregateCall(
               plannerContext,
-              rowSignature,
+              sourceRowSignature,
               null,
               rexBuilder,
               partialQuery.getSelectProject(),
@@ -182,7 +200,7 @@ public class Windowing
               new WindowAggregate(
                   aggName,
                   aggregateCall,
-                  rowSignature,
+                  sourceRowSignature,
                   plannerContext,
                   partialQuery.getSelectProject(),
                   window.constants,
@@ -206,20 +224,37 @@ public class Windowing
         throw new ISE("No processors from Window[%s], why was this code called?", window);
       }
 
-      // The ordering required for partitioning is actually not important for the semantics.  However, it *is*
-      // important that it be consistent across the query.  Because if the incoming data is sorted descending
-      // and we try to partition on an ascending sort, we will think the data is not sorted correctly
-      ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
       ops.add(new WindowOperatorFactory(
           processors.size() == 1 ?
           processors.get(0) : new ComposingProcessor(processors.toArray(new Processor[0]))
       ));
     }
 
-    return new Windowing(
-        RowSignatures.fromRelDataType(expectedOutputColumns, window.getRowType()),
-        ops
-    );
+    // Apply windowProject, if present.
+    if (partialQuery.getWindowProject() != null) {
+      // We know windowProject is a mapping due to the isMapping() check in DruidRules. Check for null anyway,
+      // as defensive programming.
+      final Mappings.TargetMapping mapping = Preconditions.checkNotNull(
+          partialQuery.getWindowProject().getMapping(),
+          "mapping for windowProject[%s]", partialQuery.getWindowProject()
+      );
+
+      final List<String> windowProjectOutputColumns = new ArrayList<>();
+      for (int i = 0; i < mapping.size(); i++) {
+        windowProjectOutputColumns.add(windowOutputColumns.get(mapping.getSourceOpt(i)));
+      }
+
+      return new Windowing(
+          RowSignatures.fromRelDataType(windowProjectOutputColumns, partialQuery.getWindowProject().getRowType()),
+          ops
+      );
+    } else {
+      // No windowProject.
+      return new Windowing(
+          RowSignatures.fromRelDataType(windowOutputColumns, window.getRowType()),
+          ops
+      );
+    }
   }
 
   private final RowSignature signature;
@@ -400,4 +435,68 @@ public class Windowing
       return ((Number) getConstantArgument(argPosition).getValue()).intValue();
     }
   }
+
+  /**
+   * Return a list of {@link ColumnWithDirection} corresponding to a {@link RelCollation}.
+   *
+   * @param collation          collation
+   * @param sourceRowSignature signature of the collated rows
+   */
+  private static LinkedHashSet<ColumnWithDirection> computeSortColumnsFromRelCollation(
+      final RelCollation collation,
+      final RowSignature sourceRowSignature
+  )
+  {
+    final LinkedHashSet<ColumnWithDirection> retVal = new LinkedHashSet<>();
+
+    for (RelFieldCollation fieldCollation : collation.getFieldCollations()) {
+      final ColumnWithDirection.Direction direction;
+
+      switch (fieldCollation.getDirection()) {
+        case ASCENDING:
+        case STRICTLY_ASCENDING:
+          direction = ColumnWithDirection.Direction.ASC;
+          break;
+
+        case DESCENDING:
+        case STRICTLY_DESCENDING:
+          direction = ColumnWithDirection.Direction.DESC;
+          break;
+
+        default:
+          // Not a useful direction. Return whatever we've come up with so far.
+          return retVal;
+      }
+
+      final ColumnWithDirection columnWithDirection = new ColumnWithDirection(
+          sourceRowSignature.getColumnName(fieldCollation.getFieldIndex()),
+          direction
+      );
+
+      retVal.add(columnWithDirection);
+    }
+
+    return retVal;
+  }
+
+  /**
+   * Whether currentSort is a prefix of priorSort. (i.e., whether data sorted by priorSort is *also* sorted
+   * by currentSort.)
+   */
+  private static boolean sortMatches(
+      final Iterable<ColumnWithDirection> priorSort,
+      final Iterable<ColumnWithDirection> currentSort
+  )
+  {
+    final Iterator<ColumnWithDirection> priorIterator = priorSort.iterator();
+    final Iterator<ColumnWithDirection> currentIterator = currentSort.iterator();
+
+    while (currentIterator.hasNext()) {
+      if (!priorIterator.hasNext() || !currentIterator.next().equals(priorIterator.next())) {
+        return false;
+      }
+    }
+
+    return true;
+  }
 }
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidRules.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidRules.java
index 4276a48b2f..2dd94c62d7 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidRules.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidRules.java
@@ -105,6 +105,14 @@ public class DruidRules
 
     if (plannerContext.queryContext().getBoolean(DruidQuery.CTX_ENABLE_WINDOW_FNS, false)) {
       retVal.add(new DruidQueryRule<>(Window.class, PartialDruidQuery.Stage.WINDOW, PartialDruidQuery::withWindow));
+      retVal.add(
+          new DruidQueryRule<>(
+              Project.class,
+              PartialDruidQuery.Stage.WINDOW_PROJECT,
+              Project::isMapping, // We can remap fields, but not apply expressions
+              PartialDruidQuery::withWindowProject
+          )
+      );
       retVal.add(DruidOuterQueryRule.WINDOW);
     }
     return retVal;
@@ -113,12 +121,14 @@ public class DruidRules
   public static class DruidQueryRule<RelType extends RelNode> extends RelOptRule
   {
     private final PartialDruidQuery.Stage stage;
-    private final BiFunction<PartialDruidQuery, RelType, PartialDruidQuery> f;
+    private final Predicate<RelType> matchesFn;
+    private final BiFunction<PartialDruidQuery, RelType, PartialDruidQuery> applyFn;
 
     public DruidQueryRule(
         final Class<RelType> relClass,
         final PartialDruidQuery.Stage stage,
-        final BiFunction<PartialDruidQuery, RelType, PartialDruidQuery> f
+        final Predicate<RelType> matchesFn,
+        final BiFunction<PartialDruidQuery, RelType, PartialDruidQuery> applyFn
     )
     {
       super(
@@ -126,24 +136,35 @@ public class DruidRules
           StringUtils.format("%s(%s)", DruidQueryRule.class.getSimpleName(), stage)
       );
       this.stage = stage;
-      this.f = f;
+      this.matchesFn = matchesFn;
+      this.applyFn = applyFn;
+    }
+
+    public DruidQueryRule(
+        final Class<RelType> relClass,
+        final PartialDruidQuery.Stage stage,
+        final BiFunction<PartialDruidQuery, RelType, PartialDruidQuery> applyFn
+    )
+    {
+      this(relClass, stage, r -> true, applyFn);
     }
 
     @Override
     public boolean matches(final RelOptRuleCall call)
     {
-      final DruidRel druidRel = call.rel(1);
-      return druidRel.getPartialDruidQuery().canAccept(stage);
+      final RelType otherRel = call.rel(0);
+      final DruidRel<?> druidRel = call.rel(1);
+      return druidRel.getPartialDruidQuery().canAccept(stage) && matchesFn.test(otherRel);
     }
 
     @Override
     public void onMatch(final RelOptRuleCall call)
     {
       final RelType otherRel = call.rel(0);
-      final DruidRel druidRel = call.rel(1);
+      final DruidRel<?> druidRel = call.rel(1);
 
-      final PartialDruidQuery newPartialDruidQuery = f.apply(druidRel.getPartialDruidQuery(), otherRel);
-      final DruidRel newDruidRel = druidRel.withPartialQuery(newPartialDruidQuery);
+      final PartialDruidQuery newPartialDruidQuery = applyFn.apply(druidRel.getPartialDruidQuery(), otherRel);
+      final DruidRel<?> newDruidRel = druidRel.withPartialQuery(newPartialDruidQuery);
 
       if (newDruidRel.isValidDruidQuery()) {
         call.transformTo(newDruidRel);
@@ -154,6 +175,7 @@ public class DruidRules
   public abstract static class DruidOuterQueryRule extends RelOptRule
   {
     public static final RelOptRule AGGREGATE = new DruidOuterQueryRule(
+        PartialDruidQuery.Stage.AGGREGATE,
         operand(Aggregate.class, operandJ(DruidRel.class, null, CAN_BUILD_ON, any())),
         "AGGREGATE"
     )
@@ -166,7 +188,7 @@ public class DruidRules
 
         final DruidOuterQueryRel outerQueryRel = DruidOuterQueryRel.create(
             druidRel,
-            PartialDruidQuery.create(druidRel.getPartialDruidQuery().leafRel())
+            PartialDruidQuery.createOuterQuery(druidRel.getPartialDruidQuery())
                              .withAggregate(aggregate)
         );
         if (outerQueryRel.isValidDruidQuery()) {
@@ -176,6 +198,7 @@ public class DruidRules
     };
 
     public static final RelOptRule WHERE_FILTER = new DruidOuterQueryRule(
+        PartialDruidQuery.Stage.WHERE_FILTER,
         operand(Filter.class, operandJ(DruidRel.class, null, CAN_BUILD_ON, any())),
         "WHERE_FILTER"
     )
@@ -188,7 +211,7 @@ public class DruidRules
 
         final DruidOuterQueryRel outerQueryRel = DruidOuterQueryRel.create(
             druidRel,
-            PartialDruidQuery.create(druidRel.getPartialDruidQuery().leafRel())
+            PartialDruidQuery.createOuterQuery(druidRel.getPartialDruidQuery())
                              .withWhereFilter(filter)
         );
         if (outerQueryRel.isValidDruidQuery()) {
@@ -198,6 +221,7 @@ public class DruidRules
     };
 
     public static final RelOptRule SELECT_PROJECT = new DruidOuterQueryRule(
+        PartialDruidQuery.Stage.SELECT_PROJECT,
         operand(Project.class, operandJ(DruidRel.class, null, CAN_BUILD_ON, any())),
         "SELECT_PROJECT"
     )
@@ -210,7 +234,7 @@ public class DruidRules
 
         final DruidOuterQueryRel outerQueryRel = DruidOuterQueryRel.create(
             druidRel,
-            PartialDruidQuery.create(druidRel.getPartialDruidQuery().leafRel())
+            PartialDruidQuery.createOuterQuery(druidRel.getPartialDruidQuery())
                              .withSelectProject(filter)
         );
         if (outerQueryRel.isValidDruidQuery()) {
@@ -220,6 +244,7 @@ public class DruidRules
     };
 
     public static final RelOptRule SORT = new DruidOuterQueryRule(
+        PartialDruidQuery.Stage.SORT,
         operand(Sort.class, operandJ(DruidRel.class, null, CAN_BUILD_ON, any())),
         "SORT"
     )
@@ -232,7 +257,7 @@ public class DruidRules
 
         final DruidOuterQueryRel outerQueryRel = DruidOuterQueryRel.create(
             druidRel,
-            PartialDruidQuery.create(druidRel.getPartialDruidQuery().leafRel())
+            PartialDruidQuery.createOuterQuery(druidRel.getPartialDruidQuery())
                              .withSort(sort)
         );
         if (outerQueryRel.isValidDruidQuery()) {
@@ -242,6 +267,7 @@ public class DruidRules
     };
 
     public static final RelOptRule WINDOW = new DruidOuterQueryRule(
+        PartialDruidQuery.Stage.WINDOW,
         operand(Window.class, operandJ(DruidRel.class, null, CAN_BUILD_ON, any())),
         "WINDOW"
     )
@@ -254,7 +280,7 @@ public class DruidRules
 
         final DruidOuterQueryRel outerQueryRel = DruidOuterQueryRel.create(
             druidRel,
-            PartialDruidQuery.create(druidRel.getPartialDruidQuery().leafRel())
+            PartialDruidQuery.createOuterQuery(druidRel.getPartialDruidQuery())
                              .withWindow(window)
         );
         if (outerQueryRel.isValidDruidQuery()) {
@@ -263,17 +289,24 @@ public class DruidRules
       }
     };
 
-    public DruidOuterQueryRule(final RelOptRuleOperand op, final String description)
+    private final PartialDruidQuery.Stage stage;
+
+    public DruidOuterQueryRule(
+        final PartialDruidQuery.Stage stage,
+        final RelOptRuleOperand op,
+        final String description
+    )
     {
       super(op, StringUtils.format("%s(%s)", DruidOuterQueryRel.class.getSimpleName(), description));
+      this.stage = stage;
     }
 
     @Override
     public boolean matches(final RelOptRuleCall call)
     {
-      // Subquery must be a groupBy, so stage must be >= AGGREGATE.
-      final DruidRel druidRel = call.rel(call.getRelList().size() - 1);
-      return druidRel.getPartialDruidQuery().stage().compareTo(PartialDruidQuery.Stage.AGGREGATE) >= 0;
+      // Only consider doing a subquery when the stage cannot be fused into a single query.
+      final DruidRel<?> druidRel = call.rel(call.getRelList().size() - 1);
+      return !stage.canFollow(druidRel.getPartialDruidQuery().stage());
     }
   }
 }
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
index d41d56f54c..620a8cbbb7 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
@@ -68,6 +68,7 @@ import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.ResultRow;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.scan.ScanQuery;
@@ -2715,8 +2716,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
                     )
                 )
                 .intervals(querySegmentSpec(Filtration.eternity()))
-                .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING))
-                .columns("__time", "_v0")
+                .columns("__time", "v0")
                 .filters(new SelectorDimFilter("v0", "10.1", null))
                 .context(queryContext)
                 .build()
@@ -2827,9 +2827,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
                     )
                 )
                 .intervals(querySegmentSpec(Filtration.eternity()))
-                .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING))
-                .columns("__time", "_v0")
-                .filters(new SelectorDimFilter("v0", "10.1", null))
+                .columns("__time", "v0")
                 .context(queryContext)
                 .build()
         ),
@@ -3022,8 +3020,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
             )
         )
         .intervals(querySegmentSpec(Filtration.eternity()))
-        .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ColumnType.STRING))
-        .columns("__time", "_v0")
+        .columns("__time", "v0")
         .context(queryContext);
 
     testQuery(
@@ -3866,18 +3863,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
                         )
                     )
                 )
-                .setLimitSpec(
-                    new DefaultLimitSpec(
-                        ImmutableList.of(
-                            new OrderByColumnSpec(
-                                "d0",
-                                OrderByColumnSpec.Direction.ASCENDING,
-                                StringComparators.NUMERIC
-                            )
-                        ),
-                        Integer.MAX_VALUE
-                    )
-                )
+                .setLimitSpec(NoopLimitSpec.instance())
                 .setContext(QUERY_CONTEXT_DEFAULT)
                 .build()
         ),
@@ -4086,18 +4072,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
                 .setGranularity(Granularities.ALL)
                 .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
                 .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                .setLimitSpec(
-                    new DefaultLimitSpec(
-                        ImmutableList.of(
-                            new OrderByColumnSpec(
-                                "d0",
-                                OrderByColumnSpec.Direction.ASCENDING,
-                                StringComparators.LEXICOGRAPHIC
-                            )
-                        ),
-                        Integer.MAX_VALUE
-                    )
-                )
+                .setLimitSpec(NoopLimitSpec.instance())
                 .setContext(queryContext)
                 .build()
         ),
@@ -4208,18 +4183,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
                 .setGranularity(Granularities.ALL)
                 .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0")))
                 .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                .setLimitSpec(
-                    new DefaultLimitSpec(
-                        ImmutableList.of(
-                            new OrderByColumnSpec(
-                                "d0",
-                                OrderByColumnSpec.Direction.ASCENDING,
-                                StringComparators.LEXICOGRAPHIC
-                            )
-                        ),
-                        Integer.MAX_VALUE
-                    )
-                )
+                .setLimitSpec(NoopLimitSpec.instance())
                 .setContext(queryContext)
                 .build()
         ),
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index a65ec177a5..88df793934 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -93,6 +93,7 @@ import org.apache.druid.query.filter.SelectorDimFilter;
 import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.GroupByQueryConfig;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
+import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction;
 import org.apache.druid.query.lookup.RegisteredLookupExtractionFn;
@@ -2032,18 +2033,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         .setGranularity(Granularities.ALL)
                         .setDimensions(dimensions(new DefaultDimensionSpec("cnt", "d0", ColumnType.LONG)))
                         .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0")))
-                        .setLimitSpec(
-                            DefaultLimitSpec
-                                .builder()
-                                .orderBy(
-                                    new OrderByColumnSpec(
-                                        "d0",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.NUMERIC
-                                    )
-                                )
-                                .build()
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(QUERY_CONTEXT_DEFAULT)
                         .build()
         ),
@@ -5620,7 +5610,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     final Map<String, String> queries = ImmutableMap.of(
         // SELECT query with order by non-__time.
         "SELECT dim1 FROM druid.foo ORDER BY dim1",
-        "Possible error: SQL query requires order by non-time column [dim1 ASC] that is not supported.",
+        "Possible error: SQL query requires order by non-time column [dim1 ASC], which is not supported.",
 
         // JOIN condition with not-equals (<>).
         "SELECT foo.dim1, foo.dim2, l.k, l.v\n"
@@ -6947,10 +6937,15 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                                         .setDataSource(CalciteTests.DATASOURCE1)
                                         .setInterval(querySegmentSpec(Filtration.eternity()))
                                         .setGranularity(Granularities.ALL)
-                                        .setDimensions(dimensions(
-                                            new DefaultDimensionSpec("dim1", "d0"),
-                                            new DefaultDimensionSpec("m2", "d1", ColumnType.DOUBLE)
-                                        ))
+                                        .setDimensions(
+                                            useDefault ? dimensions(
+                                                new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE),
+                                                new DefaultDimensionSpec("dim1", "d1")
+                                            ) : dimensions(
+                                                new DefaultDimensionSpec("dim1", "d0"),
+                                                new DefaultDimensionSpec("m2", "d1", ColumnType.DOUBLE)
+                                            )
+                                        )
                                         .setDimFilter(new SelectorDimFilter("m1", "5.0", null))
                                         .setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time")))
                                         .setContext(QUERY_CONTEXT_DEFAULT)
@@ -6967,7 +6962,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setDimensions(dimensions(
                             new DefaultDimensionSpec("v0", "_d0", ColumnType.LONG),
-                            new DefaultDimensionSpec("d0", "_d1", ColumnType.STRING)
+                            new DefaultDimensionSpec(useDefault ? "d1" : "d0", "_d1", ColumnType.STRING)
                         ))
                         .setAggregatorSpecs(
                             aggregators(
@@ -7309,8 +7304,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setGranularity(Granularities.ALL)
                         .setAggregatorSpecs(
-                            useDefault
-                            ? aggregators(
+                            aggregators(
                                 new LongMaxAggregatorFactory("_a0", "a0"),
                                 new LongMinAggregatorFactory("_a1", "a0"),
                                 new LongSumAggregatorFactory("_a2:sum", "a0"),
@@ -7318,17 +7312,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                                 new LongMaxAggregatorFactory("_a3", "d0"),
                                 new CountAggregatorFactory("_a4")
                             )
-                            : aggregators(
-                                new LongMaxAggregatorFactory("_a0", "a0"),
-                                new LongMinAggregatorFactory("_a1", "a0"),
-                                new LongSumAggregatorFactory("_a2:sum", "a0"),
-                                new FilteredAggregatorFactory(
-                                    new CountAggregatorFactory("_a2:count"),
-                                    not(selector("a0", null, null))
-                                ),
-                                new LongMaxAggregatorFactory("_a3", "d0"),
-                                new CountAggregatorFactory("_a4")
-                            )
                         )
                         .setPostAggregatorSpecs(
                             ImmutableList.of(
@@ -9285,18 +9268,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
                         .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                        .setLimitSpec(
-                            new DefaultLimitSpec(
-                                ImmutableList.of(
-                                    new OrderByColumnSpec(
-                                        "d0",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.NUMERIC
-                                    )
-                                ),
-                                Integer.MAX_VALUE
-                            )
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(QUERY_CONTEXT_DEFAULT)
                         .build()
         ),
@@ -9334,18 +9306,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
                         .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                        .setLimitSpec(
-                            new DefaultLimitSpec(
-                                ImmutableList.of(
-                                    new OrderByColumnSpec(
-                                        "d0",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.NUMERIC
-                                    )
-                                ),
-                                Integer.MAX_VALUE
-                            )
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(QUERY_CONTEXT_DEFAULT)
                         .build()
         ),
@@ -10176,18 +10137,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG)))
                         .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                        .setLimitSpec(
-                            new DefaultLimitSpec(
-                                ImmutableList.of(
-                                    new OrderByColumnSpec(
-                                        "d0",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.NUMERIC
-                                    )
-                                ),
-                                Integer.MAX_VALUE
-                            )
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(QUERY_CONTEXT_DEFAULT)
                         .build()
         ),
@@ -10226,18 +10176,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         )
                         .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)))
                         .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                        .setLimitSpec(
-                            new DefaultLimitSpec(
-                                ImmutableList.of(
-                                    new OrderByColumnSpec(
-                                        "d0",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.LEXICOGRAPHIC
-                                    )
-                                ),
-                                Integer.MAX_VALUE
-                            )
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(QUERY_CONTEXT_DEFAULT)
                         .build()
         ),
@@ -10477,19 +10416,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                             )
                         )
                         .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt")))
-                        .setLimitSpec(
-                            new DefaultLimitSpec(
-                                ImmutableList.of(
-                                    new OrderByColumnSpec("d0", OrderByColumnSpec.Direction.ASCENDING),
-                                    new OrderByColumnSpec(
-                                        "d1",
-                                        OrderByColumnSpec.Direction.ASCENDING,
-                                        StringComparators.NUMERIC
-                                    )
-                                ),
-                                Integer.MAX_VALUE
-                            )
-                        )
+                        .setLimitSpec(NoopLimitSpec.instance())
                         .setContext(withTimestampResultContext(QUERY_CONTEXT_DEFAULT, "d1", 1, Granularities.MONTH))
                         .build()
         ),
@@ -11771,15 +11698,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                                         .setInterval(querySegmentSpec(Filtration.eternity()))
                                         .setGranularity(Granularities.ALL)
                                         .setDimensions(
-                                            useDefault
-                                            ? dimensions(
+                                            dimensions(
                                                 new DefaultDimensionSpec("__time", "d0", ColumnType.LONG),
                                                 new DefaultDimensionSpec("m2", "d1", ColumnType.DOUBLE),
                                                 new DefaultDimensionSpec("dim1", "d2")
-                                            ) : dimensions(
-                                                new DefaultDimensionSpec("__time", "d0", ColumnType.LONG),
-                                                new DefaultDimensionSpec("dim1", "d1"),
-                                                new DefaultDimensionSpec("m2", "d2", ColumnType.DOUBLE)
                                             )
                                         )
                                         .setContext(QUERY_CONTEXT_DEFAULT)
@@ -11788,13 +11710,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                         .setInterval(querySegmentSpec(Filtration.eternity()))
                         .setGranularity(Granularities.ALL)
                         .setDimensions(
-                            useDefault
-                            ? dimensions(
+                            dimensions(
                                 new DefaultDimensionSpec("d0", "_d0", ColumnType.LONG),
                                 new DefaultDimensionSpec("d2", "_d1", ColumnType.STRING)
-                            ) : dimensions(
-                                new DefaultDimensionSpec("d0", "_d0", ColumnType.LONG),
-                                new DefaultDimensionSpec("d1", "_d1", ColumnType.STRING)
                             )
                         )
                         .setAggregatorSpecs(
@@ -11803,7 +11721,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
                                 ? new CountAggregatorFactory("a0")
                                 : new FilteredAggregatorFactory(
                                     new CountAggregatorFactory("a0"),
-                                    not(selector("d2", null, null))
+                                    not(selector("d1", null, null))
                                 )
                             )
                         )
diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
index a9cf6779e1..ce84cb3f4b 100644
--- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java
@@ -1373,7 +1373,7 @@ public class SqlResourceTest extends CalciteTestBase
     Assert.assertTrue(
         exception.getMessage()
                  .contains("Query not supported. " +
-                           "Possible error: SQL query requires order by non-time column [dim1 ASC] that is not supported.")
+                           "Possible error: SQL query requires order by non-time column [dim1 ASC], which is not supported.")
     );
     checkSqlRequestLog(false);
     Assert.assertTrue(lifecycleManager.getAll("id").isEmpty());
diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest
index 3ec4a47dbf..b2b5aa49c6 100644
--- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest
+++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest
@@ -43,13 +43,13 @@ expectedResults:
   - ["CL",1442019600000,-370,-154,1]
   - ["HK",1442019600000,-113,2090,2]
   - ["MY",1442019600000,-7,1018,3]
-  - ["SV",1442019600000,-1,114,4]
-  - ["MA",1442019600000,-1,-34,5]
-  - ["TW",1442019600000,0,869,6]
-  - ["BD",1442019600000,0,930,7]
-  - ["DE",1442019600000,0,604,8]
-  - ["PY",1442019600000,1,634,9]
-  - ["AR",1442019600000,1,3159,10]
+  - ["MA",1442019600000,-1,-34,4]
+  - ["SV",1442019600000,-1,114,5]
+  - ["BD",1442019600000,0,930,6]
+  - ["DE",1442019600000,0,604,7]
+  - ["TW",1442019600000,0,869,8]
+  - ["AR",1442019600000,1,3159,9]
+  - ["PY",1442019600000,1,634,10]
   - ["PH",1442019600000,6,1375,11]
   - ["RS",1442019600000,6,19,12]
   - ["CO",1442019600000,12,478,13]
@@ -80,8 +80,8 @@ expectedResults:
   - ["GT",1442023200000,-167,7,1]
   - ["IN",1442023200000,-142,2319,2]
   - ["CN",1442023200000,-13,239,3]
-  - ["VN",1442023200000,-9,25,4]
-  - ["EC",1442023200000,-9,-346,5]
+  - ["EC",1442023200000,-9,-346,4]
+  - ["VN",1442023200000,-9,25,5]
   - ["ES",1442023200000,-5,49,6]
   - ["SE",1442023200000,3,233,7]
   - ["DO",1442023200000,8,243,8]
@@ -117,8 +117,8 @@ expectedResults:
   - ["VE",1442026800000,-17,147,5]
   - ["PE",1442026800000,-12,362,6]
   - ["HN",1442026800000,-1,-1,7]
-  - ["PA",1442026800000,0,0,8]
-  - ["IR",1442026800000,0,367,9]
+  - ["IR",1442026800000,0,367,8]
+  - ["PA",1442026800000,0,0,9]
   - ["RU",1442026800000,0,3247,10]
   - ["IE",1442026800000,1,29,11]
   - ["EG",1442026800000,16,142,12]
@@ -152,10 +152,10 @@ expectedResults:
   - ["IT",1442030400000,-17,1496,4]
   - ["AR",1442030400000,-3,3449,5]
   - ["MY",1442030400000,-3,1953,6]
-  - ["TW",1442030400000,0,1186,7]
-  - ["EC",1442030400000,0,222,8]
-  - ["SG",1442030400000,1,2821,9]
-  - ["IE",1442030400000,1,1091,10]
+  - ["EC",1442030400000,0,222,7]
+  - ["TW",1442030400000,0,1186,8]
+  - ["IE",1442030400000,1,1091,9]
+  - ["SG",1442030400000,1,2821,10]
   - ["PR",1442030400000,2,-6,11]
   - ["PH",1442030400000,26,1477,12]
   - ["BR",1442030400000,30,1620,13]
@@ -185,8 +185,8 @@ expectedResults:
   - ["ZA",1442034000000,-3,126,5]
   - ["VE",1442034000000,-2,585,6]
   - ["CL",1442034000000,-1,-315,7]
-  - ["NL",1442034000000,0,1319,8]
-  - ["GR",1442034000000,0,63,9]
+  - ["GR",1442034000000,0,63,8]
+  - ["NL",1442034000000,0,1319,9]
   - ["TH",1442034000000,0,113,10]
   - ["ID",1442034000000,19,4,11]
   - ["BR",1442034000000,21,1251,12]
@@ -217,13 +217,13 @@ expectedResults:
   - ["SA",1442037600000,-97,-72,3]
   - ["VN",1442037600000,-11,-1,4]
   - ["IT",1442037600000,-9,2705,5]
-  - ["UA",1442037600000,-1,3821,6]
-  - ["SK",1442037600000,-1,18,7]
+  - ["SK",1442037600000,-1,18,6]
+  - ["UA",1442037600000,-1,3821,7]
   - ["UY",1442037600000,1,913,8]
   - ["CL",1442037600000,2,445,9]
-  - ["SG",1442037600000,3,2898,10]
-  - ["AU",1442037600000,3,664,11]
-  - ["ES",1442037600000,3,-2,12]
+  - ["AU",1442037600000,3,664,10]
+  - ["ES",1442037600000,3,-2,11]
+  - ["SG",1442037600000,3,2898,12]
   - ["MX",1442037600000,4,4668,13]
   - ["DK",1442037600000,10,41,14]
   - ["FI",1442037600000,14,1703,15]
@@ -241,8 +241,8 @@ expectedResults:
   - ["HU",1442037600000,197,881,27]
   - ["TW",1442037600000,266,1479,28]
   - ["BR",1442037600000,267,443,29]
-  - ["GB",1442037600000,544,3469,30]
-  - ["DE",1442037600000,544,2515,31]
+  - ["DE",1442037600000,544,2515,30]
+  - ["GB",1442037600000,544,3469,31]
   - ["HK",1442037600000,636,1725,32]
   - ["JP",1442037600000,2181,7873,33]
   - ["US",1442037600000,3675,12996,34]
@@ -254,12 +254,12 @@ expectedResults:
   - ["HK",1442041200000,-15,1937,4]
   - ["CL",1442041200000,-1,418,5]
   - ["IQ",1442041200000,-1,5,6]
-  - ["VN",1442041200000,0,98,7]
-  - ["PH",1442041200000,0,219,8]
+  - ["PH",1442041200000,0,219,7]
+  - ["VN",1442041200000,0,98,8]
   - ["TR",1442041200000,1,436,9]
   - ["ID",1442041200000,2,-58,10]
-  - ["TH",1442041200000,3,91,11]
-  - ["BR",1442041200000,3,624,12]
+  - ["BR",1442041200000,3,624,11]
+  - ["TH",1442041200000,3,91,12]
   - ["CA",1442041200000,5,43,13]
   - ["GR",1442041200000,7,71,14]
   - ["BG",1442041200000,9,19136,15]
@@ -274,8 +274,8 @@ expectedResults:
   - ["SG",1442041200000,59,2950,24]
   - ["UA",1442041200000,74,3823,25]
   - ["IN",1442041200000,80,5952,26]
-  - ["SE",1442041200000,91,89,27]
-  - ["HU",1442041200000,91,1380,28]
+  - ["HU",1442041200000,91,1380,27]
+  - ["SE",1442041200000,91,89,28]
   - ["ES",1442041200000,118,53,29]
   - ["ZM",1442041200000,133,133,30]
   - ["AU",1442041200000,194,611,31]
@@ -383,13 +383,13 @@ expectedResults:
   - ["TH",1442052000000,-22,45,3]
   - ["AO",1442052000000,-18,740,4]
   - ["ES",1442052000000,-4,1684,5]
-  - ["KR",1442052000000,-3,4320,6]
-  - ["AE",1442052000000,-3,182,7]
+  - ["AE",1442052000000,-3,182,6]
+  - ["KR",1442052000000,-3,4320,7]
   - ["US",1442052000000,-2,1837,8]
   - ["BE",1442052000000,-1,279,9]
-  - ["OM",1442052000000,0,0,10]
+  - ["CN",1442052000000,0,622,10]
   - ["IQ",1442052000000,0,3,11]
-  - ["CN",1442052000000,0,622,12]
+  - ["OM",1442052000000,0,0,12]
   - ["SE",1442052000000,1,1448,13]
   - ["SK",1442052000000,13,464,14]
   - ["HK",1442052000000,15,828,15]
@@ -431,11 +431,11 @@ expectedResults:
   - ["LB",1442055600000,-67,-67,3]
   - ["AR",1442055600000,-54,475,4]
   - ["SE",1442055600000,-5,1432,5]
-  - ["JO",1442055600000,-2,2,6]
-  - ["KW",1442055600000,-2,1811,7]
-  - ["HU",1442055600000,-2,1427,8]
-  - ["TH",1442055600000,0,11,9]
-  - ["CH",1442055600000,0,50,10]
+  - ["HU",1442055600000,-2,1427,6]
+  - ["JO",1442055600000,-2,2,7]
+  - ["KW",1442055600000,-2,1811,8]
+  - ["CH",1442055600000,0,50,9]
+  - ["TH",1442055600000,0,11,10]
   - ["BY",1442055600000,1,2061,11]
   - ["MY",1442055600000,1,1459,12]
   - ["IL",1442055600000,4,4377,13]
@@ -478,13 +478,13 @@ expectedResults:
   - ["CL",1442059200000,-12,355,5]
   - ["AE",1442059200000,-11,6387,6]
   - ["UA",1442059200000,-2,15681,7]
-  - ["SA",1442059200000,0,1206,8]
+  - ["DK",1442059200000,0,490,8]
   - ["JO",1442059200000,0,2,9]
   - ["MY",1442059200000,0,525,10]
-  - ["DK",1442059200000,0,490,11]
+  - ["SA",1442059200000,0,1206,11]
   - ["HK",1442059200000,2,862,12]
-  - ["VN",1442059200000,8,1077,13]
-  - ["CN",1442059200000,8,345,14]
+  - ["CN",1442059200000,8,345,13]
+  - ["VN",1442059200000,8,1077,14]
   - ["US",1442059200000,11,518,15]
   - ["ID",1442059200000,17,-290,16]
   - ["CZ",1442059200000,21,1358,17]
@@ -523,9 +523,9 @@ expectedResults:
   - ["TH",1442062800000,-46,8,6]
   - ["NL",1442062800000,-30,311,7]
   - ["PE",1442062800000,-12,360,8]
-  - ["MA",1442062800000,0,221,9]
-  - ["CZ",1442062800000,0,3331,10]
-  - ["KZ",1442062800000,0,191,11]
+  - ["CZ",1442062800000,0,3331,9]
+  - ["KZ",1442062800000,0,191,10]
+  - ["MA",1442062800000,0,221,11]
   - ["DK",1442062800000,1,445,12]
   - ["HK",1442062800000,1,1175,13]
   - ["SK",1442062800000,6,372,14]
@@ -565,11 +565,11 @@ expectedResults:
   - ["CN",1442066400000,-15,51,4]
   - ["MX",1442066400000,-1,631,5]
   - ["AR",1442066400000,0,-29,6]
-  - ["SG",1442066400000,0,517,7]
+  - ["AT",1442066400000,0,4908,7]
   - ["KZ",1442066400000,0,-243,8]
-  - ["AT",1442066400000,0,4908,9]
+  - ["LU",1442066400000,0,606,9]
   - ["RS",1442066400000,0,832,10]
-  - ["LU",1442066400000,0,606,11]
+  - ["SG",1442066400000,0,517,11]
   - ["MY",1442066400000,1,642,12]
   - ["IL",1442066400000,3,1472,13]
   - ["PT",1442066400000,12,3692,14]
@@ -611,8 +611,8 @@ expectedResults:
   - ["RO",1442070000000,-29,872,4]
   - ["AU",1442070000000,-12,1326,5]
   - ["MX",1442070000000,-1,3561,6]
-  - ["ZA",1442070000000,0,127,7]
-  - ["TH",1442070000000,0,-67,8]
+  - ["TH",1442070000000,0,-67,7]
+  - ["ZA",1442070000000,0,127,8]
   - ["UG",1442070000000,1,1,9]
   - ["GR",1442070000000,2,-235,10]
   - ["MM",1442070000000,3,28,11]
@@ -662,10 +662,10 @@ expectedResults:
   - ["MX",1442073600000,-21,3181,5]
   - ["MV",1442073600000,-3,-3,6]
   - ["FI",1442073600000,-1,912,7]
-  - ["ME",1442073600000,0,0,8]
-  - ["HR",1442073600000,0,310,9]
-  - ["MY",1442073600000,1,732,10]
-  - ["EG",1442073600000,1,170,11]
+  - ["HR",1442073600000,0,310,8]
+  - ["ME",1442073600000,0,0,9]
+  - ["EG",1442073600000,1,170,10]
+  - ["MY",1442073600000,1,732,11]
   - ["SA",1442073600000,2,1697,12]
   - ["KG",1442073600000,6,6,13]
   - ["RO",1442073600000,15,1377,14]
@@ -754,23 +754,23 @@ expectedResults:
   - ["BR",1442080800000,-267,5316,2]
   - ["PT",1442080800000,-79,3750,3]
   - ["SI",1442080800000,-45,-36,4]
-  - ["KW",1442080800000,-33,1778,5]
-  - ["KR",1442080800000,-33,4286,6]
+  - ["KR",1442080800000,-33,4286,5]
+  - ["KW",1442080800000,-33,1778,6]
   - ["CZ",1442080800000,-28,2308,7]
   - ["GE",1442080800000,-27,-140,8]
   - ["CN",1442080800000,-10,51,9]
   - ["PE",1442080800000,-2,-276,10]
-  - ["UA",1442080800000,-1,5776,11]
-  - ["TR",1442080800000,-1,2968,12]
-  - ["IN",1442080800000,0,19268,13]
-  - ["RS",1442080800000,0,900,14]
-  - ["HK",1442080800000,0,5894,15]
+  - ["TR",1442080800000,-1,2968,11]
+  - ["UA",1442080800000,-1,5776,12]
+  - ["HK",1442080800000,0,5894,13]
+  - ["IN",1442080800000,0,19268,14]
+  - ["RS",1442080800000,0,900,15]
   - ["AR",1442080800000,1,122,16]
   - ["BE",1442080800000,1,497,17]
   - ["BO",1442080800000,4,4,18]
   - ["JO",1442080800000,4,2,19]
-  - ["PY",1442080800000,5,634,20]
-  - ["MA",1442080800000,5,207,21]
+  - ["MA",1442080800000,5,207,20]
+  - ["PY",1442080800000,5,634,21]
   - ["PL",1442080800000,7,866,22]
   - ["MR",1442080800000,10,10,23]
   - ["LT",1442080800000,12,-12,24]
@@ -805,8 +805,8 @@ expectedResults:
   - ["PE",1442084400000,-68,1597,4]
   - ["NZ",1442084400000,-52,1032,5]
   - ["KZ",1442084400000,-22,-248,6]
-  - ["VN",1442084400000,-10,1426,7]
-  - ["HR",1442084400000,-10,350,8]
+  - ["HR",1442084400000,-10,350,7]
+  - ["VN",1442084400000,-10,1426,8]
   - ["AR",1442084400000,-5,752,9]
   - ["LK",1442084400000,-3,131,10]
   - ["AT",1442084400000,-2,7254,11]
@@ -817,9 +817,9 @@ expectedResults:
   - ["UA",1442084400000,5,3655,16]
   - ["DO",1442084400000,8,264,17]
   - ["CH",1442084400000,13,457,18]
-  - ["TH",1442084400000,13,-67,19]
+  - ["JP",1442084400000,13,1931,19]
   - ["PL",1442084400000,13,1622,20]
-  - ["JP",1442084400000,13,1931,21]
+  - ["TH",1442084400000,13,-67,21]
   - ["SE",1442084400000,37,278,22]
   - ["PH",1442084400000,39,670,23]
   - ["CA",1442084400000,44,3222,24]
@@ -982,8 +982,8 @@ expectedResults:
   - ["SE",1442098800000,0,97,9]
   - ["GT",1442098800000,1,7,10]
   - ["CZ",1442098800000,2,2140,11]
-  - ["PT",1442098800000,2,345,12]
-  - ["NO",1442098800000,2,31,13]
+  - ["NO",1442098800000,2,31,12]
+  - ["PT",1442098800000,2,345,13]
   - ["IL",1442098800000,3,1847,14]
   - ["NL",1442098800000,4,105,15]
   - ["SK",1442098800000,7,367,16]
@@ -1010,4 +1010,4 @@ expectedResults:
   - ["PE",1442098800000,1861,1772,37]
   - ["US",1442098800000,3575,8184,38]
   - ["RU",1442098800000,12098,18578,39]
-  - ["",1442098800000,276159,1468959,40]
\ No newline at end of file
+  - ["",1442098800000,276159,1468959,40]
diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest
new file mode 100644
index 0000000000..c370298506
--- /dev/null
+++ b/sql/src/test/resources/calcite/tests/window/wikipediaScanWindow.sqlTest
@@ -0,0 +1,23 @@
+type: "failingTest"
+
+sql: |
+  SELECT
+    __time,
+    "user",
+    page,
+    LAG(page, 1) OVER (PARTITION BY "user" ORDER BY __time) as priorPage
+  FROM wikipedia
+
+expectedOperators:
+  - { type: "naiveSort", columns: [ { column: "user", direction: "ASC" }, { column: "__time", direction: "ASC" } ]}
+  - { type: "naivePartition",  partitionColumns: [ "user" ] }
+  - type: "window"
+    processor:
+      type: "offset"
+      inputColumn: page
+      outputColumn: w0
+      offset: -1
+
+# Not correct: there should actually be results for this query. But when the query runs, at the time I write this, it
+# does not return any results at all. Therefore, we leave expectedResults empty and mark the test as a failingTest.
+expectedResults: [ [ "to be added later, when fixing this test" ] ]
diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest
new file mode 100644
index 0000000000..d74e08d892
--- /dev/null
+++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest
@@ -0,0 +1,1024 @@
+type: "operatorValidation"
+
+# Like wikipediaSimplePartition, but requires re-sorting the input data because the order of the GROUP BY
+# does not match the required order for window partitioning. ("t" and "countryIsoCode" are flipped.)
+
+sql: |
+  SELECT
+    FLOOR(__time TO HOUR) t,
+    countryIsoCode,
+    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: "naiveSort", columns: [ { column: "d1", direction: "ASC" } ] }
+  - { type: "naivePartition",  partitionColumns: [ "d1" ] }
+  - type: "window"
+    processor:
+      type: "composing"
+      processors:
+        - { "type": "offset", "inputColumn": "d0", "outputColumn": "w1", "offset": -2 }
+        - { "type": "offset", "inputColumn": "d0", "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 ]
+  - [ 1442030400000, "AE", 118, 6547, null, 1442044800000, 118, 42, 1 ]
+  - [ 1442044800000, "AE", -7, 6547, null, 1442048400000, 118, 42, 1 ]
+  - [ 1442048400000, "AE", 39, 6547, 1442030400000, 1442052000000, 118, 42, 1 ]
+  - [ 1442052000000, "AE", -3, 6547, 1442044800000, 1442059200000, 118, 42, 2 ]
+  - [ 1442059200000, "AE", -11, 6547, 1442048400000, 1442070000000, 118, 42, 2 ]
+  - [ 1442070000000, "AE", 46, 6547, 1442052000000, 1442077200000, 118, 42, 2 ]
+  - [ 1442077200000, "AE", 6323, 6547, 1442059200000, 1442080800000, 118, 42, 3 ]
+  - [ 1442080800000, "AE", 42, 6547, 1442070000000, null, 118, 42, 3 ]
+  - [ 1442077200000, "AL", 26, 80, null, 1442091600000, 26, 54, 1 ]
+  - [ 1442091600000, "AL", 54, 80, null, null, 26, 54, 2 ]
+  - [ 1442041200000, "AO", -26, 740, null, 1442052000000, -26, 722, 1 ]
+  - [ 1442052000000, "AO", -18, 740, null, 1442088000000, -26, 722, 1 ]
+  - [ 1442088000000, "AO", 62, 740, 1442041200000, 1442098800000, -26, 722, 2 ]
+  - [ 1442098800000, "AO", 722, 740, 1442052000000, null, -26, 722, 3 ]
+  - [ 1442019600000, "AR", 1, 4450, null, 1442023200000, 1, 64, 1 ]
+  - [ 1442023200000, "AR", 2514, 4450, null, 1442026800000, 1, 64, 1 ]
+  - [ 1442026800000, "AR", 644, 4450, 1442019600000, 1442030400000, 1, 64, 1 ]
+  - [ 1442030400000, "AR", -3, 4450, 1442023200000, 1442034000000, 1, 64, 1 ]
+  - [ 1442034000000, "AR", 212, 4450, 1442026800000, 1442037600000, 1, 64, 1 ]
+  - [ 1442037600000, "AR", 81, 4450, 1442030400000, 1442055600000, 1, 64, 1 ]
+  - [ 1442055600000, "AR", -54, 4450, 1442034000000, 1442059200000, 1, 64, 2 ]
+  - [ 1442059200000, "AR", 210, 4450, 1442037600000, 1442062800000, 1, 64, 2 ]
+  - [ 1442062800000, "AR", 29, 4450, 1442055600000, 1442066400000, 1, 64, 2 ]
+  - [ 1442066400000, "AR", 0, 4450, 1442059200000, 1442070000000, 1, 64, 2 ]
+  - [ 1442070000000, "AR", 377, 4450, 1442062800000, 1442077200000, 1, 64, 2 ]
+  - [ 1442077200000, "AR", -591, 4450, 1442066400000, 1442080800000, 1, 64, 2 ]
+  - [ 1442080800000, "AR", 1, 4450, 1442070000000, 1442084400000, 1, 64, 3 ]
+  - [ 1442084400000, "AR", -5, 4450, 1442077200000, 1442091600000, 1, 64, 3 ]
+  - [ 1442091600000, "AR", 340, 4450, 1442080800000, 1442095200000, 1, 64, 3 ]
+  - [ 1442095200000, "AR", 630, 4450, 1442084400000, 1442098800000, 1, 64, 3 ]
+  - [ 1442098800000, "AR", 64, 4450, 1442091600000, null, 1, 64, 3 ]
+  - [ 1442052000000, "AT", 4793, 12047, null, 1442062800000, 4793, 89, 1 ]
+  - [ 1442062800000, "AT", -155, 12047, null, 1442066400000, 4793, 89, 1 ]
+  - [ 1442066400000, "AT", 0, 12047, 1442052000000, 1442070000000, 4793, 89, 1 ]
+  - [ 1442070000000, "AT", 272, 12047, 1442062800000, 1442084400000, 4793, 89, 2 ]
+  - [ 1442084400000, "AT", -2, 12047, 1442066400000, 1442088000000, 4793, 89, 2 ]
+  - [ 1442088000000, "AT", 7050, 12047, 1442070000000, 1442091600000, 4793, 89, 3 ]
+  - [ 1442091600000, "AT", 89, 12047, 1442084400000, null, 4793, 89, 3 ]
+  - [ 1442016000000, "AU", 0, 2700, null, 1442019600000, 0, 518, 1 ]
+  - [ 1442019600000, "AU", 253, 2700, null, 1442023200000, 0, 518, 1 ]
+  - [ 1442023200000, "AU", 52, 2700, 1442016000000, 1442026800000, 0, 518, 1 ]
+  - [ 1442026800000, "AU", 188, 2700, 1442019600000, 1442030400000, 0, 518, 1 ]
+  - [ 1442030400000, "AU", -377, 2700, 1442023200000, 1442034000000, 0, 518, 1 ]
+  - [ 1442034000000, "AU", 283, 2700, 1442026800000, 1442037600000, 0, 518, 1 ]
+  - [ 1442037600000, "AU", 3, 2700, 1442030400000, 1442041200000, 0, 518, 1 ]
+  - [ 1442041200000, "AU", 194, 2700, 1442034000000, 1442044800000, 0, 518, 2 ]
+  - [ 1442044800000, "AU", 373, 2700, 1442037600000, 1442048400000, 0, 518, 2 ]
+  - [ 1442048400000, "AU", 135, 2700, 1442041200000, 1442052000000, 0, 518, 2 ]
+  - [ 1442052000000, "AU", -643, 2700, 1442044800000, 1442055600000, 0, 518, 2 ]
+  - [ 1442055600000, "AU", 182, 2700, 1442048400000, 1442059200000, 0, 518, 2 ]
+  - [ 1442059200000, "AU", 38, 2700, 1442052000000, 1442066400000, 0, 518, 2 ]
+  - [ 1442066400000, "AU", -21, 2700, 1442055600000, 1442070000000, 0, 518, 3 ]
+  - [ 1442070000000, "AU", -12, 2700, 1442059200000, 1442077200000, 0, 518, 3 ]
+  - [ 1442077200000, "AU", 1, 2700, 1442066400000, 1442091600000, 0, 518, 3 ]
+  - [ 1442091600000, "AU", 1138, 2700, 1442070000000, 1442095200000, 0, 518, 3 ]
+  - [ 1442095200000, "AU", 395, 2700, 1442077200000, 1442098800000, 0, 518, 3 ]
+  - [ 1442098800000, "AU", 518, 2700, 1442091600000, null, 0, 518, 3 ]
+  - [ 1442048400000, "BA", -13, -178, null, 1442052000000, -13, -1, 1 ]
+  - [ 1442052000000, "BA", 38, -178, null, 1442055600000, -13, -1, 1 ]
+  - [ 1442055600000, "BA", -202, -178, 1442048400000, 1442084400000, -13, -1, 2 ]
+  - [ 1442084400000, "BA", -1, -178, 1442052000000, null, -13, -1, 3 ]
+  - [ 1442019600000, "BD", 0, 1106, null, 1442041200000, 0, -2, 1 ]
+  - [ 1442041200000, "BD", 854, 1106, null, 1442066400000, 0, -2, 1 ]
+  - [ 1442066400000, "BD", 76, 1106, 1442019600000, 1442073600000, 0, -2, 2 ]
+  - [ 1442073600000, "BD", 103, 1106, 1442041200000, 1442077200000, 0, -2, 2 ]
+  - [ 1442077200000, "BD", 75, 1106, 1442066400000, 1442091600000, 0, -2, 3 ]
+  - [ 1442091600000, "BD", -2, 1106, 1442073600000, null, 0, -2, 3 ]
+  - [ 1442030400000, "BE", -103, 795, null, 1442048400000, -103, 9, 1 ]
+  - [ 1442048400000, "BE", 59, 795, null, 1442052000000, -103, 9, 1 ]
+  - [ 1442052000000, "BE", -1, 795, 1442030400000, 1442055600000, -103, 9, 1 ]
+  - [ 1442055600000, "BE", 233, 795, 1442048400000, 1442062800000, -103, 9, 1 ]
+  - [ 1442062800000, "BE", 91, 795, 1442052000000, 1442066400000, -103, 9, 2 ]
+  - [ 1442066400000, "BE", 136, 795, 1442055600000, 1442073600000, -103, 9, 2 ]
+  - [ 1442073600000, "BE", 19, 795, 1442062800000, 1442080800000, -103, 9, 2 ]
+  - [ 1442080800000, "BE", 1, 795, 1442066400000, 1442084400000, -103, 9, 2 ]
+  - [ 1442084400000, "BE", 183, 795, 1442073600000, 1442088000000, -103, 9, 3 ]
+  - [ 1442088000000, "BE", 67, 795, 1442080800000, 1442091600000, -103, 9, 3 ]
+  - [ 1442091600000, "BE", 101, 795, 1442084400000, 1442098800000, -103, 9, 3 ]
+  - [ 1442098800000, "BE", 9, 795, 1442088000000, null, -103, 9, 3 ]
+  - [ 1442041200000, "BG", 9, 19592, null, 1442052000000, 9, 401, 1 ]
+  - [ 1442052000000, "BG", 18936, 19592, null, 1442059200000, 9, 401, 1 ]
+  - [ 1442059200000, "BG", 191, 19592, 1442041200000, 1442070000000, 9, 401, 2 ]
+  - [ 1442070000000, "BG", 55, 19592, 1442052000000, 1442084400000, 9, 401, 2 ]
+  - [ 1442084400000, "BG", 401, 19592, 1442059200000, null, 9, 401, 3 ]
+  - [ 1442052000000, "BH", 44, 44, null, null, 44, 44, 1 ]
+  - [ 1442080800000, "BO", 4, 4, null, 1442088000000, 4, -4, 1 ]
+  - [ 1442088000000, "BO", 4, 4, null, 1442095200000, 4, -4, 2 ]
+  - [ 1442095200000, "BO", -4, 4, 1442080800000, null, 4, -4, 3 ]
+  - [ 1442016000000, "BR", -248, 8550, null, 1442019600000, -248, -645, 1 ]
+  - [ 1442019600000, "BR", 372, 8550, null, 1442023200000, -248, -645, 1 ]
+  - [ 1442023200000, "BR", 879, 8550, 1442016000000, 1442026800000, -248, -645, 1 ]
+  - [ 1442026800000, "BR", 51, 8550, 1442019600000, 1442030400000, -248, -645, 1 ]
+  - [ 1442030400000, "BR", 30, 8550, 1442023200000, 1442034000000, -248, -645, 1 ]
+  - [ 1442034000000, "BR", 21, 8550, 1442026800000, 1442037600000, -248, -645, 1 ]
+  - [ 1442037600000, "BR", 267, 8550, 1442030400000, 1442041200000, -248, -645, 1 ]
+  - [ 1442041200000, "BR", 3, 8550, 1442034000000, 1442044800000, -248, -645, 1 ]
+  - [ 1442044800000, "BR", 71, 8550, 1442037600000, 1442052000000, -248, -645, 2 ]
+  - [ 1442052000000, "BR", 232, 8550, 1442041200000, 1442055600000, -248, -645, 2 ]
+  - [ 1442055600000, "BR", 242, 8550, 1442044800000, 1442059200000, -248, -645, 2 ]
+  - [ 1442059200000, "BR", 73, 8550, 1442052000000, 1442062800000, -248, -645, 2 ]
+  - [ 1442062800000, "BR", 93, 8550, 1442055600000, 1442066400000, -248, -645, 2 ]
+  - [ 1442066400000, "BR", 1034, 8550, 1442059200000, 1442070000000, -248, -645, 2 ]
+  - [ 1442070000000, "BR", 536, 8550, 1442062800000, 1442073600000, -248, -645, 2 ]
+  - [ 1442073600000, "BR", 2087, 8550, 1442066400000, 1442077200000, -248, -645, 2 ]
+  - [ 1442077200000, "BR", 2253, 8550, 1442070000000, 1442080800000, -248, -645, 3 ]
+  - [ 1442080800000, "BR", -267, 8550, 1442073600000, 1442084400000, -248, -645, 3 ]
+  - [ 1442084400000, "BR", 492, 8550, 1442077200000, 1442088000000, -248, -645, 3 ]
+  - [ 1442088000000, "BR", 215, 8550, 1442080800000, 1442091600000, -248, -645, 3 ]
+  - [ 1442091600000, "BR", 11, 8550, 1442084400000, 1442095200000, -248, -645, 3 ]
+  - [ 1442095200000, "BR", 748, 8550, 1442088000000, 1442098800000, -248, -645, 3 ]
+  - [ 1442098800000, "BR", -645, 8550, 1442091600000, null, -248, -645, 3 ]
+  - [ 1442055600000, "BY", 1, 2153, null, 1442059200000, 1, 33, 1 ]
+  - [ 1442059200000, "BY", 1464, 2153, null, 1442073600000, 1, 33, 1 ]
+  - [ 1442073600000, "BY", 596, 2153, 1442055600000, 1442077200000, 1, 33, 1 ]
+  - [ 1442077200000, "BY", 30, 2153, 1442059200000, 1442080800000, 1, 33, 2 ]
+  - [ 1442080800000, "BY", 28, 2153, 1442073600000, 1442084400000, 1, 33, 2 ]
+  - [ 1442084400000, "BY", 1, 2153, 1442077200000, 1442088000000, 1, 33, 3 ]
+  - [ 1442088000000, "BY", 33, 2153, 1442080800000, null, 1, 33, 3 ]
+  - [ 1442016000000, "CA", -371, 8544, null, 1442019600000, -371, 164, 1 ]
+  - [ 1442019600000, "CA", 2184, 8544, null, 1442023200000, -371, 164, 1 ]
+  - [ 1442023200000, "CA", 286, 8544, 1442016000000, 1442026800000, -371, 164, 1 ]
+  - [ 1442026800000, "CA", 2216, 8544, 1442019600000, 1442030400000, -371, 164, 1 ]
+  - [ 1442030400000, "CA", -47, 8544, 1442023200000, 1442034000000, -371, 164, 1 ]
+  - [ 1442034000000, "CA", 178, 8544, 1442026800000, 1442037600000, -371, 164, 1 ]
+  - [ 1442037600000, "CA", -132, 8544, 1442030400000, 1442041200000, -371, 164, 1 ]
+  - [ 1442041200000, "CA", 5, 8544, 1442034000000, 1442044800000, -371, 164, 1 ]
+  - [ 1442044800000, "CA", 1, 8544, 1442037600000, 1442052000000, -371, 164, 2 ]
+  - [ 1442052000000, "CA", 38, 8544, 1442041200000, 1442059200000, -371, 164, 2 ]
+  - [ 1442059200000, "CA", 1036, 8544, 1442044800000, 1442062800000, -371, 164, 2 ]
+  - [ 1442062800000, "CA", -367, 8544, 1442052000000, 1442066400000, -371, 164, 2 ]
+  - [ 1442066400000, "CA", 307, 8544, 1442059200000, 1442070000000, -371, 164, 2 ]
+  - [ 1442070000000, "CA", 185, 8544, 1442062800000, 1442073600000, -371, 164, 2 ]
+  - [ 1442073600000, "CA", 86, 8544, 1442066400000, 1442077200000, -371, 164, 2 ]
+  - [ 1442077200000, "CA", -282, 8544, 1442070000000, 1442080800000, -371, 164, 3 ]
+  - [ 1442080800000, "CA", 481, 8544, 1442073600000, 1442084400000, -371, 164, 3 ]
+  - [ 1442084400000, "CA", 44, 8544, 1442077200000, 1442088000000, -371, 164, 3 ]
+  - [ 1442088000000, "CA", 35, 8544, 1442080800000, 1442091600000, -371, 164, 3 ]
+  - [ 1442091600000, "CA", 2858, 8544, 1442084400000, 1442095200000, -371, 164, 3 ]
+  - [ 1442095200000, "CA", -361, 8544, 1442088000000, 1442098800000, -371, 164, 3 ]
+  - [ 1442098800000, "CA", 164, 8544, 1442091600000, null, -371, 164, 3 ]
+  - [ 1442037600000, "CH", 59, 753, null, 1442041200000, 59, 67, 1 ]
+  - [ 1442041200000, "CH", 198, 753, null, 1442044800000, 59, 67, 1 ]
+  - [ 1442044800000, "CH", -54, 753, 1442037600000, 1442048400000, 59, 67, 1 ]
+  - [ 1442048400000, "CH", 24, 753, 1442041200000, 1442052000000, 59, 67, 1 ]
+  - [ 1442052000000, "CH", 47, 753, 1442044800000, 1442055600000, 59, 67, 2 ]
+  - [ 1442055600000, "CH", 0, 753, 1442048400000, 1442062800000, 59, 67, 2 ]
+  - [ 1442062800000, "CH", 22, 753, 1442052000000, 1442070000000, 59, 67, 2 ]
+  - [ 1442070000000, "CH", 11, 753, 1442055600000, 1442073600000, 59, 67, 2 ]
+  - [ 1442073600000, "CH", 360, 753, 1442062800000, 1442077200000, 59, 67, 3 ]
+  - [ 1442077200000, "CH", 6, 753, 1442070000000, 1442084400000, 59, 67, 3 ]
+  - [ 1442084400000, "CH", 13, 753, 1442073600000, 1442091600000, 59, 67, 3 ]
+  - [ 1442091600000, "CH", 67, 753, 1442077200000, null, 59, 67, 3 ]
+  - [ 1442016000000, "CL", 161, 533, null, 1442019600000, 161, 9, 1 ]
+  - [ 1442019600000, "CL", -370, 533, null, 1442023200000, 161, 9, 1 ]
+  - [ 1442023200000, "CL", 15, 533, 1442016000000, 1442030400000, 161, 9, 1 ]
+  - [ 1442030400000, "CL", 40, 533, 1442019600000, 1442034000000, 161, 9, 1 ]
+  - [ 1442034000000, "CL", -1, 533, 1442023200000, 1442037600000, 161, 9, 1 ]
+  - [ 1442037600000, "CL", 2, 533, 1442030400000, 1442041200000, 161, 9, 1 ]
+  - [ 1442041200000, "CL", -1, 533, 1442034000000, 1442052000000, 161, 9, 1 ]
+  - [ 1442052000000, "CL", 390, 533, 1442037600000, 1442059200000, 161, 9, 2 ]
+  - [ 1442059200000, "CL", -12, 533, 1442041200000, 1442062800000, 161, 9, 2 ]
+  - [ 1442062800000, "CL", 17, 533, 1442052000000, 1442066400000, 161, 9, 2 ]
+  - [ 1442066400000, "CL", -41, 533, 1442059200000, 1442070000000, 161, 9, 2 ]
+  - [ 1442070000000, "CL", 13, 533, 1442062800000, 1442073600000, 161, 9, 2 ]
+  - [ 1442073600000, "CL", 153, 533, 1442066400000, 1442077200000, 161, 9, 2 ]
+  - [ 1442077200000, "CL", -15, 533, 1442070000000, 1442080800000, 161, 9, 2 ]
+  - [ 1442080800000, "CL", 17, 533, 1442073600000, 1442084400000, 161, 9, 3 ]
+  - [ 1442084400000, "CL", 126, 533, 1442077200000, 1442088000000, 161, 9, 3 ]
+  - [ 1442088000000, "CL", 286, 533, 1442080800000, 1442091600000, 161, 9, 3 ]
+  - [ 1442091600000, "CL", 20, 533, 1442084400000, 1442095200000, 161, 9, 3 ]
+  - [ 1442095200000, "CL", -276, 533, 1442088000000, 1442098800000, 161, 9, 3 ]
+  - [ 1442098800000, "CL", 9, 533, 1442091600000, null, 161, 9, 3 ]
+  - [ 1442023200000, "CN", -13, 583, null, 1442026800000, -13, -1, 1 ]
+  - [ 1442026800000, "CN", 154, 583, null, 1442037600000, -13, -1, 1 ]
+  - [ 1442037600000, "CN", 98, 583, 1442023200000, 1442048400000, -13, -1, 1 ]
+  - [ 1442048400000, "CN", 293, 583, 1442026800000, 1442052000000, -13, -1, 1 ]
+  - [ 1442052000000, "CN", 0, 583, 1442037600000, 1442055600000, -13, -1, 2 ]
+  - [ 1442055600000, "CN", 69, 583, 1442048400000, 1442059200000, -13, -1, 2 ]
+  - [ 1442059200000, "CN", 8, 583, 1442052000000, 1442066400000, -13, -1, 2 ]
+  - [ 1442066400000, "CN", -15, 583, 1442055600000, 1442080800000, -13, -1, 3 ]
+  - [ 1442080800000, "CN", -10, 583, 1442059200000, 1442084400000, -13, -1, 3 ]
+  - [ 1442084400000, "CN", -1, 583, 1442066400000, null, -13, -1, 3 ]
+  - [ 1442016000000, "CO", 16, 59611, null, 1442019600000, 16, 83, 1 ]
+  - [ 1442019600000, "CO", 12, 59611, null, 1442023200000, 16, 83, 1 ]
+  - [ 1442023200000, "CO", 9, 59611, 1442016000000, 1442030400000, 16, 83, 1 ]
+  - [ 1442030400000, "CO", 441, 59611, 1442019600000, 1442059200000, 16, 83, 1 ]
+  - [ 1442059200000, "CO", 473, 59611, 1442023200000, 1442066400000, 16, 83, 1 ]
+  - [ 1442066400000, "CO", 288, 59611, 1442030400000, 1442070000000, 16, 83, 2 ]
+  - [ 1442070000000, "CO", -45, 59611, 1442059200000, 1442073600000, 16, 83, 2 ]
+  - [ 1442073600000, "CO", 39860, 59611, 1442066400000, 1442077200000, 16, 83, 2 ]
+  - [ 1442077200000, "CO", 581, 59611, 1442070000000, 1442080800000, 16, 83, 2 ]
+  - [ 1442080800000, "CO", 25, 59611, 1442073600000, 1442084400000, 16, 83, 2 ]
+  - [ 1442084400000, "CO", 51, 59611, 1442077200000, 1442088000000, 16, 83, 3 ]
+  - [ 1442088000000, "CO", 17150, 59611, 1442080800000, 1442091600000, 16, 83, 3 ]
+  - [ 1442091600000, "CO", 377, 59611, 1442084400000, 1442095200000, 16, 83, 3 ]
+  - [ 1442095200000, "CO", 290, 59611, 1442088000000, 1442098800000, 16, 83, 3 ]
+  - [ 1442098800000, "CO", 83, 59611, 1442091600000, null, 16, 83, 3 ]
+  - [ 1442019600000, "CR", 62, 3241, null, 1442023200000, 62, 72, 1 ]
+  - [ 1442023200000, "CR", 62, 3241, null, 1442026800000, 62, 72, 1 ]
+  - [ 1442026800000, "CR", 140, 3241, 1442019600000, 1442030400000, 62, 72, 1 ]
+  - [ 1442030400000, "CR", 2497, 3241, 1442023200000, 1442041200000, 62, 72, 2 ]
+  - [ 1442041200000, "CR", 51, 3241, 1442026800000, 1442044800000, 62, 72, 2 ]
+  - [ 1442044800000, "CR", 194, 3241, 1442030400000, 1442048400000, 62, 72, 2 ]
+  - [ 1442048400000, "CR", 163, 3241, 1442041200000, 1442088000000, 62, 72, 3 ]
+  - [ 1442088000000, "CR", 72, 3241, 1442044800000, null, 62, 72, 3 ]
+  - [ 1442026800000, "CZ", -19, 3479, null, 1442034000000, -19, 2, 1 ]
+  - [ 1442034000000, "CZ", 78, 3479, null, 1442037600000, -19, 2, 1 ]
+  - [ 1442037600000, "CZ", 18, 3479, 1442026800000, 1442055600000, -19, 2, 1 ]
+  - [ 1442055600000, "CZ", 1073, 3479, 1442034000000, 1442059200000, -19, 2, 1 ]
+  - [ 1442059200000, "CZ", 21, 3479, 1442037600000, 1442062800000, -19, 2, 2 ]
+  - [ 1442062800000, "CZ", 0, 3479, 1442055600000, 1442070000000, -19, 2, 2 ]
+  - [ 1442070000000, "CZ", 168, 3479, 1442059200000, 1442073600000, -19, 2, 2 ]
+  - [ 1442073600000, "CZ", 2051, 3479, 1442062800000, 1442077200000, -19, 2, 2 ]
+  - [ 1442077200000, "CZ", 115, 3479, 1442070000000, 1442080800000, -19, 2, 3 ]
+  - [ 1442080800000, "CZ", -28, 3479, 1442073600000, 1442098800000, -19, 2, 3 ]
+  - [ 1442098800000, "CZ", 2, 3479, 1442077200000, null, -19, 2, 3 ]
+  - [ 1442016000000, "DE", 167, 25583, null, 1442019600000, 167, 329, 1 ]
+  - [ 1442019600000, "DE", 0, 25583, null, 1442023200000, 167, 329, 1 ]
+  - [ 1442023200000, "DE", 64, 25583, 1442016000000, 1442030400000, 167, 329, 1 ]
+  - [ 1442030400000, "DE", 373, 25583, 1442019600000, 1442034000000, 167, 329, 1 ]
+  - [ 1442034000000, "DE", 358, 25583, 1442023200000, 1442037600000, 167, 329, 1 ]
+  - [ 1442037600000, "DE", 544, 25583, 1442030400000, 1442041200000, 167, 329, 1 ]
+  - [ 1442041200000, "DE", 197, 25583, 1442034000000, 1442044800000, 167, 329, 1 ]
+  - [ 1442044800000, "DE", 979, 25583, 1442037600000, 1442048400000, 167, 329, 1 ]
+  - [ 1442048400000, "DE", 811, 25583, 1442041200000, 1442052000000, 167, 329, 2 ]
+  - [ 1442052000000, "DE", 1600, 25583, 1442044800000, 1442055600000, 167, 329, 2 ]
+  - [ 1442055600000, "DE", 1523, 25583, 1442048400000, 1442059200000, 167, 329, 2 ]
+  - [ 1442059200000, "DE", 289, 25583, 1442052000000, 1442062800000, 167, 329, 2 ]
+  - [ 1442062800000, "DE", 283, 25583, 1442055600000, 1442066400000, 167, 329, 2 ]
+  - [ 1442066400000, "DE", 1577, 25583, 1442059200000, 1442070000000, 167, 329, 2 ]
+  - [ 1442070000000, "DE", 1666, 25583, 1442062800000, 1442073600000, 167, 329, 2 ]
+  - [ 1442073600000, "DE", 6075, 25583, 1442066400000, 1442077200000, 167, 329, 2 ]
+  - [ 1442077200000, "DE", 2188, 25583, 1442070000000, 1442080800000, 167, 329, 3 ]
+  - [ 1442080800000, "DE", 1133, 25583, 1442073600000, 1442084400000, 167, 329, 3 ]
+  - [ 1442084400000, "DE", -125, 25583, 1442077200000, 1442088000000, 167, 329, 3 ]
+  - [ 1442088000000, "DE", 190, 25583, 1442080800000, 1442091600000, 167, 329, 3 ]
+  - [ 1442091600000, "DE", 4355, 25583, 1442084400000, 1442095200000, 167, 329, 3 ]
+  - [ 1442095200000, "DE", 1007, 25583, 1442088000000, 1442098800000, 167, 329, 3 ]
+  - [ 1442098800000, "DE", 329, 25583, 1442091600000, null, 167, 329, 3 ]
+  - [ 1442037600000, "DK", 10, 594, null, 1442044800000, 10, 0, 1 ]
+  - [ 1442044800000, "DK", 36, 594, null, 1442048400000, 10, 0, 1 ]
+  - [ 1442048400000, "DK", -5, 594, 1442037600000, 1442055600000, 10, 0, 1 ]
+  - [ 1442055600000, "DK", 42, 594, 1442044800000, 1442059200000, 10, 0, 1 ]
+  - [ 1442059200000, "DK", 0, 594, 1442048400000, 1442062800000, 10, 0, 2 ]
+  - [ 1442062800000, "DK", 1, 594, 1442055600000, 1442066400000, 10, 0, 2 ]
+  - [ 1442066400000, "DK", 416, 594, 1442059200000, 1442077200000, 10, 0, 2 ]
+  - [ 1442077200000, "DK", -9, 594, 1442062800000, 1442080800000, 10, 0, 2 ]
+  - [ 1442080800000, "DK", 61, 594, 1442066400000, 1442084400000, 10, 0, 3 ]
+  - [ 1442084400000, "DK", -97, 594, 1442077200000, 1442091600000, 10, 0, 3 ]
+  - [ 1442091600000, "DK", 139, 594, 1442080800000, 1442095200000, 10, 0, 3 ]
+  - [ 1442095200000, "DK", 0, 594, 1442084400000, null, 10, 0, 3 ]
+  - [ 1442023200000, "DO", 8, 264, null, 1442066400000, 8, 13, 1 ]
+  - [ 1442066400000, "DO", 35, 264, null, 1442073600000, 8, 13, 1 ]
+  - [ 1442073600000, "DO", 200, 264, 1442023200000, 1442084400000, 8, 13, 2 ]
+  - [ 1442084400000, "DO", 8, 264, 1442066400000, 1442095200000, 8, 13, 2 ]
+  - [ 1442095200000, "DO", 13, 264, 1442073600000, null, 8, 13, 3 ]
+  - [ 1442077200000, "DZ", -1, -1, null, null, -1, -1, 1 ]
+  - [ 1442019600000, "EC", 29, 232, null, 1442023200000, 29, 10, 1 ]
+  - [ 1442023200000, "EC", -9, 232, null, 1442030400000, 29, 10, 1 ]
+  - [ 1442030400000, "EC", 0, 232, 1442019600000, 1442077200000, 29, 10, 2 ]
+  - [ 1442077200000, "EC", -366, 232, 1442023200000, 1442084400000, 29, 10, 2 ]
+  - [ 1442084400000, "EC", 568, 232, 1442030400000, 1442095200000, 29, 10, 3 ]
+  - [ 1442095200000, "EC", 10, 232, 1442077200000, null, 29, 10, 3 ]
+  - [ 1442041200000, "EE", 37, 18, null, 1442044800000, 37, -19, 1 ]
+  - [ 1442044800000, "EE", -19, 18, null, null, 37, -19, 2 ]
+  - [ 1442026800000, "EG", 16, 170, null, 1442055600000, 16, 27, 1 ]
+  - [ 1442055600000, "EG", 14, 170, null, 1442062800000, 16, 27, 1 ]
+  - [ 1442062800000, "EG", 112, 170, 1442026800000, 1442073600000, 16, 27, 2 ]
+  - [ 1442073600000, "EG", 1, 170, 1442055600000, 1442091600000, 16, 27, 2 ]
+  - [ 1442091600000, "EG", 27, 170, 1442062800000, null, 16, 27, 3 ]
+  - [ 1442019600000, "ES", 103, 7449, null, 1442023200000, 103, 458, 1 ]
+  - [ 1442023200000, "ES", -5, 7449, null, 1442034000000, 103, 458, 1 ]
+  - [ 1442034000000, "ES", -52, 7449, 1442019600000, 1442037600000, 103, 458, 1 ]
+  - [ 1442037600000, "ES", 3, 7449, 1442023200000, 1442041200000, 103, 458, 1 ]
+  - [ 1442041200000, "ES", 118, 7449, 1442034000000, 1442044800000, 103, 458, 1 ]
+  - [ 1442044800000, "ES", -169, 7449, 1442037600000, 1442048400000, 103, 458, 1 ]
+  - [ 1442048400000, "ES", 158, 7449, 1442041200000, 1442052000000, 103, 458, 1 ]
+  - [ 1442052000000, "ES", -4, 7449, 1442044800000, 1442055600000, 103, 458, 2 ]
+  - [ 1442055600000, "ES", 495, 7449, 1442048400000, 1442059200000, 103, 458, 2 ]
+  - [ 1442059200000, "ES", 1086, 7449, 1442052000000, 1442062800000, 103, 458, 2 ]
+  - [ 1442062800000, "ES", -71, 7449, 1442055600000, 1442066400000, 103, 458, 2 ]
+  - [ 1442066400000, "ES", 461, 7449, 1442059200000, 1442070000000, 103, 458, 2 ]
+  - [ 1442070000000, "ES", 61, 7449, 1442062800000, 1442073600000, 103, 458, 2 ]
+  - [ 1442073600000, "ES", 154, 7449, 1442066400000, 1442077200000, 103, 458, 2 ]
+  - [ 1442077200000, "ES", 1240, 7449, 1442070000000, 1442084400000, 103, 458, 3 ]
+  - [ 1442084400000, "ES", 337, 7449, 1442073600000, 1442088000000, 103, 458, 3 ]
+  - [ 1442088000000, "ES", -130, 7449, 1442077200000, 1442091600000, 103, 458, 3 ]
+  - [ 1442091600000, "ES", 700, 7449, 1442084400000, 1442095200000, 103, 458, 3 ]
+  - [ 1442095200000, "ES", 2506, 7449, 1442088000000, 1442098800000, 103, 458, 3 ]
+  - [ 1442098800000, "ES", 458, 7449, 1442091600000, null, 103, 458, 3 ]
+  - [ 1442030400000, "FI", 1491, 3579, null, 1442037600000, 1491, 69, 1 ]
+  - [ 1442037600000, "FI", 14, 3579, null, 1442048400000, 1491, 69, 1 ]
+  - [ 1442048400000, "FI", 12, 3579, 1442030400000, 1442052000000, 1491, 69, 1 ]
+  - [ 1442052000000, "FI", 186, 3579, 1442037600000, 1442059200000, 1491, 69, 1 ]
+  - [ 1442059200000, "FI", 407, 3579, 1442048400000, 1442062800000, 1491, 69, 2 ]
+  - [ 1442062800000, "FI", 19, 3579, 1442052000000, 1442066400000, 1491, 69, 2 ]
+  - [ 1442066400000, "FI", 183, 3579, 1442059200000, 1442073600000, 1491, 69, 2 ]
+  - [ 1442073600000, "FI", -1, 3579, 1442062800000, 1442077200000, 1491, 69, 2 ]
+  - [ 1442077200000, "FI", 200, 3579, 1442066400000, 1442080800000, 1491, 69, 3 ]
+  - [ 1442080800000, "FI", 104, 3579, 1442073600000, 1442084400000, 1491, 69, 3 ]
+  - [ 1442084400000, "FI", 895, 3579, 1442077200000, 1442095200000, 1491, 69, 3 ]
+  - [ 1442095200000, "FI", 69, 3579, 1442080800000, null, 1491, 69, 3 ]
+  - [ 1442016000000, "FR", -1, 37281, null, 1442019600000, -1, 136, 1 ]
+  - [ 1442019600000, "FR", 585, 37281, null, 1442023200000, -1, 136, 1 ]
+  - [ 1442023200000, "FR", 628, 37281, 1442016000000, 1442026800000, -1, 136, 1 ]
+  - [ 1442026800000, "FR", 86, 37281, 1442019600000, 1442034000000, -1, 136, 1 ]
+  - [ 1442034000000, "FR", 476, 37281, 1442023200000, 1442037600000, -1, 136, 1 ]
+  - [ 1442037600000, "FR", 4174, 37281, 1442026800000, 1442041200000, -1, 136, 1 ]
+  - [ 1442041200000, "FR", 604, 37281, 1442034000000, 1442044800000, -1, 136, 1 ]
+  - [ 1442044800000, "FR", 172, 37281, 1442037600000, 1442048400000, -1, 136, 1 ]
+  - [ 1442048400000, "FR", 3027, 37281, 1442041200000, 1442052000000, -1, 136, 2 ]
+  - [ 1442052000000, "FR", 637, 37281, 1442044800000, 1442055600000, -1, 136, 2 ]
+  - [ 1442055600000, "FR", 463, 37281, 1442048400000, 1442059200000, -1, 136, 2 ]
+  - [ 1442059200000, "FR", 4650, 37281, 1442052000000, 1442062800000, -1, 136, 2 ]
+  - [ 1442062800000, "FR", 5676, 37281, 1442055600000, 1442066400000, -1, 136, 2 ]
+  - [ 1442066400000, "FR", 2516, 37281, 1442059200000, 1442070000000, -1, 136, 2 ]
+  - [ 1442070000000, "FR", 474, 37281, 1442062800000, 1442073600000, -1, 136, 2 ]
+  - [ 1442073600000, "FR", 3522, 37281, 1442066400000, 1442077200000, -1, 136, 2 ]
+  - [ 1442077200000, "FR", -444, 37281, 1442070000000, 1442080800000, -1, 136, 3 ]
+  - [ 1442080800000, "FR", 557, 37281, 1442073600000, 1442084400000, -1, 136, 3 ]
+  - [ 1442084400000, "FR", 6643, 37281, 1442077200000, 1442088000000, -1, 136, 3 ]
+  - [ 1442088000000, "FR", 1872, 37281, 1442080800000, 1442091600000, -1, 136, 3 ]
+  - [ 1442091600000, "FR", 741, 37281, 1442084400000, 1442095200000, -1, 136, 3 ]
+  - [ 1442095200000, "FR", 87, 37281, 1442088000000, 1442098800000, -1, 136, 3 ]
+  - [ 1442098800000, "FR", 136, 37281, 1442091600000, null, -1, 136, 3 ]
+  - [ 1442016000000, "GB", -44, 35857, null, 1442019600000, -44, 49, 1 ]
+  - [ 1442019600000, "GB", 54, 35857, null, 1442023200000, -44, 49, 1 ]
+  - [ 1442023200000, "GB", 1816, 35857, 1442016000000, 1442026800000, -44, 49, 1 ]
+  - [ 1442026800000, "GB", 339, 35857, 1442019600000, 1442030400000, -44, 49, 1 ]
+  - [ 1442030400000, "GB", 2524, 35857, 1442023200000, 1442034000000, -44, 49, 1 ]
+  - [ 1442034000000, "GB", -12, 35857, 1442026800000, 1442037600000, -44, 49, 1 ]
+  - [ 1442037600000, "GB", 544, 35857, 1442030400000, 1442041200000, -44, 49, 1 ]
+  - [ 1442041200000, "GB", 42, 35857, 1442034000000, 1442044800000, -44, 49, 1 ]
+  - [ 1442044800000, "GB", 32, 35857, 1442037600000, 1442048400000, -44, 49, 2 ]
+  - [ 1442048400000, "GB", 740, 35857, 1442041200000, 1442052000000, -44, 49, 2 ]
+  - [ 1442052000000, "GB", 168, 35857, 1442044800000, 1442055600000, -44, 49, 2 ]
+  - [ 1442055600000, "GB", 453, 35857, 1442048400000, 1442059200000, -44, 49, 2 ]
+  - [ 1442059200000, "GB", 16111, 35857, 1442052000000, 1442062800000, -44, 49, 2 ]
+  - [ 1442062800000, "GB", 5743, 35857, 1442055600000, 1442066400000, -44, 49, 2 ]
+  - [ 1442066400000, "GB", 671, 35857, 1442059200000, 1442070000000, -44, 49, 2 ]
+  - [ 1442070000000, "GB", 374, 35857, 1442062800000, 1442073600000, -44, 49, 2 ]
+  - [ 1442073600000, "GB", 648, 35857, 1442066400000, 1442077200000, -44, 49, 3 ]
+  - [ 1442077200000, "GB", 1135, 35857, 1442070000000, 1442080800000, -44, 49, 3 ]
+  - [ 1442080800000, "GB", 1444, 35857, 1442073600000, 1442084400000, -44, 49, 3 ]
+  - [ 1442084400000, "GB", 384, 35857, 1442077200000, 1442088000000, -44, 49, 3 ]
+  - [ 1442088000000, "GB", 1593, 35857, 1442080800000, 1442091600000, -44, 49, 3 ]
+  - [ 1442091600000, "GB", 811, 35857, 1442084400000, 1442095200000, -44, 49, 3 ]
+  - [ 1442095200000, "GB", 238, 35857, 1442088000000, 1442098800000, -44, 49, 3 ]
+  - [ 1442098800000, "GB", 49, 35857, 1442091600000, null, -44, 49, 3 ]
+  - [ 1442044800000, "GE", -21, -140, null, 1442052000000, -21, -27, 1 ]
+  - [ 1442052000000, "GE", -108, -140, null, 1442062800000, -21, -27, 1 ]
+  - [ 1442062800000, "GE", 16, -140, 1442044800000, 1442080800000, -21, -27, 2 ]
+  - [ 1442080800000, "GE", -27, -140, 1442052000000, null, -21, -27, 3 ]
+  - [ 1442088000000, "GH", 0, 0, null, null, 0, 0, 1 ]
+  - [ 1442019600000, "GR", 82, 149, null, 1442034000000, 82, 123, 1 ]
+  - [ 1442034000000, "GR", 0, 149, null, 1442041200000, 82, 123, 1 ]
+  - [ 1442041200000, "GR", 7, 149, 1442019600000, 1442048400000, 82, 123, 1 ]
+  - [ 1442048400000, "GR", -26, 149, 1442034000000, 1442062800000, 82, 123, 1 ]
+  - [ 1442062800000, "GR", 8, 149, 1442041200000, 1442070000000, 82, 123, 2 ]
+  - [ 1442070000000, "GR", 2, 149, 1442048400000, 1442073600000, 82, 123, 2 ]
+  - [ 1442073600000, "GR", -314, 149, 1442062800000, 1442080800000, 82, 123, 2 ]
+  - [ 1442080800000, "GR", 88, 149, 1442070000000, 1442084400000, 82, 123, 3 ]
+  - [ 1442084400000, "GR", 179, 149, 1442073600000, 1442091600000, 82, 123, 3 ]
+  - [ 1442091600000, "GR", 123, 149, 1442080800000, null, 82, 123, 3 ]
+  - [ 1442023200000, "GT", -167, 7, null, 1442026800000, -167, 1, 1 ]
+  - [ 1442026800000, "GT", 173, 7, null, 1442098800000, -167, 1, 2 ]
+  - [ 1442098800000, "GT", 1, 7, 1442023200000, null, -167, 1, 3 ]
+  - [ 1442019600000, "HK", -113, 10743, null, 1442023200000, -113, -1, 1 ]
+  - [ 1442023200000, "HK", 2414, 10743, null, 1442026800000, -113, -1, 1 ]
+  - [ 1442026800000, "HK", -211, 10743, 1442019600000, 1442030400000, -113, -1, 1 ]
+  - [ 1442030400000, "HK", 157, 10743, 1442023200000, 1442034000000, -113, -1, 1 ]
+  - [ 1442034000000, "HK", 1137, 10743, 1442026800000, 1442037600000, -113, -1, 1 ]
+  - [ 1442037600000, "HK", 636, 10743, 1442030400000, 1442041200000, -113, -1, 1 ]
+  - [ 1442041200000, "HK", -15, 10743, 1442034000000, 1442044800000, -113, -1, 1 ]
+  - [ 1442044800000, "HK", 21, 10743, 1442037600000, 1442048400000, -113, -1, 2 ]
+  - [ 1442048400000, "HK", 1, 10743, 1442041200000, 1442052000000, -113, -1, 2 ]
+  - [ 1442052000000, "HK", 15, 10743, 1442044800000, 1442055600000, -113, -1, 2 ]
+  - [ 1442055600000, "HK", 804, 10743, 1442048400000, 1442059200000, -113, -1, 2 ]
+  - [ 1442059200000, "HK", 2, 10743, 1442052000000, 1442062800000, -113, -1, 2 ]
+  - [ 1442062800000, "HK", 1, 10743, 1442055600000, 1442066400000, -113, -1, 2 ]
+  - [ 1442066400000, "HK", 39, 10743, 1442059200000, 1442070000000, -113, -1, 3 ]
+  - [ 1442070000000, "HK", 314, 10743, 1442062800000, 1442073600000, -113, -1, 3 ]
+  - [ 1442073600000, "HK", 5545, 10743, 1442066400000, 1442080800000, -113, -1, 3 ]
+  - [ 1442080800000, "HK", 0, 10743, 1442070000000, 1442091600000, -113, -1, 3 ]
+  - [ 1442091600000, "HK", -3, 10743, 1442073600000, 1442095200000, -113, -1, 3 ]
+  - [ 1442095200000, "HK", -1, 10743, 1442080800000, null, -113, -1, 3 ]
+  - [ 1442026800000, "HN", -1, -1, null, null, -1, -1, 1 ]
+  - [ 1442070000000, "HR", 32, 382, null, 1442073600000, 32, 82, 1 ]
+  - [ 1442073600000, "HR", 0, 382, null, 1442077200000, 32, 82, 1 ]
+  - [ 1442077200000, "HR", 58, 382, 1442070000000, 1442080800000, 32, 82, 2 ]
+  - [ 1442080800000, "HR", 220, 382, 1442073600000, 1442084400000, 32, 82, 2 ]
+  - [ 1442084400000, "HR", -10, 382, 1442077200000, 1442088000000, 32, 82, 3 ]
+  - [ 1442088000000, "HR", 82, 382, 1442080800000, null, 32, 82, 3 ]
+  - [ 1442019600000, "HU", 46, 2116, null, 1442037600000, 46, 110, 1 ]
+  - [ 1442037600000, "HU", 197, 2116, null, 1442041200000, 46, 110, 1 ]
+  - [ 1442041200000, "HU", 91, 2116, 1442019600000, 1442044800000, 46, 110, 1 ]
+  - [ 1442044800000, "HU", 547, 2116, 1442037600000, 1442048400000, 46, 110, 1 ]
+  - [ 1442048400000, "HU", 499, 2116, 1442041200000, 1442055600000, 46, 110, 1 ]
+  - [ 1442055600000, "HU", -2, 2116, 1442044800000, 1442062800000, 46, 110, 2 ]
+  - [ 1442062800000, "HU", 50, 2116, 1442048400000, 1442080800000, 46, 110, 2 ]
+  - [ 1442080800000, "HU", 242, 2116, 1442055600000, 1442084400000, 46, 110, 2 ]
+  - [ 1442084400000, "HU", 141, 2116, 1442062800000, 1442088000000, 46, 110, 2 ]
+  - [ 1442088000000, "HU", -71, 2116, 1442080800000, 1442091600000, 46, 110, 3 ]
+  - [ 1442091600000, "HU", -5, 2116, 1442084400000, 1442095200000, 46, 110, 3 ]
+  - [ 1442095200000, "HU", 271, 2116, 1442088000000, 1442098800000, 46, 110, 3 ]
+  - [ 1442098800000, "HU", 110, 2116, 1442091600000, null, 46, 110, 3 ]
+  - [ 1442023200000, "ID", 106, -255, null, 1442026800000, 106, 13, 1 ]
+  - [ 1442026800000, "ID", -416, -255, null, 1442030400000, 106, 13, 1 ]
+  - [ 1442030400000, "ID", 279, -255, 1442023200000, 1442034000000, 106, 13, 1 ]
+  - [ 1442034000000, "ID", 19, -255, 1442026800000, 1442037600000, 106, 13, 1 ]
+  - [ 1442037600000, "ID", 14, -255, 1442030400000, 1442041200000, 106, 13, 1 ]
+  - [ 1442041200000, "ID", 2, -255, 1442034000000, 1442044800000, 106, 13, 2 ]
+  - [ 1442044800000, "ID", -388, -255, 1442037600000, 1442055600000, 106, 13, 2 ]
+  - [ 1442055600000, "ID", 16, -255, 1442041200000, 1442059200000, 106, 13, 2 ]
+  - [ 1442059200000, "ID", 17, -255, 1442044800000, 1442070000000, 106, 13, 2 ]
+  - [ 1442070000000, "ID", 42, -255, 1442055600000, 1442091600000, 106, 13, 3 ]
+  - [ 1442091600000, "ID", 21, -255, 1442059200000, 1442095200000, 106, 13, 3 ]
+  - [ 1442095200000, "ID", 20, -255, 1442070000000, 1442098800000, 106, 13, 3 ]
+  - [ 1442098800000, "ID", 13, -255, 1442091600000, null, 106, 13, 3 ]
+  - [ 1442026800000, "IE", 1, 2142, null, 1442030400000, 1, -71, 1 ]
+  - [ 1442030400000, "IE", 1, 2142, null, 1442048400000, 1, -71, 1 ]
+  - [ 1442048400000, "IE", 27, 2142, 1442026800000, 1442066400000, 1, -71, 1 ]
+  - [ 1442066400000, "IE", 1062, 2142, 1442030400000, 1442070000000, 1, -71, 2 ]
+  - [ 1442070000000, "IE", -100, 2142, 1442048400000, 1442077200000, 1, -71, 2 ]
+  - [ 1442077200000, "IE", 403, 2142, 1442066400000, 1442084400000, 1, -71, 2 ]
+  - [ 1442084400000, "IE", 819, 2142, 1442070000000, 1442091600000, 1, -71, 3 ]
+  - [ 1442091600000, "IE", -71, 2142, 1442077200000, null, 1, -71, 3 ]
+  - [ 1442041200000, "IL", 35, 6617, null, 1442044800000, 35, 3, 1 ]
+  - [ 1442044800000, "IL", 218, 6617, null, 1442048400000, 35, 3, 1 ]
+  - [ 1442048400000, "IL", 25, 6617, 1442041200000, 1442052000000, 35, 3, 1 ]
+  - [ 1442052000000, "IL", 2745, 6617, 1442044800000, 1442055600000, 35, 3, 1 ]
+  - [ 1442055600000, "IL", 4, 6617, 1442048400000, 1442059200000, 35, 3, 1 ]
+  - [ 1442059200000, "IL", 1205, 6617, 1442052000000, 1442062800000, 35, 3, 1 ]
+  - [ 1442062800000, "IL", 180, 6617, 1442055600000, 1442066400000, 35, 3, 2 ]
+  - [ 1442066400000, "IL", 3, 6617, 1442059200000, 1442070000000, 35, 3, 2 ]
+  - [ 1442070000000, "IL", 49, 6617, 1442062800000, 1442073600000, 35, 3, 2 ]
+  - [ 1442073600000, "IL", 31, 6617, 1442066400000, 1442077200000, 35, 3, 2 ]
+  - [ 1442077200000, "IL", 187, 6617, 1442070000000, 1442080800000, 35, 3, 2 ]
+  - [ 1442080800000, "IL", 88, 6617, 1442073600000, 1442084400000, 35, 3, 3 ]
+  - [ 1442084400000, "IL", 1137, 6617, 1442077200000, 1442091600000, 35, 3, 3 ]
+  - [ 1442091600000, "IL", 707, 6617, 1442080800000, 1442095200000, 35, 3, 3 ]
+  - [ 1442095200000, "IL", 0, 6617, 1442084400000, 1442098800000, 35, 3, 3 ]
+  - [ 1442098800000, "IL", 3, 6617, 1442091600000, null, 35, 3, 3 ]
+  - [ 1442016000000, "IN", 1, 29166, null, 1442019600000, 1, 4, 1 ]
+  - [ 1442019600000, "IN", 38, 29166, null, 1442023200000, 1, 4, 1 ]
+  - [ 1442023200000, "IN", -142, 29166, 1442016000000, 1442026800000, 1, 4, 1 ]
+  - [ 1442026800000, "IN", 974, 29166, 1442019600000, 1442030400000, 1, 4, 1 ]
+  - [ 1442030400000, "IN", 1448, 29166, 1442023200000, 1442034000000, 1, 4, 1 ]
+  - [ 1442034000000, "IN", 1350, 29166, 1442026800000, 1442037600000, 1, 4, 1 ]
+  - [ 1442037600000, "IN", 135, 29166, 1442030400000, 1442041200000, 1, 4, 1 ]
+  - [ 1442041200000, "IN", 80, 29166, 1442034000000, 1442044800000, 1, 4, 1 ]
+  - [ 1442044800000, "IN", 2677, 29166, 1442037600000, 1442048400000, 1, 4, 2 ]
+  - [ 1442048400000, "IN", 262, 29166, 1442041200000, 1442052000000, 1, 4, 2 ]
+  - [ 1442052000000, "IN", 534, 29166, 1442044800000, 1442055600000, 1, 4, 2 ]
+  - [ 1442055600000, "IN", 166, 29166, 1442048400000, 1442059200000, 1, 4, 2 ]
+  - [ 1442059200000, "IN", 708, 29166, 1442052000000, 1442062800000, 1, 4, 2 ]
+  - [ 1442062800000, "IN", 1547, 29166, 1442055600000, 1442066400000, 1, 4, 2 ]
+  - [ 1442066400000, "IN", 116, 29166, 1442059200000, 1442070000000, 1, 4, 2 ]
+  - [ 1442070000000, "IN", 12091, 29166, 1442062800000, 1442073600000, 1, 4, 3 ]
+  - [ 1442073600000, "IN", 1170, 29166, 1442066400000, 1442077200000, 1, 4, 3 ]
+  - [ 1442077200000, "IN", 5699, 29166, 1442070000000, 1442080800000, 1, 4, 3 ]
+  - [ 1442080800000, "IN", 0, 29166, 1442073600000, 1442084400000, 1, 4, 3 ]
+  - [ 1442084400000, "IN", 187, 29166, 1442077200000, 1442088000000, 1, 4, 3 ]
+  - [ 1442088000000, "IN", 121, 29166, 1442080800000, 1442095200000, 1, 4, 3 ]
+  - [ 1442095200000, "IN", 4, 29166, 1442084400000, null, 1, 4, 3 ]
+  - [ 1442041200000, "IQ", -1, 3, null, 1442044800000, -1, -2, 1 ]
+  - [ 1442044800000, "IQ", 6, 3, null, 1442052000000, -1, -2, 1 ]
+  - [ 1442052000000, "IQ", 0, 3, 1442041200000, 1442095200000, -1, -2, 2 ]
+  - [ 1442095200000, "IQ", -2, 3, 1442044800000, null, -1, -2, 3 ]
+  - [ 1442026800000, "IR", 0, 2698, null, 1442030400000, 0, 0, 1 ]
+  - [ 1442030400000, "IR", 375, 2698, null, 1442034000000, 0, 0, 1 ]
+  - [ 1442034000000, "IR", -8, 2698, 1442026800000, 1442041200000, 0, 0, 1 ]
+  - [ 1442041200000, "IR", -79, 2698, 1442030400000, 1442044800000, 0, 0, 1 ]
+  - [ 1442044800000, "IR", 306, 2698, 1442034000000, 1442052000000, 0, 0, 1 ]
+  - [ 1442052000000, "IR", 155, 2698, 1442041200000, 1442055600000, 0, 0, 2 ]
+  - [ 1442055600000, "IR", -124, 2698, 1442044800000, 1442059200000, 0, 0, 2 ]
+  - [ 1442059200000, "IR", 1455, 2698, 1442052000000, 1442073600000, 0, 0, 2 ]
+  - [ 1442073600000, "IR", -193, 2698, 1442055600000, 1442077200000, 0, 0, 2 ]
+  - [ 1442077200000, "IR", -34, 2698, 1442059200000, 1442080800000, 0, 0, 3 ]
+  - [ 1442080800000, "IR", 131, 2698, 1442073600000, 1442088000000, 0, 0, 3 ]
+  - [ 1442088000000, "IR", 714, 2698, 1442077200000, 1442091600000, 0, 0, 3 ]
+  - [ 1442091600000, "IR", 0, 2698, 1442080800000, null, 0, 0, 3 ]
+  - [ 1442016000000, "IT", 0, 39091, null, 1442019600000, 0, 565, 1 ]
+  - [ 1442019600000, "IT", 183, 39091, null, 1442023200000, 0, 565, 1 ]
+  - [ 1442023200000, "IT", 111, 39091, 1442016000000, 1442026800000, 0, 565, 1 ]
+  - [ 1442026800000, "IT", 222, 39091, 1442019600000, 1442030400000, 0, 565, 1 ]
+  - [ 1442030400000, "IT", -17, 39091, 1442023200000, 1442034000000, 0, 565, 1 ]
+  - [ 1442034000000, "IT", 1006, 39091, 1442026800000, 1442037600000, 0, 565, 1 ]
+  - [ 1442037600000, "IT", -9, 39091, 1442030400000, 1442041200000, 0, 565, 1 ]
+  - [ 1442041200000, "IT", 20, 39091, 1442034000000, 1442044800000, 0, 565, 1 ]
+  - [ 1442044800000, "IT", 1483, 39091, 1442037600000, 1442048400000, 0, 565, 2 ]
+  - [ 1442048400000, "IT", 676, 39091, 1442041200000, 1442052000000, 0, 565, 2 ]
+  - [ 1442052000000, "IT", 1880, 39091, 1442044800000, 1442055600000, 0, 565, 2 ]
+  - [ 1442055600000, "IT", 6240, 39091, 1442048400000, 1442059200000, 0, 565, 2 ]
+  - [ 1442059200000, "IT", 542, 39091, 1442052000000, 1442062800000, 0, 565, 2 ]
+  - [ 1442062800000, "IT", 1938, 39091, 1442055600000, 1442066400000, 0, 565, 2 ]
+  - [ 1442066400000, "IT", 4155, 39091, 1442059200000, 1442070000000, 0, 565, 2 ]
+  - [ 1442070000000, "IT", 81, 39091, 1442062800000, 1442073600000, 0, 565, 2 ]
+  - [ 1442073600000, "IT", 2586, 39091, 1442066400000, 1442077200000, 0, 565, 3 ]
+  - [ 1442077200000, "IT", 2188, 39091, 1442070000000, 1442080800000, 0, 565, 3 ]
+  - [ 1442080800000, "IT", 5544, 39091, 1442073600000, 1442084400000, 0, 565, 3 ]
+  - [ 1442084400000, "IT", 2660, 39091, 1442077200000, 1442088000000, 0, 565, 3 ]
+  - [ 1442088000000, "IT", 3746, 39091, 1442080800000, 1442091600000, 0, 565, 3 ]
+  - [ 1442091600000, "IT", 351, 39091, 1442084400000, 1442095200000, 0, 565, 3 ]
+  - [ 1442095200000, "IT", 2940, 39091, 1442088000000, 1442098800000, 0, 565, 3 ]
+  - [ 1442098800000, "IT", 565, 39091, 1442091600000, null, 0, 565, 3 ]
+  - [ 1442070000000, "JM", 30, 30, null, null, 30, 30, 1 ]
+  - [ 1442055600000, "JO", -2, 2, null, 1442059200000, -2, 4, 1 ]
+  - [ 1442059200000, "JO", 0, 2, null, 1442080800000, -2, 4, 2 ]
+  - [ 1442080800000, "JO", 4, 2, 1442055600000, null, -2, 4, 3 ]
+  - [ 1442016000000, "JP", -113, 20378, null, 1442019600000, -113, -6, 1 ]
+  - [ 1442019600000, "JP", 2002, 20378, null, 1442023200000, -113, -6, 1 ]
+  - [ 1442023200000, "JP", 1959, 20378, 1442016000000, 1442026800000, -113, -6, 1 ]
+  - [ 1442026800000, "JP", 1035, 20378, 1442019600000, 1442030400000, -113, -6, 1 ]
+  - [ 1442030400000, "JP", 805, 20378, 1442023200000, 1442034000000, -113, -6, 1 ]
+  - [ 1442034000000, "JP", 910, 20378, 1442026800000, 1442037600000, -113, -6, 1 ]
+  - [ 1442037600000, "JP", 2181, 20378, 1442030400000, 1442041200000, -113, -6, 1 ]
+  - [ 1442041200000, "JP", 1373, 20378, 1442034000000, 1442044800000, -113, -6, 1 ]
+  - [ 1442044800000, "JP", 1569, 20378, 1442037600000, 1442048400000, -113, -6, 2 ]
+  - [ 1442048400000, "JP", 1981, 20378, 1442041200000, 1442052000000, -113, -6, 2 ]
+  - [ 1442052000000, "JP", 2789, 20378, 1442044800000, 1442055600000, -113, -6, 2 ]
+  - [ 1442055600000, "JP", 998, 20378, 1442048400000, 1442059200000, -113, -6, 2 ]
+  - [ 1442059200000, "JP", -85, 20378, 1442052000000, 1442062800000, -113, -6, 2 ]
+  - [ 1442062800000, "JP", 803, 20378, 1442055600000, 1442066400000, -113, -6, 2 ]
+  - [ 1442066400000, "JP", 167, 20378, 1442059200000, 1442070000000, -113, -6, 2 ]
+  - [ 1442070000000, "JP", 79, 20378, 1442062800000, 1442073600000, -113, -6, 2 ]
+  - [ 1442073600000, "JP", 1162, 20378, 1442066400000, 1442077200000, -113, -6, 3 ]
+  - [ 1442077200000, "JP", 51, 20378, 1442070000000, 1442080800000, -113, -6, 3 ]
+  - [ 1442080800000, "JP", 420, 20378, 1442073600000, 1442084400000, -113, -6, 3 ]
+  - [ 1442084400000, "JP", 13, 20378, 1442077200000, 1442088000000, -113, -6, 3 ]
+  - [ 1442088000000, "JP", 57, 20378, 1442080800000, 1442091600000, -113, -6, 3 ]
+  - [ 1442091600000, "JP", 228, 20378, 1442084400000, 1442095200000, -113, -6, 3 ]
+  - [ 1442095200000, "JP", 0, 20378, 1442088000000, 1442098800000, -113, -6, 3 ]
+  - [ 1442098800000, "JP", -6, 20378, 1442091600000, null, -113, -6, 3 ]
+  - [ 1442044800000, "KE", -1, -1, null, null, -1, -1, 1 ]
+  - [ 1442073600000, "KG", 6, 6, null, null, 6, 6, 1 ]
+  - [ 1442016000000, "KR", 1024, 13597, null, 1442019600000, 1024, -36, 1 ]
+  - [ 1442019600000, "KR", 445, 13597, null, 1442023200000, 1024, -36, 1 ]
+  - [ 1442023200000, "KR", 319, 13597, 1442016000000, 1442026800000, 1024, -36, 1 ]
+  - [ 1442026800000, "KR", -179, 13597, 1442019600000, 1442030400000, 1024, -36, 1 ]
+  - [ 1442030400000, "KR", 1035, 13597, 1442023200000, 1442034000000, 1024, -36, 1 ]
+  - [ 1442034000000, "KR", 434, 13597, 1442026800000, 1442037600000, 1024, -36, 1 ]
+  - [ 1442037600000, "KR", 26, 13597, 1442030400000, 1442041200000, 1024, -36, 1 ]
+  - [ 1442041200000, "KR", 20, 13597, 1442034000000, 1442044800000, 1024, -36, 1 ]
+  - [ 1442044800000, "KR", 829, 13597, 1442037600000, 1442048400000, 1024, -36, 2 ]
+  - [ 1442048400000, "KR", -374, 13597, 1442041200000, 1442052000000, 1024, -36, 2 ]
+  - [ 1442052000000, "KR", -3, 13597, 1442044800000, 1442055600000, 1024, -36, 2 ]
+  - [ 1442055600000, "KR", 3640, 13597, 1442048400000, 1442059200000, 1024, -36, 2 ]
+  - [ 1442059200000, "KR", 208, 13597, 1442052000000, 1442062800000, 1024, -36, 2 ]
+  - [ 1442062800000, "KR", 1096, 13597, 1442055600000, 1442066400000, 1024, -36, 2 ]
+  - [ 1442066400000, "KR", 3299, 13597, 1442059200000, 1442070000000, 1024, -36, 2 ]
+  - [ 1442070000000, "KR", 222, 13597, 1442062800000, 1442077200000, 1024, -36, 3 ]
+  - [ 1442077200000, "KR", -40, 13597, 1442066400000, 1442080800000, 1024, -36, 3 ]
+  - [ 1442080800000, "KR", -33, 13597, 1442070000000, 1442084400000, 1024, -36, 3 ]
+  - [ 1442084400000, "KR", 314, 13597, 1442077200000, 1442088000000, 1024, -36, 3 ]
+  - [ 1442088000000, "KR", 524, 13597, 1442080800000, 1442095200000, 1024, -36, 3 ]
+  - [ 1442095200000, "KR", 827, 13597, 1442084400000, 1442098800000, 1024, -36, 3 ]
+  - [ 1442098800000, "KR", -36, 13597, 1442088000000, null, 1024, -36, 3 ]
+  - [ 1442055600000, "KW", -2, 1778, null, 1442070000000, -2, -33, 1 ]
+  - [ 1442070000000, "KW", 1815, 1778, null, 1442077200000, -2, -33, 1 ]
+  - [ 1442077200000, "KW", -2, 1778, 1442055600000, 1442080800000, -2, -33, 2 ]
+  - [ 1442080800000, "KW", -33, 1778, 1442070000000, null, -2, -33, 3 ]
+  - [ 1442034000000, "KZ", 161, 1261, null, 1442044800000, 161, 91, 1 ]
+  - [ 1442044800000, "KZ", 401, 1261, null, 1442048400000, 161, 91, 1 ]
+  - [ 1442048400000, "KZ", 439, 1261, 1442034000000, 1442052000000, 161, 91, 1 ]
+  - [ 1442052000000, "KZ", 412, 1261, 1442044800000, 1442055600000, 161, 91, 1 ]
+  - [ 1442055600000, "KZ", 63, 1261, 1442048400000, 1442059200000, 161, 91, 2 ]
+  - [ 1442059200000, "KZ", 33, 1261, 1442052000000, 1442062800000, 161, 91, 2 ]
+  - [ 1442062800000, "KZ", 0, 1261, 1442055600000, 1442066400000, 161, 91, 2 ]
+  - [ 1442066400000, "KZ", 0, 1261, 1442059200000, 1442077200000, 161, 91, 2 ]
+  - [ 1442077200000, "KZ", -317, 1261, 1442062800000, 1442084400000, 161, 91, 3 ]
+  - [ 1442084400000, "KZ", -22, 1261, 1442066400000, 1442095200000, 161, 91, 3 ]
+  - [ 1442095200000, "KZ", 91, 1261, 1442077200000, null, 161, 91, 3 ]
+  - [ 1442055600000, "LB", -67, -67, null, null, -67, -67, 1 ]
+  - [ 1442026800000, "LK", 79, 131, null, 1442048400000, 79, -3, 1 ]
+  - [ 1442048400000, "LK", 8, 131, null, 1442052000000, 79, -3, 1 ]
+  - [ 1442052000000, "LK", 47, 131, 1442026800000, 1442084400000, 79, -3, 2 ]
+  - [ 1442084400000, "LK", -3, 131, 1442048400000, null, 79, -3, 3 ]
+  - [ 1442080800000, "LT", 12, -12, null, 1442098800000, 12, -24, 1 ]
+  - [ 1442098800000, "LT", -24, -12, null, null, 12, -24, 2 ]
+  - [ 1442059200000, "LU", 79, 606, null, 1442066400000, 79, 2, 1 ]
+  - [ 1442066400000, "LU", 0, 606, null, 1442077200000, 79, 2, 1 ]
+  - [ 1442077200000, "LU", 525, 606, 1442059200000, 1442095200000, 79, 2, 2 ]
+  - [ 1442095200000, "LU", 2, 606, 1442066400000, null, 79, 2, 3 ]
+  - [ 1442095200000, "LV", 0, 0, null, null, 0, 0, 1 ]
+  - [ 1442019600000, "MA", -1, 229, null, 1442055600000, -1, 8, 1 ]
+  - [ 1442055600000, "MA", 23, 229, null, 1442059200000, -1, 8, 1 ]
+  - [ 1442059200000, "MA", -56, 229, 1442019600000, 1442062800000, -1, 8, 1 ]
+  - [ 1442062800000, "MA", 0, 229, 1442055600000, 1442077200000, -1, 8, 2 ]
+  - [ 1442077200000, "MA", 250, 229, 1442059200000, 1442080800000, -1, 8, 2 ]
+  - [ 1442080800000, "MA", 5, 229, 1442062800000, 1442098800000, -1, 8, 3 ]
+  - [ 1442098800000, "MA", 8, 229, 1442077200000, null, -1, 8, 3 ]
+  - [ 1442077200000, "MD", 6916, 6916, null, null, 6916, 6916, 1 ]
+  - [ 1442073600000, "ME", 0, 0, null, null, 0, 0, 1 ]
+  - [ 1442052000000, "MH", 40, 40, null, null, 40, 40, 1 ]
+  - [ 1442077200000, "MK", -72, -72, null, null, -72, -72, 1 ]
+  - [ 1442070000000, "MM", 3, 28, null, 1442073600000, 3, 25, 1 ]
+  - [ 1442073600000, "MM", 25, 28, null, null, 3, 25, 2 ]
+  - [ 1442034000000, "MO", 30, 48, null, 1442070000000, 30, 18, 1 ]
+  - [ 1442070000000, "MO", 18, 48, null, null, 30, 18, 2 ]
+  - [ 1442080800000, "MR", 10, 10, null, null, 10, 10, 1 ]
+  - [ 1442048400000, "MT", -1, -1, null, null, -1, -1, 1 ]
+  - [ 1442073600000, "MV", -3, -3, null, null, -3, -3, 1 ]
+  - [ 1442016000000, "MX", -67, 10472, null, 1442023200000, -67, 28, 1 ]
+  - [ 1442023200000, "MX", 549, 10472, null, 1442026800000, -67, 28, 1 ]
+  - [ 1442026800000, "MX", 3642, 10472, 1442016000000, 1442030400000, -67, 28, 1 ]
+  - [ 1442030400000, "MX", 373, 10472, 1442023200000, 1442034000000, -67, 28, 1 ]
+  - [ 1442034000000, "MX", 944, 10472, 1442026800000, 1442037600000, -67, 28, 1 ]
+  - [ 1442037600000, "MX", 4, 10472, 1442030400000, 1442041200000, -67, 28, 1 ]
+  - [ 1442041200000, "MX", -294, 10472, 1442034000000, 1442066400000, -67, 28, 2 ]
+  - [ 1442066400000, "MX", -1, 10472, 1442037600000, 1442070000000, -67, 28, 2 ]
+  - [ 1442070000000, "MX", -1, 10472, 1442041200000, 1442073600000, -67, 28, 2 ]
+  - [ 1442073600000, "MX", -21, 10472, 1442066400000, 1442077200000, -67, 28, 2 ]
+  - [ 1442077200000, "MX", 3874, 10472, 1442070000000, 1442080800000, -67, 28, 2 ]
+  - [ 1442080800000, "MX", -376, 10472, 1442073600000, 1442084400000, -67, 28, 2 ]
+  - [ 1442084400000, "MX", 981, 10472, 1442077200000, 1442088000000, -67, 28, 3 ]
+  - [ 1442088000000, "MX", 494, 10472, 1442080800000, 1442091600000, -67, 28, 3 ]
+  - [ 1442091600000, "MX", 799, 10472, 1442084400000, 1442095200000, -67, 28, 3 ]
+  - [ 1442095200000, "MX", -456, 10472, 1442088000000, 1442098800000, -67, 28, 3 ]
+  - [ 1442098800000, "MX", 28, 10472, 1442091600000, null, -67, 28, 3 ]
+  - [ 1442019600000, "MY", -7, 3207, null, 1442030400000, -7, 739, 1 ]
+  - [ 1442030400000, "MY", -3, 3207, null, 1442034000000, -7, 739, 1 ]
+  - [ 1442034000000, "MY", 1028, 3207, 1442019600000, 1442041200000, -7, 739, 1 ]
+  - [ 1442041200000, "MY", 935, 3207, 1442030400000, 1442044800000, -7, 739, 1 ]
+  - [ 1442044800000, "MY", -127, 3207, 1442034000000, 1442048400000, -7, 739, 2 ]
+  - [ 1442048400000, "MY", 649, 3207, 1442041200000, 1442055600000, -7, 739, 2 ]
+  - [ 1442055600000, "MY", 1, 3207, 1442044800000, 1442059200000, -7, 739, 2 ]
+  - [ 1442059200000, "MY", 0, 3207, 1442048400000, 1442066400000, -7, 739, 2 ]
+  - [ 1442066400000, "MY", 1, 3207, 1442055600000, 1442073600000, -7, 739, 3 ]
+  - [ 1442073600000, "MY", 1, 3207, 1442059200000, 1442077200000, -7, 739, 3 ]
+  - [ 1442077200000, "MY", -10, 3207, 1442066400000, 1442098800000, -7, 739, 3 ]
+  - [ 1442098800000, "MY", 739, 3207, 1442073600000, null, -7, 739, 3 ]
+  - [ 1442052000000, "NG", 208, 214, null, 1442070000000, 208, 6, 1 ]
+  - [ 1442070000000, "NG", 6, 214, null, null, 208, 6, 2 ]
+  - [ 1442034000000, "NL", 0, 12162, null, 1442044800000, 0, 4, 1 ]
+  - [ 1442044800000, "NL", 16, 12162, null, 1442048400000, 0, 4, 1 ]
+  - [ 1442048400000, "NL", 1303, 12162, 1442034000000, 1442052000000, 0, 4, 1 ]
+  - [ 1442052000000, "NL", 53, 12162, 1442044800000, 1442055600000, 0, 4, 1 ]
+  - [ 1442055600000, "NL", 105, 12162, 1442048400000, 1442059200000, 0, 4, 1 ]
+  - [ 1442059200000, "NL", 206, 12162, 1442052000000, 1442062800000, 0, 4, 1 ]
+  - [ 1442062800000, "NL", -30, 12162, 1442055600000, 1442066400000, 0, 4, 2 ]
+  - [ 1442066400000, "NL", 61, 12162, 1442059200000, 1442070000000, 0, 4, 2 ]
+  - [ 1442070000000, "NL", -84, 12162, 1442062800000, 1442073600000, 0, 4, 2 ]
+  - [ 1442073600000, "NL", 166, 12162, 1442066400000, 1442077200000, 0, 4, 2 ]
+  - [ 1442077200000, "NL", 878, 12162, 1442070000000, 1442080800000, 0, 4, 2 ]
+  - [ 1442080800000, "NL", 8947, 12162, 1442073600000, 1442084400000, 0, 4, 2 ]
+  - [ 1442084400000, "NL", 436, 12162, 1442077200000, 1442088000000, 0, 4, 3 ]
+  - [ 1442088000000, "NL", 12, 12162, 1442080800000, 1442091600000, 0, 4, 3 ]
+  - [ 1442091600000, "NL", 19, 12162, 1442084400000, 1442095200000, 0, 4, 3 ]
+  - [ 1442095200000, "NL", 70, 12162, 1442088000000, 1442098800000, 0, 4, 3 ]
+  - [ 1442098800000, "NL", 4, 12162, 1442091600000, null, 0, 4, 3 ]
+  - [ 1442019600000, "NO", 48, 432, null, 1442048400000, 48, 2, 1 ]
+  - [ 1442048400000, "NO", -447, 432, null, 1442052000000, 48, 2, 1 ]
+  - [ 1442052000000, "NO", 447, 432, 1442019600000, 1442055600000, 48, 2, 1 ]
+  - [ 1442055600000, "NO", 29, 432, 1442048400000, 1442066400000, 48, 2, 1 ]
+  - [ 1442066400000, "NO", 71, 432, 1442052000000, 1442073600000, 48, 2, 2 ]
+  - [ 1442073600000, "NO", 222, 432, 1442055600000, 1442080800000, 48, 2, 2 ]
+  - [ 1442080800000, "NO", 31, 432, 1442066400000, 1442088000000, 48, 2, 2 ]
+  - [ 1442088000000, "NO", 15, 432, 1442073600000, 1442091600000, 48, 2, 2 ]
+  - [ 1442091600000, "NO", 15, 432, 1442080800000, 1442095200000, 48, 2, 3 ]
+  - [ 1442095200000, "NO", -1, 432, 1442088000000, 1442098800000, 48, 2, 3 ]
+  - [ 1442098800000, "NO", 2, 432, 1442091600000, null, 48, 2, 3 ]
+  - [ 1442048400000, "NP", 61, 61, null, null, 61, 61, 1 ]
+  - [ 1442019600000, "NZ", 28, 1693, null, 1442026800000, 28, -2, 1 ]
+  - [ 1442026800000, "NZ", 635, 1693, null, 1442037600000, 28, -2, 1 ]
+  - [ 1442037600000, "NZ", 66, 1693, 1442019600000, 1442048400000, 28, -2, 1 ]
+  - [ 1442048400000, "NZ", 189, 1693, 1442026800000, 1442059200000, 28, -2, 2 ]
+  - [ 1442059200000, "NZ", 428, 1693, 1442037600000, 1442084400000, 28, -2, 2 ]
+  - [ 1442084400000, "NZ", -52, 1693, 1442048400000, 1442088000000, 28, -2, 2 ]
+  - [ 1442088000000, "NZ", 405, 1693, 1442059200000, 1442095200000, 28, -2, 3 ]
+  - [ 1442095200000, "NZ", -4, 1693, 1442084400000, 1442098800000, 28, -2, 3 ]
+  - [ 1442098800000, "NZ", -2, 1693, 1442088000000, null, 28, -2, 3 ]
+  - [ 1442052000000, "OM", 0, 0, null, null, 0, 0, 1 ]
+  - [ 1442026800000, "PA", 0, 0, null, null, 0, 0, 1 ]
+  - [ 1442019600000, "PE", 523, 2134, null, 1442023200000, 523, 1861, 1 ]
+  - [ 1442023200000, "PE", 26, 2134, null, 1442026800000, 523, 1861, 1 ]
+  - [ 1442026800000, "PE", -12, 2134, 1442019600000, 1442062800000, 523, 1861, 1 ]
+  - [ 1442062800000, "PE", -12, 2134, 1442023200000, 1442077200000, 523, 1861, 2 ]
+  - [ 1442077200000, "PE", -163, 2134, 1442026800000, 1442080800000, 523, 1861, 2 ]
+  - [ 1442080800000, "PE", -2, 2134, 1442062800000, 1442084400000, 523, 1861, 2 ]
+  - [ 1442084400000, "PE", -68, 2134, 1442077200000, 1442095200000, 523, 1861, 3 ]
+  - [ 1442095200000, "PE", -19, 2134, 1442080800000, 1442098800000, 523, 1861, 3 ]
+  - [ 1442098800000, "PE", 1861, 2134, 1442084400000, null, 523, 1861, 3 ]
+  - [ 1442019600000, "PH", 6, 6613, null, 1442023200000, 6, 8, 1 ]
+  - [ 1442023200000, "PH", 459, 6613, null, 1442026800000, 6, 8, 1 ]
+  - [ 1442026800000, "PH", 910, 6613, 1442019600000, 1442030400000, 6, 8, 1 ]
+  - [ 1442030400000, "PH", 26, 6613, 1442023200000, 1442034000000, 6, 8, 1 ]
+  - [ 1442034000000, "PH", 59, 6613, 1442026800000, 1442037600000, 6, 8, 1 ]
+  - [ 1442037600000, "PH", 17, 6613, 1442030400000, 1442041200000, 6, 8, 1 ]
+  - [ 1442041200000, "PH", 0, 6613, 1442034000000, 1442044800000, 6, 8, 1 ]
+  - [ 1442044800000, "PH", 55, 6613, 1442037600000, 1442048400000, 6, 8, 2 ]
+  - [ 1442048400000, "PH", 62, 6613, 1442041200000, 1442052000000, 6, 8, 2 ]
+  - [ 1442052000000, "PH", 22, 6613, 1442044800000, 1442055600000, 6, 8, 2 ]
+  - [ 1442055600000, "PH", 1969, 6613, 1442048400000, 1442059200000, 6, 8, 2 ]
+  - [ 1442059200000, "PH", 273, 6613, 1442052000000, 1442062800000, 6, 8, 2 ]
+  - [ 1442062800000, "PH", 171, 6613, 1442055600000, 1442066400000, 6, 8, 2 ]
+  - [ 1442066400000, "PH", 1880, 6613, 1442059200000, 1442070000000, 6, 8, 2 ]
+  - [ 1442070000000, "PH", 34, 6613, 1442062800000, 1442073600000, 6, 8, 3 ]
+  - [ 1442073600000, "PH", -227, 6613, 1442066400000, 1442077200000, 6, 8, 3 ]
+  - [ 1442077200000, "PH", 2, 6613, 1442070000000, 1442080800000, 6, 8, 3 ]
+  - [ 1442080800000, "PH", 32, 6613, 1442073600000, 1442084400000, 6, 8, 3 ]
+  - [ 1442084400000, "PH", 39, 6613, 1442077200000, 1442091600000, 6, 8, 3 ]
+  - [ 1442091600000, "PH", 816, 6613, 1442080800000, 1442098800000, 6, 8, 3 ]
+  - [ 1442098800000, "PH", 8, 6613, 1442084400000, null, 6, 8, 3 ]
+  - [ 1442019600000, "PK", 335, 641, null, 1442026800000, 335, 43, 1 ]
+  - [ 1442026800000, "PK", 101, 641, null, 1442037600000, 335, 43, 1 ]
+  - [ 1442037600000, "PK", 100, 641, 1442019600000, 1442041200000, 335, 43, 1 ]
+  - [ 1442041200000, "PK", 24, 641, 1442026800000, 1442048400000, 335, 43, 2 ]
+  - [ 1442048400000, "PK", 15, 641, 1442037600000, 1442062800000, 335, 43, 2 ]
+  - [ 1442062800000, "PK", 23, 641, 1442041200000, 1442070000000, 335, 43, 3 ]
+  - [ 1442070000000, "PK", 43, 641, 1442048400000, null, 335, 43, 3 ]
+  - [ 1442037600000, "PL", 95, 9815, null, 1442041200000, 95, -9, 1 ]
+  - [ 1442041200000, "PL", 281, 9815, null, 1442044800000, 95, -9, 1 ]
+  - [ 1442044800000, "PL", 319, 9815, 1442037600000, 1442048400000, 95, -9, 1 ]
+  - [ 1442048400000, "PL", 366, 9815, 1442041200000, 1442052000000, 95, -9, 1 ]
+  - [ 1442052000000, "PL", 330, 9815, 1442044800000, 1442055600000, 95, -9, 1 ]
+  - [ 1442055600000, "PL", 410, 9815, 1442048400000, 1442059200000, 95, -9, 1 ]
+  - [ 1442059200000, "PL", 199, 9815, 1442052000000, 1442062800000, 95, -9, 2 ]
+  - [ 1442062800000, "PL", 4171, 9815, 1442055600000, 1442066400000, 95, -9, 2 ]
+  - [ 1442066400000, "PL", 34, 9815, 1442059200000, 1442070000000, 95, -9, 2 ]
+  - [ 1442070000000, "PL", 146, 9815, 1442062800000, 1442073600000, 95, -9, 2 ]
+  - [ 1442073600000, "PL", 30, 9815, 1442066400000, 1442077200000, 95, -9, 2 ]
+  - [ 1442077200000, "PL", 324, 9815, 1442070000000, 1442080800000, 95, -9, 2 ]
+  - [ 1442080800000, "PL", 7, 9815, 1442073600000, 1442084400000, 95, -9, 3 ]
+  - [ 1442084400000, "PL", 13, 9815, 1442077200000, 1442088000000, 95, -9, 3 ]
+  - [ 1442088000000, "PL", 346, 9815, 1442080800000, 1442091600000, 95, -9, 3 ]
+  - [ 1442091600000, "PL", 902, 9815, 1442084400000, 1442095200000, 95, -9, 3 ]
+  - [ 1442095200000, "PL", 1851, 9815, 1442088000000, 1442098800000, 95, -9, 3 ]
+  - [ 1442098800000, "PL", -9, 9815, 1442091600000, null, 95, -9, 3 ]
+  - [ 1442026800000, "PR", 22, 23, null, 1442030400000, 22, 29, 1 ]
+  - [ 1442030400000, "PR", 2, 23, null, 1442059200000, 22, 29, 1 ]
+  - [ 1442059200000, "PR", -35, 23, 1442026800000, 1442077200000, 22, 29, 2 ]
+  - [ 1442077200000, "PR", 5, 23, 1442030400000, 1442095200000, 22, 29, 2 ]
+  - [ 1442095200000, "PR", 29, 23, 1442059200000, null, 22, 29, 3 ]
+  - [ 1442019600000, "PT", 172, 4037, null, 1442044800000, 172, 2, 1 ]
+  - [ 1442044800000, "PT", 11, 4037, null, 1442052000000, 172, 2, 1 ]
+  - [ 1442052000000, "PT", 102, 4037, 1442019600000, 1442066400000, 172, 2, 1 ]
+  - [ 1442066400000, "PT", 12, 4037, 1442044800000, 1442070000000, 172, 2, 1 ]
+  - [ 1442070000000, "PT", 3470, 4037, 1442052000000, 1442077200000, 172, 2, 2 ]
+  - [ 1442077200000, "PT", -75, 4037, 1442066400000, 1442080800000, 172, 2, 2 ]
+  - [ 1442080800000, "PT", -79, 4037, 1442070000000, 1442088000000, 172, 2, 2 ]
+  - [ 1442088000000, "PT", 403, 4037, 1442077200000, 1442095200000, 172, 2, 3 ]
+  - [ 1442095200000, "PT", 19, 4037, 1442080800000, 1442098800000, 172, 2, 3 ]
+  - [ 1442098800000, "PT", 2, 4037, 1442088000000, null, 172, 2, 3 ]
+  - [ 1442019600000, "PY", 1, 634, null, 1442080800000, 1, 628, 1 ]
+  - [ 1442080800000, "PY", 5, 634, null, 1442084400000, 1, 628, 2 ]
+  - [ 1442084400000, "PY", 628, 634, 1442019600000, null, 1, 628, 3 ]
+  - [ 1442041200000, "QA", 13, 13, null, null, 13, 13, 1 ]
+  - [ 1442034000000, "RO", 68, 2893, null, 1442041200000, 68, 824, 1 ]
+  - [ 1442041200000, "RO", 845, 2893, null, 1442044800000, 68, 824, 1 ]
+  - [ 1442044800000, "RO", 284, 2893, 1442034000000, 1442052000000, 68, 824, 1 ]
+  - [ 1442052000000, "RO", 319, 2893, 1442041200000, 1442055600000, 68, 824, 1 ]
+  - [ 1442055600000, "RO", 26, 2893, 1442044800000, 1442062800000, 68, 824, 2 ]
+  - [ 1442062800000, "RO", 541, 2893, 1442052000000, 1442070000000, 68, 824, 2 ]
+  - [ 1442070000000, "RO", -29, 2893, 1442055600000, 1442073600000, 68, 824, 2 ]
+  - [ 1442073600000, "RO", 15, 2893, 1442062800000, 1442091600000, 68, 824, 3 ]
+  - [ 1442091600000, "RO", 0, 2893, 1442070000000, 1442095200000, 68, 824, 3 ]
+  - [ 1442095200000, "RO", 824, 2893, 1442073600000, null, 68, 824, 3 ]
+  - [ 1442019600000, "RS", 6, 906, null, 1442062800000, 6, -15, 1 ]
+  - [ 1442062800000, "RS", 13, 906, null, 1442066400000, 6, -15, 1 ]
+  - [ 1442066400000, "RS", 0, 906, 1442019600000, 1442073600000, 6, -15, 1 ]
+  - [ 1442073600000, "RS", 813, 906, 1442062800000, 1442080800000, 6, -15, 2 ]
+  - [ 1442080800000, "RS", 0, 906, 1442066400000, 1442084400000, 6, -15, 2 ]
+  - [ 1442084400000, "RS", 89, 906, 1442073600000, 1442091600000, 6, -15, 3 ]
+  - [ 1442091600000, "RS", -15, 906, 1442080800000, null, 6, -15, 3 ]
+  - [ 1442019600000, "RU", 2214, 48104, null, 1442023200000, 2214, 12098, 1 ]
+  - [ 1442023200000, "RU", 299, 48104, null, 1442026800000, 2214, 12098, 1 ]
+  - [ 1442026800000, "RU", 0, 48104, 1442019600000, 1442030400000, 2214, 12098, 1 ]
+  - [ 1442030400000, "RU", 76, 48104, 1442023200000, 1442034000000, 2214, 12098, 1 ]
+  - [ 1442034000000, "RU", 658, 48104, 1442026800000, 1442037600000, 2214, 12098, 1 ]
+  - [ 1442037600000, "RU", -324, 48104, 1442030400000, 1442041200000, 2214, 12098, 1 ]
+  - [ 1442041200000, "RU", 580, 48104, 1442034000000, 1442044800000, 2214, 12098, 1 ]
+  - [ 1442044800000, "RU", 2564, 48104, 1442037600000, 1442048400000, 2214, 12098, 1 ]
+  - [ 1442048400000, "RU", 1027, 48104, 1442041200000, 1442052000000, 2214, 12098, 2 ]
+  - [ 1442052000000, "RU", 1214, 48104, 1442044800000, 1442055600000, 2214, 12098, 2 ]
+  - [ 1442055600000, "RU", 499, 48104, 1442048400000, 1442059200000, 2214, 12098, 2 ]
+  - [ 1442059200000, "RU", 3902, 48104, 1442052000000, 1442062800000, 2214, 12098, 2 ]
+  - [ 1442062800000, "RU", 168, 48104, 1442055600000, 1442066400000, 2214, 12098, 2 ]
+  - [ 1442066400000, "RU", 2047, 48104, 1442059200000, 1442070000000, 2214, 12098, 2 ]
+  - [ 1442070000000, "RU", 4706, 48104, 1442062800000, 1442073600000, 2214, 12098, 2 ]
+  - [ 1442073600000, "RU", 1618, 48104, 1442066400000, 1442077200000, 2214, 12098, 2 ]
+  - [ 1442077200000, "RU", 1162, 48104, 1442070000000, 1442080800000, 2214, 12098, 3 ]
+  - [ 1442080800000, "RU", 655, 48104, 1442073600000, 1442084400000, 2214, 12098, 3 ]
+  - [ 1442084400000, "RU", 6461, 48104, 1442077200000, 1442088000000, 2214, 12098, 3 ]
+  - [ 1442088000000, "RU", 2596, 48104, 1442080800000, 1442091600000, 2214, 12098, 3 ]
+  - [ 1442091600000, "RU", 3449, 48104, 1442084400000, 1442095200000, 2214, 12098, 3 ]
+  - [ 1442095200000, "RU", 435, 48104, 1442088000000, 1442098800000, 2214, 12098, 3 ]
+  - [ 1442098800000, "RU", 12098, 48104, 1442091600000, null, 2214, 12098, 3 ]
+  - [ 1442037600000, "SA", -97, 1614, null, 1442048400000, -97, 458, 1 ]
+  - [ 1442048400000, "SA", 14, 1614, null, 1442055600000, -97, 458, 1 ]
+  - [ 1442055600000, "SA", 11, 1614, 1442037600000, 1442059200000, -97, 458, 1 ]
+  - [ 1442059200000, "SA", 0, 1614, 1442048400000, 1442066400000, -97, 458, 2 ]
+  - [ 1442066400000, "SA", 1276, 1614, 1442055600000, 1442073600000, -97, 458, 2 ]
+  - [ 1442073600000, "SA", 2, 1614, 1442059200000, 1442077200000, -97, 458, 2 ]
+  - [ 1442077200000, "SA", -50, 1614, 1442066400000, 1442084400000, -97, 458, 3 ]
+  - [ 1442084400000, "SA", 458, 1614, 1442073600000, null, -97, 458, 3 ]
+  - [ 1442019600000, "SE", 109, 1838, null, 1442023200000, 109, 0, 1 ]
+  - [ 1442023200000, "SE", 3, 1838, null, 1442030400000, 109, 0, 1 ]
+  - [ 1442030400000, "SE", 30, 1838, 1442019600000, 1442041200000, 109, 0, 1 ]
+  - [ 1442041200000, "SE", 91, 1838, 1442023200000, 1442048400000, 109, 0, 1 ]
+  - [ 1442048400000, "SE", -145, 1838, 1442030400000, 1442052000000, 109, 0, 1 ]
+  - [ 1442052000000, "SE", 1, 1838, 1442041200000, 1442055600000, 109, 0, 2 ]
+  - [ 1442055600000, "SE", -5, 1838, 1442048400000, 1442059200000, 109, 0, 2 ]
+  - [ 1442059200000, "SE", 1476, 1838, 1442052000000, 1442066400000, 109, 0, 2 ]
+  - [ 1442066400000, "SE", 14, 1838, 1442055600000, 1442070000000, 109, 0, 2 ]
+  - [ 1442070000000, "SE", 78, 1838, 1442059200000, 1442080800000, 109, 0, 2 ]
+  - [ 1442080800000, "SE", 89, 1838, 1442066400000, 1442084400000, 109, 0, 3 ]
+  - [ 1442084400000, "SE", 37, 1838, 1442070000000, 1442091600000, 109, 0, 3 ]
+  - [ 1442091600000, "SE", -1, 1838, 1442080800000, 1442095200000, 109, 0, 3 ]
+  - [ 1442095200000, "SE", 61, 1838, 1442084400000, 1442098800000, 109, 0, 3 ]
+  - [ 1442098800000, "SE", 0, 1838, 1442091600000, null, 109, 0, 3 ]
+  - [ 1442026800000, "SG", 2758, 3338, null, 1442030400000, 2758, 0, 1 ]
+  - [ 1442030400000, "SG", 1, 3338, null, 1442037600000, 2758, 0, 1 ]
+  - [ 1442037600000, "SG", 3, 3338, 1442026800000, 1442041200000, 2758, 0, 1 ]
+  - [ 1442041200000, "SG", 59, 3338, 1442030400000, 1442044800000, 2758, 0, 2 ]
+  - [ 1442044800000, "SG", 77, 3338, 1442037600000, 1442048400000, 2758, 0, 2 ]
+  - [ 1442048400000, "SG", 52, 3338, 1442041200000, 1442062800000, 2758, 0, 2 ]
+  - [ 1442062800000, "SG", 388, 3338, 1442044800000, 1442066400000, 2758, 0, 3 ]
+  - [ 1442066400000, "SG", 0, 3338, 1442048400000, null, 2758, 0, 3 ]
+  - [ 1442080800000, "SI", -45, -36, null, 1442091600000, -45, 9, 1 ]
+  - [ 1442091600000, "SI", 9, -36, null, null, -45, 9, 2 ]
+  - [ 1442037600000, "SK", -1, 379, null, 1442052000000, -1, 7, 1 ]
+  - [ 1442052000000, "SK", 13, 379, null, 1442062800000, -1, 7, 1 ]
+  - [ 1442062800000, "SK", 6, 379, 1442037600000, 1442073600000, -1, 7, 2 ]
+  - [ 1442073600000, "SK", 446, 379, 1442052000000, 1442084400000, -1, 7, 2 ]
+  - [ 1442084400000, "SK", -92, 379, 1442062800000, 1442098800000, -1, 7, 3 ]
+  - [ 1442098800000, "SK", 7, 379, 1442073600000, null, -1, 7, 3 ]
+  - [ 1442019600000, "SV", -1, 114, null, 1442084400000, -1, 9, 1 ]
+  - [ 1442084400000, "SV", 106, 114, null, 1442088000000, -1, 9, 2 ]
+  - [ 1442088000000, "SV", 9, 114, 1442019600000, null, -1, 9, 3 ]
+  - [ 1442034000000, "TH", 0, 24, null, 1442041200000, 0, 13, 1 ]
+  - [ 1442041200000, "TH", 3, 24, null, 1442044800000, 0, 13, 1 ]
+  - [ 1442044800000, "TH", 110, 24, 1442034000000, 1442052000000, 0, 13, 1 ]
+  - [ 1442052000000, "TH", -22, 24, 1442041200000, 1442055600000, 0, 13, 2 ]
+  - [ 1442055600000, "TH", 0, 24, 1442044800000, 1442062800000, 0, 13, 2 ]
+  - [ 1442062800000, "TH", -46, 24, 1442052000000, 1442066400000, 0, 13, 2 ]
+  - [ 1442066400000, "TH", -34, 24, 1442055600000, 1442070000000, 0, 13, 3 ]
+  - [ 1442070000000, "TH", 0, 24, 1442062800000, 1442084400000, 0, 13, 3 ]
+  - [ 1442084400000, "TH", 13, 24, 1442066400000, null, 0, 13, 3 ]
+  - [ 1442048400000, "TJ", 1471, 1471, null, null, 1471, 1471, 1 ]
+  - [ 1442098800000, "TN", -9, -9, null, null, -9, -9, 1 ]
+  - [ 1442023200000, "TR", 306, 7078, null, 1442041200000, 306, -29, 1 ]
+  - [ 1442041200000, "TR", 1, 7078, null, 1442044800000, 306, -29, 1 ]
+  - [ 1442044800000, "TR", 41, 7078, 1442023200000, 1442048400000, 306, -29, 1 ]
+  - [ 1442048400000, "TR", 88, 7078, 1442041200000, 1442052000000, 306, -29, 1 ]
+  - [ 1442052000000, "TR", 41, 7078, 1442044800000, 1442055600000, 306, -29, 1 ]
+  - [ 1442055600000, "TR", 299, 7078, 1442048400000, 1442062800000, 306, -29, 2 ]
+  - [ 1442062800000, "TR", 315, 7078, 1442052000000, 1442066400000, 306, -29, 2 ]
+  - [ 1442066400000, "TR", 85, 7078, 1442055600000, 1442070000000, 306, -29, 2 ]
+  - [ 1442070000000, "TR", 236, 7078, 1442062800000, 1442077200000, 306, -29, 2 ]
+  - [ 1442077200000, "TR", 89, 7078, 1442066400000, 1442080800000, 306, -29, 2 ]
+  - [ 1442080800000, "TR", -1, 7078, 1442070000000, 1442084400000, 306, -29, 3 ]
+  - [ 1442084400000, "TR", 170, 7078, 1442077200000, 1442088000000, 306, -29, 3 ]
+  - [ 1442088000000, "TR", 2389, 7078, 1442080800000, 1442091600000, 306, -29, 3 ]
+  - [ 1442091600000, "TR", 3048, 7078, 1442084400000, 1442095200000, 306, -29, 3 ]
+  - [ 1442095200000, "TR", -29, 7078, 1442088000000, null, 306, -29, 3 ]
+  - [ 1442088000000, "TT", 9, 9, null, null, 9, 9, 1 ]
+  - [ 1442016000000, "TW", 92, 3656, null, 1442019600000, 92, -60, 1 ]
+  - [ 1442019600000, "TW", 0, 3656, null, 1442023200000, 92, -60, 1 ]
+  - [ 1442023200000, "TW", 97, 3656, 1442016000000, 1442026800000, 92, -60, 1 ]
+  - [ 1442026800000, "TW", 680, 3656, 1442019600000, 1442030400000, 92, -60, 1 ]
+  - [ 1442030400000, "TW", 0, 3656, 1442023200000, 1442034000000, 92, -60, 1 ]
+  - [ 1442034000000, "TW", 143, 3656, 1442026800000, 1442037600000, 92, -60, 1 ]
+  - [ 1442037600000, "TW", 266, 3656, 1442030400000, 1442041200000, 92, -60, 1 ]
+  - [ 1442041200000, "TW", 366, 3656, 1442034000000, 1442044800000, 92, -60, 1 ]
+  - [ 1442044800000, "TW", 24, 3656, 1442037600000, 1442048400000, 92, -60, 2 ]
+  - [ 1442048400000, "TW", 75, 3656, 1442041200000, 1442052000000, 92, -60, 2 ]
+  - [ 1442052000000, "TW", 24, 3656, 1442044800000, 1442055600000, 92, -60, 2 ]
+  - [ 1442055600000, "TW", 48, 3656, 1442048400000, 1442059200000, 92, -60, 2 ]
+  - [ 1442059200000, "TW", -157, 3656, 1442052000000, 1442062800000, 92, -60, 2 ]
+  - [ 1442062800000, "TW", -272, 3656, 1442055600000, 1442066400000, 92, -60, 2 ]
+  - [ 1442066400000, "TW", 624, 3656, 1442059200000, 1442070000000, 92, -60, 2 ]
+  - [ 1442070000000, "TW", 485, 3656, 1442062800000, 1442073600000, 92, -60, 3 ]
+  - [ 1442073600000, "TW", 772, 3656, 1442066400000, 1442077200000, 92, -60, 3 ]
+  - [ 1442077200000, "TW", 502, 3656, 1442070000000, 1442080800000, 92, -60, 3 ]
+  - [ 1442080800000, "TW", 24, 3656, 1442073600000, 1442084400000, 92, -60, 3 ]
+  - [ 1442084400000, "TW", 0, 3656, 1442077200000, 1442095200000, 92, -60, 3 ]
+  - [ 1442095200000, "TW", -77, 3656, 1442080800000, 1442098800000, 92, -60, 3 ]
+  - [ 1442098800000, "TW", -60, 3656, 1442084400000, null, 92, -60, 3 ]
+  - [ 1442034000000, "UA", 3468, 24898, null, 1442037600000, 3468, 38, 1 ]
+  - [ 1442037600000, "UA", -1, 24898, null, 1442041200000, 3468, 38, 1 ]
+  - [ 1442041200000, "UA", 74, 24898, 1442034000000, 1442044800000, 3468, 38, 1 ]
+  - [ 1442044800000, "UA", 280, 24898, 1442037600000, 1442048400000, 3468, 38, 1 ]
+  - [ 1442048400000, "UA", 2, 24898, 1442041200000, 1442052000000, 3468, 38, 1 ]
+  - [ 1442052000000, "UA", 410, 24898, 1442044800000, 1442055600000, 3468, 38, 1 ]
+  - [ 1442055600000, "UA", 14202, 24898, 1442048400000, 1442059200000, 3468, 38, 1 ]
+  - [ 1442059200000, "UA", -2, 24898, 1442052000000, 1442062800000, 3468, 38, 2 ]
+  - [ 1442062800000, "UA", 773, 24898, 1442055600000, 1442066400000, 3468, 38, 2 ]
+  - [ 1442066400000, "UA", 296, 24898, 1442059200000, 1442070000000, 3468, 38, 2 ]
+  - [ 1442070000000, "UA", 1733, 24898, 1442062800000, 1442073600000, 3468, 38, 2 ]
+  - [ 1442073600000, "UA", 4241, 24898, 1442066400000, 1442077200000, 3468, 38, 2 ]
+  - [ 1442077200000, "UA", -181, 24898, 1442070000000, 1442080800000, 3468, 38, 2 ]
+  - [ 1442080800000, "UA", -1, 24898, 1442073600000, 1442084400000, 3468, 38, 3 ]
+  - [ 1442084400000, "UA", 5, 24898, 1442077200000, 1442088000000, 3468, 38, 3 ]
+  - [ 1442088000000, "UA", -21, 24898, 1442080800000, 1442091600000, 3468, 38, 3 ]
+  - [ 1442091600000, "UA", -388, 24898, 1442084400000, 1442095200000, 3468, 38, 3 ]
+  - [ 1442095200000, "UA", -30, 24898, 1442088000000, 1442098800000, 3468, 38, 3 ]
+  - [ 1442098800000, "UA", 38, 24898, 1442091600000, null, 3468, 38, 3 ]
+  - [ 1442070000000, "UG", 1, 1, null, null, 1, 1, 1 ]
+  - [ 1442016000000, "US", 0, 38882, null, 1442019600000, 0, 3575, 1 ]
+  - [ 1442019600000, "US", 1043, 38882, null, 1442023200000, 0, 3575, 1 ]
+  - [ 1442023200000, "US", 2844, 38882, 1442016000000, 1442026800000, 0, 3575, 1 ]
+  - [ 1442026800000, "US", 1512, 38882, 1442019600000, 1442030400000, 0, 3575, 1 ]
+  - [ 1442030400000, "US", 2023, 38882, 1442023200000, 1442034000000, 0, 3575, 1 ]
+  - [ 1442034000000, "US", 3648, 38882, 1442026800000, 1442037600000, 0, 3575, 1 ]
+  - [ 1442037600000, "US", 3675, 38882, 1442030400000, 1442041200000, 0, 3575, 1 ]
+  - [ 1442041200000, "US", 1999, 38882, 1442034000000, 1442044800000, 0, 3575, 1 ]
+  - [ 1442044800000, "US", 139, 38882, 1442037600000, 1442048400000, 0, 3575, 2 ]
+  - [ 1442048400000, "US", -466, 38882, 1442041200000, 1442052000000, 0, 3575, 2 ]
+  - [ 1442052000000, "US", -2, 38882, 1442044800000, 1442055600000, 0, 3575, 2 ]
+  - [ 1442055600000, "US", 156, 38882, 1442048400000, 1442059200000, 0, 3575, 2 ]
+  - [ 1442059200000, "US", 11, 38882, 1442052000000, 1442062800000, 0, 3575, 2 ]
+  - [ 1442062800000, "US", 47, 38882, 1442055600000, 1442066400000, 0, 3575, 2 ]
+  - [ 1442066400000, "US", 772, 38882, 1442059200000, 1442070000000, 0, 3575, 2 ]
+  - [ 1442070000000, "US", 3505, 38882, 1442062800000, 1442073600000, 0, 3575, 2 ]
+  - [ 1442073600000, "US", 1100, 38882, 1442066400000, 1442077200000, 0, 3575, 3 ]
+  - [ 1442077200000, "US", 2168, 38882, 1442070000000, 1442080800000, 0, 3575, 3 ]
+  - [ 1442080800000, "US", 4001, 38882, 1442073600000, 1442084400000, 0, 3575, 3 ]
+  - [ 1442084400000, "US", 2523, 38882, 1442077200000, 1442088000000, 0, 3575, 3 ]
+  - [ 1442088000000, "US", 1691, 38882, 1442080800000, 1442091600000, 0, 3575, 3 ]
+  - [ 1442091600000, "US", 2502, 38882, 1442084400000, 1442095200000, 0, 3575, 3 ]
+  - [ 1442095200000, "US", 416, 38882, 1442088000000, 1442098800000, 0, 3575, 3 ]
+  - [ 1442098800000, "US", 3575, 38882, 1442091600000, null, 0, 3575, 3 ]
+  - [ 1442019600000, "UY", 77, 936, null, 1442023200000, 77, 23, 1 ]
+  - [ 1442023200000, "UY", 517, 936, null, 1442026800000, 77, 23, 1 ]
+  - [ 1442026800000, "UY", 76, 936, 1442019600000, 1442037600000, 77, 23, 1 ]
+  - [ 1442037600000, "UY", 1, 936, 1442023200000, 1442070000000, 77, 23, 2 ]
+  - [ 1442070000000, "UY", 284, 936, 1442026800000, 1442073600000, 77, 23, 2 ]
+  - [ 1442073600000, "UY", -42, 936, 1442037600000, 1442077200000, 77, 23, 3 ]
+  - [ 1442077200000, "UY", 23, 936, 1442070000000, null, 77, 23, 3 ]
+  - [ 1442044800000, "UZ", 1369, 1369, null, null, 1369, 1369, 1 ]
+  - [ 1442023200000, "VE", 115, 1101, null, 1442026800000, 115, 9, 1 ]
+  - [ 1442026800000, "VE", -17, 1101, null, 1442030400000, 115, 9, 1 ]
+  - [ 1442030400000, "VE", 51, 1101, 1442023200000, 1442034000000, 115, 9, 1 ]
+  - [ 1442034000000, "VE", -2, 1101, 1442026800000, 1442066400000, 115, 9, 1 ]
+  - [ 1442066400000, "VE", 18, 1101, 1442030400000, 1442070000000, 115, 9, 2 ]
+  - [ 1442070000000, "VE", 420, 1101, 1442034000000, 1442077200000, 115, 9, 2 ]
+  - [ 1442077200000, "VE", 412, 1101, 1442066400000, 1442084400000, 115, 9, 2 ]
+  - [ 1442084400000, "VE", 60, 1101, 1442070000000, 1442095200000, 115, 9, 3 ]
+  - [ 1442095200000, "VE", 35, 1101, 1442077200000, 1442098800000, 115, 9, 3 ]
+  - [ 1442098800000, "VE", 9, 1101, 1442084400000, null, 115, 9, 3 ]
+  - [ 1442062800000, "VG", -238, -238, null, null, -238, -238, 1 ]
+  - [ 1442023200000, "VN", -9, 1560, null, 1442026800000, -9, -10, 1 ]
+  - [ 1442026800000, "VN", 63, 1560, null, 1442034000000, -9, -10, 1 ]
+  - [ 1442034000000, "VN", -29, 1560, 1442023200000, 1442037600000, -9, -10, 1 ]
+  - [ 1442037600000, "VN", -11, 1560, 1442026800000, 1442041200000, -9, -10, 1 ]
+  - [ 1442041200000, "VN", 0, 1560, 1442034000000, 1442048400000, -9, -10, 1 ]
+  - [ 1442048400000, "VN", -15, 1560, 1442037600000, 1442052000000, -9, -10, 2 ]
+  - [ 1442052000000, "VN", 90, 1560, 1442041200000, 1442055600000, -9, -10, 2 ]
+  - [ 1442055600000, "VN", 37, 1560, 1442048400000, 1442059200000, -9, -10, 2 ]
+  - [ 1442059200000, "VN", 8, 1560, 1442052000000, 1442062800000, -9, -10, 2 ]
+  - [ 1442062800000, "VN", 146, 1560, 1442055600000, 1442066400000, -9, -10, 3 ]
+  - [ 1442066400000, "VN", 811, 1560, 1442059200000, 1442070000000, -9, -10, 3 ]
+  - [ 1442070000000, "VN", 479, 1560, 1442062800000, 1442084400000, -9, -10, 3 ]
+  - [ 1442084400000, "VN", -10, 1560, 1442066400000, null, -9, -10, 3 ]
+  - [ 1442034000000, "ZA", -3, 127, null, 1442048400000, -3, 1, 1 ]
+  - [ 1442048400000, "ZA", 79, 127, null, 1442059200000, -3, 1, 1 ]
+  - [ 1442059200000, "ZA", 50, 127, 1442034000000, 1442070000000, -3, 1, 2 ]
+  - [ 1442070000000, "ZA", 0, 127, 1442048400000, 1442091600000, -3, 1, 2 ]
+  - [ 1442091600000, "ZA", 1, 127, 1442059200000, null, -3, 1, 3 ]
+  - [ 1442041200000, "ZM", 133, 133, null, null, 133, 133, 1 ]
+  - [ 1442044800000, "ZW", 0, 254, null, 1442048400000, 0, 254, 1 ]
+  - [ 1442048400000, "ZW", 254, 254, null, null, 0, 254, 2 ]
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org