From f4a74710e6dc1eccfe376fcbfc1f488ea0fad8f5 Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Sun, 29 Oct 2023 12:10:49 +0100 Subject: [PATCH] Process pure ordering changes with windowing operators (#15241) - 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 --- .../java/org/apache/druid/query/Druids.java | 5 + .../NaivePartitioningOperatorFactory.java | 20 + .../query/operator/NaiveSortOperator.java | 11 +- .../operator/NaiveSortOperatorFactory.java | 34 +- .../druid/query/operator/OffsetLimit.java | 143 +++ .../druid/query/operator/ScanOperator.java | 10 +- .../query/operator/ScanOperatorFactory.java | 42 +- .../query/operator/WindowOperatorQuery.java | 19 +- .../window/WindowOperatorFactory.java | 23 + .../window/ranking/WindowRankProcessor.java | 26 + .../ranking/WindowRankingProcessorBase.java | 24 + .../LazilyDecoratedRowsAndColumns.java | 35 +- .../DefaultColumnSelectorFactoryMaker.java | 3 + .../DefaultRowsAndColumnsDecorator.java | 19 +- .../semantic/RowsAndColumnsDecorator.java | 3 +- .../apache/druid/query/scan/ScanQuery.java | 7 + .../query/scan/ScanQueryQueryToolChest.java | 21 +- .../druid/segment/column/TypeStrategies.java | 3 + .../NaivePartitioningOperatorFactoryTest.java | 34 + .../NaiveSortOperatorFactoryTest.java | 34 + .../query/operator/NaiveSortOperatorTest.java | 98 ++ .../druid/query/operator/OffsetLimitTest.java | 109 ++ .../operator/ScanOperatorFactoryTest.java | 20 +- .../operator/WindowOperatorFactoryTest.java | 34 + .../operator/WindowOperatorQueryTest.java | 8 +- .../concrete/FrameRowsAndColumnsTest.java | 3 +- .../semantic/RowsAndColumnsDecoratorTest.java | 13 +- .../semantic/TestRowsAndColumnsDecorator.java | 11 +- ...ualColumnEvaluationRowsAndColumnsTest.java | 3 +- .../segment/column/TypeStrategiesTest.java | 10 +- .../calcite/planner/CalciteRulesManager.java | 14 +- .../sql/calcite/planner/OffsetLimit.java | 14 + .../sql/calcite/rel/DruidOuterQueryRel.java | 9 +- .../druid/sql/calcite/rel/DruidQuery.java | 91 +- .../druid/query/OperatorFactoryBuilders.java | 102 ++ .../query/WindowOperatorQueryBuilder.java | 91 ++ .../sql/calcite/CalciteArraysQueryTest.java | 81 +- .../sql/calcite/CalciteJoinQueryTest.java | 37 +- .../druid/sql/calcite/CalciteQueryTest.java | 335 ++++-- .../sql/calcite/CalciteWindowQueryTest.java | 23 +- .../sql/calcite/DrillWindowQueryTest.java | 58 +- .../druid/sql/calcite/NotYetSupported.java | 3 +- .../druid/sql/http/SqlResourceTest.java | 2 +- .../tests/window/offsetNotDiscarded.sqlTest | 31 + ...iaAggregationsMultipleOrderingDesc.sqlTest | 1004 ++++++++++++++++- 45 files changed, 2371 insertions(+), 349 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/operator/OffsetLimit.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaivePartitioningOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorFactoryTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/NaiveSortOperatorTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/OffsetLimitTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/operator/WindowOperatorFactoryTest.java create mode 100644 sql/src/test/java/org/apache/druid/query/OperatorFactoryBuilders.java create mode 100644 sql/src/test/java/org/apache/druid/query/WindowOperatorQueryBuilder.java create mode 100644 sql/src/test/resources/calcite/tests/window/offsetNotDiscarded.sqlTest 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 sortColumns; + private final List sortColumns; public NaiveSortOperator( Operator child, - ArrayList sortColumns + List 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 sortColumns; + private final List sortColumns; @JsonCreator public NaiveSortOperatorFactory( - @JsonProperty("columns") ArrayList sortColumns + @JsonProperty("columns") List sortColumns ) { this.sortColumns = sortColumns; } @JsonProperty("columns") - public ArrayList getSortColumns() + public List 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 projectedColumns; private final VirtualColumns virtualColumns; private final List ordering; @@ -55,7 +55,7 @@ public class ScanOperator implements Operator Interval timeRange, Filter filter, List 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 projectedColumns; private final VirtualColumns virtualColumns; private final List 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 projectedColumns, @JsonProperty("virtualColumns") final VirtualColumns virtualColumns, @JsonProperty("ordering") final List 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. *

@@ -122,14 +124,17 @@ public class WindowOperatorQuery extends BaseQuery ) ); } + 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 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 ", 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 viewableColumns; private List ordering; @@ -82,7 +83,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns Interval interval, Filter filter, VirtualColumns virtualColumns, - int limit, + OffsetLimit limit, List ordering, LinkedHashSet 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 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 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 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 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 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 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 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 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 projectedColumns; private VirtualColumns virtualColumns; private List 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 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 ordering; private List 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 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.naturalNullsFirst().thenComparing(Longs::compare).compare(this.lhs, o.lhs); } } - + public static class NullableLongPairTypeStrategy implements TypeStrategy { @@ -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 b944f3cd535..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 @@ -177,11 +177,7 @@ public class CalciteRulesManager private static final List ABSTRACT_RELATIONAL_RULES = ImmutableList.of( AbstractConverter.ExpandConversionRule.INSTANCE, - // Removing CoreRules.AGGREGATE_REMOVE rule here - // as after the Calcite upgrade, it would plan queries to a scan over a group by - // with ordering on a non-time column - // which is not allowed in Druid. We should add that rule back - // once Druid starts to support non-time ordering over scan queries + CoreRules.AGGREGATE_REMOVE, CoreRules.UNION_TO_DISTINCT, CoreRules.PROJECT_REMOVE, CoreRules.AGGREGATE_JOIN_TRANSPOSE, @@ -237,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 { - 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 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 sortColumns = getColumnWithDirectionsFromOrderBys(sorting.getOrderBys()); + RowSignature signature = getOutputRowSignature(); + List 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 getColumnWithDirectionsFromOrderBys(List orderBys) + { + ArrayList 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 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 context; + private RowSignature rowSignature; + private List operators; + private List 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/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index ec4dda0dc69..3a5da7d325f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -50,6 +50,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.SubstringDimExtractionFn; @@ -3171,55 +3172,45 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest public void testArrayAggGroupByArrayAggFromSubquery() { cannotVectorize(); - skipVectorize(); + testQuery( "SELECT dim2, arr, COUNT(*) FROM (SELECT dim2, ARRAY_AGG(DISTINCT dim1) as arr FROM foo WHERE dim1 is not null GROUP BY 1 LIMIT 5) GROUP BY 1,2", QUERY_CONTEXT_NO_STRINGIFY_ARRAY, ImmutableList.of( - GroupByQuery.builder() - .setDataSource(new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .dimension(new DefaultDimensionSpec( - "dim2", - "d0", - ColumnType.STRING - )) - .metric(new DimensionTopNMetricSpec( - null, - StringComparators.LEXICOGRAPHIC - )) - .filters(notNull("dim1")) - .threshold(5) - .aggregators(new ExpressionLambdaAggregatorFactory( - "a0", - ImmutableSet.of("dim1"), - "__acc", - "ARRAY[]", - "ARRAY[]", - true, - true, - false, - "array_set_add(\"__acc\", \"dim1\")", - "array_set_add_all(\"__acc\", \"a0\")", - null, - null, - new HumanReadableBytes(1024), - ExprMacroTable.nil() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) - .build() - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.STRING), - new DefaultDimensionSpec("a0", "_d1", ColumnType.STRING_ARRAY) - ) - .setAggregatorSpecs(new CountAggregatorFactory("_a0")) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .dimension(new DefaultDimensionSpec( + "dim2", + "d0", + ColumnType.STRING + )) + .metric(new DimensionTopNMetricSpec( + null, + StringComparators.LEXICOGRAPHIC + )) + .filters(notNull("dim1")) + .threshold(5) + .aggregators(new ExpressionLambdaAggregatorFactory( + "a0", + ImmutableSet.of("dim1"), + "__acc", + "ARRAY[]", + "ARRAY[]", + true, + true, + false, + "array_set_add(\"__acc\", \"dim1\")", + "array_set_add_all(\"__acc\", \"a0\")", + null, + null, + new HumanReadableBytes(1024), + ExprMacroTable.nil() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .postAggregators(new ExpressionPostAggregator("s0", "1", null, ExprMacroTable.nil())) + .build() ), useDefault ? ImmutableList.of( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index ba55f7b4e4a..0882f3c9cb1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -5381,8 +5381,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest @Test public void testPlanWithInFilterMoreThanInSubQueryThreshold() { - skipVectorize(); - cannotVectorize(); String query = "SELECT l1 FROM numfoo WHERE l1 IN (4842, 4844, 4845, 14905, 4853, 29064)"; Map queryContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); @@ -5399,32 +5397,21 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .dataSource( JoinDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(InlineDataSource.fromIterable( - ImmutableList.of( - new Object[]{4842L}, - new Object[]{4844L}, - new Object[]{4845L}, - new Object[]{14905L}, - new Object[]{4853L}, - new Object[]{29064L} - ), - RowSignature.builder() - .add("ROW_VALUE", ColumnType.LONG) - .build() - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimensions( - new DefaultDimensionSpec("ROW_VALUE", "d0", ColumnType.LONG) - ) - .setGranularity(Granularities.ALL) - .setLimitSpec(NoopLimitSpec.instance()) + InlineDataSource.fromIterable( + ImmutableList.of( + new Object[]{4842L}, + new Object[]{4844L}, + new Object[]{4845L}, + new Object[]{14905L}, + new Object[]{4853L}, + new Object[]{29064L} + ), + RowSignature.builder() + .add("ROW_VALUE", ColumnType.LONG) .build() ), "j0.", - "(\"l1\" == \"j0.d0\")", + "(\"l1\" == \"j0.ROW_VALUE\")", JoinType.INNER, null, ExprMacroTable.nil(), 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 948eea4c1ff..b67db5dce41 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 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,31 +13952,14 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "group by 1", ImmutableList.of( GroupByQuery.builder() - .setDataSource(GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE3) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setGranularity(Granularities.ALL) - .addDimension(new DefaultDimensionSpec( - "dim1", - "_d0", - ColumnType.STRING - )) - .addAggregator(new LongSumAggregatorFactory("a0", "l1")) - .build() - ) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setDimensions(new DefaultDimensionSpec("_d0", "d0", ColumnType.STRING)) - .setAggregatorSpecs(aggregators( - new FilteredAggregatorFactory( - new CountAggregatorFactory("_a0"), - useDefault ? - selector("a0", "0") : - equality("a0", 0, ColumnType.LONG) - ) - )) - .setGranularity(Granularities.ALL) - .setContext(QUERY_CONTEXT_DEFAULT) - .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( @@ -14303,72 +14289,83 @@ 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(); - testQuery( - "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", - QUERY_CONTEXT_DEFAULT, - ImmutableList.of( - new GroupByQuery.Builder() + + 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") + .aggregators( + aggregators( + useDefault + ? new CountAggregatorFactory("a0") + : new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + notNull("m1") + ) ) - )) + ) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .context(OUTER_LIMIT_CONTEXT) .build() ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.DOUBLE) - ) - .setDimFilter( - useDefault ? - bound("d0", "2", null, true, false, null, StringComparators.NUMERIC) : - new RangeFilter("d0", ColumnType.LONG, 2L, null, true, false, null) - ) - .setAggregatorSpecs(aggregators( - new LongMaxAggregatorFactory("_a0", "a0") - )) - .setLimitSpec( - DefaultLimitSpec - .builder() - .orderBy(new OrderByColumnSpec("_a0", Direction.DESCENDING, StringComparators.NUMERIC)) + .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() ) - .setContext(OUTER_LIMIT_CONTEXT) .build() - ), - ImmutableList.of( - new Object[]{3.0D, 1L}, - new Object[]{4.0D, 1L}, - new Object[]{5.0D, 1L}, - new Object[]{6.0D, 1L} ) - ); + .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 @@ -14376,8 +14373,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest { skipVectorize(); cannotVectorize(); - testQuery( - "with t AS (SELECT m2 as mo, COUNT(m1) as trend_score\n" + 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" @@ -14385,56 +14381,167 @@ public class CalciteQueryTest extends BaseCalciteQueryTest + "select mo, (MAX(trend_score)) from t\n" + "where mo > 2\n" + "GROUP BY 1 \n" - + "ORDER BY 2 DESC LIMIT 2\n", - QUERY_CONTEXT_DEFAULT, - ImmutableList.of( - new GroupByQuery.Builder() + + "ORDER BY 2 DESC LIMIT 2 OFFSET 1\n"; + ImmutableList 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") + .aggregators( + aggregators( + useDefault + ? new CountAggregatorFactory("a0") + : new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + notNull("m1") + ) ) - )) + ) .metric(new NumericTopNMetricSpec("a0")) .context(OUTER_LIMIT_CONTEXT) .build() ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions( - new DefaultDimensionSpec("d0", "_d0", ColumnType.DOUBLE) - ) - .setDimFilter( - useDefault ? - bound("d0", "2", null, true, false, null, StringComparators.NUMERIC) : - new RangeFilter("d0", ColumnType.LONG, 2L, null, true, false, null) - ) - .setAggregatorSpecs(aggregators( - new LongMaxAggregatorFactory("_a0", "a0") - )) - .setLimitSpec( - DefaultLimitSpec - .builder() - .orderBy(new OrderByColumnSpec("_a0", Direction.DESCENDING, StringComparators.NUMERIC)) - .limit(2) + .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() ) - .setContext(OUTER_LIMIT_CONTEXT) .build() - ), - ImmutableList.of( - new Object[]{3.0D, 1L}, - new Object[]{4.0D, 1L} ) - ); + .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 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 58138ffd1ee..55014a2f414 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 @@ -383,7 +383,7 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest private boolean isOrdered(QueryResults queryResults) { - SqlNode sqlNode = ((PlannerCaptureHook) queryResults.capture).getSqlNode(); + SqlNode sqlNode = queryResults.capture.getSqlNode(); return SqlToRelConverter.isOrdered(sqlNode); } } @@ -4364,6 +4364,7 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } + @NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL) @DrillTest("nestedAggs/multiWin_5") @Test public void test_nestedAggs_multiWin_5() @@ -4477,7 +4478,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("aggregates/aggOWnFn_3") @Test public void test_aggregates_aggOWnFn_3() @@ -4485,7 +4485,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) @DrillTest("aggregates/aggOWnFn_4") @Test public void test_aggregates_aggOWnFn_4() @@ -4493,7 +4492,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() @@ -4501,7 +4499,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() @@ -4509,7 +4506,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() @@ -4525,7 +4522,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() @@ -4533,7 +4529,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() @@ -4541,7 +4536,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() @@ -4549,7 +4544,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() @@ -4557,7 +4552,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() @@ -4565,7 +4560,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() @@ -4581,7 +4576,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() @@ -4589,7 +4583,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() @@ -4597,7 +4590,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() @@ -4605,7 +4598,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() @@ -4613,7 +4606,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() @@ -4621,7 +4614,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() @@ -4629,7 +4622,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() @@ -4637,7 +4630,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() @@ -4645,7 +4638,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() @@ -4653,7 +4646,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() @@ -4661,7 +4654,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() @@ -4669,7 +4662,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() @@ -4677,7 +4670,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() @@ -4685,7 +4678,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() @@ -6697,7 +6690,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() @@ -6822,7 +6814,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() @@ -6846,7 +6837,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() @@ -6910,7 +6900,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() @@ -7012,7 +7001,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() @@ -7083,7 +7071,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() @@ -7161,7 +7148,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() @@ -7192,7 +7178,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() @@ -7249,7 +7234,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() @@ -7257,7 +7241,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_53") @Test public void test_frameclause_subQueries_frmInSubQry_53() @@ -7265,7 +7248,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_54") @Test public void test_frameclause_subQueries_frmInSubQry_54() @@ -7273,7 +7255,6 @@ public class DrillWindowQueryTest extends BaseCalciteQueryTest windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("frameclause/subQueries/frmInSubQry_55") @Test public void test_frameclause_subQueries_frmInSubQry_55() @@ -7623,7 +7604,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 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]