diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java
index 388abc901c3934..93ff54e4583049 100644
--- a/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateMaterializedViewTest.java
@@ -78,6 +78,7 @@
import org.apache.hadoop.util.ThreadUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.assertj.core.util.Sets;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -90,13 +91,13 @@
import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestName;
-import java.time.Instant;
import java.time.LocalDateTime;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Set;
import java.util.stream.Collectors;
import static com.starrocks.sql.optimizer.MVTestUtils.waitingRollupJobV2Finish;
@@ -118,8 +119,7 @@ public class CreateMaterializedViewTest {
private static Database testDb;
private static GlobalStateMgr currentState;
- private static long startSuiteTime = 0;
- private long startCaseTime = 0;
+ private static Set
existedTables = Sets.newHashSet();
@BeforeClass
public static void beforeClass() throws Exception {
@@ -133,7 +133,6 @@ public static void beforeClass() throws Exception {
// create connect context
connectContext = UtFrameUtils.createDefaultCtx();
starRocksAssert = new StarRocksAssert(connectContext);
- startSuiteTime = Instant.now().getEpochSecond();
if (!starRocksAssert.databaseExist("_statistics_")) {
StatisticsMetaManager m = new StatisticsMetaManager();
@@ -327,22 +326,22 @@ public static void beforeClass() throws Exception {
testDb = currentState.getLocalMetastore().getDb("test");
UtFrameUtils.setUpForPersistTest();
+ Thread.sleep(1000);
}
@AfterClass
public static void afterClass() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
}
@Before
public void before() {
- startCaseTime = Instant.now().getEpochSecond();
+ PlanTestBase.collectTables(starRocksAssert, existedTables);
}
@After
public void after() throws Exception {
// cleanup mv after each case
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
+ PlanTestBase.cleanup(starRocksAssert, existedTables);
}
private static void dropMv(String mvName) throws Exception {
diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java
index 26a9cf393360b8..1f4a48db2cf2de 100644
--- a/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/analysis/RefreshMaterializedViewTest.java
@@ -29,8 +29,6 @@
import com.starrocks.catalog.Table;
import com.starrocks.catalog.Tablet;
import com.starrocks.clone.DynamicPartitionScheduler;
-import com.starrocks.metric.MaterializedViewMetricsEntity;
-import com.starrocks.metric.MaterializedViewMetricsRegistry;
import com.starrocks.qe.StmtExecutor;
import com.starrocks.schema.MTable;
import com.starrocks.server.GlobalStateMgr;
@@ -1279,9 +1277,6 @@ public void testTruncateTableInDiffDb() throws Exception {
executeInsertSql(connectContext, "insert into trunc_db.trunc_db_t1 values(2, 10)");
MaterializedView mv1 = getMv("mv_db", "test_mv");
- MaterializedViewMetricsEntity mvEntity =
- (MaterializedViewMetricsEntity) MaterializedViewMetricsRegistry.getInstance().getMetricsEntity(mv1.getMvId());
- long origCount = mvEntity.histRefreshJobDuration.getCount();
Table table = getTable("trunc_db", "trunc_db_t1");
// Simulate writing to a non-existent MV
@@ -1289,13 +1284,8 @@ public void testTruncateTableInDiffDb() throws Exception {
String truncateStr = "truncate table trunc_db.trunc_db_t1;";
TruncateTableStmt truncateTableStmt = (TruncateTableStmt) UtFrameUtils.parseStmtWithNewParser(truncateStr, connectContext);
GlobalStateMgr.getCurrentState().getLocalMetastore().truncateTable(truncateTableStmt, connectContext);
- starRocksAssert.waitRefreshFinished(mv1.getId());
+ Assert.assertTrue(starRocksAssert.waitRefreshFinished(mv1.getId()));
- mvEntity =
- (MaterializedViewMetricsEntity) MaterializedViewMetricsRegistry.getInstance().getMetricsEntity(mv1.getMvId());
- System.out.println(mvEntity.histRefreshJobDuration);
- long count = mvEntity.histRefreshJobDuration.getCount();
- Assert.assertEquals(origCount + 1, count);
starRocksAssert.dropTable("trunc_db.trunc_db_t1");
starRocksAssert.dropMaterializedView("mv_db.test_mv");
}
@@ -1328,21 +1318,13 @@ public void testDropPartitionTableInDiffDb() throws Exception {
executeInsertSql(connectContext, "insert into drop_db.tbl_with_mv partition(p2) values(\"2022-02-20\", 2, 10)");
MaterializedView mv1 = getMv("drop_mv_db", "test_mv");
- MaterializedViewMetricsEntity mvEntity =
- (MaterializedViewMetricsEntity) MaterializedViewMetricsRegistry.getInstance().getMetricsEntity(mv1.getMvId());
- long origCount = mvEntity.histRefreshJobDuration.getCount();
-
OlapTable table = (OlapTable) getTable("drop_db", "tbl_with_mv");
Partition p1 = table.getPartition("p1");
DropPartitionClause dropPartitionClause = new DropPartitionClause(false, p1.getName(), false, true);
dropPartitionClause.setResolvedPartitionNames(ImmutableList.of(p1.getName()));
Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("drop_db");
GlobalStateMgr.getCurrentState().getLocalMetastore().dropPartition(db, table, dropPartitionClause);
- starRocksAssert.waitRefreshFinished(mv1.getId());
- mvEntity =
- (MaterializedViewMetricsEntity) MaterializedViewMetricsRegistry.getInstance().getMetricsEntity(mv1.getMvId());
- long count = mvEntity.histRefreshJobDuration.getCount();
- Assert.assertEquals(origCount + 1, count);
+ Assert.assertTrue(starRocksAssert.waitRefreshFinished(mv1.getId()));
starRocksAssert.dropTable("drop_db.tbl_with_mv");
starRocksAssert.dropMaterializedView("drop_mv_db.test_mv");
}
diff --git a/fe/fe-core/src/test/java/com/starrocks/benchmark/MvRefreshConcurrencyTest.java b/fe/fe-core/src/test/java/com/starrocks/benchmark/MvRefreshConcurrencyTest.java
index 09591cc5a9cdb0..5b90371fb2439c 100644
--- a/fe/fe-core/src/test/java/com/starrocks/benchmark/MvRefreshConcurrencyTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/benchmark/MvRefreshConcurrencyTest.java
@@ -29,6 +29,7 @@
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestRule;
@@ -41,8 +42,9 @@
* x dbs which contains one table
* y mvs which contains x/2 tables and uses `union all` to concatenate them
*
- * refresh mvs with concurrency to test lock and preformance
+ * refresh mvs with concurrency to test lock and performance
*/
+@Ignore
public class MvRefreshConcurrencyTest extends MvRewriteTestBase {
@Rule
diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprListTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprListTest.java
index ea369450f652c4..6e488626c9ecfb 100644
--- a/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprListTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprListTest.java
@@ -144,14 +144,6 @@ public static void beforeClass() throws Exception {
public static void afterClass() throws Exception {
}
- @Before
- public void before() {
- }
-
- @After
- public void after() throws Exception {
- }
-
public static void executeInsertSql(String sql) throws Exception {
connectContext.setQueryId(UUIDUtil.genUUID());
StatementBase statement = SqlParser.parseSingleStatement(sql, connectContext.getSessionVariable().getSqlMode());
diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprRangeTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprRangeTest.java
index 3f66d36ccd3bef..a0a8c026bcad49 100644
--- a/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprRangeTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/catalog/DropPartitionWithExprRangeTest.java
@@ -34,10 +34,8 @@
import mockit.MockUp;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -99,14 +97,6 @@ public static void beforeClass() throws Exception {
public static void afterClass() throws Exception {
}
- @Before
- public void before() {
- }
-
- @After
- public void after() throws Exception {
- }
-
public static void executeInsertSql(String sql) throws Exception {
connectContext.setQueryId(UUIDUtil.genUUID());
StatementBase statement = SqlParser.parseSingleStatement(sql, connectContext.getSessionVariable().getSqlMode());
diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/combinator/AggStateCombinatorTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/combinator/AggStateCombinatorTest.java
index c214ffb8f73c1e..5df63b2f81c1c4 100644
--- a/fe/fe-core/src/test/java/com/starrocks/catalog/combinator/AggStateCombinatorTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/catalog/combinator/AggStateCombinatorTest.java
@@ -43,6 +43,7 @@
import com.starrocks.sql.optimizer.statistics.EmptyStatisticStorage;
import com.starrocks.sql.parser.NodePosition;
import com.starrocks.sql.plan.PlanTestBase;
+import com.starrocks.statistic.StatisticsMetaManager;
import com.starrocks.utframe.StarRocksAssert;
import com.starrocks.utframe.UtFrameUtils;
import org.apache.kudu.shaded.com.google.common.collect.Streams;
@@ -61,6 +62,7 @@
import java.util.stream.Stream;
public class AggStateCombinatorTest extends MvRewriteTestBase {
+ private static final int MAX_AGG_FUNC_NUM_IN_TEST = 20;
@BeforeClass
public static void beforeClass() throws Exception {
@@ -75,6 +77,15 @@ public static void beforeClass() throws Exception {
connectContext = UtFrameUtils.createDefaultCtx();
starRocksAssert = new StarRocksAssert(connectContext);
starRocksAssert.withEnableMV().withDatabase("test").useDatabase("test");
+
+ // set default config for timeliness mvs
+ UtFrameUtils.mockTimelinessForAsyncMVTest(connectContext);
+ if (!starRocksAssert.databaseExist("_statistics_")) {
+ StatisticsMetaManager m = new StatisticsMetaManager();
+ m.createStatisticsTablesForTest();
+ }
+ Config.enable_materialized_view_text_based_rewrite = false;
+ setGlobalVariableVariable("cbo_push_down_aggregate_mode", "-1");
}
private static final Set SUPPORTED_AGG_STATE_FUNCTIONS = ImmutableSet.of(
@@ -253,7 +264,6 @@ private void buildTableT1(List funcNames,
String[] parts = colType.split(" ");
String colName = parts[0];
String type = colType.substring(colName.length() + 1);
- System.out.println("ColName:" + colName + ", Type:" + type);
colTypes.put(type, colName);
}
var builtInAggregateFunctions = getBuiltInAggFunctions();
@@ -285,7 +295,6 @@ private void buildTableT1(List funcNames,
") DUPLICATE KEY(k1) \n" +
"DISTRIBUTED BY HASH(k1) \n" +
"PROPERTIES ( \"replication_num\" = \"1\");";
- System.out.println(sql);
starRocksAssert.withTable(sql);
} catch (Exception e) {
Assert.fail(e.getMessage());
@@ -414,6 +423,9 @@ public void testCreateAggStateTableUnionWithPerTable() {
if (!AggStateUtils.isSupportedAggStateFunction(aggFunc)) {
continue;
}
+ if (i >= MAX_AGG_FUNC_NUM_IN_TEST) {
+ break;
+ }
String colName = "v" + i;
colNames.add(colName);
funcNames.add(aggFunc.functionName());
@@ -441,9 +453,7 @@ public void testCreateAggStateTableUnionWithPerTable() {
String colName = colNames.get(i);
String unionFnName = String.format("%s(%s)", fnName, colName);
String sql1 = "select k1, " + unionFnName + " from test_agg_state_table group by k1";
- // System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- // System.out.println(plan);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
}
@@ -464,6 +474,9 @@ public void testCreateAggStateTableMergeWithPerTable() {
if (!AggStateUtils.isSupportedAggStateFunction(aggFunc)) {
continue;
}
+ if (i > MAX_AGG_FUNC_NUM_IN_TEST) {
+ break;
+ }
String colName = "v" + i;
colNames.add(colName);
funcNames.add(aggFunc.functionName());
@@ -491,7 +504,6 @@ public void testCreateAggStateTableMergeWithPerTable() {
String colName = colNames.get(i);
String unionFnName = String.format("%s(%s)", fnName, colName);
String sql1 = "select k1, " + unionFnName + " from test_agg_state_table group by k1";
- // System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
@@ -515,6 +527,9 @@ public void testCreateAggStateTableWithAllFunctions() {
if (!AggStateUtils.isSupportedAggStateFunction(aggFunc)) {
continue;
}
+ if (i > MAX_AGG_FUNC_NUM_IN_TEST) {
+ break;
+ }
String colName = "v" + i;
colNames.add(colName);
funcNames.add(aggFunc.functionName());
@@ -543,7 +558,6 @@ public void testCreateAggStateTableWithAllFunctions() {
.collect(Collectors.toList());
String sql1 = "select k1, " +
Joiner.on(",").join(unionColumns) + " from test_agg_state_table group by k1";
- // System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
@@ -557,7 +571,6 @@ public void testCreateAggStateTableWithAllFunctions() {
.collect(Collectors.toList());
String sql1 = "select k1, " +
Joiner.on(",").join(mergeColumns) + " from test_agg_state_table group by k1";
- // System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
@@ -581,7 +594,6 @@ public void testCreateAggStateTable2() throws Exception {
String col = String.format("%s(%s)", FunctionSet.getAggStateName(fnName), arg);
stateColumns.add(col);
String sql1 = "select k1, " + Joiner.on(", ").join(stateColumns) + " from t1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: t1, rollup: t1");
@@ -606,7 +618,6 @@ public void testCreateAggStateTable3() throws Exception {
stateColumns.add(col);
}
String sql1 = "select k1, " + Joiner.on(", ").join(stateColumns) + " from t1";
- // System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: t1, rollup: t1");
@@ -632,9 +643,7 @@ public void testCreateAggStateTableWithApproxTopK() throws Exception {
String col = String.format("%s(%s)", FunctionSet.getAggStateName(fnName), arg);
stateColumns.add(col);
String sql1 = "select k1, " + Joiner.on(", ").join(stateColumns) + " from t1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "| 31 <-> approx_top_k_state[([27: c6, DOUBLE, true], 10, 100); " +
"args: DOUBLE,INT,INT; result: VARBINARY; args nullable: true; result nullable: true]");
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
@@ -687,14 +696,11 @@ public void testCreateAggStateTableWithMultiDistinctSum() throws Exception {
") DISTRIBUTED BY HASH(k1) \n" +
"PROPERTIES ( \"replication_num\" = \"1\");";
starRocksAssert.withTable(sql);
- System.out.println(sql);
// multi_distinct_sum_state
{
String sql1 = "select k1, " + Joiner.on(", ").join(stateColumns) + " from t1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, " 1:Project\n" +
" | output columns:\n" +
" | 1 <-> [1: k1, DATE, true]\n" +
@@ -725,9 +731,7 @@ public void testCreateAggStateTableWithMultiDistinctSum() throws Exception {
{
String sql1 = "select k1, " + Joiner.on(", ").join(unionColumns)
+ " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "| aggregate: multi_distinct_sum_union[([6: v4, VARBINARY, true]); " +
"args: VARBINARY; result: VARBINARY; args nullable: true; " +
"result nullable: true], multi_distinct_sum_union[([7: v5, VARBINARY, true]); args: VARBINARY; " +
@@ -758,9 +762,7 @@ public void testCreateAggStateTableWithMultiDistinctSum() throws Exception {
{
String sql1 = "select k1, " + Joiner.on(", ").join(mergeColumns)
+ " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "| aggregate: multi_distinct_sum_merge[([6: v4, VARBINARY, true]); " +
"args: VARBINARY; result: BIGINT; args nullable: true; result nullable: true], " +
"multi_distinct_sum_merge[([7: v5, VARBINARY, true]); args: VARBINARY; " +
@@ -833,14 +835,11 @@ public void testCreateAggStateTableWithArrayAgg() throws Exception {
") DISTRIBUTED BY HASH(k1) \n" +
"PROPERTIES ( \"replication_num\" = \"1\");";
starRocksAssert.withTable(sql);
- System.out.println(sql);
// multi_distinct_sum_state
{
String sql1 = "select k1, " + Joiner.on(", ").join(stateColumns) + " from t1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "32 <-> array_agg_state[([26: c24, VARCHAR, true]); args: VARCHAR; " +
"result: struct>; args nullable: true; result nullable: true]");
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
@@ -851,9 +850,7 @@ public void testCreateAggStateTableWithArrayAgg() throws Exception {
{
String sql1 = "select k1, " + Joiner.on(", ").join(unionColumns)
+ " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "| aggregate: array_agg_union[([2: v0, " +
"struct>, true]); args: INVALID_TYPE; result: " +
"struct>; args nullable: true; result nullable: true]");
@@ -865,9 +862,7 @@ public void testCreateAggStateTableWithArrayAgg() throws Exception {
{
String sql1 = "select k1, " + Joiner.on(", ").join(mergeColumns)
+ " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
- System.out.println(plan);
PlanTestBase.assertContains(plan, "| aggregate: " +
"array_agg_merge[([2: v0, struct>, true]); args: INVALID_TYPE; " +
"result: ARRAY; args nullable: true; result nullable: true]");
@@ -892,7 +887,6 @@ public void testGenerateSqlTesterTestsTotal() throws Exception {
Joiner.on(",\n").join(columns) +
") DISTRIBUTED BY HASH(k1) \n" +
"PROPERTIES ( \"replication_num\" = \"1\");";
- System.out.println(sql);
starRocksAssert.withTable(sql,
() -> {
Table table = starRocksAssert.getCtx().getGlobalStateMgr().getLocalMetastore()
@@ -912,7 +906,6 @@ public void testGenerateSqlTesterTestsTotal() throws Exception {
}
String sql1 = "insert into test_agg_state_table select k1, " +
Joiner.on(", ").join(stateColumns) + " from t1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: t1, rollup: t1");
@@ -925,7 +918,6 @@ public void testGenerateSqlTesterTestsTotal() throws Exception {
.collect(Collectors.toList());
String sql1 = "select k1, " +
Joiner.on(", ").join(unionColumns) + " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
@@ -939,7 +931,6 @@ public void testGenerateSqlTesterTestsTotal() throws Exception {
.collect(Collectors.toList());
String sql1 = "select k1, " +
Joiner.on(",").join(mergeColumns) + " from test_agg_state_table group by k1;";
- System.out.println(sql1);
String plan = UtFrameUtils.getVerboseFragmentPlan(starRocksAssert.getCtx(), sql1);
PlanTestBase.assertContains(plan, " 0:OlapScanNode\n" +
" table: test_agg_state_table, rollup: test_agg_state_table");
@@ -978,20 +969,17 @@ public void testGenerateCreateSyncMVWithMultiCountDistinct() throws Exception {
// create sync mv with all agg functions
String sql1 = "CREATE MATERIALIZED VIEW test_mv1 as select k1, " +
Joiner.on(", ").join(stateColumns) + " from t1 group by k1;";
- System.out.println(sql1);
starRocksAssert.withMaterializedView(sql1);
// no rollup
{
String query = String.format("select k1, %s from t1 group by k1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
// rollup
{
String query = String.format("select %s from t1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
@@ -1026,20 +1014,17 @@ private void testCreateSyncMVWithSpecificAggFunc(String aggFuncName) throws Exce
// create sync mv with all agg functions
String sql1 = "CREATE MATERIALIZED VIEW test_mv1 as select k1, " +
Joiner.on(", ").join(stateColumns) + " from t1 group by k1;";
- System.out.println(sql1);
starRocksAssert.withMaterializedView(sql1);
// no rollup
{
String query = String.format("select k1, %s from t1 group by k1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
// rollup
{
String query = String.format("select %s from t1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
@@ -1075,6 +1060,9 @@ public void testGenerateCreateAsyncMVWithAggState() throws Exception {
if (fnName.equalsIgnoreCase(FunctionSet.ARRAY_AGG_DISTINCT)) {
continue;
}
+ if (i > MAX_AGG_FUNC_NUM_IN_TEST) {
+ break;
+ }
List argTypes = aggArgTypes.get(i);
String arg = buildAggFuncArgs(fnName, argTypes, colTypes);
String col = String.format("%s(%s(%s)) as agg%s",
@@ -1088,20 +1076,17 @@ public void testGenerateCreateAsyncMVWithAggState() throws Exception {
// create async mv with all agg functions
String sql1 = "CREATE MATERIALIZED VIEW test_mv1 REFRESH MANUAL as select k1, " +
Joiner.on(", ").join(stateColumns) + " from t1 group by k1;";
- System.out.println(sql1);
- starRocksAssert.withRefreshedMaterializedView(sql1);
+ starRocksAssert.withMaterializedView(sql1);
// no rollup
{
String query = String.format("select k1, %s from t1 group by k1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
// rollup
{
String query = String.format("select %s from t1;", Joiner.on(", ").join(queryColumns));
- System.out.println(query);
String plan = getFragmentPlan(query);
PlanTestBase.assertContains(plan, "test_mv1");
}
@@ -1138,8 +1123,7 @@ public void testGenerateCreateAsyncMVWithCountFunctions() throws Exception {
// create async mv with all agg functions
String sql1 = "CREATE MATERIALIZED VIEW test_mv1 REFRESH MANUAL as select k1, " +
Joiner.on(", ").join(stateColumns) + " from t1 group by k1;";
- System.out.println(sql1);
- starRocksAssert.withRefreshedMaterializedView(sql1);
+ starRocksAssert.withMaterializedView(sql1);
MaterializedView mv = starRocksAssert.getMv("test", "test_mv1");
List mvCols = mv.getColumns();
// count agg function's output should be always not nullable.
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/MVRefreshTestBase.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/MVRefreshTestBase.java
index e11c062e509ced..e8afa4fb11d33e 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/MVRefreshTestBase.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/MVRefreshTestBase.java
@@ -35,6 +35,7 @@
import com.starrocks.sql.optimizer.rule.transformation.materialization.MvRewriteTestBase;
import com.starrocks.sql.parser.SqlParser;
import com.starrocks.sql.plan.ExecPlan;
+import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.statistic.StatisticsMetaManager;
import com.starrocks.thrift.TExplainLevel;
import com.starrocks.utframe.StarRocksAssert;
@@ -42,8 +43,9 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import org.junit.AfterClass;
+import org.junit.After;
import org.junit.Assert;
+import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.rules.TemporaryFolder;
@@ -60,9 +62,7 @@ public class MVRefreshTestBase {
@ClassRule
public static TemporaryFolder temp = new TemporaryFolder();
- protected static long startSuiteTime = 0;
- protected long startCaseTime = 0;
-
+ protected static Set existedTables = Sets.newHashSet();
protected static final String TEST_DB_NAME = "test";
@BeforeClass
@@ -82,8 +82,14 @@ public static void beforeClass() throws Exception {
starRocksAssert.useDatabase("test");
}
- @AfterClass
- public static void tearDown() throws Exception {
+ @Before
+ public void before() {
+ PlanTestBase.collectTables(starRocksAssert, existedTables);
+ }
+
+ @After
+ public void after() throws Exception {
+ PlanTestBase.cleanup(starRocksAssert, existedTables);
}
public static void executeInsertSql(String sql) throws Exception {
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PCTRefreshListPartitionOlapTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PCTRefreshListPartitionOlapTest.java
index 42b78f4f85f324..1ce4ab5c3ffb37 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PCTRefreshListPartitionOlapTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PCTRefreshListPartitionOlapTest.java
@@ -33,16 +33,12 @@
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.thrift.TExplainLevel;
import com.starrocks.utframe.UtFrameUtils;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.time.LocalDateTime;
import java.time.format.DateTimeFormatter;
import java.util.Collection;
@@ -51,7 +47,6 @@
import java.util.Set;
import java.util.stream.Collectors;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
import static com.starrocks.utframe.UtFrameUtils.getFragmentPlan;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@@ -179,21 +174,6 @@ public static void beforeClass() throws Exception {
"DISTRIBUTED BY RANDOM\n";
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private ExecPlan getExecPlan(TaskRun taskRun) {
try {
PartitionBasedMvRefreshProcessor processor = getProcessor(taskRun);
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorHiveTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorHiveTest.java
index 9b8c6b481f176c..32878546228458 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorHiveTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorHiveTest.java
@@ -34,16 +34,12 @@
import com.starrocks.thrift.TExplainLevel;
import mockit.Mock;
import mockit.MockUp;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
@@ -52,7 +48,6 @@
import static com.starrocks.scheduler.TaskRun.PARTITION_END;
import static com.starrocks.scheduler.TaskRun.PARTITION_START;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class PartitionBasedMvRefreshProcessorHiveTest extends MVRefreshTestBase {
@@ -122,21 +117,6 @@ public static void beforeClass() throws Exception {
"`hive0`.`partitioned_db`.`t1_par` using (par_col)");
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
@Test
public void testAutoRefreshPartitionLimitWithHiveTable() throws Exception {
starRocksAssert.useDatabase("test").withMaterializedView("CREATE MATERIALIZED VIEW `hive_parttbl_mv1`\n" +
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorIcebergTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorIcebergTest.java
index 1ad4f3c8310f9b..fb0f5ae78a6248 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorIcebergTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorIcebergTest.java
@@ -30,16 +30,12 @@
import com.starrocks.sql.plan.ExecPlan;
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.utframe.UtFrameUtils;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
@@ -47,8 +43,6 @@
import java.util.Set;
import java.util.stream.Collectors;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
-
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class PartitionBasedMvRefreshProcessorIcebergTest extends MVRefreshTestBase {
@@ -58,21 +52,6 @@ public static void beforeClass() throws Exception {
ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME);
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private static void triggerRefreshMv(Database testDb, MaterializedView partitionedMaterializedView)
throws Exception {
Task task = TaskBuilder.buildMvTask(partitionedMaterializedView, testDb.getFullName());
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorJdbcTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorJdbcTest.java
index aab37958742b19..fc81f417d29171 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorJdbcTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorJdbcTest.java
@@ -27,16 +27,12 @@
import com.starrocks.server.GlobalStateMgr;
import com.starrocks.sql.plan.ConnectorPlanTestBase;
import org.jetbrains.annotations.NotNull;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
@@ -44,8 +40,6 @@
import java.util.Map;
import java.util.stream.Collectors;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
-
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class PartitionBasedMvRefreshProcessorJdbcTest extends MVRefreshTestBase {
@@ -107,21 +101,6 @@ public static void beforeClass() throws Exception {
"as select str2date(d,'%Y%m%d') ss, a, b, c from jdbc0.partitioned_db0.tbl1;");
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
@Test
public void testJDBCProtocolType() {
JDBCTable table = new JDBCTable();
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapPart2Test.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapPart2Test.java
index b13d7920c02f53..8c8eac57b00aaa 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapPart2Test.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapPart2Test.java
@@ -39,16 +39,12 @@
import mockit.MockUp;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.time.LocalDateTime;
import java.time.format.DateTimeFormatter;
import java.util.HashMap;
@@ -57,7 +53,6 @@
import java.util.Set;
import static com.starrocks.scheduler.TaskRun.MV_ID;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
import static com.starrocks.utframe.UtFrameUtils.getFragmentPlan;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@@ -96,21 +91,6 @@ public static void beforeClass() throws Exception {
"PROPERTIES('replication_num' = '1');";
}
- @AfterClass
- public static void tearDown() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
@Test
public void testMVRefreshWithTheSameTables1() {
starRocksAssert.withMTables(List.of(
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapTest.java
index a12e9e39ce30d7..2819fc5f8a14b1 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorOlapTest.java
@@ -56,16 +56,12 @@
import org.apache.commons.collections4.SetUtils;
import org.apache.commons.lang3.StringUtils;
import org.jetbrains.annotations.NotNull;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
@@ -77,7 +73,6 @@
import java.util.concurrent.atomic.AtomicInteger;
import static com.starrocks.scheduler.TaskRun.MV_ID;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class PartitionBasedMvRefreshProcessorOlapTest extends MVRefreshTestBase {
@@ -235,21 +230,6 @@ public static void beforeClass() throws Exception {
"PROPERTIES('replication_num' = '1');");
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
protected void assertPlanContains(ExecPlan execPlan, String... explain) throws Exception {
String explainString = execPlan.getExplainString(TExplainLevel.NORMAL);
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorPaimonTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorPaimonTest.java
index 14f18af2320472..b3e27ba74f8a2f 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorPaimonTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshProcessorPaimonTest.java
@@ -21,21 +21,16 @@
import com.starrocks.server.GlobalStateMgr;
import com.starrocks.sql.plan.ConnectorPlanTestBase;
import com.starrocks.sql.plan.PlanTestBase;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.Collection;
import java.util.Map;
import static com.starrocks.sql.plan.ConnectorPlanTestBase.MOCK_PAIMON_CATALOG_NAME;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
import static com.starrocks.utframe.UtFrameUtils.getFragmentPlan;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
@@ -47,21 +42,6 @@ public static void beforeClass() throws Exception {
ConnectorPlanTestBase.mockCatalog(connectContext, MOCK_PAIMON_CATALOG_NAME, temp.newFolder().toURI().toString());
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
@Test
public void testcreateUnpartitionedPmnMaterializeView() {
//unparitioned
diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshTest.java
index 0956e4006b700e..8b9f986eb1c8bb 100644
--- a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedMvRefreshTest.java
@@ -29,23 +29,17 @@
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.thrift.TExplainLevel;
import org.apache.commons.lang3.StringUtils;
-import org.junit.After;
-import org.junit.AfterClass;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
-import java.time.Instant;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
-import static com.starrocks.sql.plan.PlanTestBase.cleanupEphemeralMVs;
-
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class PartitionBasedMvRefreshTest extends MVRefreshTestBase {
@@ -112,21 +106,6 @@ public static void beforeClass() throws Exception {
" ('2020-10-22','2020-10-25 12:12:12','k3','k4',0,1,2,3,4,5,1.1,1.12,2.889)");
}
- @AfterClass
- public static void afterClass() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startSuiteTime);
- }
-
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
protected void assertPlanContains(ExecPlan execPlan, String... explain) throws Exception {
String explainString = execPlan.getExplainString(TExplainLevel.NORMAL);
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePartialPartitionTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePartialPartitionTest.java
index fbd240020a797d..f77005146bc2bb 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePartialPartitionTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePartialPartitionTest.java
@@ -23,14 +23,10 @@
import com.starrocks.server.GlobalStateMgr;
import com.starrocks.sql.plan.ConnectorPlanTestBase;
import com.starrocks.sql.plan.PlanTestBase;
-import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import java.time.Instant;
-
public class MvRewritePartialPartitionTest extends MvRewriteTestBase {
private static MockedHiveMetadata mockedHiveMetadata;
@@ -52,16 +48,6 @@ public static void beforeClass() throws Exception {
ImmutableList.of("l_shipdate=" + HiveMetaClient.PARTITION_NULL_VALUE));
}
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
@Test
public void testPartialPartition1() throws Exception {
createAndRefreshMv("create materialized view partial_mv" +
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePreprocessorTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePreprocessorTest.java
index db7a0af1d4bc43..3a543f3d83a838 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePreprocessorTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewritePreprocessorTest.java
@@ -71,7 +71,8 @@
public class MvRewritePreprocessorTest extends MvRewriteTestBase {
@BeforeClass
- public static void before() throws Exception {
+ public static void beforeClass() throws Exception {
+ MvRewriteTestBase.beforeClass();
starRocksAssert.useTable("t0");
starRocksAssert.useTable("t1");
starRocksAssert.useTable("t2");
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewriteTestBase.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewriteTestBase.java
index 7d4bfe6fce866d..617555ad71f5f6 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewriteTestBase.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvRewriteTestBase.java
@@ -17,6 +17,8 @@
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.starrocks.analysis.StringLiteral;
import com.starrocks.analysis.TableName;
import com.starrocks.catalog.Database;
import com.starrocks.catalog.FunctionSet;
@@ -24,6 +26,7 @@
import com.starrocks.catalog.MvRefreshArbiter;
import com.starrocks.catalog.MvUpdateInfo;
import com.starrocks.catalog.Table;
+import com.starrocks.common.DdlException;
import com.starrocks.common.Pair;
import com.starrocks.common.util.UUIDUtil;
import com.starrocks.pseudocluster.PseudoCluster;
@@ -35,6 +38,7 @@
import com.starrocks.sql.ast.QueryRelation;
import com.starrocks.sql.ast.QueryStatement;
import com.starrocks.sql.ast.StatementBase;
+import com.starrocks.sql.ast.SystemVariable;
import com.starrocks.sql.optimizer.CachingMvPlanContextBuilder;
import com.starrocks.sql.optimizer.OptExpression;
import com.starrocks.sql.optimizer.Optimizer;
@@ -47,19 +51,21 @@
import com.starrocks.sql.parser.ParsingException;
import com.starrocks.sql.parser.SqlParser;
import com.starrocks.sql.plan.ExecPlan;
+import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.thrift.TExplainLevel;
import com.starrocks.utframe.StarRocksAssert;
import com.starrocks.utframe.UtFrameUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
+import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.rules.TemporaryFolder;
import java.sql.SQLException;
-import java.time.Instant;
import java.util.List;
import java.util.Set;
@@ -71,15 +77,12 @@ public class MvRewriteTestBase {
@ClassRule
public static TemporaryFolder temp = new TemporaryFolder();
- protected static long startSuiteTime = 0;
- protected long startCaseTime = 0;
+ protected static Set existedTables = Sets.newHashSet();
protected static String DB_NAME = "test";
@BeforeClass
public static void beforeClass() throws Exception {
- startSuiteTime = Instant.now().getEpochSecond();
-
CachingMvPlanContextBuilder.getInstance().rebuildCache();
PseudoCluster.getOrCreateWithRandomPort(true, 1);
GlobalStateMgr.getCurrentState().getTabletChecker().setInterval(500);
@@ -102,6 +105,16 @@ public static void tearDown() throws Exception {
}
}
+ @Before
+ public void before() {
+ PlanTestBase.collectTables(starRocksAssert, existedTables);
+ }
+
+ @After
+ public void after() throws Exception {
+ PlanTestBase.cleanup(starRocksAssert, existedTables);
+ }
+
public String getFragmentPlan(String sql) throws Exception {
String s = UtFrameUtils.getPlanAndFragment(connectContext, sql).second.
getExplainString(TExplainLevel.NORMAL);
@@ -277,4 +290,9 @@ public static String getAggFunction(String funcName, String aggArg) {
}
return funcName;
}
+
+ public static void setGlobalVariableVariable(String key, String value) throws DdlException {
+ GlobalStateMgr.getCurrentState().getVariableMgr().setSystemVariable(connectContext.getSessionVariable(),
+ new SystemVariable(key, new StringLiteral(value)), true);
+ }
}
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithHiveTableTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithHiveTableTest.java
index b72149fc460c0d..5b5b788f1724be 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithHiveTableTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithHiveTableTest.java
@@ -19,13 +19,9 @@
import com.starrocks.sql.plan.ConnectorPlanTestBase;
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.utframe.StarRocksAssert;
-import org.junit.After;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import java.time.Instant;
-
public class MvTransparentRewriteWithHiveTableTest extends MvRewriteTestBase {
private static MockedHiveMetadata mockedHiveMetadata;
@@ -38,16 +34,6 @@ public static void beforeClass() throws Exception {
getOptionalMetadata(MockedHiveMetadata.MOCKED_HIVE_CATALOG_NAME).get();
}
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private void withPartialScanMv(StarRocksAssert.ExceptionRunnable runner) {
starRocksAssert.withMaterializedView("CREATE MATERIALIZED VIEW mv0\n" +
"PARTITION BY (`l_shipdate`)\n" +
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java
index fbcf1daf984ac1..76a2b9198a7741 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentRewriteWithOlapTableTest.java
@@ -21,13 +21,10 @@
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.thrift.TExplainLevel;
import com.starrocks.utframe.StarRocksAssert;
-import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import java.time.Instant;
import java.util.Set;
public class MvTransparentRewriteWithOlapTableTest extends MvRewriteTestBase {
@@ -133,16 +130,6 @@ public static void beforeClass() throws Exception {
"DISTRIBUTED BY RANDOM\n";
}
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private void withPartialScanMv(StarRocksAssert.ExceptionRunnable runner) {
starRocksAssert.withTable(m1, () -> {
cluster.runSql("test", "insert into m1 values (1,1,1,1,1), (4,2,1,1,1);");
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteHiveTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteHiveTest.java
index d93ab3af1852a6..967819cb97e188 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteHiveTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteHiveTest.java
@@ -19,13 +19,9 @@
import com.starrocks.sql.plan.ConnectorPlanTestBase;
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.utframe.StarRocksAssert;
-import org.junit.After;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import java.time.Instant;
-
public class MvTransparentUnionRewriteHiveTest extends MvRewriteTestBase {
private static MockedHiveMetadata mockedHiveMetadata;
@@ -39,16 +35,6 @@ public static void beforeClass() throws Exception {
connectContext.getSessionVariable().setEnableMaterializedViewTransparentUnionRewrite(true);
}
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private void withPartialScanMv(StarRocksAssert.ExceptionRunnable runner) {
starRocksAssert.withMaterializedView("CREATE MATERIALIZED VIEW mv0\n" +
"PARTITION BY (`l_shipdate`)\n" +
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteOlapTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteOlapTest.java
index 5489ee89938438..f1bc23539ff687 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteOlapTest.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/optimizer/rule/transformation/materialization/MvTransparentUnionRewriteOlapTest.java
@@ -19,13 +19,10 @@
import com.starrocks.schema.MTable;
import com.starrocks.sql.plan.PlanTestBase;
import com.starrocks.utframe.StarRocksAssert;
-import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
-import java.time.Instant;
import java.util.Set;
public class MvTransparentUnionRewriteOlapTest extends MvRewriteTestBase {
@@ -67,16 +64,6 @@ public static void beforeClass() throws Exception {
connectContext.getSessionVariable().setEnableMaterializedViewTransparentUnionRewrite(true);
}
- @Before
- public void before() {
- startCaseTime = Instant.now().getEpochSecond();
- }
-
- @After
- public void after() throws Exception {
- PlanTestBase.cleanupEphemeralMVs(starRocksAssert, startCaseTime);
- }
-
private void withPartialScanMv(StarRocksAssert.ExceptionRunnable runner) {
starRocksAssert.withTable(m1, () -> {
cluster.runSql("test", "insert into m1 values (1,1,1,1), (4,2,1,1);");
diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/plan/PlanTestBase.java b/fe/fe-core/src/test/java/com/starrocks/sql/plan/PlanTestBase.java
index 46b217a1bc70e8..dcbd001cc4ba33 100644
--- a/fe/fe-core/src/test/java/com/starrocks/sql/plan/PlanTestBase.java
+++ b/fe/fe-core/src/test/java/com/starrocks/sql/plan/PlanTestBase.java
@@ -15,7 +15,7 @@
package com.starrocks.sql.plan;
import com.starrocks.catalog.Database;
-import com.starrocks.catalog.MaterializedView;
+import com.starrocks.catalog.Table;
import com.starrocks.common.FeConstants;
import com.starrocks.planner.TpchSQL;
import com.starrocks.qe.DefaultCoordinator;
@@ -35,6 +35,7 @@
import java.util.ArrayList;
import java.util.List;
+import java.util.Set;
import java.util.stream.Collectors;
public class PlanTestBase extends PlanTestNoneDBBase {
@@ -1010,14 +1011,30 @@ public static void afterAll() throws Exception {
afterClass();
}
- public static void cleanupEphemeralMVs(StarRocksAssert starRocksAssert, long startTime) throws Exception {
+ public static void collectTables(StarRocksAssert starRocksAssert, Set tables) {
String currentDb = starRocksAssert.getCtx().getDatabase();
if (StringUtils.isNotEmpty(currentDb)) {
Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(currentDb);
- for (MaterializedView mv : ListUtils.emptyIfNull(testDb.getMaterializedViews())) {
- if (startTime > 0 && mv.getCreateTime() > startTime) {
- starRocksAssert.dropMaterializedView(mv.getName());
- LOG.warn("cleanup mv after test case: {}", mv.getName());
+ tables.addAll(ListUtils.emptyIfNull(testDb.getTables()));
+ }
+ }
+
+ public static void cleanup(StarRocksAssert starRocksAssert, Set existedTables) throws Exception {
+ String currentDb = starRocksAssert.getCtx().getDatabase();
+ if (StringUtils.isNotEmpty(currentDb)) {
+ Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(currentDb);
+ List tables = ListUtils.emptyIfNull(testDb.getTables());
+ for (Table table : tables) {
+ if (!table.isNativeTableOrMaterializedView()) {
+ continue;
+ }
+ if (!existedTables.contains(table)) {
+ if (table.isNativeTable()) {
+ starRocksAssert.dropTable(table.getName());
+ } else {
+ starRocksAssert.dropMaterializedView(table.getName());
+ }
+ LOG.warn("cleanup table after test case: {}", table.getName());
}
}
if (CollectionUtils.isNotEmpty(testDb.getMaterializedViews())) {