Changing the queryFrameWork in Calcite*Tests may have sideeffects (#15428)

changes how its configured a bit to use an annotation instead of methods
This commit is contained in:
Zoltan Haindrich 2023-12-03 20:08:01 +01:00 committed by GitHub
parent b854058491
commit a1aa4340d0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 166 additions and 81 deletions

View File

@ -35,7 +35,6 @@ import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.hll.VersionOneHyperLogLogCollector;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.io.Closer;
@ -74,7 +73,6 @@ import org.apache.druid.query.scan.ScanQuery;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.query.timeseries.TimeseriesQuery;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -115,9 +113,10 @@ import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.Interval;
import org.joda.time.chrono.ISOChronology;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
@ -284,11 +283,8 @@ public class BaseCalciteQueryTest extends CalciteTestBase
public static final Map<String, Object> OUTER_LIMIT_CONTEXT = new HashMap<>(QUERY_CONTEXT_DEFAULT);
public static int minTopNThreshold = TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD;
@Nullable
public final SqlEngine engine0;
private static SqlTestFramework queryFramework;
final boolean useDefault = NullHandling.replaceWithDefault();
@Rule(order = 1)
@ -613,26 +609,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase
.legacy(false);
}
@BeforeClass
public static void setUpClass()
{
resetFramework();
}
@AfterClass
public static void tearDownClass()
{
resetFramework();
}
protected static void resetFramework()
{
if (queryFramework != null) {
queryFramework.close();
}
queryFramework = null;
}
protected static DruidExceptionMatcher invalidSqlIs(String s)
{
return DruidExceptionMatcher.invalidSqlInput().expectMessageIs(s);
@ -654,42 +630,23 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return queryLogHook = new QueryLogHook(() -> queryFramework().queryJsonMapper());
}
@ClassRule
public static SqlTestFrameworkConfig.ClassRule queryFrameworkClassRule = new SqlTestFrameworkConfig.ClassRule();
@Rule
public SqlTestFrameworkConfig.MethodRule queryFrameworkRule = queryFrameworkClassRule.methodRule(this);
public SqlTestFramework queryFramework()
{
if (queryFramework == null) {
createFramework(0);
}
return queryFramework;
return queryFrameworkRule.get();
}
/**
* Creates the query planning/execution framework. The logic is somewhat
* round-about: the builder creates the structure, but delegates back to
* this class for the parts that the Calcite tests customize. This class,
* in turn, delegates back to a standard class to create components. However,
* subclasses do override each method to customize components for specific
* tests.
*/
private void createFramework(int mergeBufferCount)
{
resetFramework();
try {
baseComponentSupplier = new StandardComponentSupplier(
temporaryFolder.newFolder()
);
}
catch (IOException e) {
throw new RE(e);
}
SqlTestFramework.Builder builder = new SqlTestFramework.Builder(this)
.minTopNThreshold(minTopNThreshold)
.mergeBufferCount(mergeBufferCount);
configureBuilder(builder);
queryFramework = builder.build();
}
protected void configureBuilder(Builder builder)
@Before
public void before() throws Exception
{
baseComponentSupplier = new StandardComponentSupplier(
temporaryFolder.newFolder()
);
}
@Override
@ -1456,14 +1413,6 @@ public class BaseCalciteQueryTest extends CalciteTestBase
return newContext;
}
/**
* Reset the conglomerate, walker, and engine with required number of merge buffers. Default value is 2.
*/
protected void requireMergeBuffers(int numMergeBuffers)
{
createFramework(numMergeBuffers);
}
protected Map<String, Object> withTimestampResultContext(
Map<String, Object> input,
String timestampResultField,

View File

@ -3415,10 +3415,10 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testArrayAggGroupByArrayAggOfLongsFromSubquery()
{
requireMergeBuffers(3);
cannotVectorize();
testQuery(
"select cntarray, count(*) from ( select dim1, dim2, ARRAY_AGG(cnt) as cntarray from ( select dim1, dim2, dim3, count(*) as cnt from foo group by 1, 2, 3 ) group by 1, 2 ) group by 1",
@ -3488,10 +3488,10 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testArrayAggGroupByArrayAggOfStringsFromSubquery()
{
requireMergeBuffers(3);
cannotVectorize();
testQuery(
"select cntarray, count(*) from ( select dim1, dim2, ARRAY_AGG(cnt) as cntarray from ( select dim1, dim2, dim3, cast( count(*) as VARCHAR ) as cnt from foo group by 1, 2, 3 ) group by 1, 2 ) group by 1",
@ -3554,10 +3554,10 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testArrayAggGroupByArrayAggOfDoubleFromSubquery()
{
requireMergeBuffers(3);
cannotVectorize();
testQuery(
"select cntarray, count(*) from ( select dim1, dim2, ARRAY_AGG(cnt) as cntarray from ( select dim1, dim2, dim3, cast( count(*) as DOUBLE ) as cnt from foo group by 1, 2, 3 ) group by 1, 2 ) group by 1",

View File

@ -125,10 +125,11 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
this.sortBasedJoin = sortBasedJoin;
}
@SqlTestFrameworkConfig(minTopNThreshold = 1)
@Test
public void testInnerJoinWithLimitAndAlias()
{
minTopNThreshold = 1;
Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
context.put(PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, false);
testQuery(
@ -183,12 +184,12 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
}
// Adjust topN threshold, so that the topN engine keeps only 1 slot for aggregates, which should be enough
// to compute the query with limit 1.
@SqlTestFrameworkConfig(minTopNThreshold = 1)
@Test
public void testExactTopNOnInnerJoinWithLimit()
{
// Adjust topN threshold, so that the topN engine keeps only 1 slot for aggregates, which should be enough
// to compute the query with limit 1.
minTopNThreshold = 1;
Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
context.put(PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, false);
testQuery(
@ -5803,10 +5804,10 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(minTopNThreshold = 1)
@Test
public void testJoinWithAliasAndOrderByNoGroupBy()
{
minTopNThreshold = 1;
Map<String, Object> context = new HashMap<>(QUERY_CONTEXT_DEFAULT);
context.put(PlannerConfig.CTX_KEY_USE_APPROXIMATE_TOPN, false);
testQuery(

View File

@ -2480,6 +2480,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testExactCountDistinctWithFilter()
{
@ -2503,8 +2504,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
)
);
requireMergeBuffers(3);
if (NullHandling.sqlCompatible()) {
// Cannot vectorize due to "istrue" operator.
cannotVectorize();
@ -2581,6 +2580,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testExactCountDistinctLookup()
{
@ -2593,8 +2593,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
// ExtractionDimensionSpec cannot be vectorized
cannotVectorize();
requireMergeBuffers(3);
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
ImmutableMap.of(
@ -3334,6 +3332,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
* This test case should be in {@link CalciteUnionQueryTest}. However, there's a bug in the test framework that
* doesn't reset framework once the merge buffers
*/
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@NotYetSupported
@Test
public void testUnionAllSameTableThreeTimes()
@ -3378,6 +3377,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@NotYetSupported(Modes.NOT_ENOUGH_RULES)
@Test
public void testExactCountDistinctUsingSubqueryOnUnionAllTables()
@ -6676,11 +6676,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 4)
@Test
public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets()
{
msqIncompatible();
requireMergeBuffers(4);
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
ImmutableMap.of(
@ -6940,10 +6940,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testDoubleNestedGroupBy()
{
requireMergeBuffers(3);
testQuery(
"SELECT SUM(cnt), COUNT(*) FROM (\n"
+ " SELECT dim2, SUM(t1.cnt) cnt FROM (\n"
@ -6995,6 +6995,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testDoubleNestedGroupBy2()
{
@ -8290,13 +8291,13 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename()
{
msqIncompatible();
cannotVectorize();
skipVectorize();
requireMergeBuffers(3);
testQuery(
PLANNER_CONFIG_NO_HLL.withOverrides(
ImmutableMap.of(
@ -10332,6 +10333,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSets()
{
@ -10397,11 +10399,11 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingAggregatorDifferentOrder()
{
msqIncompatible();
requireMergeBuffers(3);
testQuery(
"SELECT dim2, gran, SUM(cnt), GROUPING(gran, dim2)\n"
@ -10667,6 +10669,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupByCube()
{
@ -10729,6 +10732,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithDummyDimension()
{
@ -10791,6 +10795,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsNoSuperset()
{
@ -10848,6 +10853,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithOrderByDimension()
{
@ -10919,6 +10925,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithOrderByAggregator()
{
@ -10988,6 +10995,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithOrderByAggregatorWithLimit()
{
@ -12582,6 +12590,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithLimit()
{
@ -12647,6 +12656,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
);
}
@SqlTestFrameworkConfig(numMergeBuffers = 3)
@Test
public void testGroupingSetsWithLimitOrderByGran()
{

View File

@ -0,0 +1,125 @@
/*
* 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.sql.calcite;
import org.apache.druid.query.topn.TopNQueryConfig;
import org.apache.druid.sql.calcite.util.SqlTestFramework;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
import org.junit.rules.ExternalResource;
import org.junit.rules.TestRule;
import org.junit.runner.Description;
import org.junit.runners.model.Statement;
import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
import java.util.HashMap;
import java.util.Map;
/**
* Annotation to specify desired framework settings.
*
* This class provides junit rule facilities to build the framework accordingly to the annotation.
* These rules also cache the previously created frameworks.
*/
@Retention(RetentionPolicy.RUNTIME)
@Target({ElementType.METHOD})
public @interface SqlTestFrameworkConfig
{
int numMergeBuffers() default 0;
int minTopNThreshold() default TopNQueryConfig.DEFAULT_MIN_TOPN_THRESHOLD;
/**
* @see {@link SqlTestFrameworkConfig}
*/
class ClassRule extends ExternalResource
{
Map<SqlTestFrameworkConfig, SqlTestFramework> frameworkMap = new HashMap<SqlTestFrameworkConfig, SqlTestFramework>();
public MethodRule methodRule(BaseCalciteQueryTest testHost)
{
return new MethodRule(this, testHost);
}
@Override
protected void after()
{
for (SqlTestFramework f : frameworkMap.values()) {
f.close();
}
frameworkMap.clear();
}
}
/**
* @see {@link SqlTestFrameworkConfig}
*/
class MethodRule implements TestRule
{
private SqlTestFrameworkConfig config;
private ClassRule classRule;
private QueryComponentSupplier testHost;
public MethodRule(ClassRule classRule, QueryComponentSupplier testHost)
{
this.classRule = classRule;
this.testHost = testHost;
}
@SqlTestFrameworkConfig
public SqlTestFrameworkConfig defaultConfig()
{
try {
return getClass()
.getMethod("defaultConfig")
.getAnnotation(SqlTestFrameworkConfig.class);
}
catch (NoSuchMethodException | SecurityException e) {
throw new RuntimeException(e);
}
}
@Override
public Statement apply(Statement base, Description description)
{
config = description.getAnnotation(SqlTestFrameworkConfig.class);
if (config == null) {
config = defaultConfig();
}
return base;
}
public SqlTestFramework get()
{
return classRule.frameworkMap.computeIfAbsent(config, this::createFramework);
}
private SqlTestFramework createFramework(SqlTestFrameworkConfig config)
{
SqlTestFramework.Builder builder = new SqlTestFramework.Builder(testHost)
.minTopNThreshold(config.minTopNThreshold())
.mergeBufferCount(config.numMergeBuffers());
return builder.build();
}
}
}