branch-2.1: [feat](regression) inject debug points need run in nonConcurrent or docker suites #49581 (#49618)

Cherry-picked from #49581, #40259
This commit is contained in:
yujun
2025-03-30 15:43:58 +08:00
committed by GitHub
parent ec25f7573e
commit ed8531b99a
32 changed files with 185 additions and 78 deletions

View File

@ -108,6 +108,7 @@ class Config {
public InetSocketAddress metaServiceHttpInetSocketAddress
public Integer parallel
public Integer suiteParallel
public Integer dockerSuiteParallel
public Integer actionParallel
public Integer times
public boolean withOutLoadData
@ -279,6 +280,7 @@ class Config {
config.forceGenerateOutputFile = cmd.hasOption(forceGenOutOpt)
config.parallel = Integer.parseInt(cmd.getOptionValue(parallelOpt, "10"))
config.suiteParallel = Integer.parseInt(cmd.getOptionValue(suiteParallelOpt, "10"))
config.dockerSuiteParallel = Integer.parseInt(cmd.getOptionValue(dockerSuiteParallelOpt, "1"))
config.actionParallel = Integer.parseInt(cmd.getOptionValue(actionParallelOpt, "10"))
config.times = Integer.parseInt(cmd.getOptionValue(timesOpt, "1"))
config.randomOrder = cmd.hasOption(randomOrderOpt)
@ -606,6 +608,11 @@ class Config {
log.info("Set suiteParallel to 1 because not specify.".toString())
}
if (config.dockerSuiteParallel == null) {
config.dockerSuiteParallel = 1
log.info("Set dockerSuiteParallel to 1 because not specify.".toString())
}
if (config.actionParallel == null) {
config.actionParallel = 10
log.info("Set actionParallel to 10 because not specify.".toString())

View File

@ -61,6 +61,7 @@ class ConfigOptions {
static Option forceGenOutOpt
static Option parallelOpt
static Option suiteParallelOpt
static Option dockerSuiteParallelOpt
static Option actionParallelOpt
static Option randomOrderOpt
static Option stopWhenFailOpt
@ -352,6 +353,14 @@ class ConfigOptions {
.longOpt("suiteParallel")
.desc("the num of threads running for suites")
.build()
dockerSuiteParallelOpt = Option.builder("dockerSuiteParallel")
.argName("dockerSuiteParallel")
.required(false)
.hasArg(true)
.type(String.class)
.longOpt("dockerSuiteParallel")
.desc("the num of threads running for docker suites")
.build()
actionParallelOpt = Option.builder("actionParallel")
.argName("parallel")
.required(false)
@ -435,6 +444,7 @@ class ConfigOptions {
.addOption(forceGenOutOpt)
.addOption(parallelOpt)
.addOption(suiteParallelOpt)
.addOption(dockerSuiteParallelOpt)
.addOption(actionParallelOpt)
.addOption(randomOrderOpt)
.addOption(stopWhenFailOpt)

View File

@ -52,16 +52,22 @@ import java.util.function.Predicate
@CompileStatic
class RegressionTest {
static enum GroupExecType {
NORMAL,
SINGLE, // group contains nonConcurrent
DOCKER, // group contains docker
}
static ClassLoader classloader
static CompilerConfiguration compileConfig
static GroovyShell shell
static ExecutorService scriptExecutors
static ExecutorService suiteExecutors
static ExecutorService singleSuiteExecutors
static ExecutorService actionExecutors
static Map<GroupExecType, ExecutorService> suiteExecutors
static ThreadLocal<Integer> threadLoadedClassNum = new ThreadLocal<>()
static final int cleanLoadedClassesThreshold = 20
static String nonConcurrentTestGroup = "nonConcurrent"
static String dockerTestGroup = "docker"
static {
ch.qos.logback.classic.Logger loggerOfSuite =
@ -113,8 +119,9 @@ class RegressionTest {
}
}
actionExecutors.shutdown()
suiteExecutors.shutdown()
singleSuiteExecutors.shutdown()
for (ExecutorService suiteExecutor : suiteExecutors.values()) {
suiteExecutor.shutdown()
}
scriptExecutors.shutdown()
log.info("Test finished")
if (!success) {
@ -135,17 +142,24 @@ class RegressionTest {
.build();
scriptExecutors = Executors.newFixedThreadPool(config.parallel, scriptFactory)
suiteExecutors = [:]
BasicThreadFactory suiteFactory = new BasicThreadFactory.Builder()
.namingPattern("suite-thread-%d")
.priority(Thread.MAX_PRIORITY)
.build();
suiteExecutors = Executors.newFixedThreadPool(config.suiteParallel, suiteFactory)
suiteExecutors[GroupExecType.NORMAL] = Executors.newFixedThreadPool(config.suiteParallel, suiteFactory)
BasicThreadFactory singleSuiteFactory = new BasicThreadFactory.Builder()
.namingPattern("non-concurrent-thread-%d")
.priority(Thread.MAX_PRIORITY)
.build();
singleSuiteExecutors = Executors.newFixedThreadPool(1, singleSuiteFactory)
suiteExecutors[GroupExecType.SINGLE] = Executors.newFixedThreadPool(1, singleSuiteFactory)
BasicThreadFactory dockerSuiteFactory = new BasicThreadFactory.Builder()
.namingPattern("docker-suite-thread-%d")
.priority(Thread.MAX_PRIORITY)
.build();
suiteExecutors[GroupExecType.DOCKER] = Executors.newFixedThreadPool(config.dockerSuiteParallel, dockerSuiteFactory)
BasicThreadFactory actionFactory = new BasicThreadFactory.Builder()
.namingPattern("action-thread-%d")
@ -183,9 +197,9 @@ class RegressionTest {
return sources
}
static void runScript(Config config, ScriptSource source, Recorder recorder, boolean isSingleThreadScript) {
static void runScript(Config config, ScriptSource source, Recorder recorder, GroupExecType grpExecType) {
def suiteFilter = { String suiteName, String groupName ->
canRun(config, suiteName, groupName, isSingleThreadScript)
canRun(config, suiteName, groupName, grpExecType)
}
def file = source.getFile()
int failureLimit = Integer.valueOf(config.otherConfigs.getOrDefault("max_failure_num", "-1").toString());
@ -196,12 +210,7 @@ class RegressionTest {
return;
}
def eventListeners = getEventListeners(config, recorder)
ExecutorService executors = null
if (isSingleThreadScript) {
executors = singleSuiteExecutors
} else {
executors = suiteExecutors
}
ExecutorService executors = suiteExecutors[grpExecType]
new ScriptContext(file, executors, actionExecutors,
config, eventListeners, suiteFilter).start { scriptContext ->
@ -227,11 +236,20 @@ class RegressionTest {
scriptSources.eachWithIndex { source, i ->
// log.info("Prepare scripts [${i + 1}/${totalFile}]".toString())
def future = scriptExecutors.submit {
runScript(config, source, recorder, false)
runScript(config, source, recorder, GroupExecType.NORMAL)
}
futures.add(future)
}
List<Future> dockerFutures = Lists.newArrayList()
scriptSources.eachWithIndex { source, i ->
// log.info("Prepare scripts [${i + 1}/${totalFile}]".toString())
def future = scriptExecutors.submit {
runScript(config, source, recorder, GroupExecType.DOCKER)
}
dockerFutures.add(future)
}
// wait all scripts
for (Future future : futures) {
try {
@ -250,12 +268,20 @@ class RegressionTest {
scriptSources.eachWithIndex { source, i ->
// log.info("Prepare scripts [${i + 1}/${totalFile}]".toString())
def future = scriptExecutors.submit {
runScript(config, source, recorder, true)
runScript(config, source, recorder, GroupExecType.SINGLE)
}
futures.add(future)
}
// wait all scripts
for (Future future : dockerFutures) {
try {
future.get()
} catch (Throwable t) {
// do nothing, because already save to Recorder
}
}
for (Future future : futures) {
try {
future.get()
@ -311,19 +337,19 @@ class RegressionTest {
return true
}
static boolean canRun(Config config, String suiteName, String group, boolean isSingleThreadScript) {
Set<String> suiteGroups = group.split(',').collect { g -> g.trim() }.toSet();
if (isSingleThreadScript) {
if (!suiteGroups.contains(nonConcurrentTestGroup)) {
return false
}
} else {
if (suiteGroups.contains(nonConcurrentTestGroup)) {
return false
}
}
static boolean canRun(Config config, String suiteName, String group, GroupExecType grpExecType) {
return getGroupExecType(group) == grpExecType && filterGroups(config, group) && filterSuites(config, suiteName)
}
return filterGroups(config, group) && filterSuites(config, suiteName)
static GroupExecType getGroupExecType(String group) {
Set<String> suiteGroups = group.split(',').collect { g -> g.trim() }.toSet();
if (suiteGroups.contains(nonConcurrentTestGroup)) {
return GroupExecType.SINGLE
} else if (suiteGroups.contains(dockerTestGroup)) {
return GroupExecType.DOCKER
} else {
return GroupExecType.NORMAL
}
}
static List<EventListener> getEventListeners(Config config, Recorder recorder) {
@ -409,7 +435,7 @@ class RegressionTest {
}
pluginPath.eachFileRecurse({ it ->
if (it.name.endsWith(".groovy")) {
ScriptContext context = new ScriptContext(it, suiteExecutors, actionExecutors,
ScriptContext context = new ScriptContext(it, suiteExecutors[GroupExecType.NORMAL], actionExecutors,
config, [], { name -> true })
File pluginFile = it
context.start({
@ -441,6 +467,9 @@ class RegressionTest {
log.warn("install doris compose requirements failed: code=${proc.exitValue()}, "
+ "output: ${sout.toString()}, error: ${serr.toString()}")
}
def pipList = 'python -m pip list'.execute().text
log.info("python library: ${pipList}")
}
static void printPassed() {

View File

@ -33,6 +33,7 @@ import groovy.json.JsonSlurper
import com.google.common.collect.ImmutableList
import org.apache.commons.lang3.ObjectUtils
import org.apache.doris.regression.Config
import org.apache.doris.regression.RegressionTest
import org.apache.doris.regression.action.BenchmarkAction
import org.apache.doris.regression.action.ProfileAction
import org.apache.doris.regression.action.WaitForAction
@ -264,6 +265,11 @@ class Suite implements GroovyInterceptable {
return
}
if (RegressionTest.getGroupExecType(group) != RegressionTest.GroupExecType.DOCKER) {
throw new Exception("Need to add 'docker' to docker suite's belong groups, "
+ "see example demo_p0/docker_action.groovy")
}
boolean pipelineIsCloud = isCloudMode()
boolean dockerIsCloud = false
if (options.cloudMode == null) {
@ -1398,6 +1404,13 @@ class Suite implements GroovyInterceptable {
}
DebugPoint GetDebugPoint() {
def execType = RegressionTest.getGroupExecType(group);
if (execType != RegressionTest.GroupExecType.SINGLE
&& execType != RegressionTest.GroupExecType.DOCKER) {
throw new Exception("Debug point must use in nonConcurrent suite or docker suite, "
+ "need add 'nonConcurrent' or 'docker' to suite's belong groups, "
+ "see example demo_p0/debugpoint_action.groovy.")
}
return debugPoint
}

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
suite("test_backup_cancelled", "backup_cancelled") {
suite("test_backup_cancelled", "backup_cancelled,nonConcurrent") {
String suiteName = "test_backup_cancelled"
String repoName = "${suiteName}_repo_" + UUID.randomUUID().toString().replace("-", "")
String dbName = "${suiteName}_db"
@ -58,36 +58,37 @@ suite("test_backup_cancelled", "backup_cancelled") {
// test failed to get tablet when truncate or drop table
GetDebugPoint().enableDebugPointForAllBEs("SnapshotManager::make_snapshot.inject_failure", [tablet_id:"${tabletId}", execute:3]);
try {
GetDebugPoint().enableDebugPointForAllBEs("SnapshotManager::make_snapshot.inject_failure", [tablet_id:"${tabletId}", execute:3]);
sql """
BACKUP SNAPSHOT ${dbName}.${snapshotName}
TO `${repoName}`
ON (${tableName})
"""
sql """
BACKUP SNAPSHOT ${dbName}.${snapshotName}
TO `${repoName}`
ON (${tableName})
"""
syncer.waitSnapshotFinish(dbName)
GetDebugPoint().disableDebugPointForAllBEs("SnapshotManager::make_snapshot.inject_failure")
syncer.waitSnapshotFinish(dbName)
} finally {
GetDebugPoint().disableDebugPointForAllBEs("SnapshotManager::make_snapshot.inject_failure")
}
// test missing versions when compaction or balance
try {
GetDebugPoint().enableDebugPointForAllBEs("Tablet::capture_consistent_versions.inject_failure", [tablet_id:"${tabletId}", execute:1]);
GetDebugPoint().enableDebugPointForAllBEs("Tablet::capture_consistent_versions.inject_failure", [tablet_id:"${tabletId}", execute:1]);
sql """
BACKUP SNAPSHOT ${dbName}.${snapshotName_1}
TO `${repoName}`
ON (${tableName})
"""
sql """
BACKUP SNAPSHOT ${dbName}.${snapshotName_1}
TO `${repoName}`
ON (${tableName})
"""
syncer.waitSnapshotFinish(dbName)
GetDebugPoint().disableDebugPointForAllBEs("Tablet::capture_consistent_versions.inject_failure");
syncer.waitSnapshotFinish(dbName)
} finally {
GetDebugPoint().disableDebugPointForAllBEs("Tablet::capture_consistent_versions.inject_failure");
}
def snapshot = syncer.getSnapshotTimestamp(repoName, snapshotName)

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
suite("test_backup_restore_atomic_with_alter", "backup_restore") {
suite("test_backup_restore_atomic_with_alter", "backup_restore,nonConcurrent") {
if (!getFeConfig("enable_debug_points").equals("true")) {
logger.info("Config.enable_debug_points=true is required")
return
@ -96,6 +96,7 @@ suite("test_backup_restore_atomic_with_alter", "backup_restore") {
sql "DROP TABLE ${dbName}.${tableNamePrefix}_0 FORCE"
// disable restore
try {
GetDebugPoint().enableDebugPointForAllFEs("FE.PAUSE_NON_PENDING_RESTORE_JOB", [value:snapshotName])
sql """
@ -239,6 +240,9 @@ suite("test_backup_restore_atomic_with_alter", "backup_restore") {
}
sql "DROP DATABASE ${dbName} FORCE"
sql "DROP REPOSITORY `${repoName}`"
} finally {
GetDebugPoint().disableDebugPointForAllFEs("FE.PAUSE_NON_PENDING_RESTORE_JOB")
}
}

View File

@ -18,7 +18,7 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
suite('test_clone_no_missing_version') {
suite('test_clone_no_missing_version', 'docker') {
def tbl = 'tbl_test_clone_no_missing_version'
def options = new ClusterOptions()
options.feConfigs += [

View File

@ -17,7 +17,7 @@
import org.apache.doris.regression.suite.ClusterOptions
suite('test_decommission_mtmv') {
suite('test_decommission_mtmv', 'docker') {
def options = new ClusterOptions()
options.feConfigs += [
'disable_balance=true',

View File

@ -18,7 +18,10 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.junit.Assert
suite('test_drop_clone_tablet_path_race') {
suite('test_drop_clone_tablet_path_race', 'docker') {
if (isCloudMode()) {
return
}
def options = new ClusterOptions()
options.enableDebugPoints()
options.feConfigs += [

View File

@ -19,7 +19,7 @@ import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.Http
import org.apache.doris.regression.util.NodeType
suite('test_compaction_with_visible_version') {
suite('test_compaction_with_visible_version', 'docker') {
def options = new ClusterOptions()
def compaction_keep_invisible_version_min_count = 50L
options.feConfigs += [

View File

@ -15,6 +15,11 @@
// specific language governing permissions and limitations
// under the License.
import org.junit.Assert
// This is a good example.
// Debug point must run in 'nonConcurrent' or 'docker' suites.
// If not a docker suite, must put nonConcurrent to the groups.
suite('debugpoint_action', 'nonConcurrent') {
try {
GetDebugPoint().enableDebugPointForAllFEs('PublishVersionDaemon.stop_publish', [timeout:1])
@ -25,3 +30,24 @@ suite('debugpoint_action', 'nonConcurrent') {
GetDebugPoint().disableDebugPointForAllBEs('Tablet.build_tablet_report_info.version_miss')
}
}
// This is a bad example.
// its group tag not contains nonConcurrent or docker.
suite('debugpoint_action_bad') {
Exception exception = null;
try {
GetDebugPoint().enableDebugPointForAllFEs('debugpoint_action_bad_xx', [timeout:1])
} catch (Exception e) {
exception = e
} finally {
// this is bad example, should disable or clear debug points after end suite
// GetDebugPoint().disableDebugPointForAllFEs('debugpoint_action_bad_xx')
}
Assert.assertNotNull(exception)
def expectMsg = "Debug point must use in nonConcurrent suite or docker suite"
def msg = exception.toString()
log.info("meet exception: ${msg}")
Assert.assertTrue("Expect exception msg contains '${expectMsg}', but meet '${msg}'",
msg.contains(expectMsg))
}

View File

@ -17,7 +17,7 @@
import org.apache.doris.regression.suite.ClusterOptions
suite('docker_action') {
suite('docker_action', 'docker') {
// run a new docker
docker {
sql '''create table tb1 (k int) DISTRIBUTED BY HASH(k) BUCKETS 10'''

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
suite('test_partial_update_during_sc') {
suite('test_partial_update_during_sc', 'nonConcurrent') {
String db = context.config.getDbNameByFile(context.file)
sql "select 1;" // to create database

View File

@ -21,7 +21,7 @@
import org.apache.doris.regression.suite.ClusterOptions
suite("replay_wal_restart_fe") {
suite("replay_wal_restart_fe", 'docker') {
def check_schema_change = { state ->
for (int i = 0; i < 30; i++) {
def jobs = sql_return_maparray "SHOW ALTER TABLE COLUMN WHERE TableName = 'tbl_2' order by CreateTime desc;"

View File

@ -19,7 +19,10 @@ import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
import org.apache.doris.regression.suite.SuiteCluster
suite("test_build_index_with_clone_by_docker"){
suite("test_build_index_with_clone_by_docker", 'docker'){
if (isCloudMode()) {
return
}
def timeout = 300000
def delta_time = 1000
def alter_res = "null"

View File

@ -34,7 +34,7 @@ class InjectCase {
}
suite('test_min_load_replica_num_complicate') {
suite('test_min_load_replica_num_complicate', 'docker') {
def beCloneCostMs = 3000
def random = new Random()

View File

@ -18,7 +18,7 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
suite('test_publish_one_succ') {
suite('test_publish_one_succ', 'docker') {
def options = new ClusterOptions()
options.enableDebugPoints()
docker(options) {

View File

@ -17,7 +17,7 @@
import org.apache.doris.regression.suite.ClusterOptions
suite('test_min_load_replica_num_simple') {
suite('test_min_load_replica_num_simple', 'docker') {
def options = new ClusterOptions()
options.feConfigs.add('tablet_checker_interval_ms=1000')
docker(options) {

View File

@ -18,7 +18,7 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.http.NoHttpResponseException
suite('test_coordidator_be_restart') {
suite('test_coordidator_be_restart', 'docker') {
def options = new ClusterOptions()
options.cloudMode = false
options.enableDebugPoints()

View File

@ -18,7 +18,10 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
suite('test_migrate_disk_with_publish_version') {
suite('test_migrate_disk_with_publish_version', 'docker') {
if (isCloudMode()) {
return
}
def checkTabletOnHDD = { isOnHdd ->
sleep 5000

View File

@ -18,7 +18,7 @@ import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
import org.apache.doris.regression.suite.SuiteCluster
suite("test_create_table_exception") {
suite("test_create_table_exception", "docker") {
def options = new ClusterOptions()
options.enableDebugPoints()
options.setFeNum(3)

View File

@ -18,7 +18,7 @@ import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
import org.apache.doris.regression.suite.SuiteCluster
suite("test_partition_create_tablet_rr") {
suite("test_partition_create_tablet_rr", "docker") {
def options = new ClusterOptions()
options.beNum = 1
options.feConfigs.add('disable_balance=true')

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
suite("test_inject_send_filter_size_fail") {
suite("test_inject_send_filter_size_fail", "nonConcurrent") {
sql "set parallel_pipeline_task_num=3"
sql "set enable_runtime_filter_prune=false"
sql "set enable_sync_runtime_filter_size=true"

View File

@ -15,7 +15,7 @@
// specific language governing permissions and limitations
// under the License.
suite("test_slow_close") {
suite("test_slow_close", "nonConcurrent") {
sql "set disable_join_reorder=true;"
sql "set runtime_filter_type='bloom_filter';"
sql "set parallel_pipeline_task_num=3"

View File

@ -19,7 +19,7 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
suite("test_forward_query") {
suite("test_forward_query", 'docker') {
def options = new ClusterOptions()
options.enableDebugPoints()
options.setFeNum(2)
@ -46,4 +46,4 @@ suite("test_forward_query") {
def ret = sql """ SELECT * FROM ${tbl} """
assertEquals(ret[0][0], 1)
}
}
}

View File

@ -18,7 +18,7 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
suite('test_schema_change_concurrent_with_txn') {
suite('test_schema_change_concurrent_with_txn', 'docker') {
def options = new ClusterOptions()
options.enableDebugPoints()
options.feConfigs.add('publish_wait_time_second=-1')

View File

@ -18,7 +18,7 @@ import org.apache.doris.regression.suite.ClusterOptions
import org.apache.doris.regression.util.NodeType
import org.apache.doris.regression.suite.SuiteCluster
suite("test_partition_default_medium") {
suite("test_partition_default_medium", 'docker') {
def options = new ClusterOptions()
options.feConfigs += [
'default_storage_medium=HDD',

View File

@ -17,7 +17,11 @@
import org.apache.doris.regression.suite.ClusterOptions
suite('test_storage_medium_has_disk') {
suite('test_storage_medium_has_disk', 'docker') {
if (isCloudMode()) {
return
}
def checkPartitionMedium = { table, isHdd ->
def partitions = sql_return_maparray "SHOW PARTITIONS FROM ${table}"
assertTrue(partitions.size() > 0)

View File

@ -17,7 +17,10 @@
import org.apache.doris.regression.suite.ClusterOptions
import org.junit.Assert
suite("test_clean_trash", "p0") {
suite("test_clean_trash", "docker") {
if (isCloudMode()) {
return
}
def options = new ClusterOptions()
options.enableDebugPoints()
options.feConfigs += [
@ -73,4 +76,4 @@ suite("test_clean_trash", "p0") {
sql """admin clean trash"""
checkFunc(true)
}
}
}

View File

@ -35,7 +35,7 @@ import org.apache.http.client.methods.CloseableHttpResponse
import org.apache.http.util.EntityUtils
import org.apache.doris.regression.suite.ClusterOptions
suite("test_partial_update_conflict_be_restart") {
suite("test_partial_update_conflict_be_restart", 'docker') {
def dbName = context.config.getDbNameByFile(context.file)
def options = new ClusterOptions()

View File

@ -29,7 +29,7 @@ import org.apache.doris.regression.util.NodeType
//
// the bug is fixed in #37001
suite('test_full_clone_exception') {
suite('test_full_clone_exception', 'docker') {
def options = new ClusterOptions()
options.feConfigs += [
'disable_tablet_scheduler=true',

View File

@ -43,6 +43,7 @@ Usage: $0 <shell_options> <framework_options>
-genOut generate .out file if not exist
-forceGenOut delete and generate .out file
-parallel run tests using specified threads
-dockerSuiteParallel run docker tests using specified threads
-randomOrder run tests in a random order
-noKillDocker don't kill container when finish docker suites
-times rum tests {times} times