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

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


The following commit(s) were added to refs/heads/28.0.0 by this push:
     new d3fad8fe514 Process pure ordering changes with windowing operators 
(#15241) (#15275)
d3fad8fe514 is described below

commit d3fad8fe5145993c5736fe4c250cdc63bb924db0
Author: Zoltan Haindrich <[email protected]>
AuthorDate: Mon Oct 30 13:56:25 2023 +0100

    Process pure ordering changes with windowing operators (#15241) (#15275)
    
    - adds a new query build path: DruidQuery#toScanAndSortQuery which:
    - builds a ScanQuery without considering the current ordering
    - builds an operator to execute the sort
    - fixes a null string to "null" literal string conversion in the frame 
serializer code
    - fixes some DrillWindowQueryTest cases
    - fix NPE in NaiveSortOperator in case there was no input
    - enables back CoreRules.AGGREGATE_REMOVE
    - adds a processing level OffsetLimit class and uses that instead of just 
the limit in the rac parts
    - earlier window expressions on top of a subquery with an offset may have 
ignored the offset
    
    (cherry picked from commit f4a74710e6dc1eccfe376fcbfc1f488ea0fad8f5)
---
 .../main/java/org/apache/druid/query/Druids.java   |    5 +
 .../operator/NaivePartitioningOperatorFactory.java |   20 +
 .../druid/query/operator/NaiveSortOperator.java    |   11 +-
 .../query/operator/NaiveSortOperatorFactory.java   |   34 +-
 .../apache/druid/query/operator/OffsetLimit.java   |  143 +++
 .../apache/druid/query/operator/ScanOperator.java  |   10 +-
 .../druid/query/operator/ScanOperatorFactory.java  |   42 +-
 .../druid/query/operator/WindowOperatorQuery.java  |   19 +-
 .../operator/window/WindowOperatorFactory.java     |   23 +
 .../window/ranking/WindowRankProcessor.java        |   26 +
 .../window/ranking/WindowRankingProcessorBase.java |   24 +
 .../rowsandcols/LazilyDecoratedRowsAndColumns.java |   35 +-
 .../DefaultColumnSelectorFactoryMaker.java         |    3 +
 .../semantic/DefaultRowsAndColumnsDecorator.java   |   19 +-
 .../semantic/RowsAndColumnsDecorator.java          |    3 +-
 .../org/apache/druid/query/scan/ScanQuery.java     |    7 +
 .../druid/query/scan/ScanQueryQueryToolChest.java  |   21 +-
 .../druid/segment/column/TypeStrategies.java       |    3 +
 .../NaivePartitioningOperatorFactoryTest.java      |   34 +
 .../operator/NaiveSortOperatorFactoryTest.java     |   34 +
 .../query/operator/NaiveSortOperatorTest.java      |   98 ++
 .../druid/query/operator/OffsetLimitTest.java      |  109 +++
 .../query/operator/ScanOperatorFactoryTest.java    |   20 +-
 .../query/operator/WindowOperatorFactoryTest.java  |   34 +
 .../query/operator/WindowOperatorQueryTest.java    |    8 +-
 .../concrete/FrameRowsAndColumnsTest.java          |    3 +-
 .../semantic/RowsAndColumnsDecoratorTest.java      |   13 +-
 .../semantic/TestRowsAndColumnsDecorator.java      |   11 +-
 ...tVirtualColumnEvaluationRowsAndColumnsTest.java |    3 +-
 .../druid/segment/column/TypeStrategiesTest.java   |   10 +-
 .../sql/calcite/planner/CalciteRulesManager.java   |    8 +-
 .../druid/sql/calcite/planner/OffsetLimit.java     |   14 +
 .../druid/sql/calcite/rel/DruidOuterQueryRel.java  |    9 +-
 .../apache/druid/sql/calcite/rel/DruidQuery.java   |   91 +-
 .../druid/query/OperatorFactoryBuilders.java       |  102 ++
 .../druid/query/WindowOperatorQueryBuilder.java    |   91 ++
 .../apache/druid/sql/calcite/CalciteQueryTest.java |  287 +++++-
 .../druid/sql/calcite/CalciteWindowQueryTest.java  |   23 +-
 .../druid/sql/calcite/DrillWindowQueryTest.java    |   55 +-
 .../apache/druid/sql/calcite/NotYetSupported.java  |    3 +-
 .../org/apache/druid/sql/http/SqlResourceTest.java |    2 +-
 .../tests/window/offsetNotDiscarded.sqlTest        |   31 +
 ...kipediaAggregationsMultipleOrderingDesc.sqlTest | 1004 +++++++++++++++++++-
 43 files changed, 2371 insertions(+), 174 deletions(-)

diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java 
b/processing/src/main/java/org/apache/druid/query/Druids.java
index 24c1f3ddc4f..f85e8daa52e 100644
--- a/processing/src/main/java/org/apache/druid/query/Druids.java
+++ b/processing/src/main/java/org/apache/druid/query/Druids.java
@@ -868,6 +868,11 @@ public class Druids
       dataSource = new TableDataSource(ds);
       return this;
     }
+    public ScanQueryBuilder dataSource(Query<?> q)
+    {
+      dataSource = new QueryDataSource(q);
+      return this;
+    }
 
     public ScanQueryBuilder dataSource(DataSource ds)
     {
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java
 
b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java
index 9bd937ab844..c836007e77e 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactory.java
@@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 public class NaivePartitioningOperatorFactory implements OperatorFactory
 {
@@ -65,4 +66,23 @@ public class NaivePartitioningOperatorFactory implements 
OperatorFactory
            "partitionColumns=" + partitionColumns +
            '}';
   }
+
+  @Override
+  public final int hashCode()
+  {
+    return Objects.hash(partitionColumns);
+  }
+
+  @Override
+  public final boolean equals(Object obj)
+  {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || obj.getClass() != getClass()) {
+      return false;
+    }
+    NaivePartitioningOperatorFactory other = 
(NaivePartitioningOperatorFactory) obj;
+    return Objects.equals(partitionColumns, other.partitionColumns);
+  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java
 
b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java
index e11da384210..486d2048282 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperator.java
@@ -24,6 +24,7 @@ import 
org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker;
 
 import java.io.Closeable;
 import java.util.ArrayList;
+import java.util.List;
 
 /**
  * A naive sort operator is an operation that sorts a stream of data in-place. 
 Generally speaking this means
@@ -33,11 +34,11 @@ import java.util.ArrayList;
 public class NaiveSortOperator implements Operator
 {
   private final Operator child;
-  private final ArrayList<ColumnWithDirection> sortColumns;
+  private final List<ColumnWithDirection> sortColumns;
 
   public NaiveSortOperator(
       Operator child,
-      ArrayList<ColumnWithDirection> sortColumns
+      List<ColumnWithDirection> sortColumns
   )
   {
     this.child = child;
@@ -57,7 +58,7 @@ public class NaiveSortOperator implements Operator
           public Signal push(RowsAndColumns rac)
           {
             if (sorter == null) {
-              sorter = NaiveSortMaker.fromRAC(rac).make(sortColumns);
+              sorter = NaiveSortMaker.fromRAC(rac).make(new 
ArrayList<>(sortColumns));
             } else {
               sorter.moreData(rac);
             }
@@ -67,7 +68,9 @@ public class NaiveSortOperator implements Operator
           @Override
           public void completed()
           {
-            receiver.push(sorter.complete());
+            if (sorter != null) {
+              receiver.push(sorter.complete());
+            }
             receiver.completed();
           }
         }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java
 
b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java
index 1ab80398b69..623d0ed0fe5 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/NaiveSortOperatorFactory.java
@@ -22,22 +22,23 @@ package org.apache.druid.query.operator;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 
-import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
 
 public class NaiveSortOperatorFactory implements OperatorFactory
 {
-  private final ArrayList<ColumnWithDirection> sortColumns;
+  private final List<ColumnWithDirection> sortColumns;
 
   @JsonCreator
   public NaiveSortOperatorFactory(
-      @JsonProperty("columns") ArrayList<ColumnWithDirection> sortColumns
+      @JsonProperty("columns") List<ColumnWithDirection> sortColumns
   )
   {
     this.sortColumns = sortColumns;
   }
 
   @JsonProperty("columns")
-  public ArrayList<ColumnWithDirection> getSortColumns()
+  public List<ColumnWithDirection> getSortColumns()
   {
     return sortColumns;
   }
@@ -56,4 +57,29 @@ public class NaiveSortOperatorFactory implements 
OperatorFactory
     }
     return false;
   }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(sortColumns);
+  }
+
+  @Override
+  public boolean equals(Object obj)
+  {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || getClass() != obj.getClass()) {
+      return false;
+    }
+    NaiveSortOperatorFactory other = (NaiveSortOperatorFactory) obj;
+    return Objects.equals(sortColumns, other.sortColumns);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "NaiveSortOperatorFactory{sortColumns=" + sortColumns + "}";
+  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java 
b/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java
new file mode 100644
index 00000000000..80fedf06115
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import java.util.Objects;
+
+public class OffsetLimit
+{
+  protected final long offset;
+  protected final long limit;
+
+  public static final OffsetLimit NONE = new OffsetLimit(0, -1);
+
+  @JsonCreator
+  public OffsetLimit(
+      @JsonProperty("offset") long offset,
+      @JsonProperty("limit") long limit)
+  {
+    Preconditions.checkArgument(offset >= 0, "offset >= 0");
+    this.offset = offset;
+    this.limit = limit < 0 ? -1 : limit;
+  }
+
+  @JsonProperty("offset")
+  public long getOffset()
+  {
+    return offset;
+  }
+
+  @JsonProperty("limit")
+  public long getLimit()
+  {
+    return limit;
+  }
+
+  public boolean isPresent()
+  {
+    return hasOffset() || hasLimit();
+  }
+
+  public boolean hasOffset()
+  {
+    return offset > 0;
+  }
+
+  public boolean hasLimit()
+  {
+    return limit >= 0;
+  }
+
+  public static OffsetLimit limit(int limit2)
+  {
+    return new OffsetLimit(0, limit2);
+  }
+
+  public long getLimitOrMax()
+  {
+    if (limit < 0) {
+      return Long.MAX_VALUE;
+    } else {
+      return limit;
+    }
+  }
+
+  @Override
+  public final boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof OffsetLimit)) {
+      return false;
+    }
+    OffsetLimit that = (OffsetLimit) o;
+    return limit == that.limit && offset == that.offset;
+  }
+
+  @Override
+  public final int hashCode()
+  {
+    return Objects.hash(limit, offset);
+  }
+
+  @Override
+  public String toString()
+  {
+    return "OffsetLimit{" +
+        "offset=" + offset +
+        ", limit=" + limit +
+        '}';
+  }
+
+  /**
+   * Returns the first row index to fetch.
+   *
+   * @param maxIndex maximal index accessible
+   */
+  public long getFromIndex(long maxIndex)
+  {
+    if (maxIndex <= offset) {
+      return 0;
+    }
+    return offset;
+  }
+
+  /**
+   * Returns the last row index to fetch (non-inclusive).
+   *
+   * @param maxIndex maximal index accessible
+   */
+  public long getToIndex(long maxIndex)
+  {
+    if (maxIndex <= offset) {
+      return 0;
+    }
+    if (hasLimit()) {
+      long toIndex = limit + offset;
+      return Math.min(maxIndex, toIndex);
+    } else {
+      return maxIndex;
+    }
+  }
+}
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java 
b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java
index b823c30d22e..fd72d1b6da9 100644
--- a/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java
+++ b/processing/src/main/java/org/apache/druid/query/operator/ScanOperator.java
@@ -43,7 +43,7 @@ public class ScanOperator implements Operator
   private final Operator subOperator;
   private final Interval timeRange;
   private final Filter filter;
-  private final int limit;
+  private final OffsetLimit offsetLimit;
   private final List<String> projectedColumns;
   private final VirtualColumns virtualColumns;
   private final List<ColumnWithDirection> ordering;
@@ -55,7 +55,7 @@ public class ScanOperator implements Operator
       Interval timeRange,
       Filter filter,
       List<ColumnWithDirection> ordering,
-      int limit
+      OffsetLimit offsetLimit
   )
   {
     this.subOperator = subOperator;
@@ -64,7 +64,7 @@ public class ScanOperator implements Operator
     this.timeRange = timeRange;
     this.filter = filter;
     this.ordering = ordering;
-    this.limit = limit;
+    this.offsetLimit = offsetLimit == null ? OffsetLimit.NONE : offsetLimit;
   }
 
   @Nullable
@@ -93,8 +93,8 @@ public class ScanOperator implements Operator
           decor.limitTimeRange(timeRange);
         }
 
-        if (limit > 0) {
-          decor.setLimit(limit);
+        if (offsetLimit.isPresent()) {
+          decor.setOffsetLimit(offsetLimit);
         }
 
         if (!(ordering == null || ordering.isEmpty())) {
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java
 
b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java
index a764984855e..99453cf415a 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/ScanOperatorFactory.java
@@ -31,7 +31,7 @@ public class ScanOperatorFactory implements OperatorFactory
 {
   private final Interval timeRange;
   private final DimFilter filter;
-  private final int limit;
+  private final OffsetLimit offsetLimit;
   private final List<String> projectedColumns;
   private final VirtualColumns virtualColumns;
   private final List<ColumnWithDirection> ordering;
@@ -39,7 +39,7 @@ public class ScanOperatorFactory implements OperatorFactory
   public ScanOperatorFactory(
       @JsonProperty("timeRange") final Interval timeRange,
       @JsonProperty("filter") final DimFilter filter,
-      @JsonProperty("limit") final Integer limit,
+      @JsonProperty("offsetLimit") final OffsetLimit offsetLimit,
       @JsonProperty("projectedColumns") final List<String> projectedColumns,
       @JsonProperty("virtualColumns") final VirtualColumns virtualColumns,
       @JsonProperty("ordering") final List<ColumnWithDirection> ordering
@@ -47,7 +47,7 @@ public class ScanOperatorFactory implements OperatorFactory
   {
     this.timeRange = timeRange;
     this.filter = filter;
-    this.limit = limit == null ? -1 : limit;
+    this.offsetLimit = offsetLimit;
     this.projectedColumns = projectedColumns;
     this.virtualColumns = virtualColumns;
     this.ordering = ordering;
@@ -66,9 +66,9 @@ public class ScanOperatorFactory implements OperatorFactory
   }
 
   @JsonProperty
-  public int getLimit()
+  public OffsetLimit getOffsetLimit()
   {
-    return limit;
+    return offsetLimit;
   }
 
   @JsonProperty
@@ -99,7 +99,7 @@ public class ScanOperatorFactory implements OperatorFactory
         timeRange,
         filter == null ? null : filter.toFilter(),
         ordering,
-        limit
+        offsetLimit
     );
   }
 
@@ -119,18 +119,32 @@ public class ScanOperatorFactory implements 
OperatorFactory
       return false;
     }
     ScanOperatorFactory that = (ScanOperatorFactory) o;
-    return limit == that.limit && Objects.equals(timeRange, that.timeRange) && 
Objects.equals(
-        filter,
-        that.filter
-    ) && Objects.equals(projectedColumns, that.projectedColumns) && 
Objects.equals(
-        virtualColumns,
-        that.virtualColumns
-    ) && Objects.equals(ordering, that.ordering);
+    return Objects.equals(offsetLimit, that.offsetLimit)
+        && Objects.equals(timeRange, that.timeRange)
+        && Objects.equals(filter, that.filter)
+        && Objects.equals(projectedColumns, that.projectedColumns)
+        && Objects.equals(virtualColumns, that.virtualColumns)
+        && Objects.equals(ordering, that.ordering);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(timeRange, filter, limit, projectedColumns, 
virtualColumns, ordering);
+    return Objects.hash(timeRange, filter, offsetLimit, projectedColumns, 
virtualColumns, ordering);
   }
+
+  @Override
+  public String toString()
+  {
+    return "ScanOperatorFactory{" +
+        "timeRange=" + timeRange +
+        ", filter=" + filter +
+        ", offsetLimit=" + offsetLimit +
+        ", projectedColumns=" + projectedColumns +
+        ", virtualColumns=" + virtualColumns +
+        ", ordering=" + ordering
+        + "}";
+  }
+
+
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java
 
