This is an automated email from the ASF dual-hosted git repository.
morrysnow pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-3.1 by this push:
new 5b2d065ffd3 branch-3.1: [fix](regress) new thread should connect to
cluster.jdbcUrl in docker regression suite #53234 (#53289)
5b2d065ffd3 is described below
commit 5b2d065ffd3b67c07ace7ccb8fd433f069549991
Author: github-actions[bot]
<41898282+github-actions[bot]@users.noreply.github.com>
AuthorDate: Tue Jul 15 19:53:23 2025 +0800
branch-3.1: [fix](regress) new thread should connect to cluster.jdbcUrl in
docker regression suite #53234 (#53289)
Cherry-picked from #53234
Co-authored-by: yujun <[email protected]>
---
.../org/apache/doris/regression/Config.groovy | 11 ----
.../doris/regression/action/WaitForAction.groovy | 8 ++-
.../org/apache/doris/regression/suite/Suite.groovy | 67 ++++++++++++----------
.../doris/regression/suite/SuiteCluster.groovy | 2 +
.../doris/regression/suite/SuiteContext.groovy | 27 ++++++++-
.../cloud_p0/multi_cluster/test_auto_start.groovy | 6 +-
.../test_fe_tablet_same_backend.groovy | 8 +--
.../multi_cluster/test_multi_follower.groovy | 6 +-
.../multi_cluster/test_no_cluster_hits.groovy | 2 +-
.../cloud_p0/multi_cluster/test_rebalance.groovy | 12 ++--
.../multi_cluster/test_warmup_rebalance.groovy | 2 +-
.../node_mgr/test_cloud_decommission.groovy | 10 ++--
.../node_mgr/test_sql_mode_node_mgr.groovy | 8 +--
.../test_retry_e-230_async_mtmv_job.groovy | 4 +-
.../test_clean_tablet_when_drop_force_table.groovy | 2 +-
.../test_clean_tablet_when_rebalance.groovy | 4 +-
.../suites/demo_p0/docker_action.groovy | 3 +-
...est_tablet_state_change_in_publish_phase.groovy | 2 +-
..._local_multi_segments_re_calc_in_publish.groovy | 2 +-
.../schema_change_p0/test_abort_txn_by_be.groovy | 2 +-
.../schema_change_p0/test_abort_txn_by_fe.groovy | 2 +-
regression-test/suites/trash_p0/clean_trash.groovy | 2 +-
22 files changed, 108 insertions(+), 84 deletions(-)
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
index da76c682c57..ceffa3529a2 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/Config.groovy
@@ -1060,17 +1060,6 @@ class Config {
return DriverManager.getConnection(jdbcUrl, 'root', '')
}
- Connection getConnectionByDbName(String dbName) {
- String dbUrl = getConnectionUrlByDbName(dbName)
- tryCreateDbIfNotExist(dbName)
- log.info("connect to ${dbUrl}".toString())
- return DriverManager.getConnection(dbUrl, jdbcUser, jdbcPassword)
- }
-
- String getConnectionUrlByDbName(String dbName) {
- return buildUrlWithDb(jdbcUrl, dbName)
- }
-
Connection getConnectionByArrowFlightSqlDbName(String dbName) {
Class.forName("org.apache.arrow.driver.jdbc.ArrowFlightJdbcDriver")
String arrowFlightSqlHost = otherConfigs.get("extArrowFlightSqlHost")
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/WaitForAction.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/WaitForAction.groovy
index 54ba7bb631b..f54d43758e1 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/WaitForAction.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/WaitForAction.groovy
@@ -62,8 +62,12 @@ class WaitForAction implements SuiteAction {
if (forRollUp) {
num = 8
}
- Awaitility.await().atMost(time,
TimeUnit.SECONDS).with().pollDelay(100, TimeUnit.MILLISECONDS).and()
- .pollInterval(100, TimeUnit.MILLISECONDS).await().until(() -> {
+ Awaitility
+ .with().pollInSameThread()
+ .await()
+ .atMost(time, TimeUnit.SECONDS)
+ .with().pollDelay(100, TimeUnit.MILLISECONDS).and()
+ .pollInterval(100, TimeUnit.MILLISECONDS).await().until({
log.info("sql is :\n${sql}")
def (result, meta) =
JdbcUtils.executeToList(context.getConnection(), sql)
String res = result.get(0).get(num)
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
index 85ec795cd0d..e7d038a44eb 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/Suite.groovy
@@ -23,8 +23,6 @@ import com.amazonaws.client.builder.AwsClientBuilder
import com.amazonaws.services.s3.AmazonS3
import com.amazonaws.services.s3.AmazonS3ClientBuilder
-import static java.util.concurrent.TimeUnit.SECONDS
-
import com.google.common.base.Strings
import com.google.common.collect.ImmutableList
import com.google.common.collect.Maps
@@ -278,13 +276,22 @@ class Suite implements GroovyInterceptable {
return context.connect(user, password,
connInfo.conn.getMetaData().getURL(), actionSupplier)
}
- public void dockerAwaitUntil(int atMostSeconds, int intervalSecond = 1,
Closure actionSupplier) {
- def connInfo = context.threadLocalConn.get()
- Awaitility.await().atMost(atMostSeconds,
SECONDS).pollInterval(intervalSecond, SECONDS).until(
- {
- connect(connInfo.username, connInfo.password,
connInfo.conn.getMetaData().getURL(), actionSupplier)
- }
- )
+ // delete 'dockerAwaitUntil', should call 'Awaitility.await()...' directly
or use 'awaitUntil(..., f)'
+ // public void dockerAwaitUntil(int atMostSeconds, int intervalSecond = 1,
Closure actionSupplier) {
+ // def connInfo = context.threadLocalConn.get()
+ // Awaitility.await().atMost(atMostSeconds,
SECONDS).pollInterval(intervalSecond, SECONDS).until(
+ // {
+ // connect(connInfo.username, connInfo.password,
connInfo.conn.getMetaData().getURL(), actionSupplier)
+ // }
+ // )
+ // }
+ public void awaitUntil(int atMostSeconds, double intervalSecond = 1,
Closure actionSupplier) {
+ Awaitility
+ .with().pollInSameThread()
+ .await()
+ .atMost(atMostSeconds, TimeUnit.SECONDS)
+ .pollInterval((int) (1000 * intervalSecond), TimeUnit.MILLISECONDS)
+ .until(actionSupplier)
}
// more explaination can see example file: demo_p0/docker_action.groovy
@@ -319,6 +326,7 @@ class Suite implements GroovyInterceptable {
private void dockerImpl(ClusterOptions options, boolean isCloud, Closure
actionSupplier) throws Exception {
logger.info("=== start run suite {} in {} mode. ===", name, (isCloud ?
"cloud" : "not_cloud"))
+ def originConnection = context.threadLocalConn.get()
try {
cluster.destroy(true)
cluster.init(options, isCloud)
@@ -345,18 +353,18 @@ class Suite implements GroovyInterceptable {
// wait be report
Thread.sleep(5000)
- def url = String.format(
+ def jdbcUrl = String.format(
"jdbc:mysql://%s:%s/?useLocalSessionState=true&allowLoadLocalInfile=false",
fe.host, fe.queryPort)
- def conn = DriverManager.getConnection(url, user, password)
- def sql = "CREATE DATABASE IF NOT EXISTS " + context.dbName
- logger.info("try create database if not exists {}", context.dbName)
- JdbcUtils.executeToList(conn, sql)
-
- url = Config.buildUrlWithDb(url, context.dbName)
- logger.info("connect to docker cluster: suite={}, url={}", name,
url)
- connect(user, password, url, actionSupplier)
+ cluster.jdbcUrl = jdbcUrl
+ context.threadLocalConn.remove()
+ actionSupplier.call()
} finally {
+ if (originConnection == null) {
+ context.threadLocalConn.remove()
+ } else {
+ context.threadLocalConn.set(originConnection)
+ }
if (!context.config.dockerEndNoKill) {
cluster.destroy(context.config.dockerEndDeleteFiles)
}
@@ -650,7 +658,7 @@ class Suite implements GroovyInterceptable {
}
String getCurDbConnectUrl() {
- return context.config.getConnectionUrlByDbName(getCurDbName())
+ return Config.buildUrlWithDb(context.getJdbcUrl(), getCurDbName())
}
long getDbId() {
@@ -1893,8 +1901,7 @@ class Suite implements GroovyInterceptable {
void waitAddFeFinished(String host, int port) {
logger.info("waiting for ${host}:${port}")
- Awaitility.await().atMost(60, TimeUnit.SECONDS).with().pollDelay(100,
TimeUnit.MILLISECONDS).and()
- .pollInterval(100, TimeUnit.MILLISECONDS).await().until(() -> {
+ awaitUntil(60, 0.1) {
def frontends = getFrontendIpEditlogPort()
logger.info("frontends ${frontends}")
boolean matched = false
@@ -1906,12 +1913,12 @@ class Suite implements GroovyInterceptable {
}
}
return matched;
- });
+ }
}
void waitDropFeFinished(String host, int port) {
- Awaitility.await().atMost(60, TimeUnit.SECONDS).with().pollDelay(100,
TimeUnit.MILLISECONDS).and()
- .pollInterval(100, TimeUnit.MILLISECONDS).await().until(() -> {
+ logger.info("waiting drop fe ${host}:${port}");
+ awaitUntil(60, 0.1) {
def frontends = getFrontendIpEditlogPort()
boolean matched = false
for (frontend: frontends) {
@@ -1920,13 +1927,12 @@ class Suite implements GroovyInterceptable {
}
}
return !matched;
- });
+ }
}
void waitAddBeFinished(String host, int port) {
logger.info("waiting ${host}:${port} added");
- Awaitility.await().atMost(60, TimeUnit.SECONDS).with().pollDelay(100,
TimeUnit.MILLISECONDS).and()
- .pollInterval(100, TimeUnit.MILLISECONDS).await().until(() -> {
+ awaitUntil(60, 0.1) {
def ipList = [:]
def portList = [:]
getBackendIpHeartbeatPort(ipList, portList)
@@ -1937,12 +1943,11 @@ class Suite implements GroovyInterceptable {
}
}
return matched;
- });
+ }
}
void waitDropBeFinished(String host, int port) {
- Awaitility.await().atMost(60, TimeUnit.SECONDS).with().pollDelay(100,
TimeUnit.MILLISECONDS).and()
- .pollInterval(100, TimeUnit.MILLISECONDS).await().until(() -> {
+ awaitUntil(60, 0.1) {
def ipList = [:]
def portList = [:]
getBackendIpHeartbeatPort(ipList, portList)
@@ -1953,7 +1958,7 @@ class Suite implements GroovyInterceptable {
}
}
return !matched;
- });
+ }
}
void waiteCreateTableFinished(String tableName) {
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
index 952cba30a88..fabddbd43ba 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteCluster.groovy
@@ -286,6 +286,8 @@ class SuiteCluster {
static final Logger logger = LoggerFactory.getLogger(this.class)
+ // dockerImpl() will set jdbcUrl
+ String jdbcUrl = ""
final String name
final Config config
private boolean running
diff --git
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
index 076566651e5..b303d1fbcd6 100644
---
a/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
+++
b/regression-test/framework/src/main/groovy/org/apache/doris/regression/suite/SuiteContext.groovy
@@ -145,7 +145,7 @@ class SuiteContext implements Closeable {
def threadConnInfo = threadLocalConn.get()
if (threadConnInfo == null) {
threadConnInfo = new ConnectionInfo()
- threadConnInfo.conn = config.getConnectionByDbName(dbName)
+ threadConnInfo.conn = getConnectionByDbName(dbName)
threadConnInfo.username = config.jdbcUser
threadConnInfo.password = config.jdbcPassword
threadLocalConn.set(threadConnInfo)
@@ -153,6 +153,31 @@ class SuiteContext implements Closeable {
return threadConnInfo.conn
}
+ Connection getConnectionByDbName(String dbName) {
+ def jdbcUrl = getJdbcUrl()
+ def jdbcConn = DriverManager.getConnection(jdbcUrl, config.jdbcUser,
config.jdbcPassword)
+ try {
+ String sql = "CREATE DATABASE IF NOT EXISTS ${dbName}"
+ log.info("Try to create db, sql: ${sql}".toString())
+ if (!config.dryRun) {
+ jdbcConn.withCloseable { conn -> JdbcUtils.executeToList(conn,
sql) }
+ }
+ } catch (Throwable t) {
+ throw new IllegalStateException("Create database failed, jdbcUrl:
${jdbcUrl}", t)
+ }
+ def dbUrl = Config.buildUrlWithDb(jdbcUrl, dbName)
+ log.info("connect to ${dbUrl}".toString())
+ return DriverManager.getConnection(dbUrl, config.jdbcUser,
config.jdbcPassword)
+ }
+
+ String getJdbcUrl() {
+ if (cluster.isRunning()) {
+ return cluster.jdbcUrl
+ } else {
+ return config.jdbcUrl
+ }
+ }
+
// like getConnection, but connect to FE master
Connection getMasterConnection() {
def threadConnInfo = threadLocalMasterConn.get()
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_auto_start.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_auto_start.groovy
index ceb8969b721..a02481dcebf 100644
--- a/regression-test/suites/cloud_p0/multi_cluster/test_auto_start.groovy
+++ b/regression-test/suites/cloud_p0/multi_cluster/test_auto_start.groovy
@@ -126,7 +126,7 @@ suite('test_auto_start_in_cloud', 'multi_cluster, docker') {
// cloud control set cluster status SUSPENDED
set_cluster_status(uniqueId, cloudClusterId, "SUSPENDED", ms)
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
tag = getCloudBeTagByName(clusterName)
logger.info("tag = {}", tag)
jsonObject = jsonSlurper.parseText(tag)
@@ -154,7 +154,7 @@ suite('test_auto_start_in_cloud', 'multi_cluster, docker') {
// cloud control
future2 = thread {
// check cluster "TO_RESUME"
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
tag = getCloudBeTagByName(clusterName)
logger.info("tag = {}", tag)
jsonObject = jsonSlurper.parseText(tag)
@@ -177,7 +177,7 @@ suite('test_auto_start_in_cloud', 'multi_cluster, docker') {
// add 1 nodes, check it status NORMAL
cluster.addBackend(1, null)
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
result = sql """SHOW BACKENDS"""
result.size() == 4
}
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_fe_tablet_same_backend.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_fe_tablet_same_backend.groovy
index 973492bd4b5..5ecc610bf5a 100644
---
a/regression-test/suites/cloud_p0/multi_cluster/test_fe_tablet_same_backend.groovy
+++
b/regression-test/suites/cloud_p0/multi_cluster/test_fe_tablet_same_backend.groovy
@@ -98,7 +98,7 @@ suite('test_fe_tablet_same_backend', 'multi_cluster,docker') {
}
def checkAllTable = { isAllBeAliveOrDeadLong ->
- dockerAwaitUntil(60) {
+ awaitUntil(60) {
checkAllTableImpl(isAllBeAliveOrDeadLong, true)
}
checkAllTableImpl(isAllBeAliveOrDeadLong, false)
@@ -128,7 +128,7 @@ suite('test_fe_tablet_same_backend',
'multi_cluster,docker') {
checkAllTable(true)
cluster.stopBackends(choseDeadBeIndex)
- dockerAwaitUntil(60) {
+ awaitUntil(60) {
def chosenBe = cluster.getBeByIndex(choseDeadBeIndex)
!chosenBe.alive
}
@@ -144,13 +144,13 @@ suite('test_fe_tablet_same_backend',
'multi_cluster,docker') {
def choseRestartFeIndex = cluster.getOneFollowerFe().index
cluster.stopFrontends(choseRestartFeIndex)
- dockerAwaitUntil(60) {
+ awaitUntil(60) {
def chosenFe = cluster.getFeByIndex(choseRestartFeIndex)
!chosenFe.alive
}
cluster.startFrontends(choseRestartFeIndex)
- dockerAwaitUntil(60) {
+ awaitUntil(60) {
def chosenFe = cluster.getFeByIndex(choseRestartFeIndex)
chosenFe.alive
}
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_multi_follower.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_multi_follower.groovy
index 792e2a98b24..1b7f33fdb1d 100644
--- a/regression-test/suites/cloud_p0/multi_cluster/test_multi_follower.groovy
+++ b/regression-test/suites/cloud_p0/multi_cluster/test_multi_follower.groovy
@@ -77,7 +77,7 @@ suite('test_multi_followr_in_cloud', 'multi_cluster, docker')
{
def toDropUniqueId
// add new follower
cluster.addFrontend(1, true)
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
def ret = sql """SHOW FRONTENDS"""
log.info("show frontends: {}", ret)
ret.size() == 4
@@ -119,7 +119,7 @@ suite('test_multi_followr_in_cloud', 'multi_cluster,
docker') {
drop_node(toDropUniqueId, toDropIP, 0,
toDropPort, toDropType, feClusterName, feClusterId, ms)
- dockerAwaitUntil(50) {
+ awaitUntil(50) {
def ret = sql """SHOW FRONTENDS"""
log.info("show frontends: {}", ret)
ret.size() == 3
@@ -155,7 +155,7 @@ suite('test_multi_followr_in_cloud', 'multi_cluster,
docker') {
drop_node(toDropUniqueId, toDropIP, 0,
toDropPort, toDropType, feClusterName, feClusterId, ms)
- dockerAwaitUntil(50) {
+ awaitUntil(50) {
def ret = sql """SHOW FRONTENDS"""
log.info("show frontends: {}", ret)
ret.size() == 3
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_no_cluster_hits.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_no_cluster_hits.groovy
index e1761a061ba..193824a6540 100644
--- a/regression-test/suites/cloud_p0/multi_cluster/test_no_cluster_hits.groovy
+++ b/regression-test/suites/cloud_p0/multi_cluster/test_no_cluster_hits.groovy
@@ -145,7 +145,7 @@ suite('test_no_cluster_hits', 'multi_cluster, docker') {
logger.info("ms addr={}, port={}", ms.host, ms.httpPort)
drop_cluster(currentCluster.cluster, cloudClusterId, ms)
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
result = sql_return_maparray """show clusters"""
logger.info("show cluster2 : {}", result)
result.size() == 0
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_rebalance.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_rebalance.groovy
index c15157308c4..83a8dc336de 100644
--- a/regression-test/suites/cloud_p0/multi_cluster/test_rebalance.groovy
+++ b/regression-test/suites/cloud_p0/multi_cluster/test_rebalance.groovy
@@ -76,13 +76,13 @@ suite('test_rebalance_in_cloud', 'multi_cluster,docker') {
// add a be
cluster.addBackend(1, null)
- dockerAwaitUntil(30) {
+ awaitUntil(30) {
def bes = sql """show backends"""
log.info("bes: {}", bes)
bes.size() == 2
}
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
def ret = sql """ADMIN SHOW REPLICA DISTRIBUTION FROM
table100"""
log.info("replica distribution table100: {}", ret)
ret.size() == 2
@@ -102,7 +102,7 @@ suite('test_rebalance_in_cloud', 'multi_cluster,docker') {
}
}
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
def ret = sql """ADMIN SHOW REPLICA DISTRIBUTION FROM table_p2
PARTITION(p1992)"""
log.info("replica distribution table_p2: {}", ret)
ret.size() == 2
@@ -181,13 +181,13 @@ suite('test_rebalance_in_cloud', 'multi_cluster,docker') {
sql """admin set frontend
config("enable_cloud_warm_up_for_rebalance"="true")"""
// test rebalance thread still work
- dockerAwaitUntil(30) {
+ awaitUntil(30) {
def bes = sql """show backends"""
log.info("bes: {}", bes)
bes.size() == 3
}
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
def ret = sql """ADMIN SHOW REPLICA DISTRIBUTION FROM
table100"""
log.info("replica distribution table100: {}", ret)
ret.size() == 3
@@ -205,7 +205,7 @@ suite('test_rebalance_in_cloud', 'multi_cluster,docker') {
sleep(1 * 1000)
}
GetDebugPoint().disableDebugPointForAllFEs("CloudTabletRebalancer.checkInflghtWarmUpCacheAsync.beNull");
- dockerAwaitUntil(10) {
+ awaitUntil(10) {
def ret = sql_return_maparray """ADMIN SHOW REPLICA
DISTRIBUTION FROM table100"""
log.info("replica distribution table100: {}", ret)
ret.any { row ->
diff --git
a/regression-test/suites/cloud_p0/multi_cluster/test_warmup_rebalance.groovy
b/regression-test/suites/cloud_p0/multi_cluster/test_warmup_rebalance.groovy
index b7de0c50161..aa35a70e121 100644
--- a/regression-test/suites/cloud_p0/multi_cluster/test_warmup_rebalance.groovy
+++ b/regression-test/suites/cloud_p0/multi_cluster/test_warmup_rebalance.groovy
@@ -98,7 +98,7 @@ suite('test_warmup_rebalance_in_cloud', 'multi_cluster,
docker') {
INSERT INTO table100 VALUES (1, 1, 100);
"""
- dockerAwaitUntil(5) {
+ awaitUntil(5) {
ret = sql """ADMIN SHOW REPLICA DISTRIBUTION FROM table100"""
log.info("replica distribution table100: {}", ret)
ret.size() == 5
diff --git
a/regression-test/suites/cloud_p0/node_mgr/test_cloud_decommission.groovy
b/regression-test/suites/cloud_p0/node_mgr/test_cloud_decommission.groovy
index a8173108f0b..dfe46054e24 100644
--- a/regression-test/suites/cloud_p0/node_mgr/test_cloud_decommission.groovy
+++ b/regression-test/suites/cloud_p0/node_mgr/test_cloud_decommission.groovy
@@ -25,7 +25,7 @@ suite("cloud_decommission", 'p0, docker') {
def checkStatus = { ms, decommissionBeUniqueId, decommissionBe ->
boolean found = false
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
found = false
def resp = get_cluster.call(decommissionBeUniqueId, ms)
resp.each { cluster ->
@@ -43,7 +43,7 @@ suite("cloud_decommission", 'p0, docker') {
def dropAndCheckBe = { host, heartbeatPort ->
sql """ ALTER SYSTEM DROPP BACKEND "${host}:${heartbeatPort}" """
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
def result = sql_return_maparray """ SHOW BACKENDS """
log.info("show backends result {}", result)
def ret = result.find {it.Host == host && it.HeartbeatPort ==
heartbeatPort}
@@ -60,7 +60,7 @@ suite("cloud_decommission", 'p0, docker') {
def result = sql """ ADMIN SHOW REPLICA DISTRIBUTION FROM
decommission_table """
assertEquals(result.size(), beNum)
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
result = sql_return_maparray """ ADMIN SHOW REPLICA DISTRIBUTION
FROM decommission_table """
if (beNum == 3) {
result.every { Integer.valueOf((String) it.ReplicaNum) >= 15
&& Integer.valueOf((String) it.ReplicaNum) <= 17 }
@@ -95,7 +95,7 @@ suite("cloud_decommission", 'p0, docker') {
d_node.call(firstDecommissionBeUniqueId, firstDecommissionBe.Host,
firstDecommissionBe.HeartbeatPort,
firstDecommissionBeClusterName,
firstDecommissionBeCloudClusterId, ms)
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
result = sql_return_maparray """ ADMIN SHOW REPLICA DISTRIBUTION
FROM decommission_table """
result.any { Integer.valueOf((String) it.ReplicaNum) == 0 }
}
@@ -127,7 +127,7 @@ suite("cloud_decommission", 'p0, docker') {
result = sql """ ADMIN SHOW REPLICA DISTRIBUTION FROM
decommission_table """
assertEquals(result.size(), beNum - 1)
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
result = sql_return_maparray """ ADMIN SHOW REPLICA DISTRIBUTION
FROM decommission_table """
log.info("show replica result {}", result)
def ret = result.findAll { Integer.valueOf((String) it.ReplicaNum)
== 0 }
diff --git
a/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy
b/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy
index b29cbef0f68..0d792950105 100644
--- a/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy
+++ b/regression-test/suites/cloud_p0/node_mgr/test_sql_mode_node_mgr.groovy
@@ -329,7 +329,7 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') {
logger.info("Dropping frontend index: {}, remove it from docker
compose", dropFeInx)
// Wait for the frontend to be fully dropped
- dockerAwaitUntil(300) {
+ awaitUntil(300) {
reconnectFe()
def currentFrontends = sql_return_maparray("SHOW FRONTENDS")
currentFrontends.size() == frontends.size() - 1
@@ -365,7 +365,7 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') {
logger.info("Adding back frontend: {}", showFes)
// Wait for the frontend to be fully added back
- dockerAwaitUntil(300, 5) {
+ awaitUntil(300, 5) {
def updatedFrontends = sql_return_maparray("SHOW FRONTENDS")
updatedFrontends.size() == frontends.size()
}
@@ -404,7 +404,7 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') {
reconnectFe()
// Wait for the frontend to be fully dropped
- dockerAwaitUntil(300, 5) {
+ awaitUntil(300, 5) {
def updatedFrontends = sql_return_maparray("SHOW FRONTENDS")
!updatedFrontends.any { it['Host'] == frontendToDrop.Host &&
it['EditLogPort'] == frontendToDrop.EditLogPort }
}
@@ -414,7 +414,7 @@ suite('test_sql_mode_node_mgr', 'multi_cluster,docker,p1') {
addList = cluster.addFrontend(1, true)
logger.info("Up a new frontend, addList: {}", addList)
- dockerAwaitUntil(300, 5) {
+ awaitUntil(300, 5) {
def updatedFrontends = sql_return_maparray("SHOW FRONTENDS")
updatedFrontends.size() == 3
}
diff --git
a/regression-test/suites/cloud_p0/query_retry/test_retry_e-230_async_mtmv_job.groovy
b/regression-test/suites/cloud_p0/query_retry/test_retry_e-230_async_mtmv_job.groovy
index 67bac61e8d9..f7ddc494ffc 100644
---
a/regression-test/suites/cloud_p0/query_retry/test_retry_e-230_async_mtmv_job.groovy
+++
b/regression-test/suites/cloud_p0/query_retry/test_retry_e-230_async_mtmv_job.groovy
@@ -86,7 +86,7 @@ suite("test_retry_e-230_async_mtmv_job", 'p0, docker') {
"""
def firstTaskId = getMvTaskId(0)
def firstTask
- dockerAwaitUntil(100) {
+ awaitUntil(100) {
firstTask = getMvTask(firstTaskId)
logger.info("firstTask = {}, Status = {}, bool = {}",
firstTask, firstTask.Status, firstTask.Status[0] == "FAILED")
firstTask.Status[0] as String == "FAILED" as String
@@ -111,7 +111,7 @@ suite("test_retry_e-230_async_mtmv_job", 'p0, docker') {
def futrue2 = thread {
def secondTaskId = getMvTaskId(1)
def secondTask
- dockerAwaitUntil(100, 5) {
+ awaitUntil(100, 5) {
secondTask = getMvTask(secondTaskId)
logger.info("secondTask = {}", secondTask)
secondTask.Status[0] == "SUCCESS"
diff --git
a/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_drop_force_table.groovy
b/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_drop_force_table.groovy
index a65f59f85a1..366ef5fa103 100644
---
a/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_drop_force_table.groovy
+++
b/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_drop_force_table.groovy
@@ -127,7 +127,7 @@ suite('test_clean_tablet_when_drop_force_table', 'docker') {
}
def start = System.currentTimeMillis() / 1000
// tablet can't find in be
- dockerAwaitUntil(500) {
+ awaitUntil(500) {
def beTablets =
getTabletAndBeHostFromBe(cluster.getAllBackends()).keySet()
logger.info("before drop tablets {}, after tablets {}",
beforeGetFromFe, beTablets)
beforeGetFromFe.keySet().every {
!getTabletAndBeHostFromBe(cluster.getAllBackends()).containsKey(it) }
diff --git
a/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_rebalance.groovy
b/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_rebalance.groovy
index caac5b73cfd..9a91707731d 100644
---
a/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_rebalance.groovy
+++
b/regression-test/suites/cloud_p0/tablets/test_clean_tablet_when_rebalance.groovy
@@ -84,7 +84,7 @@ suite('test_clean_tablet_when_rebalance', 'docker') {
}
cluster.stopBackends(choseDeadBeIndex)
- dockerAwaitUntil(50) {
+ awaitUntil(50) {
def showTablets = sql_return_maparray("SHOW TABLETS FROM ${table}")
def bes = showTablets
.collect { it.BackendId }
@@ -109,7 +109,7 @@ suite('test_clean_tablet_when_rebalance', 'docker') {
def afterGetFromBe = getTabletAndBeHostFromBe(cluster.getAllBackends())
logger.info("after stop one be, rehash fe tablets {}, be tablets {}",
afterGetFromFe, afterGetFromBe)
- dockerAwaitUntil(50) {
+ awaitUntil(50) {
def showTablets = sql_return_maparray("SHOW TABLETS FROM ${table}")
def bes = showTablets
.collect { it.BackendId }
diff --git a/regression-test/suites/demo_p0/docker_action.groovy
b/regression-test/suites/demo_p0/docker_action.groovy
index 7bea52989b5..7e111b48285 100644
--- a/regression-test/suites/demo_p0/docker_action.groovy
+++ b/regression-test/suites/demo_p0/docker_action.groovy
@@ -38,8 +38,7 @@ import org.apache.doris.regression.suite.ClusterOptions
// NOTICE:
// 1. Need add 'docker' to suite's group, and don't add 'nonConcurrent' to it;
// 2. In docker closure:
-// a. Don't use 'Awaitility.await()...until(f)', but use
'dockerAwaitUntil(..., f)';
-// b. Don't use java Thread, but use regress framework's ThreadAction(see
example demo_p0/thread_action.groovy);
+// a. remove function dockerAwaitUntil(...), should use
'Awaitility.await()...until(f)' directly or use 'awaitUntil(...)';
// 3. No need to use code ` if (isCloudMode()) { return } ` in docker suites,
// instead should use `ClusterOptions.cloudMode = true/false` is enough.
// Because when run docker suite without an external doris cluster, if suite
use code `isCloudMode()`, it need specific -runMode=cloud/not_cloud.
diff --git
a/regression-test/suites/fault_injection_p0/cloud/test_tablet_state_change_in_publish_phase.groovy
b/regression-test/suites/fault_injection_p0/cloud/test_tablet_state_change_in_publish_phase.groovy
index 6b7102ed243..c3c245ce37e 100644
---
a/regression-test/suites/fault_injection_p0/cloud/test_tablet_state_change_in_publish_phase.groovy
+++
b/regression-test/suites/fault_injection_p0/cloud/test_tablet_state_change_in_publish_phase.groovy
@@ -135,7 +135,7 @@ suite("test_tablet_state_change_in_publish_phase",
"docker") {
// let sc finish
GetDebugPoint().disableDebugPointForAllBEs("CloudSchemaChangeJob.process_alter_tablet.sleep")
- dockerAwaitUntil(30) {
+ awaitUntil(30) {
def res = sql_return_maparray """ SHOW ALTER TABLE COLUMN
WHERE TableName='${table1}' ORDER BY createtime DESC LIMIT 1 """
logger.info("alter status: ${res}")
res[0].State as String == "FINISHED"
diff --git
a/regression-test/suites/fault_injection_p0/test_local_multi_segments_re_calc_in_publish.groovy
b/regression-test/suites/fault_injection_p0/test_local_multi_segments_re_calc_in_publish.groovy
index ff1d9531fb5..c5fe06b19c3 100644
---
a/regression-test/suites/fault_injection_p0/test_local_multi_segments_re_calc_in_publish.groovy
+++
b/regression-test/suites/fault_injection_p0/test_local_multi_segments_re_calc_in_publish.groovy
@@ -155,7 +155,7 @@ suite("test_local_multi_segments_re_calc_in_publish",
"docker") {
Thread.sleep(1000)
do_streamload_2pc_commit(txnId)
- dockerAwaitUntil(30) {
+ awaitUntil(30) {
def result = sql_return_maparray "show transaction from
${dbName} where id = ${txnId}"
result[0].TransactionStatus as String == "VISIBLE"
}
diff --git
a/regression-test/suites/schema_change_p0/test_abort_txn_by_be.groovy
b/regression-test/suites/schema_change_p0/test_abort_txn_by_be.groovy
index b1af2b1fcad..127465d66d8 100644
--- a/regression-test/suites/schema_change_p0/test_abort_txn_by_be.groovy
+++ b/regression-test/suites/schema_change_p0/test_abort_txn_by_be.groovy
@@ -98,7 +98,7 @@ suite('test_abort_txn_by_be', 'docker') {
sleep 3000
} else {
def dbId = getDbId()
- dockerAwaitUntil(20, {
+ awaitUntil(20, {
def txns = sql_return_maparray("show proc
'/transactions/${dbId}/running'")
txns.size() > 0
})
diff --git
a/regression-test/suites/schema_change_p0/test_abort_txn_by_fe.groovy
b/regression-test/suites/schema_change_p0/test_abort_txn_by_fe.groovy
index d93e8a203e3..91641ae483c 100644
--- a/regression-test/suites/schema_change_p0/test_abort_txn_by_fe.groovy
+++ b/regression-test/suites/schema_change_p0/test_abort_txn_by_fe.groovy
@@ -67,7 +67,7 @@ suite('test_abort_txn_by_fe', 'docker') {
sleep 6000
} else {
def dbId = getDbId()
- dockerAwaitUntil(20, {
+ awaitUntil(20, {
def txns = sql_return_maparray("show proc
'/transactions/${dbId}/running'")
txns.any { it.Label == loadLabel }
})
diff --git a/regression-test/suites/trash_p0/clean_trash.groovy
b/regression-test/suites/trash_p0/clean_trash.groovy
index 525e8054ce5..4c03f43b807 100644
--- a/regression-test/suites/trash_p0/clean_trash.groovy
+++ b/regression-test/suites/trash_p0/clean_trash.groovy
@@ -40,7 +40,7 @@ suite("test_clean_trash", "docker") {
def checkFunc = { boolean trashZero ->
def succ = false
- dockerAwaitUntil(300) {
+ awaitUntil(300) {
def bes = sql_return_maparray """show backends"""
succ = bes.every {
if (trashZero) {
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]