diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index d7df2262cfb..422becec1c7 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -91,14 +92,23 @@ NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan, const StringRef return genNamesAndTypes(table_scan.getColumns(), column_prefix); } -std::tuple genColumnDefinesForDisaggregatedRead(const TiDBTableScan & table_scan) +std::tuple>> genColumnDefinesForDisaggregatedRead( + const TiDBTableScan & table_scan) { auto column_defines = std::make_shared(); int extra_table_id_index = InvalidColumnID; column_defines->reserve(table_scan.getColumnSize()); + std::vector> generated_column_infos; for (Int32 i = 0; i < table_scan.getColumnSize(); ++i) { const auto & column_info = table_scan.getColumns()[i]; + if (column_info.hasGeneratedColumnFlag()) + { + const auto & data_type = getDataTypeByColumnInfoForComputingLayer(column_info); + const auto & col_name = GeneratedColumnPlaceholderBlockInputStream::getColumnName(i); + generated_column_infos.push_back(std::make_tuple(i, col_name, data_type)); + continue; + } // Now the upper level seems treat disagg read as an ExchangeReceiver output, so // use this as output column prefix. // Even if the id is pk_column or extra_table_id, we still output it as @@ -114,10 +124,13 @@ std::tuple genColumnDefinesForDisaggregatedRead(const break; case ExtraTableIDColumnID: { +<<<<<<< HEAD column_defines->emplace_back(DM::ColumnDefine{ ExtraTableIDColumnID, output_name, // MutableSupport::extra_table_id_column_name MutableSupport::extra_table_id_column_type}); +======= +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) extra_table_id_index = i; break; } @@ -130,7 +143,7 @@ std::tuple genColumnDefinesForDisaggregatedRead(const break; } } - return {std::move(column_defines), extra_table_id_index}; + return {std::move(column_defines), extra_table_id_index, std::move(generated_column_infos)}; } ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types) diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 4c579662912..ab3c1f0d57a 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -33,7 +33,8 @@ NamesAndTypes genNamesAndTypes(const TiDB::ColumnInfos & column_infos, const Str ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); -// The column defines and `extra table id index` -std::tuple genColumnDefinesForDisaggregatedRead(const TiDBTableScan & table_scan); +// The column defines, `extra table id index` and `generated columns info` for disaggregated read. +std::tuple>> genColumnDefinesForDisaggregatedRead( + const TiDBTableScan & table_scan); } // namespace DB diff --git a/dbms/src/Storages/StorageDisaggregated.h b/dbms/src/Storages/StorageDisaggregated.h index c63efd91534..32bf6093836 100644 --- a/dbms/src/Storages/StorageDisaggregated.h +++ b/dbms/src/Storages/StorageDisaggregated.h @@ -105,8 +105,15 @@ class StorageDisaggregated : public IStorage std::shared_ptr buildEstablishDisaggTaskReq( const Context & db_context, const pingcap::coprocessor::BatchCopTask & batch_cop_task); +<<<<<<< HEAD DM::RSOperatorPtr buildRSOperator(const Context & db_context, const DM::ColumnDefinesPtr & columns_to_read); std::variant packSegmentReadTasks( +======= + std::tuple buildRSOperatorAndColumnRange( + const Context & db_context, + const DM::ColumnDefinesPtr & columns_to_read); + std::tuple, DM::ColumnDefinesPtr> packSegmentReadTasks( +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) const Context & db_context, DM::SegmentReadTasks && read_tasks, const DM::ColumnDefinesPtr & column_defines, @@ -154,5 +161,12 @@ class StorageDisaggregated : public IStorage const FilterConditions & filter_conditions; std::unique_ptr analyzer; +<<<<<<< HEAD +======= + static constexpr auto ZONE_LABEL_KEY = "zone"; + std::optional zone_label; + // For generated column, just need a placeholder, and TiDB will fill this column. + std::vector> generated_column_infos; +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) }; } // namespace DB diff --git a/dbms/src/Storages/StorageDisaggregatedRemote.cpp b/dbms/src/Storages/StorageDisaggregatedRemote.cpp index 058992ba5a6..f55792fc977 100644 --- a/dbms/src/Storages/StorageDisaggregatedRemote.cpp +++ b/dbms/src/Storages/StorageDisaggregatedRemote.cpp @@ -103,7 +103,18 @@ BlockInputStreams StorageDisaggregated::readThroughS3(const Context & db_context { // Build InputStream according to the remote segment read tasks DAGPipeline pipeline; +<<<<<<< HEAD buildRemoteSegmentInputStreams(db_context, buildReadTaskWithBackoff(db_context), num_streams, pipeline); +======= + buildRemoteSegmentInputStreams( + db_context, + buildReadTaskWithBackoff(db_context, scan_context), + num_streams, + pipeline, + scan_context); + // handle generated column if necessary. + executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) NamesAndTypes source_columns; source_columns.reserve(table_scan.getColumnSize()); @@ -131,8 +142,16 @@ void StorageDisaggregated::readThroughS3( exec_context, group_builder, db_context, +<<<<<<< HEAD buildReadTaskWithBackoff(db_context), num_streams); +======= + buildReadTaskWithBackoff(db_context, scan_context), + num_streams, + scan_context); + // handle generated column if necessary. + executeGeneratedColumnPlaceholder(exec_context, group_builder, generated_column_infos, log); +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) NamesAndTypes source_columns; auto header = group_builder.getCurrentHeader(); @@ -496,12 +515,23 @@ DM::RSOperatorPtr StorageDisaggregated::buildRSOperator( return DM::RSOperator::build(dag_query, table_scan.getColumns(), *columns_to_read, enable_rs_filter, log); } +<<<<<<< HEAD std::variant StorageDisaggregated::packSegmentReadTasks( const Context & db_context, DM::SegmentReadTasks && read_tasks, const DM::ColumnDefinesPtr & column_defines, size_t num_streams, int extra_table_id_index) +======= +std::tuple, DM::ColumnDefinesPtr> StorageDisaggregated:: + packSegmentReadTasks( + const Context & db_context, + DM::SegmentReadTasks && read_tasks, + const DM::ColumnDefinesPtr & column_defines, + const DM::ScanContextPtr & scan_context, + size_t num_streams, + int extra_table_id_index) +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) { const auto & executor_id = table_scan.getTableScanExecutorID(); @@ -520,20 +550,29 @@ std::variant StorageDisagg push_down_filter); const UInt64 start_ts = sender_target_mpp_task_id.gather_id.query_id.start_ts; const auto enable_read_thread = db_context.getSettingsRef().dt_enable_read_thread; +<<<<<<< HEAD +======= + const auto & final_columns_defines = push_down_executor && push_down_executor->extra_cast + ? push_down_executor->columns_after_cast + : column_defines; + RUNTIME_CHECK(num_streams > 0, num_streams); +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) LOG_INFO( log, "packSegmentReadTasks: enable_read_thread={} read_mode={} is_fast_scan={} keep_order={} task_count={} " - "num_streams={} column_defines={}", + "num_streams={} column_defines={} final_columns_defines={}", enable_read_thread, magic_enum::enum_name(read_mode), table_scan.isFastScan(), table_scan.keepOrder(), read_tasks.size(), num_streams, - *column_defines); + *column_defines, + *final_columns_defines); if (enable_read_thread) { +<<<<<<< HEAD return std::make_shared( extra_table_id_index, *column_defines, @@ -561,6 +600,44 @@ std::variant StorageDisagg .read_mode = read_mode, }, num_streams); +======= + // Under disagg arch, now we use blocking IO to read data from cloud storage. So it require more active + // segments to fully utilize the read threads. + const size_t read_thread_num_active_seg = 10 * num_streams; + return { + std::make_shared( + extra_table_id_index, + *final_columns_defines, + push_down_executor, + start_ts, + db_context.getSettingsRef().max_block_size, + read_mode, + std::move(read_tasks), + /*after_segment_read*/ [](const DM::DMContextPtr &, const DM::SegmentPtr &) {}, + log->identifier(), + /*enable_read_thread*/ true, + num_streams, + read_thread_num_active_seg, + context.getDAGContext()->getKeyspaceID(), + context.getDAGContext()->getResourceGroupName()), + final_columns_defines}; + } + else + { + return { + DM::Remote::RNWorkers::create( + db_context, + std::move(read_tasks), + { + .log = log->getChild(executor_id), + .columns_to_read = final_columns_defines, + .start_ts = start_ts, + .push_down_executor = push_down_executor, + .read_mode = read_mode, + }, + num_streams), + final_columns_defines}; +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) } } @@ -598,15 +675,29 @@ void StorageDisaggregated::buildRemoteSegmentInputStreams( DAGPipeline & pipeline) { // Build the input streams to read blocks from remote segments +<<<<<<< HEAD auto [column_defines, extra_table_id_index] = genColumnDefinesForDisaggregatedRead(table_scan); auto packed_read_tasks = packSegmentReadTasks(db_context, std::move(read_tasks), column_defines, num_streams, extra_table_id_index); RUNTIME_CHECK(num_streams > 0, num_streams); +======= + DM::ColumnDefinesPtr column_defines; + int extra_table_id_index; + std::tie(column_defines, extra_table_id_index, generated_column_infos) + = genColumnDefinesForDisaggregatedRead(table_scan); + auto [packed_read_tasks, final_column_defines] = packSegmentReadTasks( + db_context, + std::move(read_tasks), + column_defines, + scan_context, + num_streams, + extra_table_id_index); +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) pipeline.streams.reserve(num_streams); InputStreamBuilder builder{ .tracing_id = log->identifier(), - .columns_to_read = column_defines, + .columns_to_read = final_column_defines, .extra_table_id_index = extra_table_id_index, }; for (size_t stream_idx = 0; stream_idx < num_streams; ++stream_idx) @@ -661,14 +752,28 @@ void StorageDisaggregated::buildRemoteSegmentSourceOps( size_t num_streams) { // Build the input streams to read blocks from remote segments +<<<<<<< HEAD auto [column_defines, extra_table_id_index] = genColumnDefinesForDisaggregatedRead(table_scan); auto packed_read_tasks = packSegmentReadTasks(db_context, std::move(read_tasks), column_defines, num_streams, extra_table_id_index); +======= + DM::ColumnDefinesPtr column_defines; + int extra_table_id_index; + std::tie(column_defines, extra_table_id_index, generated_column_infos) + = genColumnDefinesForDisaggregatedRead(table_scan); + auto [packed_read_tasks, final_column_defines] = packSegmentReadTasks( + db_context, + std::move(read_tasks), + column_defines, + scan_context, + num_streams, + extra_table_id_index); +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)) RUNTIME_CHECK(num_streams > 0, num_streams); SrouceOpBuilder builder{ .tracing_id = log->identifier(), - .column_defines = column_defines, + .column_defines = final_column_defines, .extra_table_id_index = extra_table_id_index, .exec_context = exec_context, }; diff --git a/dbms/src/TableFunctions/CMakeLists.txt b/dbms/src/TableFunctions/CMakeLists.txt index 1f164a8602b..6b047c8d771 100644 --- a/dbms/src/TableFunctions/CMakeLists.txt +++ b/dbms/src/TableFunctions/CMakeLists.txt @@ -18,5 +18,16 @@ add_headers_and_sources(tiflash_table_functions .) list(REMOVE_ITEM tiflash_table_functions_sources ITableFunction.cpp TableFunctionFactory.cpp) list(REMOVE_ITEM tiflash_table_functions_headers ITableFunction.h TableFunctionFactory.h) +<<<<<<< HEAD:dbms/src/TableFunctions/CMakeLists.txt add_library(tiflash_table_functions ${tiflash_table_functions_sources}) target_link_libraries(tiflash_table_functions tiflash_storages_system dbms ${Poco_Foundation_LIBRARY}) +======= +tikv-worker-url = "tikv-worker0:19000" + +enable-telemetry = false +temp-dir = "/data/tmp" +[performance] +tcp-keep-alive = true +[security] +enable-sem = false +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)):tests/docker/next-gen-config/tidb.toml diff --git a/release-linux-llvm/dockerfiles/misc/install_cmake.sh b/release-linux-llvm/dockerfiles/misc/install_cmake.sh index d8346e69345..896ef4a66b6 100644 --- a/release-linux-llvm/dockerfiles/misc/install_cmake.sh +++ b/release-linux-llvm/dockerfiles/misc/install_cmake.sh @@ -13,6 +13,16 @@ # See the License for the specific language governing permissions and # limitations under the License. +<<<<<<< HEAD:release-linux-llvm/dockerfiles/misc/install_cmake.sh +======= +[storage] +# No space is reserved at all for testing +reserve-space = "0" +# Enable keyspace and ttl for next-gen +api-version = 2 +enable-ttl = true +low-space-threshold = 0 +>>>>>>> a5e14033f8 (Fix three schema mismatch bugs under disaggregated arch (#10530)):tests/docker/next-gen-config/tikv.toml # Install cmake for CI/CD. # Require: wget diff --git a/tests/docker/next-gen-config/tikv-worker.toml b/tests/docker/next-gen-config/tikv-worker.toml new file mode 100644 index 00000000000..bb2a52ba88f --- /dev/null +++ b/tests/docker/next-gen-config/tikv-worker.toml @@ -0,0 +1,28 @@ +# Copyright 2025 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# object storage for next-gen +[dfs] +prefix = "tikv" +s3-endpoint = "http://minio0:9000" +s3-key-id = "minioadmin" +s3-secret-key = "minioadmin" +s3-bucket = "tiflash-test" +s3-region = "local" + +[schema-manager] +dir = "/data/schemas" +enabled = true +keyspace-refresh-interval = "10s" +schema-refresh-threshold = 1 diff --git a/tests/docker/next-gen-yaml/cluster.yaml b/tests/docker/next-gen-yaml/cluster.yaml new file mode 100644 index 00000000000..c0acdf2418c --- /dev/null +++ b/tests/docker/next-gen-yaml/cluster.yaml @@ -0,0 +1,75 @@ +# Copyright 2025 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +version: '2.3' + +services: + minio0: + image: ${MINIO_IMAGE:-hub.pingcap.net/test-infra/minio:latest} + security_opt: + - seccomp:unconfined + volumes: + - ./data/minio:/data + - ./log/minio:/log + # Using command can not set the init bucket and logging to file. + #command: server /data --console-address ':9001' > /log/minio.log 2>&1 + # Use entrypoint instead, create bucket "tiflash-test" on minio at startup + entrypoint: sh -c 'mkdir -p /data/tiflash-test && minio server /data --console-address ":9001" > /log/minio.log 2>&1' + pd0: + image: ${PD_IMAGE:-us-docker.pkg.dev/pingcap-testing-account/hub/tikv/pd/image:master-next-gen} + security_opt: + - seccomp:unconfined + volumes: + - ./next-gen-config/pd.toml:/pd.toml:ro + - ./data/pd0:/data + - ./log/pd0:/log + command: --name=pd0 --client-urls=http://0.0.0.0:2379 --peer-urls=http://0.0.0.0:2380 --advertise-client-urls=http://pd0:2379 --advertise-peer-urls=http://pd0:2380 --initial-cluster=pd0=http://pd0:2380 --config=/pd.toml --data-dir=/data --log-file=/log/pd.log + restart: on-failure + tikv0: + image: ${TIKV_IMAGE:-us-docker.pkg.dev/pingcap-testing-account/hub/tikv/tikv/image:dedicated-next-gen} + security_opt: + - seccomp:unconfined + volumes: + - ./next-gen-config/tikv.toml:/tikv.toml:ro + - ./data/tikv0:/data + - ./log/tikv0:/log + command: --addr=0.0.0.0:20160 --advertise-addr=tikv0:20160 --status-addr=0.0.0.0:20180 --advertise-status-addr=tikv0:20180 --pd=pd0:2379 --config=/tikv.toml --data-dir=/data --log-file=/log/tikv.log + depends_on: + - "pd0" + - "minio0" + restart: on-failure + tikv-worker0: + image: ${TIKV_IMAGE:-us-docker.pkg.dev/pingcap-testing-account/hub/tikv/tikv/image:dedicated-next-gen} + security_opt: + - seccomp:unconfined + volumes: + - ./next-gen-config/tikv-worker.toml:/tikv-worker.toml:ro + - ./data/tikv-worker0:/data + - ./log/tikv-worker0:/log + entrypoint: /tikv-worker + command: --addr=0.0.0.0:19000 --pd-endpoints=pd0:2379 --config=/tikv-worker.toml --data-dir=/data --log-file=/log/tikv-worker.log + depends_on: + - "pd0" + restart: on-failure + tidb0: + image: ${TIDB_IMAGE:-us-docker.pkg.dev/pingcap-testing-account/hub/pingcap/tidb/images/tidb-server:master-next-gen} + security_opt: + - seccomp:unconfined + volumes: + - ./next-gen-config/tidb.toml:/tidb.toml:ro + - ./log/tidb0:/log + command: --store=tikv --path=pd0:2379 --config=/tidb.toml --log-file=/log/tidb.log --advertise-address=tidb0 --tidb-service-scope dxf_service + depends_on: + - "tikv0" + restart: on-failure diff --git a/tests/fullstack-test-next-gen/run.sh b/tests/fullstack-test-next-gen/run.sh new file mode 100755 index 00000000000..5f828cf4b66 --- /dev/null +++ b/tests/fullstack-test-next-gen/run.sh @@ -0,0 +1,89 @@ +#!/bin/bash +# Copyright 2025 PingCAP, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +source ../docker/util.sh + +set_branch + +set -xe + +export verbose=${verbose:-"false"} + +check_env +check_docker_compose + +DISAGG_TIFLASH_YAML="disagg_tiflash.yaml" +if grep -q "Rocky Linux release 9" /etc/redhat-release; then + # replace the base image for running under Rocky Linux 9 + sed 's/tiflash-ci-base:rocky8-20241028/tiflash-ci-base:rocky9-20250529/g' \ + "${DISAGG_TIFLASH_YAML}" > "disagg_tiflash.rocky9.yaml" + DISAGG_TIFLASH_YAML="disagg_tiflash.rocky9.yaml" + echo "Using ${DISAGG_TIFLASH_YAML} for Rocky Linux 9" +fi + +if [[ -n "$ENABLE_NEXT_GEN" && "$ENABLE_NEXT_GEN" != "false" && "$ENABLE_NEXT_GEN" != "0" ]]; then + echo "Running fullstack test on next-gen TiFlash" + + # set images for next-gen TiFlash cluster + HUB_ADDR="us-docker.pkg.dev/pingcap-testing-account/hub" + if [[ -z "${PD_BRANCH}" || "${PD_BRANCH}" == "master" ]]; then + PD_BRANCH="master-next-gen" + fi + if [[ -z "${TIKV_BRANCH}" || "${TIKV_BRANCH}" == "dedicated" ]]; then + TIKV_BRANCH="dedicated-next-gen" + fi + if [[ -z "${TIDB_BRANCH}" || "${TIDB_BRANCH}" == "master" ]]; then + TIDB_BRANCH="master-next-gen" + fi + export PD_IMAGE="${HUB_ADDR}/tikv/pd/image:${PD_BRANCH}" + export TIKV_IMAGE="${HUB_ADDR}/tikv/tikv/image:${TIKV_BRANCH}" + export TIDB_IMAGE="${HUB_ADDR}/pingcap/tidb/images/tidb-server:${TIDB_BRANCH}" + + # clean up previous docker instances, data and log + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" down + clean_data_log + + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" up -d + echo "PD version:" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T pd0 bash -c '/pd-server -V' + echo "TiDB version:" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tidb0 bash -c '/tidb-server -V' + echo "TiKV version:" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tikv0 bash -c '/tikv-server -V' + + # run fullstack-tests + wait_next_gen_env + ENV_ARGS="ENABLE_NEXT_GEN=true verbose=${verbose} " + # most failpoints are expected to be set on the compute layer, use tiflash-cn0 to run tests + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test/sample.test" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test-index" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test-next-gen/placement" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test2/clustered_index" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test2/dml" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test2/variables" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test2/mpp" + # maybe we need to split them into parallel pipelines because they take too long to run. + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test/expr" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" exec -T tiflash-cn0 bash -c "cd /tests ; ${ENV_ARGS} ./run-test.sh fullstack-test/mpp" + ${COMPOSE} -f next-gen-cluster.yaml -f "${DISAGG_TIFLASH_YAML}" down + clean_data_log + + exit 0 +fi + +# classic TiFlash +echo "Should not run classic TiFlash on this test environment" +exit 1