b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java
index 5ecba3f2242..50289b9851b 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java
@@ -34,11 +34,13 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
 import org.apache.druid.segment.column.RowSignature;
 
 import javax.annotation.Nullable;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+
 /**
  * A query that can compute window functions on top of a completely in-memory 
inline datasource or query results.
  * <p>
@@ -122,14 +124,17 @@ public class WindowOperatorQuery extends 
BaseQuery<RowsAndColumns>
                 )
             );
           }
+          if (ordering.isEmpty()) {
+            ordering = null;
+          }
 
           this.leafOperators.add(
               new ScanOperatorFactory(
                   null,
                   scan.getFilter(),
-                  (int) scan.getScanRowsLimit(),
+                  scan.getOffsetLimit(),
                   scan.getColumns(),
-                  scan.getVirtualColumns(),
+                  scan.getVirtualColumns().isEmpty() ? null : 
scan.getVirtualColumns(),
                   ordering
               )
           );
@@ -242,16 +247,15 @@ public class WindowOperatorQuery extends 
BaseQuery<RowsAndColumns>
       return false;
     }
     WindowOperatorQuery that = (WindowOperatorQuery) o;
-    return Objects.equals(rowSignature, that.rowSignature) && Objects.equals(
-        operators,
-        that.operators
-    );
+    return Objects.equals(rowSignature, that.rowSignature)
+        && Objects.equals(operators, that.operators)
+        && Objects.equals(leafOperators, that.leafOperators);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(super.hashCode(), rowSignature, operators);
+    return Objects.hash(super.hashCode(), rowSignature, operators, 
leafOperators);
   }
 
   @Override
@@ -263,6 +267,7 @@ public class WindowOperatorQuery extends 
BaseQuery<RowsAndColumns>
            ", context=" + getContext() +
            ", rowSignature=" + rowSignature +
            ", operators=" + operators +
+           ", leafOperators=" + leafOperators +
            '}';
   }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java
 
b/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java
index ea44001d507..88054bc1270 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/window/WindowOperatorFactory.java
@@ -26,6 +26,8 @@ import org.apache.druid.query.operator.Operator;
 import org.apache.druid.query.operator.OperatorFactory;
 import org.apache.druid.query.operator.WindowProcessorOperator;
 
+import java.util.Objects;
+
 public class WindowOperatorFactory implements OperatorFactory
 {
   private Processor processor;
@@ -67,4 +69,25 @@ public class WindowOperatorFactory implements OperatorFactory
            "processor=" + processor +
            '}';
   }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(processor);
+  }
+
+  @Override
+  public boolean equals(Object obj)
+  {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null || obj.getClass() != getClass()) {
+      return false;
+    }
+    WindowOperatorFactory other = (WindowOperatorFactory) obj;
+    return Objects.equals(processor, other.processor);
+  }
+
+
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java
 
b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java
index 2d9b21863de..b193398dfe1 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankProcessor.java
@@ -28,6 +28,7 @@ import 
org.apache.druid.query.rowsandcols.column.IntArrayColumn;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Objects;
 
 /**
  * This Processor assumes that data has already been sorted for it.  It does 
not re-sort the data and if it is given
@@ -105,4 +106,29 @@ public class WindowRankProcessor extends 
WindowRankingProcessorBase
            ", asPercent=" + asPercent +
            '}';
   }
+
+  @Override
+  public int hashCode()
+  {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + Objects.hash(asPercent);
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj)
+  {
+    if (this == obj) {
+      return true;
+    }
+    if (!super.equals(obj)) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    WindowRankProcessor other = (WindowRankProcessor) obj;
+    return asPercent == other.asPercent;
+  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
 
b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
index 4bff17174d6..fb5bedf9519 100644
--- 
a/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
+++ 
b/processing/src/main/java/org/apache/druid/query/operator/window/ranking/WindowRankingProcessorBase.java
@@ -28,6 +28,7 @@ import 
org.apache.druid.query.rowsandcols.semantic.ClusteredGroupPartitioner;
 import 
org.apache.druid.query.rowsandcols.semantic.DefaultClusteredGroupPartitioner;
 
 import java.util.List;
+import java.util.Objects;
 import java.util.function.Function;
 
 /**
@@ -100,4 +101,27 @@ public abstract class WindowRankingProcessorBase 
implements Processor
     return "groupingCols=" + groupingCols +
            ", outputColumn='" + outputColumn + '\'';
   }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(groupingCols, outputColumn);
+  }
+
+  @Override
+  public boolean equals(Object obj)
+  {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    WindowRankingProcessorBase other = (WindowRankingProcessorBase) obj;
+    return Objects.equals(groupingCols, other.groupingCols) && 
Objects.equals(outputColumn, other.outputColumn);
+  }
+
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
index 79505dcdd41..3e938eb2d9d 100644
--- 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
+++ 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java
@@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.ValueMatcher;
 import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.column.Column;
 import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
 import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns;
@@ -73,7 +74,7 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
   private Interval interval;
   private Filter filter;
   private VirtualColumns virtualColumns;
-  private int limit;
+  private OffsetLimit limit;
   private LinkedHashSet<String> viewableColumns;
   private List<ColumnWithDirection> ordering;
 
@@ -82,7 +83,7 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
       Interval interval,
       Filter filter,
       VirtualColumns virtualColumns,
-      int limit,
+      OffsetLimit limit,
       List<ColumnWithDirection> ordering,
       LinkedHashSet<String> viewableColumns
   )
@@ -175,7 +176,7 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
 
   private boolean needsMaterialization()
   {
-    return interval != null || filter != null || limit != -1 || ordering != 
null || virtualColumns != null;
+    return interval != null || filter != null || limit.isPresent() || ordering 
!= null || virtualColumns != null;
   }
 
   private Pair<byte[], RowSignature> materialize()
@@ -198,7 +199,7 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
     interval = null;
     filter = null;
     virtualColumns = null;
-    limit = -1;
+    limit = OffsetLimit.NONE;
     viewableColumns = null;
     ordering = null;
   }
@@ -238,7 +239,8 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
         throw new ISE("accumulated[%s] non-null, why did we get multiple 
cursors?", accumulated);
       }
 
-      int theLimit = limit == -1 ? Integer.MAX_VALUE : limit;
+      long remainingRowsToSkip = limit.getOffset();
+      long remainingRowsToFetch = limit.getLimitOrMax();
 
       final ColumnSelectorFactory columnSelectorFactory = 
in.getColumnSelectorFactory();
       final RowSignature.Builder sigBob = RowSignature.builder();
@@ -284,12 +286,12 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
       );
 
       final FrameWriter frameWriter = 
frameWriterFactory.newFrameWriter(columnSelectorFactory);
-      while (!in.isDoneOrInterrupted()) {
+      for (; !in.isDoneOrInterrupted() && remainingRowsToSkip > 0; 
remainingRowsToSkip--) {
+        in.advance();
+      }
+      for (; !in.isDoneOrInterrupted() && remainingRowsToFetch > 0; 
remainingRowsToFetch--) {
         frameWriter.addSelection();
         in.advance();
-        if (--theLimit <= 0) {
-          break;
-        }
       }
 
       return frameWriter;
@@ -390,12 +392,8 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
       sigBob.add(column, racColumn.toAccessor().getType());
     }
 
-    final int limitedNumRows;
-    if (limit == -1) {
-      limitedNumRows = Integer.MAX_VALUE;
-    } else {
-      limitedNumRows = limit;
-    }
+    long remainingRowsToSkip = limit.getOffset();
+    long remainingRowsToFetch = limit.getLimitOrMax();
 
     final FrameWriter frameWriter = FrameWriters.makeFrameWriterFactory(
         FrameType.COLUMNAR,
@@ -405,11 +403,16 @@ public class LazilyDecoratedRowsAndColumns implements 
RowsAndColumns
     ).newFrameWriter(selectorFactory);
 
     rowId.set(0);
-    for (; rowId.get() < numRows && frameWriter.getNumRows() < limitedNumRows; 
rowId.incrementAndGet()) {
+    for (; rowId.get() < numRows && remainingRowsToFetch > 0; 
rowId.incrementAndGet()) {
       final int theId = rowId.get();
       if (rowsToSkip != null && rowsToSkip.get(theId)) {
         continue;
       }
+      if (remainingRowsToSkip > 0) {
+        remainingRowsToSkip--;
+        continue;
+      }
+      remainingRowsToFetch--;
       frameWriter.addSelection();
     }
 
diff --git 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
index 4611280ce89..3c6d3cc08c9 100644
--- 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
+++ 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java
@@ -106,6 +106,9 @@ public class DefaultColumnSelectorFactoryMaker implements 
ColumnSelectorFactoryM
             protected String getValue()
             {
               final Object retVal = 
columnAccessor.getObject(cellIdSupplier.get());
+              if (retVal == null) {
+                return null;
+              }
               if (retVal instanceof ByteBuffer) {
                 return StringUtils.fromUtf8(((ByteBuffer) 
retVal).asReadOnlyBuffer());
               }
diff --git 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java
 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java
index fd81491112d..3cfcfeec614 100644
--- 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java
+++ 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultRowsAndColumnsDecorator.java
@@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
 
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns;
 import org.apache.druid.query.rowsandcols.RowsAndColumns;
 import org.apache.druid.segment.VirtualColumn;
@@ -39,14 +40,14 @@ public class DefaultRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
   private Interval interval;
   private Filter filter;
   private VirtualColumns virtualColumns;
-  private int limit;
+  private OffsetLimit offsetLimit;
   private List<ColumnWithDirection> ordering;
 
   public DefaultRowsAndColumnsDecorator(
       RowsAndColumns base
   )
   {
-    this(base, null, null, null, -1, null);
+    this(base, null, null, null, OffsetLimit.NONE, null);
   }
 
   public DefaultRowsAndColumnsDecorator(
@@ -54,7 +55,7 @@ public class DefaultRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
       Interval interval,
       Filter filter,
       VirtualColumns virtualColumns,
-      int limit,
+      OffsetLimit limit,
       List<ColumnWithDirection> ordering
   )
   {
@@ -62,7 +63,7 @@ public class DefaultRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
     this.interval = interval;
     this.filter = filter;
     this.virtualColumns = virtualColumns;
-    this.limit = limit;
+    this.offsetLimit = limit;
     this.ordering = ordering;
   }
 
@@ -111,13 +112,9 @@ public class DefaultRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
   }
 
   @Override
-  public void setLimit(int numRows)
+  public void setOffsetLimit(OffsetLimit offsetLimit)
   {
-    if (this.limit == -1) {
-      this.limit = numRows;
-    } else {
-      this.limit = Math.min(limit, numRows);
-    }
+    this.offsetLimit = offsetLimit;
   }
 
   @Override
@@ -134,7 +131,7 @@ public class DefaultRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
         interval,
         filter,
         virtualColumns,
-        limit,
+        offsetLimit,
         ordering,
         columns == null ? null : new LinkedHashSet<>(columns)
     );
diff --git 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java
 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java
index 77d8e2068dd..b066fbe64f2 100644
--- 
a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java
+++ 
b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecorator.java
@@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
 
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.RowsAndColumns;
 import org.apache.druid.segment.VirtualColumns;
 import org.joda.time.Interval;
@@ -61,7 +62,7 @@ public interface RowsAndColumnsDecorator
 
   void addVirtualColumns(VirtualColumns virtualColumn);
 
-  void setLimit(int numRows);
+  void setOffsetLimit(OffsetLimit offsetLimit);
 
   void setOrdering(List<ColumnWithDirection> ordering);
 
diff --git 
a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java 
b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
index 57f32bc4398..b897811d607 100644
--- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
+++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java
@@ -37,6 +37,7 @@ import org.apache.druid.query.DataSource;
 import org.apache.druid.query.Druids;
 import org.apache.druid.query.Queries;
 import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.spec.QuerySegmentSpec;
 import org.apache.druid.segment.VirtualColumns;
 import org.apache.druid.segment.column.ColumnHolder;
@@ -325,6 +326,11 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
     return scanRowsLimit;
   }
 
+  public OffsetLimit getOffsetLimit()
+  {
+    return new OffsetLimit(scanRowsOffset, scanRowsLimit);
+  }
+
   /**
    * Returns whether this query is limited or not. Because {@link 
Long#MAX_VALUE} is used to signify unlimitedness,
    * this is equivalent to {@code getScanRowsLimit() != Long.Max_VALUE}.
@@ -667,4 +673,5 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
       return obj instanceof Integer && (int) obj == DEFAULT_BATCH_SIZE;
     }
   }
+
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
 
b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
index 4d0885da00d..8a21e7f9fb3 100644
--- 
a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
+++ 
b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java
@@ -42,8 +42,10 @@ import org.apache.druid.java.util.common.guava.BaseSequence;
 import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.java.util.common.guava.Sequences;
 import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.query.DataSource;
 import org.apache.druid.query.FrameSignaturePair;
 import org.apache.druid.query.GenericQueryMetricsFactory;
+import org.apache.druid.query.InlineDataSource;
 import org.apache.druid.query.IterableRowsCursorHelper;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryMetrics;
@@ -57,6 +59,8 @@ import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.utils.CloseableUtils;
 
+import javax.annotation.Nullable;
+
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -196,8 +200,7 @@ public class ScanQueryQueryToolChest extends 
QueryToolChest<ScanResultValue, Sca
           final ColumnCapabilities capabilities = virtualColumn.capabilities(c 
-> null, columnName);
           columnType = capabilities != null ? capabilities.toColumnType() : 
null;
         } else {
-          // Unknown type. In the future, it would be nice to have a way to 
fill these in.
-          columnType = null;
+          columnType = getDataSourceColumnType(query.getDataSource(), 
columnName);
         }
 
         builder.add(columnName, columnType);
@@ -207,6 +210,20 @@ public class ScanQueryQueryToolChest extends 
QueryToolChest<ScanResultValue, Sca
     }
   }
 
+  @Nullable
+  private ColumnType getDataSourceColumnType(DataSource dataSource, String 
columnName)
+  {
+    if (dataSource instanceof InlineDataSource) {
+      InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
+      ColumnCapabilities caps = 
inlineDataSource.getRowSignature().getColumnCapabilities(columnName);
+      if (caps != null) {
+        return caps.toColumnType();
+      }
+    }
+    // Unknown type. In the future, it would be nice to have a way to fill 
these in.
+    return null;
+  }
+
   /**
    * This batches the fetched {@link ScanResultValue}s which have similar 
signatures and are consecutives. In best case
    * it would return a single frame, and in the worst case, it would return as 
many frames as the number of {@link ScanResultValue}
diff --git 
a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java 
b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
index 20507c597b0..3afcfdb7074 100644
--- 
a/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
+++ 
b/processing/src/main/java/org/apache/druid/segment/column/TypeStrategies.java
@@ -53,6 +53,9 @@ public class TypeStrategies
   @Nullable
   public static TypeStrategy<?> getComplex(String typeName)
   {
+    if (typeName == null) {
+      return null;
+    }
     return COMPLEX_STRATEGIES.get(typeName);
   }
 
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java
new file mode 100644
index 00000000000..123e6b4198e
--- /dev/null
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.junit.Test;
+
+public class NaivePartitioningOperatorFactoryTest
+{
+  @Test
+  public void testEquals()
+  {
+    EqualsVerifier.forClass(NaivePartitioningOperatorFactory.class)
+        .usingGetClass()
+        .verify();
+  }
+}
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java
new file mode 100644
index 00000000000..4c299c2e8c3
--- /dev/null
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.junit.Test;
+
+public class NaiveSortOperatorFactoryTest
+{
+  @Test
+  public void testEquals()
+  {
+    EqualsVerifier.forClass(NaiveSortOperatorFactory.class)
+        .usingGetClass()
+        .verify();
+  }
+}
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java
new file mode 100644
index 00000000000..3a54dd5f853
--- /dev/null
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.query.operator.Operator.Signal;
+import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
+import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
+import org.apache.druid.query.rowsandcols.RowsAndColumns;
+import org.apache.druid.query.rowsandcols.column.Column;
+import org.apache.druid.query.rowsandcols.column.IntArrayColumn;
+import org.junit.Test;
+
+public class NaiveSortOperatorTest
+{
+  @Test
+  public void testNoInputisHandledCorrectly()
+  {
+    NaiveSortOperator op = new NaiveSortOperator(
+        InlineScanOperator.make(),
+        ImmutableList.of(ColumnWithDirection.ascending("someColumn"))
+    );
+
+    new OperatorTestHelper()
+        .withPushFn(() -> (someRac) -> Signal.GO)
+        .runToCompletion(op);
+  }
+
+  @Test
+  public void testSortAscending()
+  {
+    RowsAndColumns rac1 = racForColumn("c", new int[] {5, 3, 1});
+    RowsAndColumns rac2 = racForColumn("c", new int[] {2, 6, 4});
+
+    NaiveSortOperator op = new NaiveSortOperator(
+        InlineScanOperator.make(rac1, rac2),
+        ImmutableList.of(ColumnWithDirection.ascending("c"))
+    );
+
+    new OperatorTestHelper()
+        .expectAndStopAfter(
+            new RowsAndColumnsHelper()
+                .expectColumn("c", new int[] {1, 2, 3, 4, 5, 6})
+        )
+        .runToCompletion(op);
+  }
+
+  @Test
+  public void testSortDescending()
+  {
+    RowsAndColumns rac1 = racForColumn("c", new int[] {5, 3, 1});
+    RowsAndColumns rac2 = racForColumn("c", new int[] {2, 6, 4});
+
+    NaiveSortOperator op = new NaiveSortOperator(
+        InlineScanOperator.make(rac1, rac2),
+        ImmutableList.of(ColumnWithDirection.descending("c"))
+    );
+
+    new OperatorTestHelper()
+        .expectAndStopAfter(
+            new RowsAndColumnsHelper()
+                .expectColumn("c", new int[] {6, 5, 4, 3, 2, 1})
+        )
+        .runToCompletion(op);
+  }
+
+  private MapOfColumnsRowsAndColumns racForColumn(String k1, Object arr)
+  {
+    if (int.class.equals(arr.getClass().getComponentType())) {
+      return racForColumn(k1, new IntArrayColumn((int[]) arr));
+    }
+    throw new IllegalArgumentException("Not yet supported");
+  }
+
+  private MapOfColumnsRowsAndColumns racForColumn(String k1, Column v1)
+  {
+    return MapOfColumnsRowsAndColumns.fromMap(ImmutableMap.of(k1, v1));
+  }
+
+}
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java 
b/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java
new file mode 100644
index 00000000000..f6fc6cd32c9
--- /dev/null
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class OffsetLimitTest
+{
+  @Test
+  public void testNone()
+  {
+    assertFalse(OffsetLimit.NONE.isPresent());
+    assertFalse(OffsetLimit.NONE.hasOffset());
+    assertFalse(OffsetLimit.NONE.hasLimit());
+  }
+
+  @Test
+  public void testOffset()
+  {
+    int offset = 3;
+    OffsetLimit ol = new OffsetLimit(offset, -1);
+    assertTrue(ol.hasOffset());
+    assertFalse(ol.hasLimit());
+    assertEquals(offset, ol.getOffset());
+    assertEquals(-1, ol.getLimit());
+    assertEquals(Long.MAX_VALUE, ol.getLimitOrMax());
+    assertEquals(offset, ol.getFromIndex(Long.MAX_VALUE));
+    assertEquals(Long.MAX_VALUE, ol.getToIndex(Long.MAX_VALUE));
+    assertEquals(0, ol.getFromIndex(1));
+    assertEquals(0, ol.getFromIndex(offset));
+    assertEquals(0, ol.getToIndex(offset));
+  }
+
+  @Test
+  public void testLimit()
+  {
+    OffsetLimit ol = new OffsetLimit(0, 4);
+    assertFalse(ol.hasOffset());
+    assertTrue(ol.hasLimit());
+    assertEquals(0, ol.getOffset());
+    assertEquals(4, ol.getLimit());
+    assertEquals(4, ol.getLimitOrMax());
+    assertEquals(0, ol.getFromIndex(Long.MAX_VALUE));
+    assertEquals(4, ol.getToIndex(Long.MAX_VALUE));
+    assertEquals(0, ol.getFromIndex(2));
+    assertEquals(2, ol.getToIndex(2));
+  }
+
+  @Test
+  public void testOffsetLimit()
+  {
+    int offset = 3;
+    int limit = 10;
+    OffsetLimit ol = new OffsetLimit(offset, limit);
+    assertTrue(ol.hasOffset());
+    assertTrue(ol.hasLimit());
+    assertEquals(offset, ol.getOffset());
+    assertEquals(limit, ol.getLimit());
+    assertEquals(limit, ol.getLimitOrMax());
+    assertEquals(offset, ol.getFromIndex(Long.MAX_VALUE));
+    assertEquals(offset + limit, ol.getToIndex(Long.MAX_VALUE));
+    assertEquals(0, ol.getFromIndex(offset));
+    assertEquals(0, ol.getToIndex(offset));
+    assertEquals(offset, ol.getFromIndex(offset + 1));
+    assertEquals(offset + 1, ol.getToIndex(offset + 1));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testInvalidOffset()
+  {
+    new OffsetLimit(-1, -1);
+  }
+
+  @Test
+  public void testNegativeLimitsAreNotDifferent()
+  {
+    OffsetLimit ol1 = new OffsetLimit(1, -1);
+    OffsetLimit ol2 = new OffsetLimit(1, -2);
+    assertEquals(ol1, ol2);
+  }
+
+  @Test
+  public void testEquals()
+  {
+    EqualsVerifier.forClass(OffsetLimit.class).verify();
+  }
+}
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java
index 2b02d161d7a..74bb3048565 100644
--- 
a/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/ScanOperatorFactoryTest.java
@@ -61,7 +61,7 @@ public class ScanOperatorFactoryTest
     final Builder bob = new Builder();
     bob.timeRange = Intervals.utc(0, 6);
     bob.filter = DimFilters.dimEquals("abc", "b");
-    bob.limit = 48;
+    bob.offsetLimit = OffsetLimit.limit(48);
     bob.projectedColumns = Arrays.asList("a", "b");
     bob.virtualColumns = VirtualColumns.EMPTY;
     bob.ordering = 
Collections.singletonList(ColumnWithDirection.ascending("a"));
@@ -72,7 +72,7 @@ public class ScanOperatorFactoryTest
 
     Assert.assertNotEquals(factory, bob.copy().setTimeRange(null).build());
     Assert.assertNotEquals(factory, bob.copy().setFilter(null).build());
-    Assert.assertNotEquals(factory, bob.copy().setLimit(null).build());
+    Assert.assertNotEquals(factory, bob.copy().setOffsetLimit(null).build());
     Assert.assertNotEquals(factory, 
bob.copy().setProjectedColumns(null).build());
     Assert.assertNotEquals(factory, 
bob.copy().setVirtualColumns(null).build());
     Assert.assertNotEquals(factory, bob.copy().setOrdering(null).build());
@@ -132,7 +132,7 @@ public class ScanOperatorFactoryTest
                     "interval[%s], filter[%s], limit[%s], ordering[%s], 
projection[%s], virtual[%s]",
                     interval,
                     filter,
-                    limit,
+                    OffsetLimit.limit(limit),
                     ordering,
                     projection,
                     virtual
@@ -141,7 +141,7 @@ public class ScanOperatorFactoryTest
                 ScanOperatorFactory factory = new ScanOperatorFactory(
                     interval,
                     filter,
-                    limit,
+                    OffsetLimit.limit(limit),
                     projection,
                     virtual,
                     ordering
@@ -182,7 +182,7 @@ public class ScanOperatorFactoryTest
                             
(TestRowsAndColumnsDecorator.DecoratedRowsAndColumns) inRac;
 
                         Assert.assertEquals(msg, factory.getTimeRange(), 
rac.getTimeRange());
-                        Assert.assertEquals(msg, factory.getLimit(), 
rac.getLimit());
+                        Assert.assertEquals(msg, factory.getOffsetLimit(), 
rac.getOffsetLimit());
                         Assert.assertEquals(msg, factory.getVirtualColumns(), 
rac.getVirtualColumns());
                         validateList(msg, factory.getOrdering(), 
rac.getOrdering());
                         validateList(msg, factory.getProjectedColumns(), 
rac.getProjectedColumns());
@@ -228,7 +228,7 @@ public class ScanOperatorFactoryTest
   {
     private Interval timeRange;
     private DimFilter filter;
-    private Integer limit;
+    private OffsetLimit offsetLimit;
     private List<String> projectedColumns;
     private VirtualColumns virtualColumns;
     private List<ColumnWithDirection> ordering;
@@ -245,9 +245,9 @@ public class ScanOperatorFactoryTest
       return this;
     }
 
-    public Builder setLimit(Integer limit)
+    public Builder setOffsetLimit(OffsetLimit offsetLimit)
     {
-      this.limit = limit;
+      this.offsetLimit = offsetLimit;
       return this;
     }
 
@@ -274,7 +274,7 @@ public class ScanOperatorFactoryTest
       Builder retVal = new Builder();
       retVal.timeRange = timeRange;
       retVal.filter = filter;
-      retVal.limit = limit;
+      retVal.offsetLimit = offsetLimit;
       retVal.projectedColumns = projectedColumns;
       retVal.virtualColumns = virtualColumns;
       retVal.ordering = ordering;
@@ -286,7 +286,7 @@ public class ScanOperatorFactoryTest
       return new ScanOperatorFactory(
           timeRange,
           filter,
-          limit,
+          offsetLimit,
           projectedColumns,
           virtualColumns,
           ordering
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java
new file mode 100644
index 00000000000..7f1c6b00387
--- /dev/null
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query.operator;
+
+import nl.jqno.equalsverifier.EqualsVerifier;
+import org.junit.Test;
+
+public class WindowOperatorFactoryTest
+{
+  @Test
+  public void testEquals()
+  {
+    EqualsVerifier.forClass(NaivePartitioningOperatorFactory.class)
+        .usingGetClass()
+        .verify();
+  }
+}
diff --git 
a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java
 
b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java
index dcd969e57fa..48b52cf781c 100644
--- 
a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.query.operator;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import nl.jqno.equalsverifier.EqualsVerifier;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.query.InlineDataSource;
 import org.apache.druid.query.QueryContext;
@@ -131,8 +132,11 @@ public class WindowOperatorQueryTest
   @Test
   public void testEquals()
   {
-    Assert.assertEquals(query, query);
-    Assert.assertEquals(query, query.withDataSource(query.getDataSource()));
+    EqualsVerifier.simple().forClass(WindowOperatorQuery.class)
+        .withNonnullFields("duration", "querySegmentSpec")
+        .usingGetClass()
+        .verify();
+
     Assert.assertNotEquals(query, query.toString());
   }
 }
diff --git 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java
 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java
index 837e30185e2..9bd529b195f 100644
--- 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.query.rowsandcols.concrete;
 
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns;
 import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
 import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase;
@@ -38,7 +39,7 @@ public class FrameRowsAndColumnsTest extends 
RowsAndColumnsTestBase
 
   private static FrameRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns 
input)
   {
-    LazilyDecoratedRowsAndColumns rac = new 
LazilyDecoratedRowsAndColumns(input, null, null, null, Integer.MAX_VALUE, null, 
null);
+    LazilyDecoratedRowsAndColumns rac = new 
LazilyDecoratedRowsAndColumns(input, null, null, null, 
OffsetLimit.limit(Integer.MAX_VALUE), null, null);
 
     rac.numRows(); // materialize
 
diff --git 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
index 3426dd00946..ad8967c1b5b 100644
--- 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/RowsAndColumnsDecoratorTest.java
@@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.filter.InDimFilter;
 import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
 import org.apache.druid.query.rowsandcols.RowsAndColumns;
 import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
@@ -121,7 +122,7 @@ public class RowsAndColumnsDecoratorTest extends 
SemanticTestBase
         for (int k = 0; k <= limits.length; ++k) {
           int limit = (k == 0 ? -1 : limits[k - 1]);
           for (int l = 0; l <= orderings.length; ++l) {
-            validateDecorated(base, siggy, vals, interval, filter, limit, l == 
0 ? null : orderings[l - 1]);
+            validateDecorated(base, siggy, vals, interval, filter, 
OffsetLimit.limit(limit), l == 0 ? null : orderings[l - 1]);
           }
         }
       }
@@ -134,7 +135,7 @@ public class RowsAndColumnsDecoratorTest extends 
SemanticTestBase
       Object[][] originalVals,
       Interval interval,
       Filter filter,
-      int limit,
+      OffsetLimit limit,
       List<ColumnWithDirection> ordering
   )
   {
@@ -211,10 +212,10 @@ public class RowsAndColumnsDecoratorTest extends 
SemanticTestBase
       vals.sort(comparator);
     }
 
-    if (limit != -1) {
-      decor.setLimit(limit);
-
-      vals = vals.subList(0, Math.min(vals.size(), limit));
+    if (limit.isPresent()) {
+      decor.setOffsetLimit(limit);
+      int size = vals.size();
+      vals = vals.subList((int) limit.getFromIndex(size), (int) 
limit.getToIndex(vals.size()));
     }
 
     if (ordering != null) {
diff --git 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java
 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java
index ffddbb3f743..79b4b51acf5 100644
--- 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java
+++ 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestRowsAndColumnsDecorator.java
@@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
 
 import org.apache.druid.query.filter.Filter;
 import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.rowsandcols.RowsAndColumns;
 import org.apache.druid.query.rowsandcols.column.Column;
 import org.apache.druid.segment.VirtualColumns;
@@ -35,7 +36,7 @@ public class TestRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
   private Interval timeRange;
   private Filter filter;
   private VirtualColumns virtualColumns;
-  private int limit = -1;
+  private OffsetLimit offsetLimit = OffsetLimit.NONE;
   private List<ColumnWithDirection> ordering;
   private List<String> projectedColumns;
 
@@ -58,9 +59,9 @@ public class TestRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
   }
 
   @Override
-  public void setLimit(int numRows)
+  public void setOffsetLimit(OffsetLimit offsetLimit)
   {
-    this.limit = numRows;
+    this.offsetLimit = offsetLimit;
   }
 
   @Override
@@ -99,9 +100,9 @@ public class TestRowsAndColumnsDecorator implements 
RowsAndColumnsDecorator
       return virtualColumns;
     }
 
-    public int getLimit()
+    public OffsetLimit getOffsetLimit()
     {
-      return limit;
+      return offsetLimit;
     }
 
     public List<ColumnWithDirection> getOrdering()
diff --git 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java
 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java
index e53850bccc9..c26508694d4 100644
--- 
a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
 
 import com.google.common.collect.Lists;
 import org.apache.druid.query.expression.TestExprMacroTable;
+import org.apache.druid.query.operator.OffsetLimit;
 import org.apache.druid.query.operator.window.RowsAndColumnsHelper;
 import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns;
 import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
@@ -74,7 +75,7 @@ public class TestVirtualColumnEvaluationRowsAndColumnsTest 
extends SemanticTestB
             "val * 2",
             ColumnType.LONG,
             TestExprMacroTable.INSTANCE)),
-        Integer.MAX_VALUE,
+        OffsetLimit.NONE,
         null,
         null);
 
diff --git 
a/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
 
b/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
index 66f4adcdd02..19b49212ecf 100644
--- 
a/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
+++ 
b/processing/src/test/java/org/apache/druid/segment/column/TypeStrategiesTest.java
@@ -35,6 +35,8 @@ import javax.annotation.Nullable;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
+import static org.junit.Assert.assertNull;
+
 public class TypeStrategiesTest
 {
   ByteBuffer buffer = ByteBuffer.allocate(1 << 16);
@@ -637,7 +639,7 @@ public class TypeStrategiesTest
       return 
Comparators.<Long>naturalNullsFirst().thenComparing(Longs::compare).compare(this.lhs,
 o.lhs);
     }
   }
-  
+
   public static class NullableLongPairTypeStrategy implements 
TypeStrategy<NullableLongPair>
   {
 
@@ -692,4 +694,10 @@ public class TypeStrategiesTest
       return read(ByteBuffer.wrap(value));
     }
   }
+
+  @Test
+  public void getComplexTypeNull()
+  {
+    assertNull(TypeStrategies.getComplex(null));
+  }
 }
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java
 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java
index 9a4abf5b226..7095ea9275f 100644
--- 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java
+++ 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/CalciteRulesManager.java
@@ -233,7 +233,13 @@ public class CalciteRulesManager
 
     boolean isDebug = plannerContext.queryContext().isDebug();
     return ImmutableList.of(
-        Programs.sequence(preProgram, 
Programs.ofRules(druidConventionRuleSet(plannerContext))),
+        Programs.sequence(
+            new LoggingProgram("Start", isDebug),
+            preProgram,
+            new LoggingProgram("After PreProgram", isDebug),
+            Programs.ofRules(druidConventionRuleSet(plannerContext)),
+            new LoggingProgram("After volcano planner program", isDebug)
+        ),
         Programs.sequence(preProgram, 
Programs.ofRules(bindableConventionRuleSet(plannerContext))),
         Programs.sequence(
             // currently, adding logging program after every stage for easier 
debugging
diff --git 
a/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java 
b/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java
index 8d4a375d558..21d7e87b12d 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/OffsetLimit.java
@@ -76,6 +76,11 @@ public class OffsetLimit
     return limit != null;
   }
 
+  public boolean isNone()
+  {
+    return !hasLimit() && !hasOffset();
+  }
+
   public long getLimit()
   {
     Preconditions.checkState(limit != null, "limit is not present");
@@ -162,4 +167,13 @@ public class OffsetLimit
            ", limit=" + limit +
            '}';
   }
+
+  public org.apache.druid.query.operator.OffsetLimit toOperatorOffsetLimit()
+  {
+    if (hasLimit()) {
+      return new org.apache.druid.query.operator.OffsetLimit(offset, limit);
+    } else {
+      return new org.apache.druid.query.operator.OffsetLimit(offset, -1);
+    }
+  }
 }
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 c49fc6bd04e..da828ce61ba 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
@@ -47,7 +47,14 @@ import java.util.Set;
  */
 public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
 {
-  private static final TableDataSource DUMMY_DATA_SOURCE = new 
TableDataSource("__subquery__");
+  private static final TableDataSource DUMMY_DATA_SOURCE = new 
TableDataSource("__subquery__")
+  {
+    @Override
+    public boolean isConcrete()
+    {
+      return false;
+    }
+  };
 
   private static final QueryDataSource DUMMY_QUERY_DATA_SOURCE = new 
QueryDataSource(
       
Druids.newScanQueryBuilder().dataSource("__subquery__").eternityInterval().build()
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 b670d682222..54ac7c364e2 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
@@ -67,6 +67,9 @@ import org.apache.druid.query.groupby.GroupByQuery;
 import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
 import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
 import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
+import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.ColumnWithDirection.Direction;
+import org.apache.druid.query.operator.NaiveSortOperatorFactory;
 import org.apache.druid.query.operator.OperatorFactory;
 import org.apache.druid.query.operator.ScanOperatorFactory;
 import org.apache.druid.query.operator.WindowOperatorQuery;
@@ -1014,11 +1017,16 @@ public class DruidQuery
       return groupByQuery;
     }
 
-    final ScanQuery scanQuery = toScanQuery();
+    final ScanQuery scanQuery = toScanQuery(true);
     if (scanQuery != null) {
       return scanQuery;
     }
 
+    final WindowOperatorQuery scanAndSortQuery = toScanAndSortQuery();
+    if (scanAndSortQuery != null) {
+      return scanAndSortQuery;
+    }
+
     throw new CannotBuildQueryException("Cannot convert query parts into an 
actual query");
   }
 
@@ -1439,6 +1447,11 @@ public class DruidQuery
     if (windowing == null) {
       return null;
     }
+
+    // This is not yet supported
+    if (dataSource.isConcrete()) {
+      return null;
+    }
     if (dataSource instanceof TableDataSource) {
       // We need a scan query to pull the results up for us before applying 
the window
       // Returning null here to ensure that the planner generates that 
alternative
@@ -1473,13 +1486,83 @@ public class DruidQuery
     );
   }
 
