mirror of https://github.com/apache/druid.git
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
This commit is contained in:
parent
737947754d
commit
f4a74710e6
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.query.rowsandcols.semantic.NaiveSortMaker;
|
|||
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* A naive sort operator is an operation that sorts a stream of data in-place. Generally speaking this means
|
||||
|
@ -33,11 +34,11 @@ import java.util.ArrayList;
|
|||
public class NaiveSortOperator implements Operator
|
||||
{
|
||||
private final Operator child;
|
||||
private final ArrayList<ColumnWithDirection> sortColumns;
|
||||
private final List<ColumnWithDirection> sortColumns;
|
||||
|
||||
public NaiveSortOperator(
|
||||
Operator child,
|
||||
ArrayList<ColumnWithDirection> sortColumns
|
||||
List<ColumnWithDirection> sortColumns
|
||||
)
|
||||
{
|
||||
this.child = child;
|
||||
|
@ -57,7 +58,7 @@ public class NaiveSortOperator implements Operator
|
|||
public Signal push(RowsAndColumns rac)
|
||||
{
|
||||
if (sorter == null) {
|
||||
sorter = NaiveSortMaker.fromRAC(rac).make(sortColumns);
|
||||
sorter = NaiveSortMaker.fromRAC(rac).make(new ArrayList<>(sortColumns));
|
||||
} else {
|
||||
sorter.moreData(rac);
|
||||
}
|
||||
|
@ -67,7 +68,9 @@ public class NaiveSortOperator implements Operator
|
|||
@Override
|
||||
public void completed()
|
||||
{
|
||||
if (sorter != null) {
|
||||
receiver.push(sorter.complete());
|
||||
}
|
||||
receiver.completed();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,22 +22,23 @@ package org.apache.druid.query.operator;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class NaiveSortOperatorFactory implements OperatorFactory
|
||||
{
|
||||
private final ArrayList<ColumnWithDirection> sortColumns;
|
||||
private final List<ColumnWithDirection> sortColumns;
|
||||
|
||||
@JsonCreator
|
||||
public NaiveSortOperatorFactory(
|
||||
@JsonProperty("columns") ArrayList<ColumnWithDirection> sortColumns
|
||||
@JsonProperty("columns") List<ColumnWithDirection> sortColumns
|
||||
)
|
||||
{
|
||||
this.sortColumns = sortColumns;
|
||||
}
|
||||
|
||||
@JsonProperty("columns")
|
||||
public ArrayList<ColumnWithDirection> getSortColumns()
|
||||
public List<ColumnWithDirection> getSortColumns()
|
||||
{
|
||||
return sortColumns;
|
||||
}
|
||||
|
@ -56,4 +57,29 @@ public class NaiveSortOperatorFactory implements OperatorFactory
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(sortColumns);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj)
|
||||
{
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
if (obj == null || getClass() != obj.getClass()) {
|
||||
return false;
|
||||
}
|
||||
NaiveSortOperatorFactory other = (NaiveSortOperatorFactory) obj;
|
||||
return Objects.equals(sortColumns, other.sortColumns);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "NaiveSortOperatorFactory{sortColumns=" + sortColumns + "}";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -43,7 +43,7 @@ public class ScanOperator implements Operator
|
|||
private final Operator subOperator;
|
||||
private final Interval timeRange;
|
||||
private final Filter filter;
|
||||
private final int limit;
|
||||
private final OffsetLimit offsetLimit;
|
||||
private final List<String> projectedColumns;
|
||||
private final VirtualColumns virtualColumns;
|
||||
private final List<ColumnWithDirection> ordering;
|
||||
|
@ -55,7 +55,7 @@ public class ScanOperator implements Operator
|
|||
Interval timeRange,
|
||||
Filter filter,
|
||||
List<ColumnWithDirection> ordering,
|
||||
int limit
|
||||
OffsetLimit offsetLimit
|
||||
)
|
||||
{
|
||||
this.subOperator = subOperator;
|
||||
|
@ -64,7 +64,7 @@ public class ScanOperator implements Operator
|
|||
this.timeRange = timeRange;
|
||||
this.filter = filter;
|
||||
this.ordering = ordering;
|
||||
this.limit = limit;
|
||||
this.offsetLimit = offsetLimit == null ? OffsetLimit.NONE : offsetLimit;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
@ -93,8 +93,8 @@ public class ScanOperator implements Operator
|
|||
decor.limitTimeRange(timeRange);
|
||||
}
|
||||
|
||||
if (limit > 0) {
|
||||
decor.setLimit(limit);
|
||||
if (offsetLimit.isPresent()) {
|
||||
decor.setOffsetLimit(offsetLimit);
|
||||
}
|
||||
|
||||
if (!(ordering == null || ordering.isEmpty())) {
|
||||
|
|
|
@ -31,7 +31,7 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
{
|
||||
private final Interval timeRange;
|
||||
private final DimFilter filter;
|
||||
private final int limit;
|
||||
private final OffsetLimit offsetLimit;
|
||||
private final List<String> projectedColumns;
|
||||
private final VirtualColumns virtualColumns;
|
||||
private final List<ColumnWithDirection> ordering;
|
||||
|
@ -39,7 +39,7 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
public ScanOperatorFactory(
|
||||
@JsonProperty("timeRange") final Interval timeRange,
|
||||
@JsonProperty("filter") final DimFilter filter,
|
||||
@JsonProperty("limit") final Integer limit,
|
||||
@JsonProperty("offsetLimit") final OffsetLimit offsetLimit,
|
||||
@JsonProperty("projectedColumns") final List<String> projectedColumns,
|
||||
@JsonProperty("virtualColumns") final VirtualColumns virtualColumns,
|
||||
@JsonProperty("ordering") final List<ColumnWithDirection> ordering
|
||||
|
@ -47,7 +47,7 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
{
|
||||
this.timeRange = timeRange;
|
||||
this.filter = filter;
|
||||
this.limit = limit == null ? -1 : limit;
|
||||
this.offsetLimit = offsetLimit;
|
||||
this.projectedColumns = projectedColumns;
|
||||
this.virtualColumns = virtualColumns;
|
||||
this.ordering = ordering;
|
||||
|
@ -66,9 +66,9 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
public int getLimit()
|
||||
public OffsetLimit getOffsetLimit()
|
||||
{
|
||||
return limit;
|
||||
return offsetLimit;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
|
@ -99,7 +99,7 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
timeRange,
|
||||
filter == null ? null : filter.toFilter(),
|
||||
ordering,
|
||||
limit
|
||||
offsetLimit
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -119,18 +119,32 @@ public class ScanOperatorFactory implements OperatorFactory
|
|||
return false;
|
||||
}
|
||||
ScanOperatorFactory that = (ScanOperatorFactory) o;
|
||||
return limit == that.limit && Objects.equals(timeRange, that.timeRange) && Objects.equals(
|
||||
filter,
|
||||
that.filter
|
||||
) && Objects.equals(projectedColumns, that.projectedColumns) && Objects.equals(
|
||||
virtualColumns,
|
||||
that.virtualColumns
|
||||
) && Objects.equals(ordering, that.ordering);
|
||||
return Objects.equals(offsetLimit, that.offsetLimit)
|
||||
&& Objects.equals(timeRange, that.timeRange)
|
||||
&& Objects.equals(filter, that.filter)
|
||||
&& Objects.equals(projectedColumns, that.projectedColumns)
|
||||
&& Objects.equals(virtualColumns, that.virtualColumns)
|
||||
&& Objects.equals(ordering, that.ordering);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(timeRange, filter, limit, projectedColumns, virtualColumns, ordering);
|
||||
return Objects.hash(timeRange, filter, offsetLimit, projectedColumns, virtualColumns, ordering);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ScanOperatorFactory{" +
|
||||
"timeRange=" + timeRange +
|
||||
", filter=" + filter +
|
||||
", offsetLimit=" + offsetLimit +
|
||||
", projectedColumns=" + projectedColumns +
|
||||
", virtualColumns=" + virtualColumns +
|
||||
", ordering=" + ordering
|
||||
+ "}";
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -34,11 +34,13 @@ import org.apache.druid.query.spec.QuerySegmentSpec;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
|
||||
/**
|
||||
* A query that can compute window functions on top of a completely in-memory inline datasource or query results.
|
||||
* <p>
|
||||
|
@ -122,14 +124,17 @@ public class WindowOperatorQuery extends BaseQuery<RowsAndColumns>
|
|||
)
|
||||
);
|
||||
}
|
||||
if (ordering.isEmpty()) {
|
||||
ordering = null;
|
||||
}
|
||||
|
||||
this.leafOperators.add(
|
||||
new ScanOperatorFactory(
|
||||
null,
|
||||
scan.getFilter(),
|
||||
(int) scan.getScanRowsLimit(),
|
||||
scan.getOffsetLimit(),
|
||||
scan.getColumns(),
|
||||
scan.getVirtualColumns(),
|
||||
scan.getVirtualColumns().isEmpty() ? null : scan.getVirtualColumns(),
|
||||
ordering
|
||||
)
|
||||
);
|
||||
|
@ -242,16 +247,15 @@ public class WindowOperatorQuery extends BaseQuery<RowsAndColumns>
|
|||
return false;
|
||||
}
|
||||
WindowOperatorQuery that = (WindowOperatorQuery) o;
|
||||
return Objects.equals(rowSignature, that.rowSignature) && Objects.equals(
|
||||
operators,
|
||||
that.operators
|
||||
);
|
||||
return Objects.equals(rowSignature, that.rowSignature)
|
||||
&& Objects.equals(operators, that.operators)
|
||||
&& Objects.equals(leafOperators, that.leafOperators);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), rowSignature, operators);
|
||||
return Objects.hash(super.hashCode(), rowSignature, operators, leafOperators);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -263,6 +267,7 @@ public class WindowOperatorQuery extends BaseQuery<RowsAndColumns>
|
|||
", context=" + getContext() +
|
||||
", rowSignature=" + rowSignature +
|
||||
", operators=" + operators +
|
||||
", leafOperators=" + leafOperators +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns;
|
||||
|
@ -73,7 +74,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
private Interval interval;
|
||||
private Filter filter;
|
||||
private VirtualColumns virtualColumns;
|
||||
private int limit;
|
||||
private OffsetLimit limit;
|
||||
private LinkedHashSet<String> viewableColumns;
|
||||
private List<ColumnWithDirection> ordering;
|
||||
|
||||
|
@ -82,7 +83,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
Interval interval,
|
||||
Filter filter,
|
||||
VirtualColumns virtualColumns,
|
||||
int limit,
|
||||
OffsetLimit limit,
|
||||
List<ColumnWithDirection> ordering,
|
||||
LinkedHashSet<String> viewableColumns
|
||||
)
|
||||
|
@ -175,7 +176,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
|
||||
private boolean needsMaterialization()
|
||||
{
|
||||
return interval != null || filter != null || limit != -1 || ordering != null || virtualColumns != null;
|
||||
return interval != null || filter != null || limit.isPresent() || ordering != null || virtualColumns != null;
|
||||
}
|
||||
|
||||
private Pair<byte[], RowSignature> materialize()
|
||||
|
@ -198,7 +199,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
interval = null;
|
||||
filter = null;
|
||||
virtualColumns = null;
|
||||
limit = -1;
|
||||
limit = OffsetLimit.NONE;
|
||||
viewableColumns = null;
|
||||
ordering = null;
|
||||
}
|
||||
|
@ -238,7 +239,8 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
throw new ISE("accumulated[%s] non-null, why did we get multiple cursors?", accumulated);
|
||||
}
|
||||
|
||||
int theLimit = limit == -1 ? Integer.MAX_VALUE : limit;
|
||||
long remainingRowsToSkip = limit.getOffset();
|
||||
long remainingRowsToFetch = limit.getLimitOrMax();
|
||||
|
||||
final ColumnSelectorFactory columnSelectorFactory = in.getColumnSelectorFactory();
|
||||
final RowSignature.Builder sigBob = RowSignature.builder();
|
||||
|
@ -284,12 +286,12 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
);
|
||||
|
||||
final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(columnSelectorFactory);
|
||||
while (!in.isDoneOrInterrupted()) {
|
||||
for (; !in.isDoneOrInterrupted() && remainingRowsToSkip > 0; remainingRowsToSkip--) {
|
||||
in.advance();
|
||||
}
|
||||
for (; !in.isDoneOrInterrupted() && remainingRowsToFetch > 0; remainingRowsToFetch--) {
|
||||
frameWriter.addSelection();
|
||||
in.advance();
|
||||
if (--theLimit <= 0) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return frameWriter;
|
||||
|
@ -390,12 +392,8 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
sigBob.add(column, racColumn.toAccessor().getType());
|
||||
}
|
||||
|
||||
final int limitedNumRows;
|
||||
if (limit == -1) {
|
||||
limitedNumRows = Integer.MAX_VALUE;
|
||||
} else {
|
||||
limitedNumRows = limit;
|
||||
}
|
||||
long remainingRowsToSkip = limit.getOffset();
|
||||
long remainingRowsToFetch = limit.getLimitOrMax();
|
||||
|
||||
final FrameWriter frameWriter = FrameWriters.makeFrameWriterFactory(
|
||||
FrameType.COLUMNAR,
|
||||
|
@ -405,11 +403,16 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
).newFrameWriter(selectorFactory);
|
||||
|
||||
rowId.set(0);
|
||||
for (; rowId.get() < numRows && frameWriter.getNumRows() < limitedNumRows; rowId.incrementAndGet()) {
|
||||
for (; rowId.get() < numRows && remainingRowsToFetch > 0; rowId.incrementAndGet()) {
|
||||
final int theId = rowId.get();
|
||||
if (rowsToSkip != null && rowsToSkip.get(theId)) {
|
||||
continue;
|
||||
}
|
||||
if (remainingRowsToSkip > 0) {
|
||||
remainingRowsToSkip--;
|
||||
continue;
|
||||
}
|
||||
remainingRowsToFetch--;
|
||||
frameWriter.addSelection();
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
|
|||
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.VirtualColumn;
|
||||
|
@ -39,14 +40,14 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
private Interval interval;
|
||||
private Filter filter;
|
||||
private VirtualColumns virtualColumns;
|
||||
private int limit;
|
||||
private OffsetLimit offsetLimit;
|
||||
private List<ColumnWithDirection> ordering;
|
||||
|
||||
public DefaultRowsAndColumnsDecorator(
|
||||
RowsAndColumns base
|
||||
)
|
||||
{
|
||||
this(base, null, null, null, -1, null);
|
||||
this(base, null, null, null, OffsetLimit.NONE, null);
|
||||
}
|
||||
|
||||
public DefaultRowsAndColumnsDecorator(
|
||||
|
@ -54,7 +55,7 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
Interval interval,
|
||||
Filter filter,
|
||||
VirtualColumns virtualColumns,
|
||||
int limit,
|
||||
OffsetLimit limit,
|
||||
List<ColumnWithDirection> ordering
|
||||
)
|
||||
{
|
||||
|
@ -62,7 +63,7 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
this.interval = interval;
|
||||
this.filter = filter;
|
||||
this.virtualColumns = virtualColumns;
|
||||
this.limit = limit;
|
||||
this.offsetLimit = limit;
|
||||
this.ordering = ordering;
|
||||
}
|
||||
|
||||
|
@ -111,13 +112,9 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setLimit(int numRows)
|
||||
public void setOffsetLimit(OffsetLimit offsetLimit)
|
||||
{
|
||||
if (this.limit == -1) {
|
||||
this.limit = numRows;
|
||||
} else {
|
||||
this.limit = Math.min(limit, numRows);
|
||||
}
|
||||
this.offsetLimit = offsetLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -134,7 +131,7 @@ public class DefaultRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
interval,
|
||||
filter,
|
||||
virtualColumns,
|
||||
limit,
|
||||
offsetLimit,
|
||||
ordering,
|
||||
columns == null ? null : new LinkedHashSet<>(columns)
|
||||
);
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
|
|||
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -61,7 +62,7 @@ public interface RowsAndColumnsDecorator
|
|||
|
||||
void addVirtualColumns(VirtualColumns virtualColumn);
|
||||
|
||||
void setLimit(int numRows);
|
||||
void setOffsetLimit(OffsetLimit offsetLimit);
|
||||
|
||||
void setOrdering(List<ColumnWithDirection> ordering);
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.apache.druid.query.DataSource;
|
|||
import org.apache.druid.query.Druids;
|
||||
import org.apache.druid.query.Queries;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.spec.QuerySegmentSpec;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
|
@ -325,6 +326,11 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
return scanRowsLimit;
|
||||
}
|
||||
|
||||
public OffsetLimit getOffsetLimit()
|
||||
{
|
||||
return new OffsetLimit(scanRowsOffset, scanRowsLimit);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns whether this query is limited or not. Because {@link Long#MAX_VALUE} is used to signify unlimitedness,
|
||||
* this is equivalent to {@code getScanRowsLimit() != Long.Max_VALUE}.
|
||||
|
@ -667,4 +673,5 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
|
|||
return obj instanceof Integer && (int) obj == DEFAULT_BATCH_SIZE;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -42,8 +42,10 @@ import org.apache.druid.java.util.common.guava.BaseSequence;
|
|||
import org.apache.druid.java.util.common.guava.Sequence;
|
||||
import org.apache.druid.java.util.common.guava.Sequences;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.query.DataSource;
|
||||
import org.apache.druid.query.FrameSignaturePair;
|
||||
import org.apache.druid.query.GenericQueryMetricsFactory;
|
||||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.IterableRowsCursorHelper;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryMetrics;
|
||||
|
@ -57,6 +59,8 @@ import org.apache.druid.segment.column.ColumnType;
|
|||
import org.apache.druid.segment.column.RowSignature;
|
||||
import org.apache.druid.utils.CloseableUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
|
@ -196,8 +200,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||
final ColumnCapabilities capabilities = virtualColumn.capabilities(c -> null, columnName);
|
||||
columnType = capabilities != null ? capabilities.toColumnType() : null;
|
||||
} else {
|
||||
// Unknown type. In the future, it would be nice to have a way to fill these in.
|
||||
columnType = null;
|
||||
columnType = getDataSourceColumnType(query.getDataSource(), columnName);
|
||||
}
|
||||
|
||||
builder.add(columnName, columnType);
|
||||
|
@ -207,6 +210,20 @@ public class ScanQueryQueryToolChest extends QueryToolChest<ScanResultValue, Sca
|
|||
}
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private ColumnType getDataSourceColumnType(DataSource dataSource, String columnName)
|
||||
{
|
||||
if (dataSource instanceof InlineDataSource) {
|
||||
InlineDataSource inlineDataSource = (InlineDataSource) dataSource;
|
||||
ColumnCapabilities caps = inlineDataSource.getRowSignature().getColumnCapabilities(columnName);
|
||||
if (caps != null) {
|
||||
return caps.toColumnType();
|
||||
}
|
||||
}
|
||||
// Unknown type. In the future, it would be nice to have a way to fill these in.
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case
|
||||
* it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue}
|
||||
|
|
|
@ -53,6 +53,9 @@ public class TypeStrategies
|
|||
@Nullable
|
||||
public static TypeStrategy<?> getComplex(String typeName)
|
||||
{
|
||||
if (typeName == null) {
|
||||
return null;
|
||||
}
|
||||
return COMPLEX_STRATEGIES.get(typeName);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -61,7 +61,7 @@ public class ScanOperatorFactoryTest
|
|||
final Builder bob = new Builder();
|
||||
bob.timeRange = Intervals.utc(0, 6);
|
||||
bob.filter = DimFilters.dimEquals("abc", "b");
|
||||
bob.limit = 48;
|
||||
bob.offsetLimit = OffsetLimit.limit(48);
|
||||
bob.projectedColumns = Arrays.asList("a", "b");
|
||||
bob.virtualColumns = VirtualColumns.EMPTY;
|
||||
bob.ordering = Collections.singletonList(ColumnWithDirection.ascending("a"));
|
||||
|
@ -72,7 +72,7 @@ public class ScanOperatorFactoryTest
|
|||
|
||||
Assert.assertNotEquals(factory, bob.copy().setTimeRange(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setFilter(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setLimit(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setOffsetLimit(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setProjectedColumns(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setVirtualColumns(null).build());
|
||||
Assert.assertNotEquals(factory, bob.copy().setOrdering(null).build());
|
||||
|
@ -132,7 +132,7 @@ public class ScanOperatorFactoryTest
|
|||
"interval[%s], filter[%s], limit[%s], ordering[%s], projection[%s], virtual[%s]",
|
||||
interval,
|
||||
filter,
|
||||
limit,
|
||||
OffsetLimit.limit(limit),
|
||||
ordering,
|
||||
projection,
|
||||
virtual
|
||||
|
@ -141,7 +141,7 @@ public class ScanOperatorFactoryTest
|
|||
ScanOperatorFactory factory = new ScanOperatorFactory(
|
||||
interval,
|
||||
filter,
|
||||
limit,
|
||||
OffsetLimit.limit(limit),
|
||||
projection,
|
||||
virtual,
|
||||
ordering
|
||||
|
@ -182,7 +182,7 @@ public class ScanOperatorFactoryTest
|
|||
(TestRowsAndColumnsDecorator.DecoratedRowsAndColumns) inRac;
|
||||
|
||||
Assert.assertEquals(msg, factory.getTimeRange(), rac.getTimeRange());
|
||||
Assert.assertEquals(msg, factory.getLimit(), rac.getLimit());
|
||||
Assert.assertEquals(msg, factory.getOffsetLimit(), rac.getOffsetLimit());
|
||||
Assert.assertEquals(msg, factory.getVirtualColumns(), rac.getVirtualColumns());
|
||||
validateList(msg, factory.getOrdering(), rac.getOrdering());
|
||||
validateList(msg, factory.getProjectedColumns(), rac.getProjectedColumns());
|
||||
|
@ -228,7 +228,7 @@ public class ScanOperatorFactoryTest
|
|||
{
|
||||
private Interval timeRange;
|
||||
private DimFilter filter;
|
||||
private Integer limit;
|
||||
private OffsetLimit offsetLimit;
|
||||
private List<String> projectedColumns;
|
||||
private VirtualColumns virtualColumns;
|
||||
private List<ColumnWithDirection> ordering;
|
||||
|
@ -245,9 +245,9 @@ public class ScanOperatorFactoryTest
|
|||
return this;
|
||||
}
|
||||
|
||||
public Builder setLimit(Integer limit)
|
||||
public Builder setOffsetLimit(OffsetLimit offsetLimit)
|
||||
{
|
||||
this.limit = limit;
|
||||
this.offsetLimit = offsetLimit;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -274,7 +274,7 @@ public class ScanOperatorFactoryTest
|
|||
Builder retVal = new Builder();
|
||||
retVal.timeRange = timeRange;
|
||||
retVal.filter = filter;
|
||||
retVal.limit = limit;
|
||||
retVal.offsetLimit = offsetLimit;
|
||||
retVal.projectedColumns = projectedColumns;
|
||||
retVal.virtualColumns = virtualColumns;
|
||||
retVal.ordering = ordering;
|
||||
|
@ -286,7 +286,7 @@ public class ScanOperatorFactoryTest
|
|||
return new ScanOperatorFactory(
|
||||
timeRange,
|
||||
filter,
|
||||
limit,
|
||||
offsetLimit,
|
||||
projectedColumns,
|
||||
virtualColumns,
|
||||
ordering
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.guava.Sequence;
|
|||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.filter.InDimFilter;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
|
||||
|
@ -121,7 +122,7 @@ public class RowsAndColumnsDecoratorTest extends SemanticTestBase
|
|||
for (int k = 0; k <= limits.length; ++k) {
|
||||
int limit = (k == 0 ? -1 : limits[k - 1]);
|
||||
for (int l = 0; l <= orderings.length; ++l) {
|
||||
validateDecorated(base, siggy, vals, interval, filter, limit, l == 0 ? null : orderings[l - 1]);
|
||||
validateDecorated(base, siggy, vals, interval, filter, OffsetLimit.limit(limit), l == 0 ? null : orderings[l - 1]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -134,7 +135,7 @@ public class RowsAndColumnsDecoratorTest extends SemanticTestBase
|
|||
Object[][] originalVals,
|
||||
Interval interval,
|
||||
Filter filter,
|
||||
int limit,
|
||||
OffsetLimit limit,
|
||||
List<ColumnWithDirection> ordering
|
||||
)
|
||||
{
|
||||
|
@ -211,10 +212,10 @@ public class RowsAndColumnsDecoratorTest extends SemanticTestBase
|
|||
vals.sort(comparator);
|
||||
}
|
||||
|
||||
if (limit != -1) {
|
||||
decor.setLimit(limit);
|
||||
|
||||
vals = vals.subList(0, Math.min(vals.size(), limit));
|
||||
if (limit.isPresent()) {
|
||||
decor.setOffsetLimit(limit);
|
||||
int size = vals.size();
|
||||
vals = vals.subList((int) limit.getFromIndex(size), (int) limit.getToIndex(vals.size()));
|
||||
}
|
||||
|
||||
if (ordering != null) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.druid.query.rowsandcols.semantic;
|
|||
|
||||
import org.apache.druid.query.filter.Filter;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.rowsandcols.RowsAndColumns;
|
||||
import org.apache.druid.query.rowsandcols.column.Column;
|
||||
import org.apache.druid.segment.VirtualColumns;
|
||||
|
@ -35,7 +36,7 @@ public class TestRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
private Interval timeRange;
|
||||
private Filter filter;
|
||||
private VirtualColumns virtualColumns;
|
||||
private int limit = -1;
|
||||
private OffsetLimit offsetLimit = OffsetLimit.NONE;
|
||||
private List<ColumnWithDirection> ordering;
|
||||
private List<String> projectedColumns;
|
||||
|
||||
|
@ -58,9 +59,9 @@ public class TestRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
}
|
||||
|
||||
@Override
|
||||
public void setLimit(int numRows)
|
||||
public void setOffsetLimit(OffsetLimit offsetLimit)
|
||||
{
|
||||
this.limit = numRows;
|
||||
this.offsetLimit = offsetLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -99,9 +100,9 @@ public class TestRowsAndColumnsDecorator implements RowsAndColumnsDecorator
|
|||
return virtualColumns;
|
||||
}
|
||||
|
||||
public int getLimit()
|
||||
public OffsetLimit getOffsetLimit()
|
||||
{
|
||||
return limit;
|
||||
return offsetLimit;
|
||||
}
|
||||
|
||||
public List<ColumnWithDirection> getOrdering()
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
@ -692,4 +694,10 @@ public class TypeStrategiesTest
|
|||
return read(ByteBuffer.wrap(value));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void getComplexTypeNull()
|
||||
{
|
||||
assertNull(TypeStrategies.getComplex(null));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -177,11 +177,7 @@ public class CalciteRulesManager
|
|||
private static final List<RelOptRule> 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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -47,7 +47,14 @@ import java.util.Set;
|
|||
*/
|
||||
public class DruidOuterQueryRel extends DruidRel<DruidOuterQueryRel>
|
||||
{
|
||||
private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__subquery__");
|
||||
private static final TableDataSource DUMMY_DATA_SOURCE = new TableDataSource("__subquery__")
|
||||
{
|
||||
@Override
|
||||
public boolean isConcrete()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
private static final QueryDataSource DUMMY_QUERY_DATA_SOURCE = new QueryDataSource(
|
||||
Druids.newScanQueryBuilder().dataSource("__subquery__").eternityInterval().build()
|
||||
|
|
|
@ -67,6 +67,9 @@ import org.apache.druid.query.groupby.GroupByQuery;
|
|||
import org.apache.druid.query.groupby.having.DimFilterHavingSpec;
|
||||
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection.Direction;
|
||||
import org.apache.druid.query.operator.NaiveSortOperatorFactory;
|
||||
import org.apache.druid.query.operator.OperatorFactory;
|
||||
import org.apache.druid.query.operator.ScanOperatorFactory;
|
||||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
|
@ -1014,11 +1017,16 @@ public class DruidQuery
|
|||
return groupByQuery;
|
||||
}
|
||||
|
||||
final ScanQuery scanQuery = toScanQuery();
|
||||
final ScanQuery scanQuery = toScanQuery(true);
|
||||
if (scanQuery != null) {
|
||||
return scanQuery;
|
||||
}
|
||||
|
||||
final WindowOperatorQuery scanAndSortQuery = toScanAndSortQuery();
|
||||
if (scanAndSortQuery != null) {
|
||||
return scanAndSortQuery;
|
||||
}
|
||||
|
||||
throw new CannotBuildQueryException("Cannot convert query parts into an actual query");
|
||||
}
|
||||
|
||||
|
@ -1439,6 +1447,11 @@ public class DruidQuery
|
|||
if (windowing == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// This is not yet supported
|
||||
if (dataSource.isConcrete()) {
|
||||
return null;
|
||||
}
|
||||
if (dataSource instanceof TableDataSource) {
|
||||
// We need a scan query to pull the results up for us before applying the window
|
||||
// Returning null here to ensure that the planner generates that alternative
|
||||
|
@ -1473,13 +1486,83 @@ public class DruidQuery
|
|||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an OperatorQuery which runs an order on top of a scan.
|
||||
*/
|
||||
@Nullable
|
||||
private WindowOperatorQuery toScanAndSortQuery()
|
||||
{
|
||||
if (sorting == null
|
||||
|| sorting.getOrderBys().isEmpty()
|
||||
|| sorting.getProjection() != null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
ScanQuery scan = toScanQuery(false);
|
||||
if (scan == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
if (dataSource.isConcrete()) {
|
||||
// Currently only non-time orderings of subqueries are allowed.
|
||||
List<String> orderByColumnNames = sorting.getOrderBys()
|
||||
.stream().map(OrderByColumnSpec::getDimension)
|
||||
.collect(Collectors.toList());
|
||||
plannerContext.setPlanningError(
|
||||
"SQL query requires ordering a table by non-time column [%s], which is not supported.",
|
||||
orderByColumnNames
|
||||
);
|
||||
return null;
|
||||
}
|
||||
|
||||
QueryDataSource newDataSource = new QueryDataSource(scan);
|
||||
List<ColumnWithDirection> sortColumns = getColumnWithDirectionsFromOrderBys(sorting.getOrderBys());
|
||||
RowSignature signature = getOutputRowSignature();
|
||||
List<OperatorFactory> operators = new ArrayList<>();
|
||||
|
||||
operators.add(new NaiveSortOperatorFactory(sortColumns));
|
||||
if (!sorting.getOffsetLimit().isNone()) {
|
||||
operators.add(
|
||||
new ScanOperatorFactory(
|
||||
null,
|
||||
null,
|
||||
sorting.getOffsetLimit().toOperatorOffsetLimit(),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
return new WindowOperatorQuery(
|
||||
newDataSource,
|
||||
new LegacySegmentSpec(Intervals.ETERNITY),
|
||||
plannerContext.queryContextMap(),
|
||||
signature,
|
||||
operators,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
||||
private ArrayList<ColumnWithDirection> getColumnWithDirectionsFromOrderBys(List<OrderByColumnSpec> orderBys)
|
||||
{
|
||||
ArrayList<ColumnWithDirection> ordering = new ArrayList<>();
|
||||
for (OrderByColumnSpec orderBySpec : orderBys) {
|
||||
Direction direction = orderBySpec.getDirection() == OrderByColumnSpec.Direction.ASCENDING
|
||||
? ColumnWithDirection.Direction.ASC
|
||||
: ColumnWithDirection.Direction.DESC;
|
||||
ordering.add(new ColumnWithDirection(orderBySpec.getDimension(), direction));
|
||||
}
|
||||
return ordering;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return this query as a Scan query, or null if this query is not compatible with Scan.
|
||||
*
|
||||
* @param considerSorting can be used to ignore the current sorting requirements {@link #toScanAndSortQuery()} uses it to produce the non-sorted part
|
||||
* @return query or null
|
||||
*/
|
||||
@Nullable
|
||||
private ScanQuery toScanQuery()
|
||||
private ScanQuery toScanQuery(final boolean considerSorting)
|
||||
{
|
||||
if (grouping != null || windowing != null) {
|
||||
// Scan cannot GROUP BY or do windows.
|
||||
|
@ -1504,7 +1587,7 @@ public class DruidQuery
|
|||
long scanOffset = 0L;
|
||||
long scanLimit = 0L;
|
||||
|
||||
if (sorting != null) {
|
||||
if (considerSorting && sorting != null) {
|
||||
scanOffset = sorting.getOffsetLimit().getOffset();
|
||||
|
||||
if (sorting.getOffsetLimit().hasLimit()) {
|
||||
|
|
|
@ -0,0 +1,102 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.query.filter.DimFilter;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection.Direction;
|
||||
import org.apache.druid.query.operator.NaivePartitioningOperatorFactory;
|
||||
import org.apache.druid.query.operator.NaiveSortOperatorFactory;
|
||||
import org.apache.druid.query.operator.OffsetLimit;
|
||||
import org.apache.druid.query.operator.OperatorFactory;
|
||||
import org.apache.druid.query.operator.ScanOperatorFactory;
|
||||
import org.apache.druid.query.operator.window.ComposingProcessor;
|
||||
import org.apache.druid.query.operator.window.Processor;
|
||||
import org.apache.druid.query.operator.window.WindowOperatorFactory;
|
||||
import org.apache.druid.query.operator.window.ranking.WindowRankProcessor;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class OperatorFactoryBuilders
|
||||
{
|
||||
|
||||
public static ScanOperatorFactoryBuilder scanOperatorFactoryBuilder()
|
||||
{
|
||||
return new ScanOperatorFactoryBuilder();
|
||||
}
|
||||
|
||||
public static class ScanOperatorFactoryBuilder
|
||||
{
|
||||
private OffsetLimit offsetLimit;
|
||||
private DimFilter filter;
|
||||
private List<String> projectedColumns;
|
||||
|
||||
public OperatorFactory build()
|
||||
{
|
||||
return new ScanOperatorFactory(null, filter, offsetLimit, projectedColumns, null, null);
|
||||
}
|
||||
|
||||
public ScanOperatorFactoryBuilder setOffsetLimit(long offset, long limit)
|
||||
{
|
||||
offsetLimit = new OffsetLimit(offset, limit);
|
||||
return this;
|
||||
}
|
||||
|
||||
public ScanOperatorFactoryBuilder setFilter(DimFilter filter)
|
||||
{
|
||||
this.filter = filter;
|
||||
return this;
|
||||
}
|
||||
|
||||
public ScanOperatorFactoryBuilder setProjectedColumns(String... columns)
|
||||
{
|
||||
this.projectedColumns = Arrays.asList(columns);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public static OperatorFactory naiveSortOperator(ColumnWithDirection... colWithDirs)
|
||||
{
|
||||
return new NaiveSortOperatorFactory(Arrays.asList(colWithDirs));
|
||||
}
|
||||
|
||||
public static OperatorFactory naiveSortOperator(String column, Direction direction)
|
||||
{
|
||||
return naiveSortOperator(new ColumnWithDirection(column, direction));
|
||||
}
|
||||
|
||||
public static OperatorFactory naivePartitionOperator(String... columns)
|
||||
{
|
||||
return new NaivePartitioningOperatorFactory(Arrays.asList(columns));
|
||||
}
|
||||
|
||||
public static WindowOperatorFactory windowOperators(Processor... processors)
|
||||
{
|
||||
Preconditions.checkArgument(processors.length > 0, "You must specify at least one processor!");
|
||||
return new WindowOperatorFactory(processors.length == 1 ? processors[0] : new ComposingProcessor(processors));
|
||||
}
|
||||
|
||||
public static Processor rankProcessor(String outputColumn, String... groupingColumns)
|
||||
{
|
||||
return new WindowRankProcessor(Arrays.asList(groupingColumns), outputColumn, false);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,91 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.query;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.query.operator.OperatorFactory;
|
||||
import org.apache.druid.query.operator.WindowOperatorQuery;
|
||||
import org.apache.druid.query.spec.LegacySegmentSpec;
|
||||
import org.apache.druid.query.spec.QuerySegmentSpec;
|
||||
import org.apache.druid.segment.column.RowSignature;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class WindowOperatorQueryBuilder
|
||||
{
|
||||
private DataSource dataSource;
|
||||
private QuerySegmentSpec intervals = new LegacySegmentSpec(Intervals.ETERNITY);
|
||||
private Map<String, Object> context;
|
||||
private RowSignature rowSignature;
|
||||
private List<OperatorFactory> operators;
|
||||
private List<OperatorFactory> leafOperators;
|
||||
|
||||
public static WindowOperatorQueryBuilder builder()
|
||||
{
|
||||
return new WindowOperatorQueryBuilder();
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setDataSource(DataSource dataSource)
|
||||
{
|
||||
this.dataSource = dataSource;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setDataSource(String dataSource)
|
||||
{
|
||||
return setDataSource(new TableDataSource(dataSource));
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setDataSource(Query<?> query)
|
||||
{
|
||||
return setDataSource(new QueryDataSource(query));
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setSignature(RowSignature rowSignature)
|
||||
{
|
||||
this.rowSignature = rowSignature;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Query<?> build()
|
||||
{
|
||||
return new WindowOperatorQuery(
|
||||
dataSource,
|
||||
intervals,
|
||||
context,
|
||||
rowSignature,
|
||||
operators,
|
||||
leafOperators);
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setOperators(OperatorFactory... operators)
|
||||
{
|
||||
this.operators = Lists.newArrayList(operators);
|
||||
return this;
|
||||
}
|
||||
|
||||
public WindowOperatorQueryBuilder setLeafOperators(OperatorFactory... operators)
|
||||
{
|
||||
this.leafOperators = Lists.newArrayList(operators);
|
||||
return this;
|
||||
}
|
||||
}
|
|
@ -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,13 +3172,12 @@ 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()
|
||||
new TopNQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.dimension(new DefaultDimensionSpec(
|
||||
"dim2",
|
||||
|
@ -3209,16 +3209,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
|
|||
.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)
|
||||
.postAggregators(new ExpressionPostAggregator("s0", "1", null, ExprMacroTable.nil()))
|
||||
.build()
|
||||
),
|
||||
useDefault ?
|
||||
|
|
|
@ -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<String, Object> queryContext = new HashMap<>(QUERY_CONTEXT_DEFAULT);
|
||||
|
@ -5399,9 +5397,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
.dataSource(
|
||||
JoinDataSource.create(
|
||||
new TableDataSource(CalciteTests.DATASOURCE3),
|
||||
new QueryDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(InlineDataSource.fromIterable(
|
||||
InlineDataSource.fromIterable(
|
||||
ImmutableList.of(
|
||||
new Object[]{4842L},
|
||||
new Object[]{4844L},
|
||||
|
@ -5413,18 +5409,9 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
|
|||
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())
|
||||
.build()
|
||||
),
|
||||
"j0.",
|
||||
"(\"l1\" == \"j0.d0\")",
|
||||
"(\"l1\" == \"j0.ROW_VALUE\")",
|
||||
JoinType.INNER,
|
||||
null,
|
||||
ExprMacroTable.nil(),
|
||||
|
|
|
@ -39,11 +39,13 @@ import org.apache.druid.query.Druids;
|
|||
import org.apache.druid.query.InlineDataSource;
|
||||
import org.apache.druid.query.JoinDataSource;
|
||||
import org.apache.druid.query.LookupDataSource;
|
||||
import org.apache.druid.query.OperatorFactoryBuilders;
|
||||
import org.apache.druid.query.Query;
|
||||
import org.apache.druid.query.QueryContexts;
|
||||
import org.apache.druid.query.QueryDataSource;
|
||||
import org.apache.druid.query.TableDataSource;
|
||||
import org.apache.druid.query.UnionDataSource;
|
||||
import org.apache.druid.query.WindowOperatorQueryBuilder;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||
|
@ -96,6 +98,7 @@ import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
|
|||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec.Direction;
|
||||
import org.apache.druid.query.lookup.RegisteredLookupExtractionFn;
|
||||
import org.apache.druid.query.operator.ColumnWithDirection;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.query.scan.ScanQuery;
|
||||
import org.apache.druid.query.scan.ScanQuery.ResultFormat;
|
||||
|
@ -2725,7 +2728,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_CONVERT)
|
||||
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
|
||||
@Test
|
||||
public void testGroupByWithSelectAndOrderByProjections()
|
||||
{
|
||||
|
@ -2810,7 +2813,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_CONVERT)
|
||||
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
|
||||
@Test
|
||||
public void testTopNWithSelectAndOrderByProjections()
|
||||
{
|
||||
|
@ -4692,7 +4695,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_CONVERT)
|
||||
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
|
||||
@Test
|
||||
public void testGroupByWithSortOnPostAggregationDefault()
|
||||
{
|
||||
|
@ -4724,7 +4727,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_CONVERT)
|
||||
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
|
||||
@Test
|
||||
public void testGroupByWithSortOnPostAggregationNoTopNConfig()
|
||||
{
|
||||
|
@ -4768,7 +4771,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
);
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_CONVERT)
|
||||
@NotYetSupported(Modes.CANNOT_APPLY_VIRTUAL_COL)
|
||||
@Test
|
||||
public void testGroupByWithSortOnPostAggregationNoTopNContext()
|
||||
{
|
||||
|
@ -5370,7 +5373,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
final Map<String, String> queries = ImmutableMap.of(
|
||||
// SELECT query with order by non-__time.
|
||||
"SELECT dim1 FROM druid.foo ORDER BY dim1",
|
||||
"SQL query requires order by non-time column [[dim1 ASC]], which is not supported.",
|
||||
"SQL query requires ordering a table by non-time column [[dim1]], which is not supported.",
|
||||
|
||||
// JOIN condition with not-equals (<>).
|
||||
"SELECT foo.dim1, foo.dim2, l.k, l.v\n"
|
||||
|
@ -13949,30 +13952,13 @@ 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
|
||||
))
|
||||
.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)
|
||||
.setPostAggregatorSpecs(ImmutableList.of(
|
||||
expressionPostAgg("p0", "case_searched((\"a0\" == 0),1,0)")))
|
||||
.build()
|
||||
|
||||
),
|
||||
|
@ -14309,7 +14295,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
{
|
||||
skipVectorize();
|
||||
cannotVectorize();
|
||||
testQuery(
|
||||
|
||||
testBuilder()
|
||||
.sql(
|
||||
"with t AS (SELECT m2, COUNT(m1) as trend_score\n"
|
||||
+ "FROM \"foo\"\n"
|
||||
+ "GROUP BY 1 \n"
|
||||
|
@ -14318,57 +14306,66 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
|||
+ "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()
|
||||
+ "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(
|
||||
.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()
|
||||
)
|
||||
.setContext(OUTER_LIMIT_CONTEXT)
|
||||
.setOperators(
|
||||
OperatorFactoryBuilders.naiveSortOperator("a0", ColumnWithDirection.Direction.DESC)
|
||||
)
|
||||
.setLeafOperators(
|
||||
OperatorFactoryBuilders.scanOperatorFactoryBuilder()
|
||||
.setOffsetLimit(0, Long.MAX_VALUE)
|
||||
.setFilter(
|
||||
range(
|
||||
"d0",
|
||||
ColumnType.LONG,
|
||||
2L,
|
||||
null,
|
||||
true,
|
||||
false
|
||||
)
|
||||
)
|
||||
.setProjectedColumns("a0", "d0")
|
||||
.build()
|
||||
),
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.expectedResults(
|
||||
ImmutableList.of(
|
||||
new Object[] {3.0D, 1L},
|
||||
new Object[] {4.0D, 1L},
|
||||
new Object[] {5.0D, 1L},
|
||||
new Object[] {6.0D, 1L}
|
||||
)
|
||||
);
|
||||
)
|
||||
.run();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -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<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[] {4.0D, 1L},
|
||||
new Object[] {5.0D, 1L}
|
||||
);
|
||||
|
||||
testBuilder()
|
||||
.sql(sql)
|
||||
.expectedQuery(
|
||||
WindowOperatorQueryBuilder.builder()
|
||||
.setDataSource(
|
||||
new TopNQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(querySegmentSpec(Filtration.eternity()))
|
||||
.dimension(new DefaultDimensionSpec("m2", "d0", ColumnType.DOUBLE))
|
||||
.threshold(10)
|
||||
.aggregators(aggregators(
|
||||
.aggregators(
|
||||
aggregators(
|
||||
useDefault
|
||||
? new CountAggregatorFactory("a0")
|
||||
: new FilteredAggregatorFactory(
|
||||
new CountAggregatorFactory("a0"),
|
||||
notNull("m1")
|
||||
)
|
||||
))
|
||||
)
|
||||
)
|
||||
.metric(new NumericTopNMetricSpec("a0"))
|
||||
.context(OUTER_LIMIT_CONTEXT)
|
||||
.build()
|
||||
)
|
||||
.setSignature(
|
||||
RowSignature.builder()
|
||||
.add("d0", ColumnType.DOUBLE)
|
||||
.add("a0", ColumnType.LONG)
|
||||
.build()
|
||||
)
|
||||
.setOperators(
|
||||
OperatorFactoryBuilders.naiveSortOperator("a0", ColumnWithDirection.Direction.DESC),
|
||||
OperatorFactoryBuilders.scanOperatorFactoryBuilder()
|
||||
.setOffsetLimit(1, 2)
|
||||
.build()
|
||||
)
|
||||
.setLeafOperators(
|
||||
OperatorFactoryBuilders.scanOperatorFactoryBuilder()
|
||||
.setOffsetLimit(0, Long.MAX_VALUE)
|
||||
.setFilter(
|
||||
range(
|
||||
"d0",
|
||||
ColumnType.LONG,
|
||||
2L,
|
||||
null,
|
||||
true,
|
||||
false
|
||||
)
|
||||
)
|
||||
.setProjectedColumns("a0", "d0")
|
||||
.build()
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.expectedResults(expectedResults)
|
||||
.run();
|
||||
}
|
||||
|
||||
@NotYetSupported(Modes.CANNOT_TRANSLATE)
|
||||
@Test
|
||||
public void testWindowingWithScanAndSort()
|
||||
{
|
||||
skipVectorize();
|
||||
cannotVectorize();
|
||||
msqIncompatible();
|
||||
String sql = "with t AS (\n"
|
||||
+ "SELECT \n"
|
||||
+ " RANK() OVER (PARTITION BY m2 ORDER BY m2 ASC) \n"
|
||||
+ " AS ranking,\n"
|
||||
+ " COUNT(m1) as trend_score\n"
|
||||
+ "FROM foo\n"
|
||||
+ "GROUP BY m2,m1 LIMIT 10\n"
|
||||
+ ")\n"
|
||||
+ "select ranking, trend_score from t ORDER BY trend_score";
|
||||
ImmutableList<Object[]> expectedResults = ImmutableList.of(
|
||||
new Object[] {1L, 1L},
|
||||
new Object[] {1L, 1L},
|
||||
new Object[] {1L, 1L},
|
||||
new Object[] {1L, 1L},
|
||||
new Object[] {1L, 1L},
|
||||
new Object[] {1L, 1L}
|
||||
);
|
||||
|
||||
testBuilder()
|
||||
.sql(sql)
|
||||
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
|
||||
.expectedQuery(
|
||||
WindowOperatorQueryBuilder.builder()
|
||||
.setDataSource(
|
||||
Druids.newScanQueryBuilder()
|
||||
.dataSource(
|
||||
new WindowOperatorQueryBuilder()
|
||||
.setDataSource(
|
||||
GroupByQuery.builder()
|
||||
.setDataSource(CalciteTests.DATASOURCE1)
|
||||
.setInterval(querySegmentSpec(Filtration.eternity()))
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
new DefaultDimensionSpec("d0", "_d0", ColumnType.DOUBLE)
|
||||
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")
|
||||
)
|
||||
)
|
||||
.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)
|
||||
.build()
|
||||
)
|
||||
.setContext(OUTER_LIMIT_CONTEXT)
|
||||
.build()
|
||||
),
|
||||
ImmutableList.of(
|
||||
new Object[]{3.0D, 1L},
|
||||
new Object[]{4.0D, 1L}
|
||||
.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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -128,6 +128,7 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
|
|||
Assert.assertEquals(1, results.recordedQueries.size());
|
||||
|
||||
maybeDumpActualResults(results.results);
|
||||
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);
|
||||
|
@ -139,7 +140,9 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
|
|||
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();
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -89,7 +89,8 @@ public @interface NotYetSupported
|
|||
// at least c7 is represented oddly in the parquet file
|
||||
T_ALLTYPES_ISSUES(AssertionError.class, "(t_alltype|allTypsUniq|fewRowsAllData).parquet.*Verifier.verify"),
|
||||
RESULT_MISMATCH(AssertionError.class, "assertResultsEquals"),
|
||||
UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)");
|
||||
UNSUPPORTED_NULL_ORDERING(DruidException.class, "(A|DE)SCENDING ordering with NULLS (LAST|FIRST)"),
|
||||
CANNOT_TRANSLATE(DruidException.class, "Cannot translate reference");
|
||||
|
||||
public Class<? extends Throwable> throwableClass;
|
||||
public String regex;
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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]
|
File diff suppressed because it is too large
Load Diff
Loading…
Reference in New Issue