From a5f7294807d5e0394040f099354b24ae7e31eb1c Mon Sep 17 00:00:00 2001 From: Zoltan Haindrich Date: Mon, 23 Oct 2023 14:39:58 +0200 Subject: [PATCH] Fix VirtualColumn related issues in window expressions (#15119) (#15230) for some exotic queries like: SELECT '_'||dim1, MIN(cast(0 as double)) OVER (), MIN(cast((cnt||cnt) as bigint)) OVER () FROM foo the compilation have resulted in NPE -s mostly because VirtualColumn -s were not handled properly (cherry picked from commit b95035f183e193f24ceee57cc41d295918fe87ac) --- .../write/columnar/FrameColumnWriters.java | 5 +- .../druid/query/operator/OperatorFactory.java | 1 + .../query/operator/WindowOperatorQuery.java | 14 ++- .../WindowOperatorQueryQueryToolChest.java | 10 +- .../LazilyDecoratedRowsAndColumns.java | 45 +++++++-- .../concrete/FrameRowsAndColumns.java | 8 ++ .../DefaultColumnSelectorFactoryMaker.java | 14 ++- .../apache/druid/segment/VirtualColumns.java | 17 ++++ .../druid/error/DruidExceptionMatcher.java | 9 ++ .../operator/WindowOperatorQueryTest.java | 21 +++++ .../rowsandcols/RowsAndColumnsTestBase.java | 5 +- .../concrete/FrameRowsAndColumnsTest.java | 47 ++++++++++ .../CombinedSemanticInterfacesTest.java | 30 ++++++ ...ualColumnEvaluationRowsAndColumnsTest.java | 92 +++++++++++++++++++ .../virtual/ExpressionVirtualColumnTest.java | 4 +- .../segment/virtual/VirtualColumnsTest.java | 24 +++++ .../druid/sql/calcite/rel/DruidQuery.java | 25 ++++- .../calcite/rel/VirtualColumnRegistry.java | 27 ++++++ .../druid/sql/calcite/rel/Windowing.java | 6 +- .../sql/calcite/BaseCalciteQueryTest.java | 3 +- .../sql/calcite/DrillWindowQueryTest.java | 68 +++++++------- .../druid/sql/calcite/NotYetSupported.java | 2 +- .../tests/window/virtualColumns.sqlTest | 33 +++++++ 23 files changed, 449 insertions(+), 61 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java create mode 100644 sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest diff --git a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java index bc1180cb8e38..596fa77646f3 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java +++ b/processing/src/main/java/org/apache/druid/frame/write/columnar/FrameColumnWriters.java @@ -19,7 +19,6 @@ package org.apache.druid.frame.write.columnar; -import org.apache.druid.common.config.NullHandling; import org.apache.druid.frame.allocation.MemoryAllocator; import org.apache.druid.frame.write.UnsupportedColumnTypeException; import org.apache.druid.java.util.common.ISE; @@ -167,9 +166,7 @@ private static ComplexFrameColumnWriter makeComplexWriter( private static boolean hasNullsForNumericWriter(final ColumnCapabilities capabilities) { - if (NullHandling.replaceWithDefault()) { - return false; - } else if (capabilities == null) { + if (capabilities == null) { return true; } else if (capabilities.getType().isNumeric()) { return capabilities.hasNulls().isMaybeTrue(); diff --git a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java index 90bd30862c9f..a97c332505cf 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/operator/OperatorFactory.java @@ -33,6 +33,7 @@ @JsonSubTypes.Type(name = "naivePartition", value = NaivePartitioningOperatorFactory.class), @JsonSubTypes.Type(name = "naiveSort", value = NaiveSortOperatorFactory.class), @JsonSubTypes.Type(name = "window", value = WindowOperatorFactory.class), + @JsonSubTypes.Type(name = "scan", value = ScanOperatorFactory.class), }) public interface OperatorFactory { diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java index a07bad0d082c..5ecba3f22420 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQuery.java @@ -196,7 +196,7 @@ public Query withQuerySegmentSpec(QuerySegmentSpec spec) { return new WindowOperatorQuery( getDataSource(), - getQuerySegmentSpec(), + spec, getContext(), rowSignature, operators, @@ -217,6 +217,18 @@ public Query withDataSource(DataSource dataSource) ); } + public Query withOperators(List operators) + { + return new WindowOperatorQuery( + getDataSource(), + getQuerySegmentSpec(), + getContext(), + rowSignature, + operators, + leafOperators + ); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChest.java index 650bd0ae4521..bec529eedefa 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/operator/WindowOperatorQueryQueryToolChest.java @@ -54,9 +54,17 @@ public QueryRunner mergeResults(QueryRunner runn (queryPlus, responseContext) -> { final WindowOperatorQuery query = (WindowOperatorQuery) queryPlus.getQuery(); final List opFactories = query.getOperators(); + if (opFactories.isEmpty()) { + return runner.run(queryPlus, responseContext); + } Supplier opSupplier = () -> { - Operator retVal = new SequenceOperator(runner.run(queryPlus, responseContext)); + Operator retVal = new SequenceOperator( + runner.run( + queryPlus.withQuery(query.withOperators(new ArrayList())), + responseContext + ) + ); for (OperatorFactory operatorFactory : opFactories) { retVal = operatorFactory.wrap(retVal); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index c1a13c504414..79505dcdd419 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -19,6 +19,7 @@ package org.apache.druid.query.rowsandcols; +import com.google.common.collect.ImmutableList; import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; @@ -101,6 +102,11 @@ public Collection getColumnNames() return viewableColumns == null ? base.getColumnNames() : viewableColumns; } + public RowsAndColumns getBase() + { + return base; + } + @Override public int numRows() { @@ -115,7 +121,6 @@ public Column findColumn(String name) if (viewableColumns != null && !viewableColumns.contains(name)) { return null; } - maybeMaterialize(); return base.findColumn(name); } @@ -158,7 +163,7 @@ public WireTransferable toWireTransferable() private void maybeMaterialize() { - if (!(interval == null && filter == null && limit == -1 && ordering == null)) { + if (needsMaterialization()) { final Pair thePair = materialize(); if (thePair == null) { reset(new EmptyRowsAndColumns()); @@ -168,6 +173,11 @@ private void maybeMaterialize() } } + private boolean needsMaterialization() + { + return interval != null || filter != null || limit != -1 || ordering != null || virtualColumns != null; + } + private Pair materialize() { if (ordering != null) { @@ -180,7 +190,6 @@ private Pair materialize() } else { return materializeStorageAdapter(as); } - } private void reset(RowsAndColumns rac) @@ -200,13 +209,26 @@ private Pair materializeStorageAdapter(StorageAdapter as) final Sequence cursors = as.makeCursors( filter, interval == null ? Intervals.ETERNITY : interval, - virtualColumns, + virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns, Granularities.ALL, false, null ); - Collection cols = viewableColumns == null ? base.getColumnNames() : viewableColumns; + + final Collection cols; + if (viewableColumns != null) { + cols = viewableColumns; + } else { + if (virtualColumns == null) { + cols = base.getColumnNames(); + } else { + cols = ImmutableList.builder() + .addAll(base.getColumnNames()) + .addAll(virtualColumns.getColumnNames()) + .build(); + } + } AtomicReference siggy = new AtomicReference<>(null); FrameWriter writer = cursors.accumulate(null, (accumulated, in) -> { @@ -222,9 +244,18 @@ private Pair materializeStorageAdapter(StorageAdapter as) final RowSignature.Builder sigBob = RowSignature.builder(); for (String col : cols) { - final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(col); + ColumnCapabilities capabilities; + capabilities = columnSelectorFactory.getColumnCapabilities(col); if (capabilities != null) { sigBob.add(col, capabilities.toColumnType()); + continue; + } + if (virtualColumns != null) { + capabilities = virtualColumns.getColumnCapabilities(columnSelectorFactory, col); + if (capabilities != null) { + sigBob.add(col, capabilities.toColumnType()); + continue; + } } } final RowSignature signature = sigBob.build(); @@ -350,12 +381,12 @@ private Pair naiveMaterialize(RowsAndColumns rac) final RowSignature.Builder sigBob = RowSignature.builder(); final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(200 << 20); + for (String column : columnsToGenerate) { final Column racColumn = rac.findColumn(column); if (racColumn == null) { continue; } - sigBob.add(column, racColumn.toAccessor().getType()); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java index 0eefa288df2f..59a97f12bf51 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java @@ -21,10 +21,14 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.read.FrameReader; import org.apache.druid.frame.read.columnar.FrameColumnReaders; +import org.apache.druid.frame.segment.FrameStorageAdapter; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -77,10 +81,14 @@ public Column findColumn(String name) } + @SuppressWarnings("unchecked") @Nullable @Override public T as(Class clazz) { + if (StorageAdapter.class.equals(clazz)) { + return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); + } return null; } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java index a8f3570b9ad6..4611280ce89e 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultColumnSelectorFactoryMaker.java @@ -141,7 +141,7 @@ public ColumnValueSelector makeColumnValueSelector(@Nonnull String columnName) { return withColumnAccessor(columnName, columnAccessor -> { if (columnAccessor == null) { - return DimensionSelector.constant(null); + return DimensionSelector.nilSelector(); } else { final ColumnType type = columnAccessor.getType(); switch (type.getType()) { @@ -160,16 +160,22 @@ public ColumnValueSelector makeColumnValueSelector(@Nonnull String columnName) @Override public ColumnCapabilities getColumnCapabilities(String column) { - return withColumnAccessor(column, columnAccessor -> - new ColumnCapabilitiesImpl() + return withColumnAccessor(column, columnAccessor -> { + if (columnAccessor == null) { + return null; + } else { + return new ColumnCapabilitiesImpl() .setType(columnAccessor.getType()) .setHasMultipleValues(false) .setDictionaryEncoded(false) - .setHasBitmapIndexes(false)); + .setHasBitmapIndexes(false); + } + }); } private T withColumnAccessor(String column, Function fn) { + @Nullable ColumnAccessor retVal = accessorCache.get(column); if (retVal == null) { Column racColumn = rac.findColumn(column); diff --git a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java index f26e297684b5..be8775662629 100644 --- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java +++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java @@ -48,10 +48,12 @@ import javax.annotation.Nullable; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; /** * Class allowing lookup and usage of virtual columns. @@ -112,6 +114,11 @@ public static VirtualColumns create(List virtualColumns) return new VirtualColumns(ImmutableList.copyOf(virtualColumns), withDotSupport, withoutDotSupport); } + public static VirtualColumns create(VirtualColumn... virtualColumns) + { + return create(Arrays.asList(virtualColumns)); + } + public static VirtualColumns nullToEmpty(@Nullable VirtualColumns virtualColumns) { return virtualColumns == null ? EMPTY : virtualColumns; @@ -519,4 +526,14 @@ public boolean equals(Object obj) ((VirtualColumns) obj).virtualColumns.isEmpty(); } } + + public boolean isEmpty() + { + return virtualColumns.isEmpty(); + } + + public List getColumnNames() + { + return virtualColumns.stream().map(v -> v.getOutputName()).collect(Collectors.toList()); + } } diff --git a/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java index d3d4e057c5e0..75166543f552 100644 --- a/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java +++ b/processing/src/test/java/org/apache/druid/error/DruidExceptionMatcher.java @@ -45,6 +45,15 @@ public static DruidExceptionMatcher invalidSqlInput() return invalidInput().expectContext("sourceType", "sql"); } + public static DruidExceptionMatcher defensive() + { + return new DruidExceptionMatcher( + DruidException.Persona.DEVELOPER, + DruidException.Category.DEFENSIVE, + "general" + ); + } + private final AllOf delegate; private final ArrayList> matcherList; diff --git a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java index 69712c06493c..dcd969e57fa5 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/WindowOperatorQueryTest.java @@ -19,18 +19,23 @@ package org.apache.druid.query.operator; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.QueryContext; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.spec.LegacySegmentSpec; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.column.RowSignature; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Set; /** @@ -107,6 +112,22 @@ public void withDataSource() Assert.assertSame(newDs, query.withDataSource(newDs).getDataSource()); } + @Test + public void withQuerySpec() + { + QuerySegmentSpec spec = new MultipleIntervalSegmentSpec(Collections.emptyList()); + Assert.assertSame(spec, ((WindowOperatorQuery) query.withQuerySegmentSpec(spec)).getQuerySegmentSpec()); + } + + @Test + public void withOperators() + { + List operators = ImmutableList.builder() + .add(new NaivePartitioningOperatorFactory(Collections.singletonList("some"))) + .build(); + Assert.assertSame(operators, ((WindowOperatorQuery) query.withOperators(operators)).getOperators()); + } + @Test public void testEquals() { diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java index 36a621ff8c8d..16b84f82728f 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/RowsAndColumnsTestBase.java @@ -22,6 +22,8 @@ import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumns; +import org.apache.druid.query.rowsandcols.concrete.FrameRowsAndColumnsTest; import org.junit.Assert; import org.junit.Test; @@ -63,7 +65,8 @@ private static ArrayList getMakers() new Object[]{MapOfColumnsRowsAndColumns.class, Function.identity()}, new Object[]{ArrayListRowsAndColumns.class, ArrayListRowsAndColumnsTest.MAKER}, new Object[]{ConcatRowsAndColumns.class, ConcatRowsAndColumnsTest.MAKER}, - new Object[]{RearrangedRowsAndColumns.class, RearrangedRowsAndColumnsTest.MAKER} + new Object[]{RearrangedRowsAndColumns.class, RearrangedRowsAndColumnsTest.MAKER}, + new Object[]{FrameRowsAndColumns.class, FrameRowsAndColumnsTest.MAKER} ); } diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java new file mode 100644 index 000000000000..837e30185e2e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumnsTest.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.rowsandcols.concrete; + +import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumnsTestBase; +import java.util.function.Function; + +public class FrameRowsAndColumnsTest extends RowsAndColumnsTestBase +{ + public FrameRowsAndColumnsTest() + { + super(FrameRowsAndColumns.class); + } + + public static Function MAKER = input -> { + + return buildFrame(input); + }; + + private static FrameRowsAndColumns buildFrame(MapOfColumnsRowsAndColumns input) + { + LazilyDecoratedRowsAndColumns rac = new LazilyDecoratedRowsAndColumns(input, null, null, null, Integer.MAX_VALUE, null, null); + + rac.numRows(); // materialize + + return (FrameRowsAndColumns) rac.getBase(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/CombinedSemanticInterfacesTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/CombinedSemanticInterfacesTest.java index 9a5bce17c58b..b7c91b854e4d 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/CombinedSemanticInterfacesTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/CombinedSemanticInterfacesTest.java @@ -30,6 +30,8 @@ import org.apache.druid.query.rowsandcols.column.ColumnAccessor; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; import org.junit.Test; @@ -38,8 +40,12 @@ import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + /** * Place where tests can live that are testing the interactions of multiple semantic interfaces */ @@ -54,6 +60,30 @@ public CombinedSemanticInterfacesTest( super(name, fn); } + @Test + public void testColumnSelectorFactoryMakeColumnValueSelectorNonExistentColumn() + { + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "some", new IntArrayColumn(new int[] {3, 54, 21, 1, 5, 54, 2, 3, 92})))); + AtomicInteger currRow = new AtomicInteger(); + ColumnSelectorFactory csfm = ColumnSelectorFactoryMaker.fromRAC(rac).make(currRow); + + assertEquals(DimensionSelector.nilSelector(), csfm.makeColumnValueSelector("nonexistent")); + } + + @Test + public void testColumnSelectorFactoryGetColumnCapabilitiesNonExistentColumn() + { + RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( + ImmutableMap.of( + "some", new IntArrayColumn(new int[] {3, 54, 21, 1, 5, 54, 2, 3, 92})))); + AtomicInteger currRow = new AtomicInteger(); + ColumnSelectorFactory csfm = ColumnSelectorFactoryMaker.fromRAC(rac).make(currRow); + + assertNull(csfm.getColumnCapabilities("nonexistent")); + } + /** * Tests a relatively common series of operations for window functions: partition -> aggregate -> sort */ diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java new file mode 100644 index 000000000000..e53850bccc9a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/TestVirtualColumnEvaluationRowsAndColumnsTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.query.rowsandcols.semantic; + +import com.google.common.collect.Lists; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.operator.window.RowsAndColumnsHelper; +import org.apache.druid.query.rowsandcols.LazilyDecoratedRowsAndColumns; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.junit.Test; + +import java.util.function.Function; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assume.assumeNotNull; + +public class TestVirtualColumnEvaluationRowsAndColumnsTest extends SemanticTestBase +{ + public TestVirtualColumnEvaluationRowsAndColumnsTest(String name, Function fn) + { + super(name, fn); + } + + @Test + public void testMaterializeVirtualColumns() + { + Object[][] vals = new Object[][] { + {1L, "a", 123L, 0L}, + {2L, "a", 456L, 1L}, + {3L, "b", 789L, 2L}, + {4L, "b", 123L, 3L}, + }; + + RowSignature siggy = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim", ColumnType.STRING) + .add("val", ColumnType.LONG) + .add("arrayIndex", ColumnType.LONG) + .build(); + + final RowsAndColumns base = make(MapOfColumnsRowsAndColumns.fromRowObjects(vals, siggy)); + + assumeNotNull("skipping: StorageAdapter not supported", base.as(StorageAdapter.class)); + + LazilyDecoratedRowsAndColumns ras = new LazilyDecoratedRowsAndColumns( + base, + null, + null, + VirtualColumns.create(new ExpressionVirtualColumn( + "expr", + "val * 2", + ColumnType.LONG, + TestExprMacroTable.INSTANCE)), + Integer.MAX_VALUE, + null, + null); + + // do the materialziation + ras.numRows(); + + assertEquals(Lists.newArrayList("__time", "dim", "val", "arrayIndex", "expr"), ras.getColumnNames()); + + new RowsAndColumnsHelper() + .expectColumn("expr", new long[] {123 * 2, 456L * 2, 789 * 2, 123 * 2}) + .validate(ras); + + } + +} diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index c8fd92a10793..9d35e6025ff6 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -716,8 +716,8 @@ public void testLongSelectorOfTimeColumn() CURRENT_ROW.set(ROW0); Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getLong()); - Assert.assertEquals((float) DateTimes.of("2000-01-01").getMillis(), selector.getFloat(), 0.0f); - Assert.assertEquals((double) DateTimes.of("2000-01-01").getMillis(), selector.getDouble(), 0.0d); + Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getFloat(), 0.0f); + Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getDouble(), 0.0d); Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), selector.getObject()); CURRENT_ROW.set(ROW1); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java index fbf3b1c3aee9..e13807e07e2a 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java @@ -64,8 +64,12 @@ import javax.annotation.Nullable; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class VirtualColumnsTest extends InitializedNullHandlingTest { private static final String REAL_COLUMN_NAME = "real_column"; @@ -90,6 +94,26 @@ public void testExists() Assert.assertFalse(virtualColumns.exists("bar")); } + @Test + public void testIsEmpty() + { + assertTrue(VirtualColumns.EMPTY.isEmpty()); + assertTrue(VirtualColumns.create(Collections.emptyList()).isEmpty()); + } + + @Test + public void testGetColumnNames() + { + final VirtualColumns virtualColumns = makeVirtualColumns(); + List colNames = ImmutableList.builder() + .add("expr") + .add("expr2i") + .add("expr2") + .add("foo") + .build(); + assertEquals(colNames, virtualColumns.getColumnNames()); + } + @Test public void testGetColumnCapabilitiesNilBase() { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 1cf79b6dc123..b670d682222d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -67,6 +67,8 @@ 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.OperatorFactory; +import org.apache.druid.query.operator.ScanOperatorFactory; import org.apache.druid.query.operator.WindowOperatorQuery; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.query.scan.ScanQuery; @@ -282,7 +284,8 @@ public static DruidQuery fromPartialQuery( partialQuery, plannerContext, sourceRowSignature, // Plans immediately after Scan, so safe to use the row signature from scan - rexBuilder + rexBuilder, + virtualColumnRegistry ) ); } else { @@ -1442,12 +1445,30 @@ private WindowOperatorQuery toWindowQuery() return null; } + // all virtual cols are needed - these columns are only referenced from the aggregates + VirtualColumns virtualColumns = virtualColumnRegistry.build(Collections.emptySet()); + final List operators; + + if (virtualColumns.isEmpty()) { + operators = windowing.getOperators(); + } else { + operators = ImmutableList.builder() + .add(new ScanOperatorFactory( + null, + null, + null, + null, + virtualColumns, + null)) + .addAll(windowing.getOperators()) + .build(); + } return new WindowOperatorQuery( dataSource, new LegacySegmentSpec(Intervals.ETERNITY), plannerContext.queryContextMap(), windowing.getSignature(), - windowing.getOperators(), + operators, null ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java index b44be8b43890..3503838d6fd7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/VirtualColumnRegistry.java @@ -22,6 +22,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -32,13 +33,17 @@ import javax.annotation.Nullable; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Optional; import java.util.Queue; +import java.util.Set; import java.util.stream.Collectors; /** @@ -88,6 +93,11 @@ public static VirtualColumnRegistry create( ); } + public boolean isEmpty() + { + return virtualColumnsByExpression.isEmpty(); + } + /** * Check if a {@link VirtualColumn} is defined by column name */ @@ -350,4 +360,21 @@ public int hashCode() return Objects.hash(expression, typeHint); } } + + public VirtualColumns build(Set exclude) + { + List columns = new ArrayList<>(); + if (virtualColumnsByName == null) { + return VirtualColumns.EMPTY; + } + + for (Entry entry : virtualColumnsByName.entrySet()) { + if (exclude.contains(entry.getKey())) { + continue; + } + columns.add(getVirtualColumn(entry.getKey())); + } + columns.sort(Comparator.comparing(VirtualColumn::getOutputName)); + return VirtualColumns.create(columns); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 4039ca8914ab..9f8d3d091b5d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -115,7 +115,8 @@ public static Windowing fromCalciteStuff( final PartialDruidQuery partialQuery, final PlannerContext plannerContext, final RowSignature sourceRowSignature, - final RexBuilder rexBuilder + final RexBuilder rexBuilder, + final VirtualColumnRegistry virtualColumnRegistry ) { final Window window = Preconditions.checkNotNull(partialQuery.getWindow(), "window"); @@ -172,10 +173,11 @@ public static Windowing fromCalciteStuff( ProcessorMaker maker = KNOWN_WINDOW_FNS.get(aggregateCall.getAggregation().getName()); if (maker == null) { + final Aggregation aggregation = GroupByRules.translateAggregateCall( plannerContext, sourceRowSignature, - null, + virtualColumnRegistry, rexBuilder, InputAccessor.buildFor( rexBuilder, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 1527e75fa385..6b2d12c6cce0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1335,8 +1335,7 @@ public static Query recursivelyClearContext(final Query query, ObjectM return queryJsonMapper.treeToValue(newQueryNode, Query.class); } catch (Exception e) { - Assert.fail(e.getMessage()); - return null; + throw new RuntimeException(e); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 8a737eca0355..7a2b9b70f1a6 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -4691,7 +4691,7 @@ public void test_ntile_func_ntileFn_58() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("aggregates/winFnQry_12") @Test public void test_aggregates_winFnQry_12() @@ -4699,7 +4699,7 @@ public void test_aggregates_winFnQry_12() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("aggregates/winFnQry_13") @Test public void test_aggregates_winFnQry_13() @@ -4707,7 +4707,7 @@ public void test_aggregates_winFnQry_13() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("aggregates/winFnQry_20") @Test public void test_aggregates_winFnQry_20() @@ -4715,7 +4715,7 @@ public void test_aggregates_winFnQry_20() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("aggregates/winFnQry_21") @Test public void test_aggregates_winFnQry_21() @@ -4731,7 +4731,7 @@ public void test_first_val_firstValFn_5() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/defaultFrame/RBUPACR_chr_1") @Test public void test_frameclause_defaultFrame_RBUPACR_chr_1() @@ -4739,7 +4739,7 @@ public void test_frameclause_defaultFrame_RBUPACR_chr_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/defaultFrame/RBUPACR_chr_2") @Test public void test_frameclause_defaultFrame_RBUPACR_chr_2() @@ -4747,7 +4747,7 @@ public void test_frameclause_defaultFrame_RBUPACR_chr_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/defaultFrame/RBUPACR_vchr_1") @Test public void test_frameclause_defaultFrame_RBUPACR_vchr_1() @@ -4755,7 +4755,7 @@ public void test_frameclause_defaultFrame_RBUPACR_vchr_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/defaultFrame/RBUPACR_vchr_2") @Test public void test_frameclause_defaultFrame_RBUPACR_vchr_2() @@ -4763,7 +4763,7 @@ public void test_frameclause_defaultFrame_RBUPACR_vchr_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/multipl_wnwds/max_mulwds") @Test public void test_frameclause_multipl_wnwds_max_mulwds() @@ -4771,7 +4771,7 @@ public void test_frameclause_multipl_wnwds_max_mulwds() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/multipl_wnwds/min_mulwds") @Test public void test_frameclause_multipl_wnwds_min_mulwds() @@ -4779,7 +4779,7 @@ public void test_frameclause_multipl_wnwds_min_mulwds() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBCRACR/RBCRACR_char_1") @Test public void test_frameclause_RBCRACR_RBCRACR_char_1() @@ -4787,7 +4787,7 @@ public void test_frameclause_RBCRACR_RBCRACR_char_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBCRACR/RBCRACR_char_2") @Test public void test_frameclause_RBCRACR_RBCRACR_char_2() @@ -4795,7 +4795,7 @@ public void test_frameclause_RBCRACR_RBCRACR_char_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBCRACR/RBCRACR_vchar_1") @Test public void test_frameclause_RBCRACR_RBCRACR_vchar_1() @@ -4803,7 +4803,7 @@ public void test_frameclause_RBCRACR_RBCRACR_vchar_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBCRACR/RBCRACR_vchar_2") @Test public void test_frameclause_RBCRACR_RBCRACR_vchar_2() @@ -4811,7 +4811,7 @@ public void test_frameclause_RBCRACR_RBCRACR_vchar_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPACR/RBUPACR_chr_1") @Test public void test_frameclause_RBUPACR_RBUPACR_chr_1() @@ -4819,7 +4819,7 @@ public void test_frameclause_RBUPACR_RBUPACR_chr_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPACR/RBUPACR_chr_2") @Test public void test_frameclause_RBUPACR_RBUPACR_chr_2() @@ -4827,7 +4827,7 @@ public void test_frameclause_RBUPACR_RBUPACR_chr_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPACR/RBUPACR_vchr_1") @Test public void test_frameclause_RBUPACR_RBUPACR_vchr_1() @@ -4835,7 +4835,7 @@ public void test_frameclause_RBUPACR_RBUPACR_vchr_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPACR/RBUPACR_vchr_2") @Test public void test_frameclause_RBUPACR_RBUPACR_vchr_2() @@ -4843,7 +4843,7 @@ public void test_frameclause_RBUPACR_RBUPACR_vchr_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPAUF/RBUPAUF_char_1") @Test public void test_frameclause_RBUPAUF_RBUPAUF_char_1() @@ -4851,7 +4851,7 @@ public void test_frameclause_RBUPAUF_RBUPAUF_char_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPAUF/RBUPAUF_char_2") @Test public void test_frameclause_RBUPAUF_RBUPAUF_char_2() @@ -4859,7 +4859,7 @@ public void test_frameclause_RBUPAUF_RBUPAUF_char_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPAUF/RBUPAUF_vchar_1") @Test public void test_frameclause_RBUPAUF_RBUPAUF_vchar_1() @@ -4867,7 +4867,7 @@ public void test_frameclause_RBUPAUF_RBUPAUF_vchar_1() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/RBUPAUF/RBUPAUF_vchar_2") @Test public void test_frameclause_RBUPAUF_RBUPAUF_vchar_2() @@ -4875,7 +4875,7 @@ public void test_frameclause_RBUPAUF_RBUPAUF_vchar_2() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_22") @Test public void test_frameclause_subQueries_frmInSubQry_22() @@ -4883,7 +4883,7 @@ public void test_frameclause_subQueries_frmInSubQry_22() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_23") @Test public void test_frameclause_subQueries_frmInSubQry_23() @@ -4891,7 +4891,7 @@ public void test_frameclause_subQueries_frmInSubQry_23() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_24") @Test public void test_frameclause_subQueries_frmInSubQry_24() @@ -4899,7 +4899,7 @@ public void test_frameclause_subQueries_frmInSubQry_24() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_41") @Test public void test_frameclause_subQueries_frmInSubQry_41() @@ -4907,7 +4907,7 @@ public void test_frameclause_subQueries_frmInSubQry_41() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_42") @Test public void test_frameclause_subQueries_frmInSubQry_42() @@ -4915,7 +4915,7 @@ public void test_frameclause_subQueries_frmInSubQry_42() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_43") @Test public void test_frameclause_subQueries_frmInSubQry_43() @@ -4923,7 +4923,7 @@ public void test_frameclause_subQueries_frmInSubQry_43() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_44") @Test public void test_frameclause_subQueries_frmInSubQry_44() @@ -4931,7 +4931,7 @@ public void test_frameclause_subQueries_frmInSubQry_44() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_45") @Test public void test_frameclause_subQueries_frmInSubQry_45() @@ -4939,7 +4939,7 @@ public void test_frameclause_subQueries_frmInSubQry_45() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("frameclause/subQueries/frmInSubQry_46") @Test public void test_frameclause_subQueries_frmInSubQry_46() @@ -4963,7 +4963,7 @@ public void test_last_val_lastValFn_5() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("nestedAggs/basic_10") @Test public void test_nestedAggs_basic_10() @@ -4971,7 +4971,7 @@ public void test_nestedAggs_basic_10() windowQueryTest(); } - @NotYetSupported(Modes.NPE) + @NotYetSupported(Modes.AGGREGATION_NOT_SUPPORT_TYPE) @DrillTest("nestedAggs/cte_win_01") @Test public void test_nestedAggs_cte_win_01() @@ -4979,7 +4979,7 @@ public void test_nestedAggs_cte_win_01() windowQueryTest(); } - @NotYetSupported(Modes.NPE_PLAIN) + @NotYetSupported(Modes.RESULT_MISMATCH) @DrillTest("aggregates/winFnQry_7") @Test public void test_aggregates_winFnQry_7() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index 933359e96bf6..86e5c41d6de8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -79,7 +79,7 @@ enum Modes BIGINT_TO_DATE(DruidException.class, "BIGINT to type (DATE|TIME)"), NPE_PLAIN(NullPointerException.class, "java.lang.NullPointerException"), NPE(DruidException.class, "java.lang.NullPointerException"), - AGGREGATION_NOT_SUPPORT_TYPE(DruidException.class, "Aggregation \\[(MIN|MAX)\\] does not support type"), + AGGREGATION_NOT_SUPPORT_TYPE(DruidException.class, "Aggregation \\[(MIN|MAX)\\] does not support type \\[STRING\\]"), CANNOT_APPLY_VIRTUAL_COL(UOE.class, "apply virtual columns"), MISSING_DESC(DruidException.class, "function signature DESC"), RESULT_COUNT_MISMATCH(AssertionError.class, "result count:"), diff --git a/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest new file mode 100644 index 000000000000..0a86a691e26a --- /dev/null +++ b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest @@ -0,0 +1,33 @@ +type: "operatorValidation" + +sql: | + SELECT + '_'||dim1, + MIN(cast(42 as double)) OVER (), + MIN(cast((cnt||cnt) as bigint)) OVER () + FROM foo + +expectedOperators: + - type: "scan" + limit: -1 + virtualColumns: + - type: "expression" + name: "_v0" + expression: "42.0" + outputType: "DOUBLE" + - { type: "naivePartition", partitionColumns: [ ] } + - type: "window" + processor: + type: "framedAgg" + frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + aggregations: + - { type: "doubleMin", name: "w0", fieldName: "_v0" } + - { type: "longMin", name: "w1", fieldName: "v1" } + +expectedResults: + - ["_",42.0,11] + - ["_10.1",42.0,11] + - ["_2",42.0,11] + - ["_1",42.0,11] + - ["_def",42.0,11] + - ["_abc",42.0,11]