+  /**
+   * Create an OperatorQuery which runs an order on top of a scan.
+   */
+  @Nullable
+  private WindowOperatorQuery toScanAndSortQuery()
+  {
+    if (sorting == null
+        || sorting.getOrderBys().isEmpty()
+        || sorting.getProjection() != null) {
+      return null;
+    }
+
+    ScanQuery scan = toScanQuery(false);
+    if (scan == null) {
+      return null;
+    }
+
+    if (dataSource.isConcrete()) {
+      // Currently only non-time orderings of subqueries are allowed.
+      List<String> orderByColumnNames = sorting.getOrderBys()
+          .stream().map(OrderByColumnSpec::getDimension)
+          .collect(Collectors.toList());
+      plannerContext.setPlanningError(
+          "SQL query requires ordering a table by non-time column [%s], which 
is not supported.",
+          orderByColumnNames
+      );
+      return null;
+    }
+
+    QueryDataSource newDataSource = new QueryDataSource(scan);
+    List<ColumnWithDirection> sortColumns = 
getColumnWithDirectionsFromOrderBys(sorting.getOrderBys());
+    RowSignature signature = getOutputRowSignature();
+    List<OperatorFactory> operators = new ArrayList<>();
+
+    operators.add(new NaiveSortOperatorFactory(sortColumns));
+    if (!sorting.getOffsetLimit().isNone()) {
+      operators.add(
+          new ScanOperatorFactory(
+              null,
+              null,
+              sorting.getOffsetLimit().toOperatorOffsetLimit(),
+              null,
+              null,
+              null
+          )
+      );
+    }
+
+    return new WindowOperatorQuery(
+        newDataSource,
+        new LegacySegmentSpec(Intervals.ETERNITY),
+        plannerContext.queryContextMap(),
+        signature,
+        operators,
+        null
+    );
+  }
+
+  private ArrayList<ColumnWithDirection> 
getColumnWithDirectionsFromOrderBys(List<OrderByColumnSpec> orderBys)
+  {
+    ArrayList<ColumnWithDirection> ordering = new ArrayList<>();
+    for (OrderByColumnSpec orderBySpec : orderBys) {
+      Direction direction = orderBySpec.getDirection() == 
OrderByColumnSpec.Direction.ASCENDING
+          ? ColumnWithDirection.Direction.ASC
+          : ColumnWithDirection.Direction.DESC;
+      ordering.add(new ColumnWithDirection(orderBySpec.getDimension(), 
direction));
+    }
+    return ordering;
+  }
+
   /**
    * Return this query as a Scan query, or null if this query is not 
compatible with Scan.
-   *
+   * @param considerSorting can be used to ignore the current sorting 
requirements {@link #toScanAndSortQuery()} uses it to produce the non-sorted 
part
    * @return query or null
    */
   @Nullable
