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]


Reply via email to