SQL tests: avoid mixing skip and cannot vectorize. (#16251)

* SQL tests: avoid mixing skip and cannot vectorize.

skipVectorize switches off vectorization tests completely, and
cannotVectorize turns vectorization tests into negative tests. It doesn't
make sense to use them together, so this patch makes it an error to do so,
and cleans up cases where both are mentioned.

This patch also has the effect of changing various tests from skipVectorize
to cannotVectorize, because in the past when both were mentioned,
skipVectorize would take priority.

* Fix bug with StringAnyAggregatorFactory attempting to vectorize when it cannt.

* Fix tests.
This commit is contained in:
Gian Merlino 2024-04-11 15:06:11 -07:00 committed by GitHub
parent df9e1bb97b
commit 9f358f5f4a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 66 additions and 158 deletions

View File

@ -1088,7 +1088,6 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
@Test
public void testHllEstimateAsVirtualColumnWithGroupByOrderBy()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT"
@ -1188,7 +1187,6 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
public void testResultCacheWithWindowing()
{
cannotVectorize();
skipVectorize();
for (int i = 0; i < 2; i++) {
testBuilder()
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))

View File

@ -1108,7 +1108,6 @@ public class ThetaSketchSqlAggregatorTest extends BaseCalciteQueryTest
@Test
public void testThetaEstimateAsVirtualColumnWithGroupByOrderBy()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT"

View File

@ -34,6 +34,7 @@ import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
@ -89,6 +90,7 @@ public class StringAnyAggregatorFactory extends AggregatorFactory
{
ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName);
// null capabilities mean the column doesn't exist, so in vector engines the selector will never be multi-value
// canVectorize ensures that nonnull capabilities => dict-encoded string
if (capabilities != null && capabilities.hasMultipleValues().isMaybeTrue()) {
return new StringAnyVectorAggregator(
null,
@ -109,7 +111,9 @@ public class StringAnyAggregatorFactory extends AggregatorFactory
@Override
public boolean canVectorize(ColumnInspector columnInspector)
{
return true;
final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName);
return capabilities == null
|| (capabilities.is(ValueType.STRING) && capabilities.isDictionaryEncoded().isTrue());
}
@Override

View File

@ -28,6 +28,7 @@ import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.vector.TestVectorColumnSelectorFactory;
import org.apache.druid.segment.virtual.FallbackVirtualColumnTest;
import org.apache.druid.testing.InitializedNullHandlingTest;
@ -43,8 +44,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
private static final String FIELD_NAME = "FIELD_NAME";
private static final int MAX_STRING_BYTES = 10;
private TestColumnSelectorFactory columnInspector;
private ColumnCapabilities capabilities;
private ColumnCapabilities capabilitiesMvString;
private TestVectorColumnSelectorFactory vectorSelectorFactory;
private StringAnyAggregatorFactory target;
@ -52,17 +52,37 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
public void setUp()
{
target = new StringAnyAggregatorFactory(NAME, FIELD_NAME, MAX_STRING_BYTES, true);
columnInspector = new TestColumnSelectorFactory();
capabilitiesMvString = ColumnCapabilitiesImpl.createDefault()
.setType(ColumnType.STRING)
.setDictionaryEncoded(true)
.setHasMultipleValues(true);
vectorSelectorFactory = new TestVectorColumnSelectorFactory();
capabilities = ColumnCapabilitiesImpl.createDefault().setHasMultipleValues(true);
vectorSelectorFactory.addCapabilities(FIELD_NAME, capabilities);
vectorSelectorFactory.addCapabilities(FIELD_NAME, capabilitiesMvString);
vectorSelectorFactory.addMVDVS(FIELD_NAME, new FallbackVirtualColumnTest.SameMultiVectorSelector());
}
@Test
public void canVectorizeWithoutCapabilitiesShouldReturnTrue()
{
Assert.assertTrue(target.canVectorize(columnInspector));
Assert.assertTrue(target.canVectorize(new TestColumnSelectorFactory(null)));
}
@Test
public void canVectorizeWithDictionaryEncodedMvStringShouldReturnTrue()
{
Assert.assertTrue(target.canVectorize(new TestColumnSelectorFactory(capabilitiesMvString)));
}
@Test
public void canVectorizeWithNonDictionaryEncodedStringShouldReturnFalse()
{
Assert.assertFalse(
target.canVectorize(
new TestColumnSelectorFactory(
ColumnCapabilitiesImpl.createDefault().setType(ColumnType.STRING)
)
)
);
}
@Test
@ -96,7 +116,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
@Test
public void testFactorize()
{
Aggregator res = target.factorize(new TestColumnSelectorFactory());
Aggregator res = target.factorize(new TestColumnSelectorFactory(capabilitiesMvString));
Assert.assertTrue(res instanceof StringAnyAggregator);
res.aggregate();
Assert.assertEquals(null, res.get());
@ -107,7 +127,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
@Test
public void testSvdStringAnyAggregator()
{
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory();
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory(capabilitiesMvString);
Aggregator res = target.factorize(columnSelectorFactory);
Assert.assertTrue(res instanceof StringAnyAggregator);
columnSelectorFactory.moveSelectorCursorToNext();
@ -118,7 +138,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
@Test
public void testMvdStringAnyAggregator()
{
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory();
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory(capabilitiesMvString);
Aggregator res = target.factorize(columnSelectorFactory);
Assert.assertTrue(res instanceof StringAnyAggregator);
columnSelectorFactory.moveSelectorCursorToNext();
@ -131,7 +151,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
public void testMvdStringAnyAggregatorWithAggregateMultipleToFalse()
{
StringAnyAggregatorFactory target = new StringAnyAggregatorFactory(NAME, FIELD_NAME, MAX_STRING_BYTES, false);
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory();
TestColumnSelectorFactory columnSelectorFactory = new TestColumnSelectorFactory(capabilitiesMvString);
Aggregator res = target.factorize(columnSelectorFactory);
Assert.assertTrue(res instanceof StringAnyAggregator);
columnSelectorFactory.moveSelectorCursorToNext();
@ -145,9 +165,15 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
{
List<String> mvd = Lists.newArrayList("AAAA", "AAAAB", "AAAC");
final Object[] mvds = {null, "CCCC", mvd, "BBBB", "EEEE"};
Integer maxStringBytes = 1024;
TestObjectColumnSelector<Object> objectColumnSelector = new TestObjectColumnSelector<>(mvds);
private final ColumnCapabilities capabilities;
public TestColumnSelectorFactory(ColumnCapabilities capabilities)
{
this.capabilities = capabilities;
}
@Override
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
{
@ -163,7 +189,7 @@ public class StringAnyAggregatorFactoryTest extends InitializedNullHandlingTest
@Override
public ColumnCapabilities getColumnCapabilities(String columnName)
{
return ColumnCapabilitiesImpl.createDefault().setHasMultipleValues(true);
return capabilities;
}
public void moveSelectorCursorToNext()

View File

@ -3729,7 +3729,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestInline()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT * FROM UNNEST(ARRAY[1,2,3])",
@ -3766,7 +3765,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestInlineWithCount()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT COUNT(*) FROM (select c from UNNEST(ARRAY[1,2,3]) as unnested(c))",
@ -3797,11 +3795,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnest()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -4615,7 +4608,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
public void testUnnestThriceWithFiltersOnDimAndUnnestColumnsORCombinations()
{
cannotVectorize();
skipVectorize();
String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n"
+ " ( SELECT * FROM \n"
+ " ( SELECT * FROM lotsocolumns, UNNEST(MV_TO_ARRAY(dimMultivalEnumerated)) as ut(dim3_unnest1) )"
@ -4769,11 +4761,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupBy()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) GROUP BY d3 ",
@ -4844,11 +4831,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByOrderBy()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 ORDER BY d3 DESC ",
@ -4900,11 +4882,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByOrderByWithLimit()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 ORDER BY d3 ASC LIMIT 4 ",
@ -4943,7 +4920,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByHaving()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 HAVING COUNT(*) = 1",
@ -4982,11 +4958,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithLimit()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) LIMIT 3",
@ -5017,11 +4988,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestFirstQueryOnSelect()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM (select dim1, dim2, dim3 from druid.numfoo), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5067,11 +5033,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestVirtualWithColumns1()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT strings, m1 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (strings) where (strings='a' and (m1<=10 or strings='b'))",
@ -5113,11 +5074,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestVirtualWithColumns2()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT strings, m1 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (strings) where (strings='a' or (m1=2 and strings='b'))",
@ -5156,11 +5112,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFilters()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM (select * from druid.numfoo where dim2='a'), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5193,11 +5144,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFiltersWithExpressionInInnerQuery()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT t,d3 FROM (select FLOOR(__time to hour) t, dim3 from druid.numfoo where dim2='a'), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5235,11 +5181,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithInFiltersWithExpressionInInnerQuery()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT t,d3 FROM (select FLOOR(__time to hour) t, dim3 from druid.numfoo where dim2 IN ('a','b')), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5275,11 +5216,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFiltersInnerLimit()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM (select dim2,dim3 from druid.numfoo where dim2='a' LIMIT 2), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5327,7 +5263,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFiltersInsideAndOutside()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM\n"
+ " (select * from druid.numfoo where dim2='a') as t,\n"
@ -5365,7 +5301,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFiltersInsideAndOutside1()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM\n"
+ " (select * from druid.numfoo where dim2='a'),\n"
@ -5406,7 +5342,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithFiltersOutside()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM\n"
+ " druid.numfoo t,\n"
@ -5448,11 +5384,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithInFilters()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT d3 FROM (select * from druid.numfoo where dim2 IN ('a','b','ab','abc')), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5487,11 +5418,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestVirtualWithColumns()
{
// This tells the test to skip generating (vectorize = force) path
// Generates only 1 native query with vectorize = false
skipVectorize();
// This tells that both vectorize = force and vectorize = false takes the same path of non vectorization
// Generates 2 native queries with 2 different values of vectorize
cannotVectorize();
testQuery(
"SELECT strings FROM druid.numfoo, UNNEST(ARRAY[dim4, dim5]) as unnested (strings)",
@ -5530,7 +5456,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByOrderByOnVirtualColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d24, COUNT(*) FROM druid.numfoo, UNNEST(ARRAY[dim2, dim4]) AS unnested(d24) GROUP BY d24 ORDER BY d24 DESC ",
@ -5586,7 +5511,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithJoinOnTheLeft()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 from (SELECT * from druid.numfoo JOIN (select dim2 as t from druid.numfoo where dim2 IN ('a','b','ab','abc')) ON dim2=t), UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5651,7 +5575,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
// Since there is a constant on the right,
// Druid will plan this as a join query
// as there is nothing to correlate between left and right
skipVectorize();
cannotVectorize();
testQuery(
"SELECT longs FROM druid.numfoo, UNNEST(ARRAY[1,2,3]) as unnested (longs)",
@ -5707,7 +5630,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSQLFunctionOnUnnestedColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT strlen(d3) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -5754,7 +5676,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithINFiltersWithLeftRewrite()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3 IN ('a','b') and m1 < 10",
@ -5787,7 +5708,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithINFiltersWithNoLeftRewrite()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d45 FROM druid.numfoo, UNNEST(ARRAY[dim4,dim5]) as unnested (d45) where d45 IN ('a','b')",
@ -5820,7 +5740,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithInvalidINFiltersOnUnnestedColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3 IN ('foo','bar')",
@ -5846,7 +5765,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithNotFiltersOnUnnestedColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3!='d' ",
@ -5888,7 +5806,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSelectorFiltersOnSelectedColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3='b'",
@ -5917,7 +5834,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSelectorFiltersOnVirtualColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d12 FROM druid.numfoo, UNNEST(ARRAY[m1,m2]) as unnested (d12) where d12=1",
@ -5948,7 +5864,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSelectorFiltersOnVirtualStringColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d45 FROM druid.numfoo, UNNEST(ARRAY[dim4,dim5]) as unnested (d45) where d45 IN ('a','ab')",
@ -5980,7 +5895,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleAndFiltersOnSelectedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3='b' and m1 < 10 and m2 < 10",
@ -6015,7 +5929,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnSelectedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3='b' or m1 < 2 ",
@ -6051,7 +5964,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleAndFiltersOnSelectedUnnestedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3 IN ('a','b') and d3 < 'e' ",
@ -6081,7 +5993,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnUnnestedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3='b' or d3='d' ",
@ -6111,7 +6022,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnVariationsOfUnnestedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where strlen(d3) < 2 or d3='d' ",
@ -6158,7 +6068,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnSelectedNonUnnestedColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where m1 < 2 or m2 < 2 ",
@ -6193,7 +6102,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnSelectedVirtualColumns()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d45 FROM druid.numfoo, UNNEST(ARRAY[dim4,dim5]) as unnested (d45) where d45 IN ('a','aa') or m1 < 2 ",
@ -6231,7 +6139,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where d3='b' or dim3='d' ",
@ -6267,7 +6174,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumnDiffOrdering()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT dim3, d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) where dim3='b' or d3='a' ",
@ -6304,7 +6210,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithCountOnColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT count(*) d3 FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3)",
@ -6330,7 +6235,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByHavingSelector()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) GROUP BY d3 HAVING d3='b'",
@ -6360,7 +6264,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSumOnUnnestedVirtualColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"select sum(c) col from druid.numfoo, unnest(ARRAY[m1,m2]) as u(c)",
@ -6386,7 +6289,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSumOnUnnestedColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"select sum(c) col from druid.numfoo, unnest(mv_to_array(dim3)) as u(c)",
@ -6417,7 +6319,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithSumOnUnnestedArrayColumn()
{
skipVectorize();
cannotVectorize();
testQuery(
"select sum(c) col from druid.arrays, unnest(arrayDoubleNulls) as u(c)",
@ -6443,7 +6344,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByHavingWithWhereOnAggCol()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) WHERE d3 IN ('a','c') GROUP BY d3 HAVING COUNT(*) = 1",
@ -6474,7 +6374,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByHavingWithWhereOnUnnestCol()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT d3, COUNT(*) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) AS unnested(d3) WHERE d3 IN ('a','c') GROUP BY d3 HAVING d3='a'",
@ -6504,7 +6403,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByWithWhereOnUnnestArrayCol()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT uln, COUNT(*) FROM druid.arrays, UNNEST(arrayLongNulls) AS unnested(uln) WHERE uln IN (1, 2, 3) GROUP BY uln",
@ -6535,7 +6433,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByHavingWithWhereOnUnnestArrayCol()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT uln, COUNT(*) FROM druid.arrays, UNNEST(arrayLongNulls) AS unnested(uln) WHERE uln IN (1, 2, 3) GROUP BY uln HAVING uln=1",
@ -6565,7 +6462,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestVirtualWithColumnsAndNullIf()
{
skipVectorize();
cannotVectorize();
testQuery(
"select c,m2 from druid.foo, unnest(ARRAY[\"m1\", \"m2\"]) as u(c) where NULLIF(c,m2) IS NULL",
@ -7059,7 +6955,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestWithGroupByOnExpression()
{
skipVectorize();
cannotVectorize();
testQuery(
"WITH X as \n"
@ -7211,7 +7106,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestExtractionFn()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT substring(d3,1) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) WHERE substring(d3,1) <> 'b'",
@ -7253,7 +7147,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
@Test
public void testUnnestExtractionFnNull()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT substring(d3,1) FROM druid.numfoo, UNNEST(MV_TO_ARRAY(dim3)) as unnested (d3) WHERE substring(d3,1) is not null",

View File

@ -3359,7 +3359,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources(Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"with abc as\n"
@ -4876,7 +4875,6 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
{
// Cannot vectorize JOIN operator.
cannotVectorize();
skipVectorize();
testQuery(
"SELECT dim1, dim2, COUNT(*) FROM druid.foo\n"
+ "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n"

View File

@ -1374,7 +1374,6 @@ public class CalciteNestedDataQueryTest extends BaseCalciteQueryTest
return;
}
cannotVectorize();
skipVectorize();
testBuilder()
.sql(
"SELECT "

View File

@ -577,7 +577,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testSafeDivide()
{
skipVectorize();
cannotVectorize();
final Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
@ -849,7 +848,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testAnyAggregator()
{
// Cannot vectorize virtual expressions.
skipVectorize();
cannotVectorize();
testQuery(
"SELECT "
@ -880,9 +879,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
.context(QUERY_CONTEXT_DEFAULT)
.build()
),
NullHandling.sqlCompatible()
? ImmutableList.of(new Object[]{1L, 1.0f, 1.0, "", 2L, 2.0f, "1"})
: ImmutableList.of(new Object[]{1L, 1.0f, 1.0, "", 2L, 2.0f, "1"})
ImmutableList.of(new Object[]{1L, 1.0f, 1.0, "", 2L, 2.0f, "1"})
);
}
@ -8732,7 +8729,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
{
msqIncompatible();
cannotVectorize();
skipVectorize();
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
ImmutableMap.of(
@ -9704,7 +9700,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
msqIncompatible();
cannotVectorize();
skipVectorize();
// timeseries with all granularity have a single group, so should return default results for given aggregators
testQuery(
"SELECT\n"
@ -10019,7 +10014,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testGroupByAggregatorDefaultValuesNonVectorized()
{
cannotVectorize();
skipVectorize();
testQuery(
"SELECT\n"
+ " dim2,\n"
@ -13011,7 +13005,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testTimeStampAddZeroYearPeriod()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT TIMESTAMPADD(YEAR, 0, \"__time\") FROM druid.foo",
@ -13912,7 +13906,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
public void testStringAggExpression()
{
cannotVectorize();
skipVectorize();
testQuery(
"SELECT\n"
+ " STRING_AGG(DISTINCT CONCAT(dim1, dim2), ','),\n"
@ -14224,7 +14217,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithSingleConstantDimension()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT 'A' from foo WHERE m1 = 50 AND dim1 = 'wat' GROUP BY 'foobar'",
ImmutableList.of(
@ -14278,7 +14271,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithMultipleConstantDimensions()
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT 'A', dim1 from foo WHERE m1 = 50 AND dim1 = 'wat' GROUP BY dim1",
ImmutableList.of(
@ -15022,7 +15015,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInGroupByLimitOutGroupByOrderBy()
{
skipVectorize();
cannotVectorize();
testBuilder()
@ -15100,7 +15092,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testInGroupByOrderByLimitOutGroupByOrderByLimit()
{
skipVectorize();
cannotVectorize();
String sql = "with t AS (SELECT m2 as mo, COUNT(m1) as trend_score\n"
+ "FROM \"foo\"\n"
@ -15196,7 +15187,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testWindowingWithScanAndSort()
{
skipVectorize();
cannotVectorize();
msqIncompatible();
String sql = "with t AS (\n"
@ -15554,7 +15544,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
@Test
public void testBitwiseXor()
{
skipVectorize();
cannotVectorize();
msqIncompatible();
testQuery(

View File

@ -998,8 +998,6 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
@Test
public void testReplaceWithNonExistentOrdinalInClusteredBy()
{
skipVectorize();
final String sql = "REPLACE INTO dst"
+ " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' "
+ " SELECT * FROM foo"
@ -1017,8 +1015,6 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
@Test
public void testReplaceWithNegativeOrdinalInClusteredBy()
{
skipVectorize();
final String sql = "REPLACE INTO dst"
+ " OVERWRITE WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00' "
+ " SELECT * FROM foo"

View File

@ -480,7 +480,6 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
@Test
public void testSafeDivideWithoutTable()
{
skipVectorize();
cannotVectorize();
final Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);

View File

@ -1074,7 +1074,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueFloatAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT count(*) FROM foo where m1 <= (select min(m1) + 4 from foo)",
@ -1133,7 +1132,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueDoubleAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT count(*) FROM foo where m1 >= (select max(m1) - 3.5 from foo)",
@ -1192,7 +1190,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueLongAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT count(*) FROM wikipedia where __time >= (select max(__time) - INTERVAL '10' MINUTE from wikipedia)",
@ -1254,7 +1251,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueStringAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT count(*) FROM wikipedia where channel = (select channel from wikipedia order by __time desc LIMIT 1 OFFSET 6)",
@ -1317,7 +1313,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueStringMultipleRowsAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQueryThrows(
"SELECT count(*) FROM wikipedia where channel = (select channel from wikipedia order by __time desc LIMIT 2 OFFSET 6)",
@ -1330,7 +1325,6 @@ public class CalciteSubqueryTest extends BaseCalciteQueryTest
@ParameterizedTest(name = "{0}")
public void testSingleValueEmptyInnerAgg(String testName, Map<String, Object> queryContext)
{
skipVectorize();
cannotVectorize();
testQuery(
"SELECT distinct countryName FROM wikipedia where countryName = ( select countryName from wikipedia where channel in ('abc', 'xyz'))",

View File

@ -28,6 +28,7 @@ import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.sql.SqlExplainFormat;
import org.apache.calcite.sql.SqlExplainLevel;
import org.apache.calcite.sql.SqlInsert;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.guava.Sequence;
@ -304,7 +305,8 @@ public class QueryTestRunner
private PlannerCaptureHook getCaptureHook()
{
if (builder.getQueryContext().containsKey(PlannerCaptureHook.NEED_CAPTURE_HOOK) || builder.expectedLogicalPlan != null) {
if (builder.getQueryContext().containsKey(PlannerCaptureHook.NEED_CAPTURE_HOOK)
|| builder.expectedLogicalPlan != null) {
return new PlannerCaptureHook();
}
return null;
@ -549,7 +551,8 @@ public class QueryTestRunner
"",
hook.relRoot().rel,
SqlExplainFormat.TEXT,
SqlExplainLevel.DIGEST_ATTRIBUTES);
SqlExplainLevel.DIGEST_ATTRIBUTES
);
String plan;
SqlInsert insertNode = hook.insertNode();
if (insertNode == null) {
@ -642,6 +645,16 @@ public class QueryTestRunner
);
}
// Validate: don't set both skipVectorize and cannotVectorize.
if (builder.skipVectorize && builder.queryCannotVectorize) {
throw new IAE(
"Do not set both skipVectorize and cannotVectorize. Use cannotVectorize if a query is not currently "
+ "able to vectorize, but may be able to in the future. Use skipVectorize if a query *can* vectorize, but "
+ "for some reason we need to skip the test on the vectorized code path. In most situations, cannotVectorize "
+ "is the one you want."
);
}
// Historically, a test either prepares the query (to check resources), or
// runs the query (to check the native query and results.) In the future we
// may want to do both in a single test; but we have no such tests today.