-  private ScanQuery toScanQuery()
+  private ScanQuery toScanQuery(final boolean considerSorting)
   {
     if (grouping != null || windowing != null) {
       // Scan cannot GROUP BY or do windows.
@@ -1504,7 +1587,7 @@ public class DruidQuery
     long scanOffset = 0L;
     long scanLimit = 0L;
 
-    if (sorting != null) {
+    if (considerSorting && sorting != null) {
       scanOffset = sorting.getOffsetLimit().getOffset();
 
       if (sorting.getOffsetLimit().hasLimit()) {
diff --git 
a/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java 
b/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java
new file mode 100644
index 00000000000..29aae495e8f
--- /dev/null
+++ b/sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query;
+
+import com.google.common.base.Preconditions;
+import org.apache.druid.query.filter.DimFilter;
+import org.apache.druid.query.operator.ColumnWithDirection;
+import org.apache.druid.query.operator.ColumnWithDirection.Direction;
+import org.apache.druid.query.operator.NaivePartitioningOperatorFactory;
+import org.apache.druid.query.operator.NaiveSortOperatorFactory;
+import org.apache.druid.query.operator.OffsetLimit;
+import org.apache.druid.query.operator.OperatorFactory;
+import org.apache.druid.query.operator.ScanOperatorFactory;
+import org.apache.druid.query.operator.window.ComposingProcessor;
+import org.apache.druid.query.operator.window.Processor;
+import org.apache.druid.query.operator.window.WindowOperatorFactory;
+import org.apache.druid.query.operator.window.ranking.WindowRankProcessor;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class OperatorFactoryBuilders
+{
+
+  public static ScanOperatorFactoryBuilder scanOperatorFactoryBuilder()
+  {
+    return new ScanOperatorFactoryBuilder();
+  }
+
+  public static class ScanOperatorFactoryBuilder
+  {
+    private OffsetLimit offsetLimit;
+    private DimFilter filter;
+    private List<String> projectedColumns;
+
+    public OperatorFactory build()
+    {
+      return new ScanOperatorFactory(null, filter, offsetLimit, 
projectedColumns, null, null);
+    }
+
+    public ScanOperatorFactoryBuilder setOffsetLimit(long offset, long limit)
+    {
+      offsetLimit = new OffsetLimit(offset, limit);
+      return this;
+    }
+
+    public ScanOperatorFactoryBuilder setFilter(DimFilter filter)
+    {
+      this.filter = filter;
+      return this;
+    }
+
+    public ScanOperatorFactoryBuilder setProjectedColumns(String... columns)
+    {
+      this.projectedColumns = Arrays.asList(columns);
+      return this;
+    }
+  }
+
+  public static OperatorFactory naiveSortOperator(ColumnWithDirection... 
colWithDirs)
+  {
+    return new NaiveSortOperatorFactory(Arrays.asList(colWithDirs));
+  }
+
+  public static OperatorFactory naiveSortOperator(String column, Direction 
direction)
+  {
+    return naiveSortOperator(new ColumnWithDirection(column, direction));
+  }
+
+  public static OperatorFactory naivePartitionOperator(String... columns)
+  {
+    return new NaivePartitioningOperatorFactory(Arrays.asList(columns));
+  }
+
+  public static WindowOperatorFactory windowOperators(Processor... processors)
+  {
+    Preconditions.checkArgument(processors.length > 0, "You must specify at 
least one processor!");
+    return new WindowOperatorFactory(processors.length == 1 ? processors[0] : 
new ComposingProcessor(processors));
+  }
+
+  public static Processor rankProcessor(String outputColumn, String... 
groupingColumns)
+  {
+    return new WindowRankProcessor(Arrays.asList(groupingColumns), 
outputColumn, false);
+  }
+}
diff --git 
a/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java 
b/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java
new file mode 100644
index 00000000000..91ab769898c
--- /dev/null
+++ b/sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.query;
+
+import com.google.common.collect.Lists;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.query.operator.OperatorFactory;
+import org.apache.druid.query.operator.WindowOperatorQuery;
+import org.apache.druid.query.spec.LegacySegmentSpec;
+import org.apache.druid.query.spec.QuerySegmentSpec;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.List;
+import java.util.Map;
+
+public class WindowOperatorQueryBuilder
+{
+  private DataSource dataSource;
+  private QuerySegmentSpec intervals = new 
LegacySegmentSpec(Intervals.ETERNITY);
+  private Map<String, Object> context;
+  private RowSignature rowSignature;
+  private List<OperatorFactory> operators;
+  private List<OperatorFactory> leafOperators;
+
+  public static WindowOperatorQueryBuilder builder()
+  {
+    return new WindowOperatorQueryBuilder();
+  }
+
+  public WindowOperatorQueryBuilder setDataSource(DataSource dataSource)
+  {
+    this.dataSource = dataSource;
+    return this;
+  }
+
+  public WindowOperatorQueryBuilder setDataSource(String dataSource)
+  {
+    return setDataSource(new TableDataSource(dataSource));
+  }
+
+  public WindowOperatorQueryBuilder setDataSource(Query<?> query)
+  {
+    return setDataSource(new QueryDataSource(query));
+  }
+
+  public WindowOperatorQueryBuilder setSignature(RowSignature rowSignature)
+  {
+    this.rowSignature = rowSignature;
+    return this;
+  }
+
+  public Query<?> build()
+  {
+    return new WindowOperatorQuery(
+        dataSource,
+        intervals,
+        context,
+        rowSignature,
+        operators,
+        leafOperators);
+  }
+
+  public WindowOperatorQueryBuilder setOperators(OperatorFactory... operators)
+  {
+    this.operators = Lists.newArrayList(operators);
+    return this;
+  }
+
+  public WindowOperatorQueryBuilder setLeafOperators(OperatorFactory... 
operators)
+  {
+    this.leafOperators = Lists.newArrayList(operators);
+    return this;
+  }
+}
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 d60d6b38ee8..47cf00fc466 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
@@ -39,11 +39,13 @@ import org.apache.druid.query.Druids;
 import org.apache.druid.query.InlineDataSource;
 import org.apache.druid.query.JoinDataSource;
 import org.apache.druid.query.LookupDataSource;
+import org.apache.druid.query.OperatorFactoryBuilders;
 import org.apache.druid.query.Query;
 import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryDataSource;
 import org.apache.druid.query.TableDataSource;
 import org.apache.druid.query.UnionDataSource;
+import org.apache.druid.query.WindowOperatorQueryBuilder;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory;
@@ -96,6 +98,7 @@ 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;
+import org.apache.druid.query.operator.ColumnWithDirection;
 import org.apache.druid.query.ordering.StringComparators;
 import org.apache.druid.query.scan.ScanQuery;
 import org.apache.druid.query.scan.ScanQuery.ResultFormat;
@@ -2725,7 +2728,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
-  @NotYetSupported(Modes.CANNOT_CONVERT)
+  @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
   @Test
   public void testGroupByWithSelectAndOrderByProjections()
   {
@@ -2810,7 +2813,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
-  @NotYetSupported(Modes.CANNOT_CONVERT)
+  @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
   @Test
   public void testTopNWithSelectAndOrderByProjections()
   {
@@ -4692,7 +4695,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
-  @NotYetSupported(Modes.CANNOT_CONVERT)
+  @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
   @Test
   public void testGroupByWithSortOnPostAggregationDefault()
   {
@@ -4724,7 +4727,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
-  @NotYetSupported(Modes.CANNOT_CONVERT)
+  @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
   @Test
   public void testGroupByWithSortOnPostAggregationNoTopNConfig()
   {
@@ -4768,7 +4771,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
     );
   }
 
-  @NotYetSupported(Modes.CANNOT_CONVERT)
+  @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
   @Test
   public void testGroupByWithSortOnPostAggregationNoTopNContext()
   {
@@ -5370,7 +5373,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",
-        "SQL query requires order by non-time column [[dim1 ASC]], which is 
not supported.",
+        "SQL query requires ordering a table by non-time column [[dim1]], 
which is not supported.",
 
         // JOIN condition with not-equals (<>).
         "SELECT foo.dim1, foo.dim2, l.k, l.v\n"
@@ -13949,15 +13952,14 @@ public class CalciteQueryTest extends 
BaseCalciteQueryTest
         + "group by 1",
         ImmutableList.of(
             GroupByQuery.builder()
-                        .setDataSource(CalciteTests.DATASOURCE3)
-                        .setInterval(querySegmentSpec(Intervals.ETERNITY))
-                        .setGranularity(Granularities.ALL)
-                        .addDimension(new DefaultDimensionSpec("dim1", "_d0", 
ColumnType.STRING))
-                        .addAggregator(new LongSumAggregatorFactory("a0", 
"l1"))
-                        .setPostAggregatorSpecs(ImmutableList.of(
-                            expressionPostAgg("p0", "case_searched((\"a0\" == 
0),1,0)")
-                        ))
-                        .build()
+                .setDataSource(CalciteTests.DATASOURCE3)
+                .setInterval(querySegmentSpec(Intervals.ETERNITY))
+                .setGranularity(Granularities.ALL)
+                .addDimension(new DefaultDimensionSpec("dim1", "_d0", 
ColumnType.STRING))
+                .addAggregator(new LongSumAggregatorFactory("a0", "l1"))
+                .setPostAggregatorSpecs(ImmutableList.of(
+                    expressionPostAgg("p0", "case_searched((\"a0\" == 
0),1,0)")))
+                .build()
         ),
         useDefault ? ImmutableList.of(
             new Object[]{"", 0L},
@@ -14286,4 +14288,259 @@ public class CalciteQueryTest extends 
BaseCalciteQueryTest
 
     assertThat(e, invalidSqlIs("The query contains window functions; To run 
these window functions, enable [WINDOW_FUNCTIONS] in query context. (line [1], 
column [13])"));
   }
+
+  @Test
+  public void testInGroupByLimitOutGroupByOrderBy()
+  {
+    skipVectorize();
+    cannotVectorize();
+
+    testBuilder()
+        .sql(
+            "with t AS (SELECT m2, COUNT(m1) as trend_score\n"
+                + "FROM \"foo\"\n"
+                + "GROUP BY 1 \n"
+                + "LIMIT 10\n"
+                + ")\n"
+                + "select m2, (MAX(trend_score)) from t\n"
+                + "where m2 > 2\n"
+                + "GROUP BY 1 \n"
+                + "ORDER BY 2 DESC"
+        )
+        .expectedQuery(
+            WindowOperatorQueryBuilder.builder()
+                .setDataSource(
+                    new TopNQueryBuilder()
+                        .dataSource(CalciteTests.DATASOURCE1)
+                        .intervals(querySegmentSpec(Filtration.eternity()))
+                        .dimension(new DefaultDimensionSpec("m2", "d0", 
ColumnType.DOUBLE))
+                        .threshold(10)
+                        .aggregators(
+                            aggregators(
+                                useDefault
+                                    ? new CountAggregatorFactory("a0")
+                                    : new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        notNull("m1")
+                                    )
+                            )
+                        )
+                        .metric(new DimensionTopNMetricSpec(null, 
StringComparators.NUMERIC))
+                        .context(OUTER_LIMIT_CONTEXT)
+                        .build()
+                )
+                .setSignature(
+                    RowSignature.builder()
+                        .add("d0", ColumnType.DOUBLE)
+                        .add("a0", ColumnType.LONG)
+                        .build()
+                )
+                .setOperators(
+                    OperatorFactoryBuilders.naiveSortOperator("a0", 
ColumnWithDirection.Direction.DESC)
+                )
+                .setLeafOperators(
+                    OperatorFactoryBuilders.scanOperatorFactoryBuilder()
+                        .setOffsetLimit(0, Long.MAX_VALUE)
+                        .setFilter(
+                            range(
+                                "d0",
+                                ColumnType.LONG,
+                                2L,
+                                null,
+                                true,
+                                false
+                            )
+                        )
+                        .setProjectedColumns("a0", "d0")
+                        .build()
+                )
+                .build()
+        )
+        .expectedResults(
+            ImmutableList.of(
+                new Object[] {3.0D, 1L},
+                new Object[] {4.0D, 1L},
+                new Object[] {5.0D, 1L},
+                new Object[] {6.0D, 1L}
+            )
+        )
+        .run();
+  }
+
+  @Test
+  public void testInGroupByOrderByLimitOutGroupByOrderByLimit()
+  {
+    skipVectorize();
+    cannotVectorize();
+    String sql = "with t AS (SELECT m2 as mo, COUNT(m1) as trend_score\n"
+        + "FROM \"foo\"\n"
+        + "GROUP BY 1\n"
+        + "ORDER BY trend_score DESC\n"
+        + "LIMIT 10)\n"
+        + "select mo, (MAX(trend_score)) from t\n"
+        + "where mo > 2\n"
+        + "GROUP BY 1 \n"
+        + "ORDER BY 2 DESC  LIMIT 2 OFFSET 1\n";
+    ImmutableList<Object[]> expectedResults = ImmutableList.of(
+        new Object[] {4.0D, 1L},
+        new Object[] {5.0D, 1L}
+    );
+
+    testBuilder()
+        .sql(sql)
+        .expectedQuery(
+            WindowOperatorQueryBuilder.builder()
+                .setDataSource(
+                    new TopNQueryBuilder()
+                        .dataSource(CalciteTests.DATASOURCE1)
+                        .intervals(querySegmentSpec(Filtration.eternity()))
+                        .dimension(new DefaultDimensionSpec("m2", "d0", 
ColumnType.DOUBLE))
+                        .threshold(10)
+                        .aggregators(
+                            aggregators(
+                                useDefault
+                                    ? new CountAggregatorFactory("a0")
+                                    : new FilteredAggregatorFactory(
+                                        new CountAggregatorFactory("a0"),
+                                        notNull("m1")
+                                    )
+                            )
+                        )
+                        .metric(new NumericTopNMetricSpec("a0"))
+                        .context(OUTER_LIMIT_CONTEXT)
+                        .build()
+                )
+                .setSignature(
+                    RowSignature.builder()
+                        .add("d0", ColumnType.DOUBLE)
+                        .add("a0", ColumnType.LONG)
+                        .build()
+                )
+                .setOperators(
+                    OperatorFactoryBuilders.naiveSortOperator("a0", 
ColumnWithDirection.Direction.DESC),
+                    OperatorFactoryBuilders.scanOperatorFactoryBuilder()
+                        .setOffsetLimit(1, 2)
+                        .build()
+                )
+                .setLeafOperators(
+                    OperatorFactoryBuilders.scanOperatorFactoryBuilder()
+                        .setOffsetLimit(0, Long.MAX_VALUE)
+                        .setFilter(
+                            range(
+                                "d0",
+                                ColumnType.LONG,
+                                2L,
+                                null,
+                                true,
+                                false
+                            )
+                        )
+                        .setProjectedColumns("a0", "d0")
+                        .build()
+                )
+                .build()
+        )
+        .expectedResults(expectedResults)
+        .run();
+  }
+
+  @NotYetSupported(Modes.CANNOT_TRANSLATE)
+  @Test
+  public void testWindowingWithScanAndSort()
+  {
+    skipVectorize();
+    cannotVectorize();
+    msqIncompatible();
+    String sql = "with t AS (\n"
+        + "SELECT  \n"
+        + "    RANK() OVER (PARTITION BY m2 ORDER BY m2 ASC) \n"
+        + "      AS ranking,\n"
+        + "    COUNT(m1) as trend_score\n"
+        + "FROM foo\n"
+        + "GROUP BY m2,m1 LIMIT 10\n"
+        + ")\n"
+        + "select ranking, trend_score from t ORDER BY trend_score";
+    ImmutableList<Object[]> expectedResults = ImmutableList.of(
+        new Object[] {1L, 1L},
+        new Object[] {1L, 1L},
+        new Object[] {1L, 1L},
+        new Object[] {1L, 1L},
+        new Object[] {1L, 1L},
+        new Object[] {1L, 1L}
+    );
+
+    testBuilder()
+        .sql(sql)
+        .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, 
true))
+        .expectedQuery(
+            WindowOperatorQueryBuilder.builder()
+                .setDataSource(
+                    Druids.newScanQueryBuilder()
+                        .dataSource(
+                            new WindowOperatorQueryBuilder()
+                                .setDataSource(
+                                    GroupByQuery.builder()
+                                        
.setDataSource(CalciteTests.DATASOURCE1)
+                                        
.setInterval(querySegmentSpec(Filtration.eternity()))
+                                        .setGranularity(Granularities.ALL)
+                                        .setDimensions(
+                                            dimensions(
+                                                new DefaultDimensionSpec("m2", 
"d0", ColumnType.DOUBLE),
+                                                new DefaultDimensionSpec("m1", 
"d1", ColumnType.FLOAT)
+                                            )
+                                        )
+                                        .setAggregatorSpecs(
+                                            aggregators(
+                                                useDefault
+                                                    ? new 
CountAggregatorFactory("a0")
+                                                    : new 
FilteredAggregatorFactory(
+                                                        new 
CountAggregatorFactory("a0"),
+                                                        notNull("m1")
+                                                    )
+                                            )
+                                        )
+                                        .build()
+                                )
+                                .setOperators(
+                                    
OperatorFactoryBuilders.naivePartitionOperator("d0"),
+                                    OperatorFactoryBuilders.windowOperators(
+                                        
OperatorFactoryBuilders.rankProcessor("w0", "d0")
+                                    )
+                                )
+                                .setSignature(
+                                    RowSignature.builder()
+                                        .add("w0", ColumnType.LONG)
+                                        .add("a0", ColumnType.LONG)
+                                        .build()
+                                )
+                                .build()
+                        )
+                        .intervals(querySegmentSpec(Filtration.eternity()))
+                        .columns("a0", "w0")
+                        .context(QUERY_CONTEXT_DEFAULT)
+                        
.resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
+                        .legacy(false)
+                        .limit(10)
+                        .build()
+                )
+                .setSignature(
+                    RowSignature.builder()
+                        .add("w0", ColumnType.LONG)
+                        .add("a0", ColumnType.LONG)
+                        .build()
+                )
+                .setOperators(
+                    OperatorFactoryBuilders.naiveSortOperator("a0", 
ColumnWithDirection.Direction.ASC)
+                )
+                .setLeafOperators(
+                    OperatorFactoryBuilders.scanOperatorFactoryBuilder()
+                        .setOffsetLimit(0, Long.MAX_VALUE)
+                        .setProjectedColumns("a0", "w0")
+                        .build()
+                )
+                .build()
+        )
+        .expectedResults(expectedResults)
+        .run();
+  }
 }
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
index b0172fcd0c8..765dab45b52 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
@@ -128,18 +128,21 @@ public class CalciteWindowQueryTest extends 
BaseCalciteQueryTest
       Assert.assertEquals(1, results.recordedQueries.size());
 
       maybeDumpActualResults(results.results);
-      final WindowOperatorQuery query = 
getWindowOperatorQuery(results.recordedQueries);
-      for (int i = 0; i < input.expectedOperators.size(); ++i) {
-        final OperatorFactory expectedOperator = 
input.expectedOperators.get(i);
-        final OperatorFactory actualOperator = query.getOperators().get(i);
-        if (!expectedOperator.validateEquivalent(actualOperator)) {
-          assertEquals("Operator Mismatch, index[" + i + "]",
-              queryJackson.writeValueAsString(expectedOperator),
-              queryJackson.writeValueAsString(actualOperator));
-          fail("validateEquivalent failed; but textual comparision of 
operators didn't reported the mismatch!");
+      if (input.expectedOperators != null) {
+        final WindowOperatorQuery query = 
getWindowOperatorQuery(results.recordedQueries);
+        for (int i = 0; i < input.expectedOperators.size(); ++i) {
+          final OperatorFactory expectedOperator = 
input.expectedOperators.get(i);
+          final OperatorFactory actualOperator = query.getOperators().get(i);
+          if (!expectedOperator.validateEquivalent(actualOperator)) {
+            assertEquals("Operator Mismatch, index[" + i + "]",
+                queryJackson.writeValueAsString(expectedOperator),
+                queryJackson.writeValueAsString(actualOperator));
+            fail("validateEquivalent failed; but textual comparision of 
operators didn't reported the mismatch!");
+          }
         }
       }
-      final RowSignature outputSignature = query.getRowSignature();
+
+      final RowSignature outputSignature = results.signature;
       ColumnType[] types = new ColumnType[outputSignature.size()];
       for (int i = 0; i < outputSignature.size(); ++i) {
         types[i] = outputSignature.getColumnType(i).get();
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
index 7a2b9b70f1a..7ac536c0f8d 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java
@@ -4475,7 +4475,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("aggregates/aggOWnFn_3")
   @Test
   public void test_aggregates_aggOWnFn_3()
@@ -4483,7 +4482,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("aggregates/aggOWnFn_4")
   @Test
   public void test_aggregates_aggOWnFn_4()
@@ -4491,7 +4489,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("first_val/firstValFn_29")
   @Test
   public void test_first_val_firstValFn_29()
@@ -4499,7 +4496,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("first_val/firstValFn_32")
   @Test
   public void test_first_val_firstValFn_32()
@@ -4507,7 +4503,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("first_val/firstValFn_33")
   @Test
   public void test_first_val_firstValFn_33()
@@ -4523,7 +4519,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("lag_func/lag_Fn_9")
   @Test
   public void test_lag_func_lag_Fn_9()
@@ -4531,7 +4526,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("last_val/lastValFn_29")
   @Test
   public void test_last_val_lastValFn_29()
@@ -4539,7 +4533,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("last_val/lastValFn_34")
   @Test
   public void test_last_val_lastValFn_34()
@@ -4547,7 +4541,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("last_val/lastValFn_35")
   @Test
   public void test_last_val_lastValFn_35()
@@ -4555,7 +4549,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("last_val/lastValFn_38")
   @Test
   public void test_last_val_lastValFn_38()
@@ -4563,7 +4557,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("last_val/lastValFn_39")
   @Test
   public void test_last_val_lastValFn_39()
@@ -4579,7 +4573,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("ntile_func/ntileFn_33")
   @Test
   public void test_ntile_func_ntileFn_33()
@@ -4587,7 +4580,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
   @DrillTest("ntile_func/ntileFn_34")
   @Test
   public void test_ntile_func_ntileFn_34()
@@ -4595,7 +4587,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_47")
   @Test
   public void test_ntile_func_ntileFn_47()
@@ -4603,7 +4595,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_48")
   @Test
   public void test_ntile_func_ntileFn_48()
@@ -4611,7 +4603,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_49")
   @Test
   public void test_ntile_func_ntileFn_49()
@@ -4619,7 +4611,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_50")
   @Test
   public void test_ntile_func_ntileFn_50()
@@ -4627,7 +4619,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_COUNT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_51")
   @Test
   public void test_ntile_func_ntileFn_51()
@@ -4635,7 +4627,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_52")
   @Test
   public void test_ntile_func_ntileFn_52()
@@ -4643,7 +4635,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_53")
   @Test
   public void test_ntile_func_ntileFn_53()
@@ -4651,7 +4643,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_54")
   @Test
   public void test_ntile_func_ntileFn_54()
@@ -4659,7 +4651,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_55")
   @Test
   public void test_ntile_func_ntileFn_55()
@@ -4667,7 +4659,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_56")
   @Test
   public void test_ntile_func_ntileFn_56()
@@ -4675,7 +4667,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_57")
   @Test
   public void test_ntile_func_ntileFn_57()
@@ -4683,7 +4675,7 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.NOT_ENOUGH_RULES)
+  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("ntile_func/ntileFn_58")
   @Test
   public void test_ntile_func_ntileFn_58()
@@ -6695,7 +6687,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/defaultFrame/RBUPACR_chr_3")
   @Test
   public void test_frameclause_defaultFrame_RBUPACR_chr_3()
@@ -6820,7 +6811,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/defaultFrame/RBUPACR_vchr_3")
   @Test
   public void test_frameclause_defaultFrame_RBUPACR_vchr_3()
@@ -6844,7 +6834,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/multipl_wnwds/count_mulwds")
   @Test
   public void test_frameclause_multipl_wnwds_count_mulwds()
@@ -6908,7 +6897,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBCRACR/RBCRACR_char_3")
   @Test
   public void test_frameclause_RBCRACR_RBCRACR_char_3()
@@ -7010,7 +6998,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBCRACR/RBCRACR_vchar_3")
   @Test
   public void test_frameclause_RBCRACR_RBCRACR_vchar_3()
@@ -7081,7 +7068,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBUPACR/RBUPACR_chr_3")
   @Test
   public void test_frameclause_RBUPACR_RBUPACR_chr_3()
@@ -7159,7 +7145,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBUPACR/RBUPACR_vchr_3")
   @Test
   public void test_frameclause_RBUPACR_RBUPACR_vchr_3()
@@ -7190,7 +7175,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBUPAUF/RBUPAUF_char_3")
   @Test
   public void test_frameclause_RBUPAUF_RBUPAUF_char_3()
@@ -7247,7 +7231,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/RBUPAUF/RBUPAUF_vchar_3")
   @Test
   public void test_frameclause_RBUPAUF_RBUPAUF_vchar_3()
@@ -7255,7 +7238,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/subQueries/frmInSubQry_53")
   @Test
   public void test_frameclause_subQueries_frmInSubQry_53()
@@ -7263,7 +7245,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/subQueries/frmInSubQry_54")
   @Test
   public void test_frameclause_subQueries_frmInSubQry_54()
@@ -7271,7 +7252,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("frameclause/subQueries/frmInSubQry_55")
   @Test
   public void test_frameclause_subQueries_frmInSubQry_55()
@@ -7621,7 +7601,6 @@ public class DrillWindowQueryTest extends 
BaseCalciteQueryTest
     windowQueryTest();
   }
 
-  @NotYetSupported(Modes.RESULT_MISMATCH)
   @DrillTest("nestedAggs/emtyOvrCls_13")
   @Test
   public void test_nestedAggs_emtyOvrCls_13()
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java
index 86e5c41d6de..7f4e6a06993 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java
@@ -89,7 +89,8 @@ public @interface NotYetSupported
     // at least c7 is represented oddly in the parquet file
     T_ALLTYPES_ISSUES(AssertionError.class, 
"(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"),
     RESULT_MISMATCH(AssertionError.class, "assertResultsEquals"),
-    UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering 
with NULLS (LAST|FIRST)");
+    UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering 
with NULLS (LAST|FIRST)"),
+    CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference");
 
     public Class<? extends Throwable> throwableClass;
     public String regex;
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 6e38c26e4f3..4d75cac5ae0 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
@@ -1391,7 +1391,7 @@ public class SqlResourceTest extends CalciteTestBase
         DruidException.Persona.ADMIN,
         DruidException.Category.INVALID_INPUT,
         "Query could not be planned. A possible reason is "
-        + "[SQL query requires order by non-time column [[dim1 ASC]], which is 
not supported.]"
+        + "[SQL query requires ordering a table by non-time column [[dim1]], 
which is not supported.]"
     );
     checkSqlRequestLog(false);
     Assert.assertTrue(lifecycleManager.getAll("id").isEmpty());
diff --git 
a/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest 
b/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest
new file mode 100644
index 00000000000..060366d934b
--- /dev/null
+++ b/sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest
@@ -0,0 +1,31 @@
+type: "operatorValidation"
+
+sql: |
+  SELECT
+    RANK() OVER (PARTITION BY m1 ORDER BY m2 ASC) AS ranking,
+    m1,m2,dim1,dim2
+  FROM foo
+     
+
+expectedOperators:
+  - type: "naiveSort"
+    columns:
+      - column: "m1"
+        direction: "ASC"
+      - column: "m2"
+        direction: "ASC"
+  - { type: "naivePartition", partitionColumns: [ m1 ] }
+  - type: "window"
+    processor:
+      type: "rank"
+      group: [ m2 ]
+      outputColumn: w0
+      asPercent: false
+
+expectedResults:
+  - [1,1.0,1.0,"","a"]
+  - [1,2.0,2.0,"10.1",null]
+  - [1,3.0,3.0,"2",""]
+  - [1,4.0,4.0,"1","a"]
+  - [1,5.0,5.0,"def","abc"]
+  - [1,6.0,6.0,"abc",null]
diff --git 
a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest
 
b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest
index affed86e6b8..0180f615313 100644
--- 
a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest
+++ 
b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrderingDesc.sqlTest
@@ -1,4 +1,4 @@
-type: "failingTest"
+type: "operatorValidation"
 
 sql: |
     SELECT
@@ -11,15 +11,993 @@ sql: |
     GROUP BY 1, 2
     ORDER BY 1 DESC, 2 DESC
 
-expectedOperators:
-  - { type: "naivePartition",  partitionColumns: [ "d0" ] }
-  - type: "window"
-    processor:
-      type: "framedAgg"
-      frame: { peerType: "ROWS", lowUnbounded: false, lowOffset: 3, 
uppUnbounded: false, uppOffset: 2 }
-      aggregations:
-        - { type: "longSum", name: "w0", fieldName: "a0" }
-  - { type: "naiveSort", columns: [ { column: "d1", direction: "DESC" }, { 
column: "a0", direction: "DESC"} ]}
-  - { type: "naivePartition",  partitionColumns: [ "d1" ] }
-  - type: "window"
-    processor: { type: "rowNumber", outputColumn: "w1"}
\ No newline at end of file
+# expectedOperators are not validated as the query right now have 2 
windowOperatorQuery objects
+
+expectedResults:
+  - ["ZW",1442048400000,254,254,16]
+  - ["ZW",1442044800000,0,254,33]
+  - ["ZM",1442041200000,133,133,14]
+  - ["ZA",1442091600000,1,51,32]
+  - ["ZA",1442070000000,0,130,44]
+  - ["ZA",1442059200000,50,127,25]
+  - ["ZA",1442048400000,79,127,22]
+  - ["ZA",1442034000000,-3,126,30]
+  - ["VN",1442084400000,-10,1280,46]
+  - ["VN",1442070000000,479,1426,11]
+  - ["VN",1442066400000,811,1434,11]
+  - ["VN",1442062800000,146,1471,17]
+  - ["VN",1442059200000,8,1571,32]
+  - ["VN",1442055600000,37,1077,26]
+  - ["VN",1442052000000,90,266,23]
+  - ["VN",1442048400000,-15,109,41]
+  - ["VN",1442041200000,0,72,37]
+  - ["VN",1442037600000,-11,98,33]
+  - ["VN",1442034000000,-29,-1,33]
+  - ["VN",1442026800000,63,14,22]
+  - ["VN",1442023200000,-9,14,30]
+  - ["VG",1442062800000,-238,-238,42]
+  - ["VE",1442098800000,9,104,22]
+  - ["VE",1442095200000,35,516,21]
+  - ["VE",1442084400000,60,936,27]
+  - ["VE",1442077200000,412,954,17]
+  - ["VE",1442070000000,420,943,13]
+  - ["VE",1442066400000,18,959,31]
+  - ["VE",1442034000000,-2,882,29]
+  - ["VE",1442030400000,51,585,17]
+  - ["VE",1442026800000,-17,165,32]
+  - ["VE",1442023200000,115,147,16]
+  - ["UZ",1442044800000,1369,1369,6]
+  - ["UY",1442077200000,23,265,30]
+  - ["UY",1442073600000,-42,266,41]
+  - ["UY",1442070000000,284,342,17]
+  - ["UY",1442037600000,1,859,29]
+  - ["UY",1442026800000,76,913,21]
+  - ["UY",1442023200000,517,955,10]
+  - ["UY",1442019600000,77,671,17]
+  - ["US",1442098800000,3575,6493,3]
+  - ["US",1442095200000,416,8184,11]
+  - ["US",1442091600000,2502,10707,6]
+  - ["US",1442088000000,1691,14708,8]
+  - ["US",1442084400000,2523,13301,5]
+  - ["US",1442080800000,4001,13985,4]
+  - ["US",1442077200000,2168,14988,9]
+  - ["US",1442073600000,1100,14069,13]
+  - ["US",1442070000000,3505,11593,4]
+  - ["US",1442066400000,772,7603,12]
+  - ["US",1442062800000,47,5591,22]
+  - ["US",1442059200000,11,4489,30]
+  - ["US",1442055600000,156,518,20]
+  - ["US",1442052000000,-2,-115,41]
+  - ["US",1442048400000,-466,1837,46]
+  - ["US",1442044800000,139,5501,19]
+  - ["US",1442041200000,1999,8993,2]
+  - ["US",1442037600000,3675,11018,3]
+  - ["US",1442034000000,3648,12996,2]
+  - ["US",1442030400000,2023,15701,4]
+  - ["US",1442026800000,1512,14745,5]
+  - ["US",1442023200000,2844,11070,2]
+  - ["US",1442019600000,1043,7422,5]
+  - ["US",1442016000000,0,5399,10]
+  - ["UG",1442070000000,1,1,42]
+  - ["UA",1442098800000,38,-380,17]
+  - ["UA",1442095200000,-30,-401,40]
+  - ["UA",1442091600000,-388,-396,41]
+  - ["UA",1442088000000,-21,-397,33]
+  - ["UA",1442084400000,5,-616,37]
+  - ["UA",1442080800000,-1,3655,39]
+  - ["UA",1442077200000,-181,5776,46]
+  - ["UA",1442073600000,4241,6093,5]
+  - ["UA",1442070000000,1733,6861,7]
+  - ["UA",1442066400000,296,6860,18]
+  - ["UA",1442062800000,773,21243,9]
+  - ["UA",1442059200000,-2,17412,38]
+  - ["UA",1442055600000,14202,15681,2]
+  - ["UA",1442052000000,410,15665,13]
+  - ["UA",1442048400000,2,14966,36]
+  - ["UA",1442044800000,280,14967,15]
+  - ["UA",1442041200000,74,4233,19]
+  - ["UA",1442037600000,-1,3823,31]
+  - ["UA",1442034000000,3468,3821,3]
+  - ["TW",1442098800000,-60,-137,39]
+  - ["TW",1442095200000,-77,-113,41]
+  - ["TW",1442084400000,0,389,39]
+  - ["TW",1442080800000,24,1161,24]
+  - ["TW",1442077200000,502,1706,16]
+  - ["TW",1442073600000,772,2407,15]
+  - ["TW",1442070000000,485,2135,10]
+  - ["TW",1442066400000,624,1954,14]
+  - ["TW",1442062800000,-272,1500,43]
+  - ["TW",1442059200000,-157,752,44]
+  - ["TW",1442055600000,48,342,24]
+  - ["TW",1442052000000,24,-258,32]
+  - ["TW",1442048400000,75,380,23]
+  - ["TW",1442044800000,24,803,27]
+  - ["TW",1442041200000,366,898,9]
+  - ["TW",1442037600000,266,874,9]
+  - ["TW",1442034000000,143,1479,18]
+  - ["TW",1442030400000,0,1552,26]
+  - ["TW",1442026800000,680,1186,9]
+  - ["TW",1442023200000,97,1012,19]
+  - ["TW",1442019600000,0,869,32]
+  - ["TW",1442016000000,92,869,5]
+  - ["TT",1442088000000,9,9,30]
+  - ["TR",1442095200000,-29,5408,39]
+  - ["TR",1442091600000,3048,5578,4]
+  - ["TR",1442088000000,2389,5577,6]
+  - ["TR",1442084400000,170,5666,22]
+  - ["TR",1442080800000,-1,5931,38]
+  - ["TR",1442077200000,89,2968,24]
+  - ["TR",1442070000000,236,894,19]
+  - ["TR",1442066400000,85,1023,24]
+  - ["TR",1442062800000,315,1065,12]
+  - ["TR",1442055600000,299,1064,15]
+  - ["TR",1442052000000,41,869,28]
+  - ["TR",1442048400000,88,785,21]
+  - ["TR",1442044800000,41,776,24]
+  - ["TR",1442041200000,1,477,35]
+  - ["TR",1442023200000,306,436,13]
+  - ["TN",1442098800000,-9,-9,36]
+  - ["TJ",1442048400000,1471,1471,4]
+  - ["TH",1442084400000,13,-21,35]
+  - ["TH",1442070000000,0,-67,43]
+  - ["TH",1442066400000,-34,-67,45]
+  - ["TH",1442062800000,-46,-89,39]
+  - ["TH",1442055600000,0,8,37]
+  - ["TH",1442052000000,-22,11,46]
+  - ["TH",1442044800000,110,45,20]
+  - ["TH",1442041200000,3,91,33]
+  - ["TH",1442034000000,0,91,27]
+  - ["SV",1442088000000,9,114,29]
+  - ["SV",1442084400000,106,114,25]
+  - ["SV",1442019600000,-1,114,34]
+  - ["SK",1442098800000,7,361,25]
+  - ["SK",1442084400000,-92,367,50]
+  - ["SK",1442073600000,446,380,18]
+  - ["SK",1442062800000,6,379,31]
+  - ["SK",1442052000000,13,372,35]
+  - ["SK",1442037600000,-1,464,30]
+  - ["SI",1442091600000,9,-36,31]
+  - ["SI",1442080800000,-45,-36,46]
+  - ["SG",1442066400000,0,440,41]
+  - ["SG",1442062800000,388,517,11]
+  - ["SG",1442048400000,52,576,27]
+  - ["SG",1442044800000,77,579,21]
+  - ["SG",1442041200000,59,580,20]
+  - ["SG",1442037600000,3,2950,27]
+  - ["SG",1442030400000,1,2898,24]
+  - ["SG",1442026800000,2758,2821,3]
+  - ["SE",1442098800000,0,60,32]
+  - ["SE",1442095200000,61,97,20]
+  - ["SE",1442091600000,-1,186,35]
+  - ["SE",1442084400000,37,264,31]
+  - ["SE",1442080800000,89,278,15]
+  - ["SE",1442070000000,78,1693,26]
+  - ["SE",1442066400000,14,1689,32]
+  - ["SE",1442059200000,1476,1653,5]
+  - ["SE",1442055600000,-5,1419,41]
+  - ["SE",1442052000000,1,1432,36]
+  - ["SE",1442048400000,-145,1448,43]
+  - ["SE",1442041200000,91,-25,17]
+  - ["SE",1442030400000,30,89,20]
+  - ["SE",1442023200000,3,88,27]
+  - ["SE",1442019600000,109,233,14]
+  - ["SA",1442084400000,458,410,13]
+  - ["SA",1442077200000,-50,1686,42]
+  - ["SA",1442073600000,2,1686,33]
+  - ["SA",1442066400000,1276,1697,8]
+  - ["SA",1442059200000,0,1253,37]
+  - ["SA",1442055600000,11,1206,32]
+  - ["SA",1442048400000,14,1204,33]
+  - ["SA",1442037600000,-97,-72,34]
+  - ["RU",1442098800000,12098,15982,2]
+  - ["RU",1442095200000,435,18578,10]
+  - ["RU",1442091600000,3449,25039,3]
+  - ["RU",1442088000000,2596,25694,5]
+  - ["RU",1442084400000,6461,14758,3]
+  - ["RU",1442080800000,655,15941,7]
+  - ["RU",1442077200000,1162,17198,11]
+  - ["RU",1442073600000,1618,16649,10]
+  - ["RU",1442070000000,4706,10356,3]
+  - ["RU",1442066400000,2047,13603,5]
+  - ["RU",1442062800000,168,12940,16]
+  - ["RU",1442059200000,3902,12536,4]
+  - ["RU",1442055600000,499,8857,10]
+  - ["RU",1442052000000,1214,9374,8]
+  - ["RU",1442048400000,1027,9786,6]
+  - ["RU",1442044800000,2564,5560,3]
+  - ["RU",1442041200000,580,5719,8]
+  - ["RU",1442037600000,-324,4581,36]
+  - ["RU",1442034000000,658,3554,10]
+  - ["RU",1442030400000,76,1289,16]
+  - ["RU",1442026800000,0,2923,29]
+  - ["RU",1442023200000,299,3247,14]
+  - ["RU",1442019600000,2214,2589,2]
+  - ["RS",1442091600000,-15,74,39]
+  - ["RS",1442084400000,89,887,26]
+  - ["RS",1442080800000,0,887,37]
+  - ["RS",1442073600000,813,900,14]
+  - ["RS",1442066400000,0,921,40]
+  - ["RS",1442062800000,13,832,29]
+  - ["RS",1442019600000,6,832,27]
+  - ["RO",1442095200000,824,839,7]
+  - ["RO",1442091600000,0,810,34]
+  - ["RO",1442073600000,15,1351,31]
+  - ["RO",1442070000000,-29,1377,47]
+  - ["RO",1442062800000,541,872,10]
+  - ["RO",1442055600000,26,1156,28]
+  - ["RO",1442052000000,319,1986,16]
+  - ["RO",1442044800000,284,2083,14]
+  - ["RO",1442041200000,845,1542,6]
+  - ["RO",1442034000000,68,1516,20]
+  - ["QA",1442041200000,13,13,28]
+  - ["PY",1442084400000,628,634,10]
+  - ["PY",1442080800000,5,634,30]
+  - ["PY",1442019600000,1,634,29]
+  - ["PT",1442098800000,2,424,30]
+  - ["PT",1442095200000,19,345,24]
+  - ["PT",1442088000000,403,270,14]
+  - ["PT",1442080800000,-79,3740,47]
+  - ["PT",1442077200000,-75,3750,44]
+  - ["PT",1442070000000,3470,3833,5]
+  - ["PT",1442066400000,12,3441,33]
+  - ["PT",1442052000000,102,3692,22]
+  - ["PT",1442044800000,11,3767,30]
+  - ["PT",1442019600000,172,297,13]
+  - ["PR",1442095200000,29,-1,22]
+  - ["PR",1442077200000,5,1,32]
+  - ["PR",1442059200000,-35,23,41]
+  - ["PR",1442030400000,2,23,22]
+  - ["PR",1442026800000,22,-6,24]
+  - ["PL",1442098800000,-9,2744,35]
+  - ["PL",1442095200000,1851,3090,4]
+  - ["PL",1442091600000,902,3103,8]
+  - ["PL",1442088000000,346,3110,15]
+  - ["PL",1442084400000,13,3443,34]
+  - ["PL",1442080800000,7,1622,28]
+  - ["PL",1442077200000,324,866,19]
+  - ["PL",1442073600000,30,554,28]
+  - ["PL",1442070000000,146,4712,23]
+  - ["PL",1442066400000,34,4904,30]
+  - ["PL",1442062800000,4171,4990,4]
+  - ["PL",1442059200000,199,5290,21]
+  - ["PL",1442055600000,410,5510,14]
+  - ["PL",1442052000000,330,5795,15]
+  - ["PL",1442048400000,366,1905,13]
+  - ["PL",1442044800000,319,1801,12]
+  - ["PL",1442041200000,281,1391,10]
+  - ["PL",1442037600000,95,1061,14]
+  - ["PK",1442070000000,43,81,31]
+  - ["PK",1442062800000,23,105,24]
+  - ["PK",1442048400000,15,205,32]
+  - ["PK",1442041200000,24,306,25]
+  - ["PK",1442037600000,100,598,12]
+  - ["PK",1442026800000,101,575,18]
+  - ["PK",1442019600000,335,560,10]
+  - ["PH",1442098800000,8,863,24]
+  - ["PH",1442091600000,816,895,9]
+  - ["PH",1442084400000,39,897,30]
+  - ["PH",1442080800000,32,670,20]
+  - ["PH",1442077200000,2,696,33]
+  - ["PH",1442073600000,-227,1760,43]
+  - ["PH",1442070000000,34,1892,33]
+  - ["PH",1442066400000,1880,2133,6]
+  - ["PH",1442062800000,171,4100,15]
+  - ["PH",1442059200000,273,4349,17]
+  - ["PH",1442055600000,1969,4377,5]
+  - ["PH",1442052000000,22,2552,33]
+  - ["PH",1442048400000,62,2381,24]
+  - ["PH",1442044800000,55,2125,23]
+  - ["PH",1442041200000,0,215,36]
+  - ["PH",1442037600000,17,219,20]
+  - ["PH",1442034000000,59,1067,21]
+  - ["PH",1442030400000,26,1471,21]
+  - ["PH",1442026800000,910,1477,8]
+  - ["PH",1442023200000,459,1460,11]
+  - ["PH",1442019600000,6,1401,26]
+  - ["PE",1442098800000,1861,1774,4]
+  - ["PE",1442095200000,-19,1772,38]
+  - ["PE",1442084400000,-68,1609,49]
+  - ["PE",1442080800000,-2,1597,40]
+  - ["PE",1442077200000,-163,-276,45]
+  - ["PE",1442062800000,-12,-231,37]
+  - ["PE",1442026800000,-12,360,31]
+  - ["PE",1442023200000,26,362,23]
+  - ["PE",1442019600000,523,525,7]
+  - ["PA",1442026800000,0,0,28]
+  - ["OM",1442052000000,0,0,39]
+  - ["NZ",1442098800000,-2,399,33]
+  - ["NZ",1442095200000,-4,347,37]
+  - ["NZ",1442088000000,405,775,13]
+  - ["NZ",1442084400000,-52,964,48]
+  - ["NZ",1442059200000,428,1032,14]
+  - ["NZ",1442048400000,189,1671,17]
+  - ["NZ",1442037600000,66,1294,16]
+  - ["NZ",1442026800000,635,1346,11]
+  - ["NZ",1442019600000,28,918,24]
+  - ["NP",1442048400000,61,61,25]
+  - ["NO",1442098800000,2,16,29]
+  - ["NO",1442095200000,-1,31,34]
+  - ["NO",1442091600000,15,62,29]
+  - ["NO",1442088000000,15,284,27]
+  - ["NO",1442080800000,31,353,21]
+  - ["NO",1442073600000,222,383,20]
+  - ["NO",1442066400000,71,815,26]
+  - ["NO",1442055600000,29,353,27]
+  - ["NO",1442052000000,447,370,11]
+  - ["NO",1442048400000,-447,148,45]
+  - ["NO",1442019600000,48,77,20]
+  - ["NL",1442098800000,4,93,26]
+  - ["NL",1442095200000,70,105,18]
+  - ["NL",1442091600000,19,541,28]
+  - ["NL",1442088000000,12,9488,28]
+  - ["NL",1442084400000,436,10362,14]
+  - ["NL",1442080800000,8947,10458,2]
+  - ["NL",1442077200000,878,10355,13]
+  - ["NL",1442073600000,166,10404,22]
+  - ["NL",1442070000000,-84,9938,49]
+  - ["NL",1442066400000,61,1197,27]
+  - ["NL",1442062800000,-30,424,38]
+  - ["NL",1442059200000,206,311,20]
+  - ["NL",1442055600000,105,1698,21]
+  - ["NL",1442052000000,53,1653,24]
+  - ["NL",1442048400000,1303,1683,5]
+  - ["NL",1442044800000,16,1477,29]
+  - ["NL",1442034000000,0,1372,26]
+  - ["NG",1442070000000,6,214,39]
+  - ["NG",1442052000000,208,214,18]
+  - ["MY",1442098800000,739,730,5]
+  - ["MY",1442077200000,-10,731,38]
+  - ["MY",1442073600000,1,731,35]
+  - ["MY",1442066400000,1,732,35]
+  - ["MY",1442059200000,0,642,36]
+  - ["MY",1442055600000,1,525,35]
+  - ["MY",1442048400000,649,1459,10]
+  - ["MY",1442044800000,-127,2486,39]
+  - ["MY",1442041200000,935,2483,4]
+  - ["MY",1442034000000,1028,2475,6]
+  - ["MY",1442030400000,-3,1826,28]
+  - ["MY",1442019600000,-7,1953,35]
+  - ["MX",1442098800000,28,371,18]
+  - ["MX",1442095200000,-456,865,44]
+  - ["MX",1442091600000,799,1846,11]
+  - ["MX",1442088000000,494,1470,12]
+  - ["MX",1442084400000,981,5316,7]
+  - ["MX",1442080800000,-376,5751,49]
+  - ["MX",1442077200000,3874,4951,5]
+  - ["MX",1442073600000,-21,4456,40]
+  - ["MX",1442070000000,-1,3181,45]
+  - ["MX",1442066400000,-1,3561,42]
+  - ["MX",1442041200000,-294,631,43]
+  - ["MX",1442037600000,4,1025,24]
+  - ["MX",1442034000000,944,4668,8]
+  - ["MX",1442030400000,373,5218,12]
+  - ["MX",1442026800000,3642,5445,2]
+  - ["MX",1442023200000,549,5441,9]
+  - ["MX",1442016000000,-67,4497,13]
+  - ["MV",1442073600000,-3,-3,39]
+  - ["MT",1442048400000,-1,-1,38]
+  - ["MR",1442080800000,10,10,27]
+  - ["MO",1442070000000,18,48,36]
+  - ["MO",1442034000000,30,48,22]
+  - ["MM",1442073600000,25,28,29]
+  - ["MM",1442070000000,3,28,40]
+  - ["MK",1442077200000,-72,-72,43]
+  - ["MH",1442052000000,40,40,29]
+  - ["ME",1442073600000,0,0,37]
+  - ["MD",1442077200000,6916,6916,2]
+  - ["MA",1442098800000,8,263,23]
+  - ["MA",1442080800000,5,263,29]
+  - ["MA",1442077200000,250,207,20]
+  - ["MA",1442062800000,0,230,36]
+  - ["MA",1442059200000,-56,221,42]
+  - ["MA",1442055600000,23,216,29]
+  - ["MA",1442019600000,-1,-34,33]
+  - ["LV",1442095200000,0,0,32]
+  - ["LU",1442095200000,2,527,28]
+  - ["LU",1442077200000,525,606,15]
+  - ["LU",1442066400000,0,606,39]
+  - ["LU",1442059200000,79,606,23]
+  - ["LT",1442098800000,-24,-12,37]
+  - ["LT",1442080800000,12,-12,26]
+  - ["LK",1442084400000,-3,52,43]
+  - ["LK",1442052000000,47,131,26]
+  - ["LK",1442048400000,8,131,35]
+  - ["LK",1442026800000,79,131,20]
+  - ["LB",1442055600000,-67,-67,43]
+  - ["KZ",1442095200000,91,-248,16]
+  - ["KZ",1442084400000,-22,-248,47]
+  - ["KZ",1442077200000,-317,-248,48]
+  - ["KZ",1442066400000,0,-215,38]
+  - ["KZ",1442062800000,0,-243,35]
+  - ["KZ",1442059200000,33,191,27]
+  - ["KZ",1442055600000,63,947,23]
+  - ["KZ",1442052000000,412,1348,12]
+  - ["KZ",1442048400000,439,1509,12]
+  - ["KZ",1442044800000,401,1476,10]
+  - ["KZ",1442034000000,161,1413,17]
+  - ["KW",1442080800000,-33,1780,45]
+  - ["KW",1442077200000,-2,1778,36]
+  - ["KW",1442070000000,1815,1778,6]
+  - ["KW",1442055600000,-2,1778,40]
+  - ["KR",1442098800000,-36,1315,38]
+  - ["KR",1442095200000,827,1629,6]
+  - ["KR",1442088000000,524,1596,11]
+  - ["KR",1442084400000,314,1556,18]
+  - ["KR",1442080800000,-33,1814,44]
+  - ["KR",1442077200000,-40,4286,41]
+  - ["KR",1442070000000,222,4858,20]
+  - ["KR",1442066400000,3299,4752,3]
+  - ["KR",1442062800000,1096,8425,7]
+  - ["KR",1442059200000,208,8462,19]
+  - ["KR",1442055600000,3640,7866,4]
+  - ["KR",1442052000000,-3,5396,43]
+  - ["KR",1442048400000,-374,4320,44]
+  - ["KR",1442044800000,829,4138,8]
+  - ["KR",1442041200000,20,932,27]
+  - ["KR",1442037600000,26,1970,18]
+  - ["KR",1442034000000,434,2165,12]
+  - ["KR",1442030400000,1035,1655,7]
+  - ["KR",1442026800000,-179,2080,34]
+  - ["KR",1442023200000,319,3078,12]
+  - ["KR",1442019600000,445,2644,8]
+  - ["KR",1442016000000,1024,1609,2]
+  - ["KG",1442073600000,6,6,32]
+  - ["KE",1442044800000,-1,-1,34]
+  - ["JP",1442098800000,-6,222,34]
+  - ["JP",1442095200000,0,279,31]
+  - ["JP",1442091600000,228,292,18]
+  - ["JP",1442088000000,57,712,24]
+  - ["JP",1442084400000,13,769,33]
+  - ["JP",1442080800000,420,1931,10]
+  - ["JP",1442077200000,51,1782,27]
+  - ["JP",1442073600000,1162,1892,12]
+  - ["JP",1442070000000,79,2682,25]
+  - ["JP",1442066400000,167,2177,21]
+  - ["JP",1442062800000,803,3124,8]
+  - ["JP",1442059200000,-85,4751,43]
+  - ["JP",1442055600000,998,6653,8]
+  - ["JP",1442052000000,2789,8055,4]
+  - ["JP",1442048400000,1981,8625,3]
+  - ["JP",1442044800000,1569,10891,4]
+  - ["JP",1442041200000,1373,10803,3]
+  - ["JP",1442037600000,2181,8819,4]
+  - ["JP",1442034000000,910,7873,9]
+  - ["JP",1442030400000,805,8263,8]
+  - ["JP",1442026800000,1035,8892,6]
+  - ["JP",1442023200000,1959,6598,5]
+  - ["JP",1442019600000,2002,5688,4]
+  - ["JP",1442016000000,-113,4883,14]
+  - ["JO",1442080800000,4,2,32]
+  - ["JO",1442059200000,0,2,35]
+  - ["JO",1442055600000,-2,2,39]
+  - ["JM",1442070000000,30,30,35]
+  - ["IT",1442098800000,565,3856,7]
+  - ["IT",1442095200000,2940,7602,2]
+  - ["IT",1442091600000,351,10262,16]
+  - ["IT",1442088000000,3746,15806,4]
+  - ["IT",1442084400000,2660,17429,4]
+  - ["IT",1442080800000,5544,17075,3]
+  - ["IT",1442077200000,2188,16805,8]
+  - ["IT",1442073600000,2586,17214,7]
+  - ["IT",1442070000000,81,16492,24]
+  - ["IT",1442066400000,4155,11490,2]
+  - ["IT",1442062800000,1938,15542,5]
+  - ["IT",1442059200000,542,14836,12]
+  - ["IT",1442055600000,6240,15431,3]
+  - ["IT",1442052000000,1880,12759,6]
+  - ["IT",1442048400000,676,10841,9]
+  - ["IT",1442044800000,1483,10290,5]
+  - ["IT",1442041200000,20,5056,26]
+  - ["IT",1442037600000,-9,3159,32]
+  - ["IT",1442034000000,1006,2705,7]
+  - ["IT",1442030400000,-17,1333,29]
+  - ["IT",1442026800000,222,1496,13]
+  - ["IT",1442023200000,111,1505,17]
+  - ["IT",1442019600000,183,499,12]
+  - ["IT",1442016000000,0,516,9]
+  - ["IR",1442091600000,0,845,33]
+  - ["IR",1442088000000,714,811,10]
+  - ["IR",1442080800000,131,618,13]
+  - ["IR",1442077200000,-34,2073,40]
+  - ["IR",1442073600000,-193,1949,42]
+  - ["IR",1442059200000,1455,1390,7]
+  - ["IR",1442055600000,-124,1565,44]
+  - ["IR",1442052000000,155,1520,21]
+  - ["IR",1442044800000,306,1705,13]
+  - ["IR",1442041200000,-79,625,42]
+  - ["IR",1442034000000,-8,749,31]
+  - ["IR",1442030400000,375,594,10]
+  - ["IR",1442026800000,0,288,27]
+  - ["IQ",1442095200000,-2,4,35]
+  - ["IQ",1442052000000,0,3,38]
+  - ["IQ",1442044800000,6,3,31]
+  - ["IQ",1442041200000,-1,3,39]
+  - ["IN",1442095200000,4,312,27]
+  - ["IN",1442088000000,121,312,19]
+  - ["IN",1442084400000,187,6011,19]
+  - ["IN",1442080800000,0,7181,36]
+  - ["IN",1442077200000,5699,19268,4]
+  - ["IN",1442073600000,1170,19263,11]
+  - ["IN",1442070000000,12091,20623,2]
+  - ["IN",1442066400000,116,21331,23]
+  - ["IN",1442062800000,1547,15798,6]
+  - ["IN",1442059200000,708,15162,11]
+  - ["IN",1442055600000,166,3333,19]
+  - ["IN",1442052000000,534,5894,10]
+  - ["IN",1442048400000,262,4427,15]
+  - ["IN",1442044800000,2677,3854,2]
+  - ["IN",1442041200000,80,5038,18]
+  - ["IN",1442037600000,135,5952,11]
+  - ["IN",1442034000000,1350,6664,4]
+  - ["IN",1442030400000,1448,3845,6]
+  - ["IN",1442026800000,974,3803,7]
+  - ["IN",1442023200000,-142,3669,32]
+  - ["IN",1442019600000,38,2319,22]
+  - ["IN",1442016000000,1,871,7]
+  - ["IL",1442098800000,3,710,27]
+  - ["IL",1442095200000,0,1847,30]
+  - ["IL",1442091600000,707,1935,13]
+  - ["IL",1442084400000,1137,2122,6]
+  - ["IL",1442080800000,88,2150,17]
+  - ["IL",1442077200000,187,2199,22]
+  - ["IL",1442073600000,31,1495,27]
+  - ["IL",1442070000000,49,538,29]
+  - ["IL",1442066400000,3,1655,34]
+  - ["IL",1442062800000,180,1472,14]
+  - ["IL",1442059200000,1205,4186,8]
+  - ["IL",1442055600000,4,4162,33]
+  - ["IL",1442052000000,2745,4377,5]
+  - ["IL",1442048400000,25,4232,30]
+  - ["IL",1442044800000,218,3027,16]
+  - ["IL",1442041200000,35,3023,24]
+  - ["IE",1442091600000,-71,1151,40]
+  - ["IE",1442084400000,819,1051,9]
+  - ["IE",1442077200000,403,2113,18]
+  - ["IE",1442070000000,-100,2140,50]
+  - ["IE",1442066400000,1062,2212,9]
+  - ["IE",1442048400000,27,1394,29]
+  - ["IE",1442030400000,1,991,23]
+  - ["IE",1442026800000,1,1091,26]
+  - ["ID",1442098800000,13,54,19]
+  - ["ID",1442095200000,20,96,23]
+  - ["ID",1442091600000,21,113,26]
+  - ["ID",1442070000000,42,129,32]
+  - ["ID",1442059200000,17,-272,29]
+  - ["ID",1442055600000,16,-290,30]
+  - ["ID",1442044800000,-388,-297,41]
+  - ["ID",1442041200000,2,-320,34]
+  - ["ID",1442037600000,14,-58,22]
+  - ["ID",1442034000000,19,-490,24]
+  - ["ID",1442030400000,279,4,13]
+  - ["ID",1442026800000,-416,2,36]
+  - ["ID",1442023200000,106,-12,18]
+  - ["HU",1442098800000,110,376,13]
+  - ["HU",1442095200000,271,305,14]
+  - ["HU",1442091600000,-5,446,38]
+  - ["HU",1442088000000,-71,688,34]
+  - ["HU",1442084400000,141,628,23]
+  - ["HU",1442080800000,242,355,11]
+  - ["HU",1442062800000,50,859,21]
+  - ["HU",1442055600000,-2,1477,38]
+  - ["HU",1442048400000,499,1427,11]
+  - ["HU",1442044800000,547,1382,9]
+  - ["HU",1442041200000,91,1378,16]
+  - ["HU",1442037600000,197,1380,10]
+  - ["HU",1442019600000,46,881,21]
+  - ["HR",1442088000000,82,292,20]
+  - ["HR",1442084400000,-10,350,45]
+  - ["HR",1442080800000,220,350,12]
+  - ["HR",1442077200000,58,382,26]
+  - ["HR",1442073600000,0,300,36]
+  - ["HR",1442070000000,32,310,34]
+  - ["HN",1442026800000,-1,-1,30]
+  - ["HK",1442095200000,-1,-4,33]
+  - ["HK",1442091600000,-3,5541,37]
+  - ["HK",1442080800000,0,5855,35]
+  - ["HK",1442073600000,5545,5894,4]
+  - ["HK",1442070000000,314,5896,16]
+  - ["HK",1442066400000,39,5901,28]
+  - ["HK",1442062800000,1,6705,33]
+  - ["HK",1442059200000,2,1175,33]
+  - ["HK",1442055600000,804,862,9]
+  - ["HK",1442052000000,15,844,34]
+  - ["HK",1442048400000,1,828,37]
+  - ["HK",1442044800000,21,1462,28]
+  - ["HK",1442041200000,-15,1795,40]
+  - ["HK",1442037600000,636,1937,5]
+  - ["HK",1442034000000,1137,1725,5]
+  - ["HK",1442030400000,157,4118,14]
+  - ["HK",1442026800000,-211,4020,35]
+  - ["HK",1442023200000,2414,3384,4]
+  - ["HK",1442019600000,-113,2247,36]
+  - ["GT",1442098800000,1,7,31]
+  - ["GT",1442026800000,173,7,15]
+  - ["GT",1442023200000,-167,7,33]
+  - ["GR",1442091600000,123,390,20]
+  - ["GR",1442084400000,179,76,21]
+  - ["GR",1442080800000,88,78,16]
+  - ["GR",1442073600000,-314,86,44]
+  - ["GR",1442070000000,2,-63,41]
+  - ["GR",1442062800000,8,-235,30]
+  - ["GR",1442048400000,-26,-323,42]
+  - ["GR",1442041200000,7,73,30]
+  - ["GR",1442034000000,0,71,25]
+  - ["GR",1442019600000,82,63,16]
+  - ["GH",1442088000000,0,0,32]
+  - ["GE",1442080800000,-27,-119,42]
+  - ["GE",1442062800000,16,-140,28]
+  - ["GE",1442052000000,-108,-140,47]
+  - ["GE",1442044800000,-21,-140,37]
+  - ["GB",1442098800000,49,1098,16]
+  - ["GB",1442095200000,238,2691,15]
+  - ["GB",1442091600000,811,3075,10]
+  - ["GB",1442088000000,1593,4519,9]
+  - ["GB",1442084400000,384,5605,16]
+  - ["GB",1442080800000,1444,6015,5]
+  - ["GB",1442077200000,1135,5578,12]
+  - ["GB",1442073600000,648,4656,16]
+  - ["GB",1442070000000,374,10015,15]
+  - ["GB",1442066400000,671,24682,13]
+  - ["GB",1442062800000,5743,24000,2]
+  - ["GB",1442059200000,16111,23520,2]
+  - ["GB",1442055600000,453,23886,13]
+  - ["GB",1442052000000,168,23247,20]
+  - ["GB",1442048400000,740,17546,8]
+  - ["GB",1442044800000,32,1979,26]
+  - ["GB",1442041200000,42,1514,22]
+  - ["GB",1442037600000,544,3870,7]
+  - ["GB",1442034000000,-12,3469,32]
+  - ["GB",1442030400000,2524,5253,2]
+  - ["GB",1442026800000,339,5265,12]
+  - ["GB",1442023200000,1816,4677,6]
+  - ["GB",1442019600000,54,4689,19]
+  - ["GB",1442016000000,-44,2165,12]
+  - ["FR",1442098800000,136,964,12]
+  - ["FR",1442095200000,87,2836,17]
+  - ["FR",1442091600000,741,9479,12]
+  - ["FR",1442088000000,1872,10036,7]
+  - ["FR",1442084400000,6643,9456,2]
+  - ["FR",1442080800000,557,12891,8]
+  - ["FR",1442077200000,-444,12624,50]
+  - ["FR",1442073600000,3522,13268,6]
+  - ["FR",1442070000000,474,12301,12]
+  - ["FR",1442066400000,2516,16394,4]
+  - ["FR",1442062800000,5676,17301,3]
+  - ["FR",1442059200000,4650,14416,3]
+  - ["FR",1442055600000,463,16969,12]
+  - ["FR",1442052000000,637,14625,9]
+  - ["FR",1442048400000,3027,9553,2]
+  - ["FR",1442044800000,172,9077,18]
+  - ["FR",1442041200000,604,9090,7]
+  - ["FR",1442037600000,4174,8539,2]
+  - ["FR",1442034000000,476,6140,11]
+  - ["FR",1442026800000,86,6553,19]
+  - ["FR",1442023200000,628,5948,8]
+  - ["FR",1442019600000,585,1774,6]
+  - ["FR",1442016000000,-1,1298,11]
+  - ["FI",1442095200000,69,1068,19]
+  - ["FI",1442084400000,895,1268,8]
+  - ["FI",1442080800000,104,1267,14]
+  - ["FI",1442077200000,200,1450,21]
+  - ["FI",1442073600000,-1,1400,38]
+  - ["FI",1442066400000,183,912,20]
+  - ["FI",1442062800000,19,994,26]
+  - ["FI",1442059200000,407,806,15]
+  - ["FI",1442052000000,186,821,19]
+  - ["FI",1442048400000,12,2129,34]
+  - ["FI",1442037600000,14,2110,21]
+  - ["FI",1442030400000,1491,1703,5]
+  - ["ES",1442098800000,458,3664,9]
+  - ["ES",1442095200000,2506,3534,3]
+  - ["ES",1442091600000,700,3871,14]
+  - ["ES",1442088000000,-130,5111,35]
+  - ["ES",1442084400000,337,4807,17]
+  - ["ES",1442077200000,1240,2362,10]
+  - ["ES",1442073600000,154,2123,23]
+  - ["ES",1442070000000,61,2182,27]
+  - ["ES",1442066400000,461,2931,15]
+  - ["ES",1442062800000,-71,2186,40]
+  - ["ES",1442059200000,1086,2028,9]
+  - ["ES",1442055600000,495,2125,11]
+  - ["ES",1442052000000,-4,1495,44]
+  - ["ES",1442048400000,158,1684,19]
+  - ["ES",1442044800000,-169,601,40]
+  - ["ES",1442041200000,118,54,15]
+  - ["ES",1442037600000,3,53,26]
+  - ["ES",1442034000000,-52,-2,34]
+  - ["ES",1442023200000,-5,167,28]
+  - ["ES",1442019600000,103,49,15]
+  - ["EG",1442091600000,27,140,25]
+  - ["EG",1442073600000,1,154,34]
+  - ["EG",1442062800000,112,170,18]
+  - ["EG",1442055600000,14,170,31]
+  - ["EG",1442026800000,16,143,25]
+  - ["EE",1442044800000,-19,18,36]
+  - ["EE",1442041200000,37,18,23]
+  - ["EC",1442095200000,10,212,26]
+  - ["EC",1442084400000,568,212,11]
+  - ["EC",1442077200000,-366,203,49]
+  - ["EC",1442030400000,0,232,25]
+  - ["EC",1442023200000,-9,222,29]
+  - ["EC",1442019600000,29,-346,23]
+  - ["DZ",1442077200000,-1,-1,35]
+  - ["DO",1442095200000,13,221,25]
+  - ["DO",1442084400000,8,256,36]
+  - ["DO",1442073600000,200,264,21]
+  - ["DO",1442066400000,35,264,29]
+  - ["DO",1442023200000,8,251,26]
+  - ["DK",1442095200000,0,42,29]
+  - ["DK",1442091600000,139,103,19]
+  - ["DK",1442084400000,-97,94,51]
+  - ["DK",1442080800000,61,510,18]
+  - ["DK",1442077200000,-9,511,37]
+  - ["DK",1442066400000,416,372,16]
+  - ["DK",1442062800000,1,511,32]
+  - ["DK",1442059200000,0,445,34]
+  - ["DK",1442055600000,42,490,25]
+  - ["DK",1442048400000,-5,84,39]
+  - ["DK",1442044800000,36,83,25]
+  - ["DK",1442037600000,10,83,23]
+  - ["DE",1442098800000,329,5691,10]
+  - ["DE",1442095200000,1007,5881,5]
+  - ["DE",1442091600000,4355,5756,2]
+  - ["DE",1442088000000,190,6889,18]
+  - ["DE",1442084400000,-125,8748,52]
+  - ["DE",1442080800000,1133,13816,6]
+  - ["DE",1442077200000,2188,11127,7]
+  - ["DE",1442073600000,6075,12514,3]
+  - ["DE",1442070000000,1666,12922,8]
+  - ["DE",1442066400000,1577,12078,7]
+  - ["DE",1442062800000,283,11413,13]
+  - ["DE",1442059200000,289,6938,16]
+  - ["DE",1442055600000,1523,6083,6]
+  - ["DE",1442052000000,1600,5485,7]
+  - ["DE",1442048400000,811,5399,7]
+  - ["DE",1442044800000,979,5654,7]
+  - ["DE",1442041200000,197,4489,12]
+  - ["DE",1442037600000,544,3262,6]
+  - ["DE",1442034000000,358,2515,13]
+  - ["DE",1442030400000,373,1536,11]
+  - ["DE",1442023200000,64,1506,20]
+  - ["DE",1442019600000,0,962,31]
+  - ["DE",1442016000000,167,604,3]
+  - ["CZ",1442098800000,2,89,28]
+  - ["CZ",1442080800000,-28,2140,43]
+  - ["CZ",1442077200000,115,2308,23]
+  - ["CZ",1442073600000,2051,2308,9]
+  - ["CZ",1442070000000,168,2327,22]
+  - ["CZ",1442062800000,0,3428,34]
+  - ["CZ",1442059200000,21,3331,28]
+  - ["CZ",1442055600000,1073,1358,7]
+  - ["CZ",1442037600000,18,1171,19]
+  - ["CZ",1442034000000,78,1171,19]
+  - ["CZ",1442026800000,-19,1150,33]
+  - ["CR",1442088000000,72,429,21]
+  - ["CR",1442048400000,163,480,18]
+  - ["CR",1442044800000,194,2977,17]
+  - ["CR",1442041200000,51,3117,21]
+  - ["CR",1442030400000,2497,3107,3]
+  - ["CR",1442026800000,140,3006,17]
+  - ["CR",1442023200000,62,2812,21]
+  - ["CR",1442019600000,62,2761,18]
+  - ["CO",1442098800000,83,750,14]
+  - ["CO",1442095200000,290,17900,13]
+  - ["CO",1442091600000,377,17951,15]
+  - ["CO",1442088000000,17150,17976,2]
+  - ["CO",1442084400000,51,18474,28]
+  - ["CO",1442080800000,25,58044,23]
+  - ["CO",1442077200000,581,57622,14]
+  - ["CO",1442073600000,39860,40760,2]
+  - ["CO",1442070000000,-45,41182,48]
+  - ["CO",1442066400000,288,41598,19]
+  - ["CO",1442059200000,473,41026,13]
+  - ["CO",1442030400000,441,1178,9]
+  - ["CO",1442023200000,9,1239,25]
+  - ["CO",1442019600000,12,951,25]
+  - ["CO",1442016000000,16,478,6]
+  - ["CN",1442084400000,-1,-26,41]
+  - ["CN",1442080800000,-10,-18,41]
+  - ["CN",1442066400000,-15,51,43]
+  - ["CN",1442059200000,8,51,31]
+  - ["CN",1442055600000,69,345,22]
+  - ["CN",1442052000000,0,453,37]
+  - ["CN",1442048400000,293,622,14]
+  - ["CN",1442037600000,98,601,13]
+  - ["CN",1442026800000,154,532,16]
+  - ["CN",1442023200000,-13,532,31]
+  - ["CL",1442098800000,9,-247,21]
+  - ["CL",1442095200000,-276,39,42]
+  - ["CL",1442091600000,20,165,27]
+  - ["CL",1442088000000,286,182,16]
+  - ["CL",1442084400000,126,158,24]
+  - ["CL",1442080800000,17,587,25]
+  - ["CL",1442077200000,-15,580,39]
+  - ["CL",1442073600000,153,253,24]
+  - ["CL",1442070000000,13,144,37]
+  - ["CL",1442066400000,-41,115,46]
+  - ["CL",1442062800000,17,520,27]
+  - ["CL",1442059200000,-12,366,40]
+  - ["CL",1442052000000,390,355,14]
+  - ["CL",1442041200000,-1,395,38]
+  - ["CL",1442037600000,2,418,28]
+  - ["CL",1442034000000,-1,445,28]
+  - ["CL",1442030400000,40,-315,18]
+  - ["CL",1442023200000,15,-153,24]
+  - ["CL",1442019600000,-370,-155,37]
+  - ["CL",1442016000000,161,-154,4]
+  - ["CH",1442091600000,67,86,23]
+  - ["CH",1442084400000,13,446,32]
+  - ["CH",1442077200000,6,457,31]
+  - ["CH",1442073600000,360,479,19]
+  - ["CH",1442070000000,11,412,38]
+  - ["CH",1442062800000,22,446,25]
+  - ["CH",1442055600000,0,464,36]
+  - ["CH",1442052000000,47,50,25]
+  - ["CH",1442048400000,24,237,31]
+  - ["CH",1442044800000,-54,274,38]
+  - ["CH",1442041200000,198,274,11]
+  - ["CH",1442037600000,59,227,17]
+  - ["CA",1442098800000,164,2661,11]
+  - ["CA",1442095200000,-361,2696,43]
+  - ["CA",1442091600000,2858,2740,5]
+  - ["CA",1442088000000,35,3221,25]
+  - ["CA",1442084400000,44,2775,29]
+  - ["CA",1442080800000,481,3222,9]
+  - ["CA",1442077200000,-282,549,47]
+  - ["CA",1442073600000,86,821,26]
+  - ["CA",1442070000000,185,410,21]
+  - ["CA",1442066400000,307,965,17]
+  - ["CA",1442062800000,-367,1285,44]
+  - ["CA",1442059200000,1036,1200,10]
+  - ["CA",1442052000000,38,1020,31]
+  - ["CA",1442044800000,1,581,32]
+  - ["CA",1442041200000,5,1126,31]
+  - ["CA",1442037600000,-132,43,35]
+  - ["CA",1442034000000,178,2221,16]
+  - ["CA",1442030400000,-47,2506,30]
+  - ["CA",1442026800000,2216,4685,4]
+  - ["CA",1442023200000,286,4446,15]
+  - ["CA",1442019600000,2184,4268,3]
+  - ["CA",1442016000000,-371,4315,16]
+  - ["BY",1442088000000,33,62,26]
+  - ["BY",1442084400000,1,92,38]
+  - ["BY",1442080800000,28,688,22]
+  - ["BY",1442077200000,30,2152,28]
+  - ["BY",1442073600000,596,2120,17]
+  - ["BY",1442059200000,1464,2119,6]
+  - ["BY",1442055600000,1,2091,34]
+  - ["BR",1442098800000,-645,114,40]
+  - ["BR",1442095200000,748,329,8]
+  - ["BR",1442091600000,11,821,30]
+  - ["BR",1442088000000,215,554,17]
+  - ["BR",1442084400000,492,3452,12]
+  - ["BR",1442080800000,-267,4791,48]
+  - ["BR",1442077200000,2253,5316,6]
+  - ["BR",1442073600000,2087,6135,8]
+  - ["BR",1442070000000,536,5736,9]
+  - ["BR",1442066400000,1034,6076,10]
+  - ["BR",1442062800000,93,4065,19]
+  - ["BR",1442059200000,73,2210,24]
+  - ["BR",1442055600000,242,1745,16]
+  - ["BR",1442052000000,232,714,17]
+  - ["BR",1442044800000,71,888,22]
+  - ["BR",1442041200000,3,836,32]
+  - ["BR",1442037600000,267,624,8]
+  - ["BR",1442034000000,21,443,23]
+  - ["BR",1442030400000,30,1251,19]
+  - ["BR",1442026800000,51,1620,23]
+  - ["BR",1442023200000,879,1105,7]
+  - ["BR",1442019600000,372,1084,9]
+  - ["BR",1442016000000,-248,1054,15]
+  - ["BO",1442095200000,-4,4,36]
+  - ["BO",1442088000000,4,4,31]
+  - ["BO",1442080800000,4,4,31]
+  - ["BH",1442052000000,44,44,27]
+  - ["BG",1442084400000,401,647,15]
+  - ["BG",1442070000000,55,19583,28]
+  - ["BG",1442059200000,191,19592,22]
+  - ["BG",1442052000000,18936,19592,2]
+  - ["BG",1442041200000,9,19191,29]
+  - ["BE",1442098800000,9,177,20]
+  - ["BE",1442091600000,101,360,21]
+  - ["BE",1442088000000,67,361,22]
+  - ["BE",1442084400000,183,380,20]
+  - ["BE",1442080800000,1,507,34]
+  - ["BE",1442073600000,19,497,30]
+  - ["BE",1442066400000,136,663,22]
+  - ["BE",1442062800000,91,479,20]
+  - ["BE",1442055600000,233,537,17]
+  - ["BE",1442052000000,-1,415,40]
+  - ["BE",1442048400000,59,279,26]
+  - ["BE",1442030400000,-103,188,31]
+  - ["BD",1442091600000,-2,176,36]
+  - ["BD",1442077200000,75,252,25]
+  - ["BD",1442073600000,103,1106,25]
+  - ["BD",1442066400000,76,1106,25]
+  - ["BD",1442041200000,854,1108,5]
+  - ["BD",1442019600000,0,1033,30]
+  - ["BA",1442084400000,-1,-165,40]
+  - ["BA",1442055600000,-202,-178,45]
+  - ["BA",1442052000000,38,-178,30]
+  - ["BA",1442048400000,-13,-178,40]
+  - ["AU",1442098800000,518,2051,8]
+  - ["AU",1442095200000,395,2052,12]
+  - ["AU",1442091600000,1138,2040,7]
+  - ["AU",1442077200000,1,2019,34]
+  - ["AU",1442070000000,-12,1539,46]
+  - ["AU",1442066400000,-21,1326,44]
+  - ["AU",1442059200000,38,-455,26]
+  - ["AU",1442055600000,182,-321,18]
+  - ["AU",1442052000000,-643,64,48]
+  - ["AU",1442048400000,135,279,20]
+  - ["AU",1442044800000,373,244,11]
+  - ["AU",1442041200000,194,345,13]
+  - ["AU",1442037600000,3,611,25]
+  - ["AU",1442034000000,283,664,14]
+  - ["AU",1442030400000,-377,343,32]
+  - ["AU",1442026800000,188,402,14]
+  - ["AU",1442023200000,52,399,22]
+  - ["AU",1442019600000,253,116,11]
+  - ["AU",1442016000000,0,493,8]
+  - ["AT",1442091600000,89,7137,22]
+  - ["AT",1442088000000,7050,7409,3]
+  - ["AT",1442084400000,-2,7409,42]
+  - ["AT",1442070000000,272,7254,18]
+  - ["AT",1442066400000,0,11958,37]
+  - ["AT",1442062800000,-155,4908,41]
+  - ["AT",1442052000000,4793,4910,3]
+  - ["AR",1442098800000,64,1034,15]
+  - ["AR",1442095200000,630,1029,9]
+  - ["AR",1442091600000,340,1030,17]
+  - ["AR",1442084400000,-5,439,44]
+  - ["AR",1442080800000,1,752,33]
+  - ["AR",1442077200000,-591,122,51]
+  - ["AR",1442070000000,377,-189,14]
+  - ["AR",1442066400000,0,26,36]
+  - ["AR",1442062800000,29,-29,23]
+  - ["AR",1442059200000,210,643,18]
+  - ["AR",1442055600000,-54,478,42]
+  - ["AR",1442037600000,81,475,15]
+  - ["AR",1442034000000,212,1090,15]
+  - ["AR",1442030400000,-3,3394,27]
+  - ["AR",1442026800000,644,3449,10]
+  - ["AR",1442023200000,2514,3368,3]
+  - ["AR",1442019600000,1,3156,28]
+  - ["AO",1442098800000,722,766,6]
+  - ["AO",1442088000000,62,740,23]
+  - ["AO",1442052000000,-18,740,45]
+  - ["AO",1442041200000,-26,740,41]
+  - ["AL",1442091600000,54,80,24]
+  - ["AL",1442077200000,26,80,29]
+  - ["AE",1442080800000,42,6411,19]
+  - ["AE",1442077200000,6323,6400,3]
+  - ["AE",1442070000000,46,6397,30]
+  - ["AE",1442059200000,-11,6436,39]
+  - ["AE",1442052000000,-3,6387,42]
+  - ["AE",1442048400000,39,182,28]
+  - ["AE",1442044800000,-7,136,35]
+  - ["AE",1442030400000,118,147,15]
+  - [null,1442098800000,276159,1165087,1]
+  - [null,1442095200000,374501,1468959,1]
+  - [null,1442091600000,514427,1841528,1]
+  - [null,1442088000000,303872,2294605,1]
+  - [null,1442084400000,372569,2410929,1]
+  - [null,1442080800000,453077,2411822,1]
+  - [null,1442077200000,392483,2255910,1]
+  - [null,1442073600000,375394,2303622,1]
+  - [null,1442070000000,358515,2320518,1]
+  - [null,1442066400000,351584,2326738,1]
+  - [null,1442062800000,389465,2218213,1]
+  - [null,1442059200000,459297,2630189,1]
+  - [null,1442055600000,283958,2579990,1]
+  - [null,1442052000000,787370,2544408,1]
+  - [null,1442048400000,308316,2698393,1]
+  - [null,1442044800000,316002,2439701,1]
+  - [null,1442041200000,543450,2486700,1]
+  - [null,1442037600000,200605,1866002,1]
+  - [null,1442034000000,330957,1810312,1]
+  - [null,1442030400000,166672,1893946,1]
+  - [null,1442026800000,252626,1524388,1]
+  - [null,1442023200000,399636,1353656,1]
+  - [null,1442019600000,173892,1022699,1]
+  - [null,1442016000000,29873,856027,1]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to