Skip to content

Latest commit

 

History

History
410 lines (330 loc) · 114 KB

tune-tiflash-performance.md

File metadata and controls

410 lines (330 loc) · 114 KB
title summary aliases
Tune TiFlash Performance
Learn how to tune the performance of TiFlash by planning machine resources and tuning TiDB parameters.
/docs/dev/tiflash/tune-tiflash-performance/
/docs/dev/reference/tiflash/tune-performance/

Tune TiFlash Performance

This document introduces how to tune the performance of TiFlash by properly planning machine resources and tuning TiDB parameters. By following these methods, your TiFlash cluster can achieve optimal performance.

Plan resources

If you want to save machine resources and have no requirement on isolation, you can use the method that combines the deployment of both TiKV and TiFlash. It is recommended that you save enough resources for TiKV and TiFlash respectively, and do not share disks.

Tune TiDB parameters

This section describes how to improve TiFlash performance by tuning TiDB parameters, including:

Forcibly enable the MPP mode

MPP execution plans can fully utilize distributed computing resources, thereby significantly improving the efficiency of batch data queries. When the optimizer does not generate an MPP execution plan for a query, you can forcibly enable the MPP mode:

The variable tidb_enforce_mpp controls whether to ignore the optimizer's cost estimation and to forcibly use TiFlash's MPP mode for query execution. To enable MPP mode forcibly, run the following command:

set @@tidb_enforce_mpp = ON;

The following example shows the query result before and after tidb_enforce_mpp is enabled. Before this variable is enabled, TiDB needs to read data from TiKV and execute Join and Aggregation in TiDB. After tidb_enforce_mpp is enabled, Join and Aggregation are pushed down to TiFlash. In addition, because the optimizer does not necessarily generate MPP execution plans, by enabling tidb_enforce_mpp, you can force the optimizer to generate MPP execution plans.

Before MPP mode is enabled:

mysql> explain analyze select o_orderpriority, count(*) as order_count from orders where o_orderdate >= '1995-01-01' and o_orderdate < date_add('1995-01-01', interval '3' month) and exists (select * from lineitem where l_orderkey = o_orderkey and l_commitdate < l_receiptdate) group by o_orderpriority;
+--------------------------------------+--------------+-----------+-----------+---------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------+------+
| id                                   | estRows      | actRows   | task      | access object                                           | execution info                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                              | operator info                                                                                                                                                                          | memory     | disk |
+--------------------------------------+--------------+-----------+-----------+---------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------+------+
| Projection_9                         | 1.00         | 5         | root      |                                                         | time:22.8s, loops:5, Concurrency:OFF                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                        | tpch_100.orders.o_orderpriority, Column#27                                                                                                                                             | 1016 Bytes | N/A  |
| └─HashAgg_10                         | 1.00         | 5         | root      |                                                         | time:22.8s, loops:5, partial_worker:{wall_time:22.817403422s, concurrency:5, task_num:5027, tot_wait:1m52.560509138s, tot_exec:1.523754659s, tot_time:1m54.086196522s, max:22.817363394s, p95:22.817363394s}, final_worker:{wall_time:22.817416046s, concurrency:5, task_num:20, tot_wait:1m54.086784741s, tot_exec:155.94µs, tot_time:1m54.086948982s, max:22.817396284s, p95:22.817396284s}                                                                                                                                                                                               | group by:tpch_100.orders.o_orderpriority, funcs:count(1)->Column#27, funcs:firstrow(tpch_100.orders.o_orderpriority)->tpch_100.orders.o_orderpriority                                  | 511.7 KB   | N/A  |
|   └─IndexHashJoin_18                 | 4429262.13   | 5145138   | root      |                                                         | time:22.8s, loops:5028, inner:{total:1m52.6s, concurrency:5, task:227, construct:5.17s, fetch:1m40s, build:1.22s, join:7.49s}                                                                                                                                                                                                                                                                                                                                                                                                                                                               | semi join, inner:IndexLookUp_15, outer key:tpch_100.orders.o_orderkey, inner key:tpch_100.lineitem.l_orderkey, equal cond:eq(tpch_100.orders.o_orderkey, tpch_100.lineitem.l_orderkey) | 46.8 MB    | N/A  |
|     ├─TableReader_37(Build)          | 5536577.67   | 5610951   | root      |                                                         | time:4.89s, loops:5467, cop_task: {num: 2127, max: 408.7ms, min: 277.5µs, avg: 81.6ms, p95: 249.7ms, max_proc_keys: 226272, p95_proc_keys: 220128, tot_proc: 2m50.2s, tot_wait: 40ms, rpc_num: 2127, rpc_time: 2m53.6s, copr_cache_hit_ratio: 0.02, distsql_concurrency: 15}                                                                                                                                                                                                                                                                                                                | data:Selection_36                                                                                                                                                                      | 3.42 MB    | N/A  |
|     │ └─Selection_36                 | 5536577.67   | 5610951   | cop[tikv] |                                                         | tikv_task:{proc max:402ms, min:0s, avg: 79.7ms, p80:151ms, p95:245ms, iters:155122, tasks:2127}, scan_detail: {total_process_keys: 149478336, total_process_keys_size: 22705157593, total_keys: 149480413, get_snapshot_time: 187.9ms, rocksdb: {key_skipped_count: 149478336, block: {cache_hit_count: 393690}}}                                                                                                                                                                                                                                                                           | ge(tpch_100.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch_100.orders.o_orderdate, 1995-04-01 00:00:00.000000)                                                               | N/A        | N/A  |
|     │   └─TableFullScan_35           | 150000000.00 | 150000000 | cop[tikv] | table:orders                                            | tikv_task:{proc max:382ms, min:0s, avg: 74.6ms, p80:141ms, p95:229ms, iters:155122, tasks:2127}                                                                                                                                                                                                                                                                                                                                                                                                                                                                                             | keep order:false                                                                                                                                                                       | N/A        | N/A  |
|     └─IndexLookUp_15(Probe)          | 22018496.89  | 14185840  | root      |                                                         | time:1m33.4s, loops:14204, index_task: {total_time: 1m10.8s, fetch_handle: 1m10.7s, build: 3.9ms, wait: 13.3ms}, table_task: {total_time: 1m44.4s, num: 1989, concurrency: 5}                                                                                                                                                                                                                                                                                                                                                                                                               |                                                                                                                                                                                        | 17.9 KB    | N/A  |
|       ├─IndexRangeScan_12(Build)     | 27523121.11  | 22441356  | cop[tikv] | table:lineitem, index:PRIMARY(L_ORDERKEY, L_LINENUMBER) | time:1m8.8s, loops:23635, cop_task: {num: 12537, max: 179.3ms, min: 331.4µs, avg: 22.7ms, p95: 66.9ms, max_proc_keys: 13755, p95_proc_keys: 5088, tot_proc: 3m44s, tot_wait: 129ms, rpc_num: 12537, rpc_time: 4m44s, copr_cache_hit_ratio: 0.06, distsql_concurrency: 15}, tikv_task:{proc max:170ms, min:0s, avg: 18.8ms, p80:29ms, p95:62ms, iters:68823, tasks:12537}, scan_detail: {total_process_keys: 21985403, total_process_keys_size: 1165226359, total_keys: 27492546, get_snapshot_time: 563.1ms, rocksdb: {key_skipped_count: 21985403, block: {cache_hit_count: 44004354}}}    | range: decided by [eq(tpch_100.lineitem.l_orderkey, tpch_100.orders.o_orderkey)], keep order:false                                                                                     | N/A        | N/A  |
|       └─Selection_14(Probe)          | 22018496.89  | 14185840  | cop[tikv] |                                                         | time:1m34.8s, loops:16882, cop_task: {num: 12057, max: 240.4ms, min: 328.5µs, avg: 20.4ms, p95: 73.1ms, max_proc_keys: 15005, p95_proc_keys: 6118, tot_proc: 3m39.8s, tot_wait: 300ms, rpc_num: 12057, rpc_time: 4m5.7s, copr_cache_hit_ratio: 0.28, distsql_concurrency: 15}, tikv_task:{proc max:230ms, min:0s, avg: 27.5ms, p80:45ms, p95:82ms, iters:74165, tasks:12057}, scan_detail: {total_process_keys: 19916937, total_process_keys_size: 3957799619, total_keys: 24053143, get_snapshot_time: 1.07s, rocksdb: {key_skipped_count: 19254567, block: {cache_hit_count: 37816908}}}  | lt(tpch_100.lineitem.l_commitdate, tpch_100.lineitem.l_receiptdate)                                                                                                                    | N/A        | N/A  |
|         └─TableRowIDScan_13          | 27523121.11  | 22441356  | cop[tikv] | table:lineitem                                          | tikv_task:{proc max:230ms, min:0s, avg: 27.2ms, p80:45ms, p95:81ms, iters:74165, tasks:12057}                                                                                                                                                                                                                                                                                                                                                                                                                                                                                               | keep order:false                                                                                                                                                                       | N/A        | N/A  |
+--------------------------------------+--------------+-----------+-----------+---------------------------------------------------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------+------+
10 rows in set (22.82 sec)

Enable MPP mode:

mysql> set @@tidb_enforce_mpp = ON;

Query OK, 0 rows affected (0.00 sec)

After MPP mode is enabled:

mysql> explain analyze select o_orderpriority, count(*) as order_count from orders where o_orderdate >= '1995-01-01' and o_orderdate < date_add('1995-01-01', interval '3' month) and exists (select * from lineitem where l_orderkey = o_orderkey and l_commitdate < l_receiptdate) group by o_orderpriority;
+--------------------------------------------------+--------------+-----------+--------------+----------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+------+
| id                                               | estRows      | actRows   | task         | access object  | execution info                                                                                                                                                                                                                                                                                                                                        | operator info                                                                                                                                                                 | memory | disk |
+--------------------------------------------------+--------------+-----------+--------------+----------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_59                                   | 1.00         | 5         | root         |                | time:5.97s, loops:2, cop_task: {num: 8, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                               | data:ExchangeSender_58                                                                                                                                                        | N/A    | N/A  |
| └─ExchangeSender_58                              | 1.00         | 5         | mpp[tiflash] |                | tiflash_task:{proc max:5.97s, min:5.97s, avg: 5.97s, p80:5.97s, p95:5.97s, iters:5, tasks:3, threads:60}                                                                                                                                                                                                                                              | ExchangeType: PassThrough                                                                                                                                                     | N/A    | N/A  |
|   └─Projection_10                                | 1.00         | 5         | mpp[tiflash] |                | tiflash_task:{proc max:5.97s, min:5.96s, avg: 5.97s, p80:5.97s, p95:5.97s, iters:5, tasks:3, threads:60}                                                                                                                                                                                                                                              | tpch_100.orders.o_orderpriority, Column#27                                                                                                                                    | N/A    | N/A  |
|     └─Projection_54                              | 1.00         | 5         | mpp[tiflash] |                | tiflash_task:{proc max:5.97s, min:5.96s, avg: 5.97s, p80:5.97s, p95:5.97s, iters:5, tasks:3, threads:60}                                                                                                                                                                                                                                              | Column#27, tpch_100.orders.o_orderpriority                                                                                                                                    | N/A    | N/A  |
|       └─HashAgg_55                               | 1.00         | 5         | mpp[tiflash] |                | tiflash_task:{proc max:5.97s, min:5.96s, avg: 5.97s, p80:5.97s, p95:5.97s, iters:5, tasks:3, threads:60}                                                                                                                                                                                                                                              | group by:tpch_100.orders.o_orderpriority, funcs:sum(Column#32)->Column#27, funcs:firstrow(tpch_100.orders.o_orderpriority)->tpch_100.orders.o_orderpriority, stream_count: 20 | N/A    | N/A  |
|         └─ExchangeReceiver_57                    | 1.00         | 15        | mpp[tiflash] |                | tiflash_task:{proc max:5.97s, min:5.96s, avg: 5.97s, p80:5.97s, p95:5.97s, iters:5, tasks:3, threads:60}                                                                                                                                                                                                                                              | stream_count: 20                                                                                                                                                              | N/A    | N/A  |
|           └─ExchangeSender_56                    | 1.00         | 15        | mpp[tiflash] |                | tiflash_task:{proc max:5.96s, min:0s, avg: 1.99s, p80:5.96s, p95:5.96s, iters:768, tasks:3, threads:60}                                                                                                                                                                                                                                               | ExchangeType: HashPartition, Hash Cols: [name: tpch_100.orders.o_orderpriority, collate: utf8mb4_bin], stream_count: 20                                                       | N/A    | N/A  |
|             └─HashAgg_52                         | 1.00         | 15        | mpp[tiflash] |                | tiflash_task:{proc max:5.96s, min:0s, avg: 1.99s, p80:5.96s, p95:5.96s, iters:768, tasks:3, threads:60}                                                                                                                                                                                                                                               | group by:tpch_100.orders.o_orderpriority, funcs:count(1)->Column#32                                                                                                           | N/A    | N/A  |
|               └─HashJoin_41                      | 4429262.13   | 5145138   | mpp[tiflash] |                | tiflash_task:{proc max:5.95s, min:0s, avg: 1.98s, p80:5.95s, p95:5.95s, iters:118, tasks:3, threads:60}                                                                                                                                                                                                                                               | semi join, equal:[eq(tpch_100.orders.o_orderkey, tpch_100.lineitem.l_orderkey)], stream_count: 20                                                                             | N/A    | N/A  |
|                 ├─ExchangeReceiver_26(Build)     | 480030321.60 | 379356474 | mpp[tiflash] |                | tiflash_task:{proc max:3.26s, min:0s, avg: 1.09s, p80:3.26s, p95:3.26s, iters:42469, tasks:3, threads:60}                                                                                                                                                                                                                                             | stream_count: 20                                                                                                                                                              | N/A    | N/A  |
|                 │ └─ExchangeSender_25            | 480030321.60 | 379356474 | mpp[tiflash] |                | tiflash_task:{proc max:3.37s, min:0s, avg: 1.12s, p80:3.37s, p95:3.37s, iters:9292, tasks:3, threads:60}                                                                                                                                                                                                                                              | ExchangeType: HashPartition, Hash Cols: [name: tpch_100.lineitem.l_orderkey, collate: binary], stream_count: 20                                                               | N/A    | N/A  |
|                 │   └─Projection_60              | 480030321.60 | 379356474 | mpp[tiflash] |                | tiflash_task:{proc max:819.1ms, min:0s, avg: 273ms, p80:819.1ms, p95:819.1ms, iters:9292, tasks:3, threads:60}                                                                                                                                                                                                                                        | tpch_100.lineitem.l_orderkey                                                                                                                                                  | N/A    | N/A  |
|                 │     └─Selection_24             | 480030321.60 | 379356474 | mpp[tiflash] |                | tiflash_task:{proc max:817.1ms, min:0s, avg: 272.4ms, p80:817.1ms, p95:817.1ms, iters:9292, tasks:3, threads:60}                                                                                                                                                                                                                                      | lt(tpch_100.lineitem.l_commitdate, tpch_100.lineitem.l_receiptdate)                                                                                                           | N/A    | N/A  |
|                 │       └─TableFullScan_23       | 600037902.00 | 600037902 | mpp[tiflash] | table:lineitem | tiflash_task:{proc max:328.1ms, min:0s, avg: 109.4ms, p80:328.1ms, p95:328.1ms, iters:9292, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73475, total_skipped_packs:13452, total_scanned_rows:600230532, total_skipped_rows:109887612, total_rs_index_load_time: 35ms, total_read_time: 34065ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                                                                              | N/A    | N/A  |
|                 └─ExchangeReceiver_22(Probe)     | 5536577.67   | 5610951   | mpp[tiflash] |                | tiflash_task:{proc max:2.5s, min:0s, avg: 833.7ms, p80:2.5s, p95:2.5s, iters:669, tasks:3, threads:60}                                                                                                                                                                                                                                                |                                                                                                                                                                               | N/A    | N/A  |
|                   └─ExchangeSender_21            | 5536577.67   | 5610951   | mpp[tiflash] |                | tiflash_task:{proc max:5.93s, min:0s, avg: 1.98s, p80:5.93s, p95:5.93s, iters:2336, tasks:3, threads:60}                                                                                                                                                                                                                                              | ExchangeType: HashPartition, Hash Cols: [name: tpch_100.orders.o_orderkey, collate: binary]                                                                                   | N/A    | N/A  |
|                     └─Selection_20               | 5536577.67   | 5610951   | mpp[tiflash] |                | tiflash_task:{proc max:325.5ms, min:0s, avg: 108.5ms, p80:325.5ms, p95:325.5ms, iters:2336, tasks:3, threads:60}                                                                                                                                                                                                                                      | ge(tpch_100.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch_100.orders.o_orderdate, 1995-04-01 00:00:00.000000)                                                      | N/A    | N/A  |
|                       └─TableFullScan_19         | 150000000.00 | 150000000 | mpp[tiflash] | table:orders   | tiflash_task:{proc max:238.5ms, min:0s, avg: 79.5ms, p80:238.5ms, p95:238.5ms, iters:2336, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:18541, total_skipped_packs:186, total_scanned_rows:150000611, total_skipped_rows:1513328, total_rs_index_load_time: 75ms, total_read_time: 11454ms}, total_create_snapshot_time: 0ms}      | keep order:false                                                                                                                                                              | N/A    | N/A  |
+--------------------------------------------------+--------------+-----------+--------------+----------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------+------+
18 rows in set (6.00 sec)

Push down aggregate functions to a position before Join or Union

By pushing down aggregate operations to the position before Join or Union, you can reduce the data to be processed in the Join or Union operation, thereby improving performance.

The variable tidb_opt_agg_push_down controls whether the optimizer executes the optimization operation of pushing down the aggregate function to the position before Join or Union. When the aggregate operations are quite slow in the query, you can set this variable to ON.

set @@tidb_opt_agg_push_down = ON;

The following example shows the query result before and after the tidb_opt_agg_push_down variable is enabled. Before this variable is enabled, the HashAgg_58 operation is executed following the HashJoin_41 operation. After this variable is enabled, newly generated HashAgg_21 and HashAgg_32 operations are executed before the HashJoin_76 operation. This significantly reduces the data to be processed by the Join operation.

Before tidb_opt_agg_push_down is enabled:

mysql> explain analyze select count(*) from t1 join t2 where t1.a = t2.b group by t1.a;
+------------------------------------------+--------------+-----------+--------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+--------+------+
| id                                       | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                                | operator info                                                                                      | memory | disk |
+------------------------------------------+--------------+-----------+--------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+--------+------+
| TableReader_66                           | 100450000.00 | 20        | root         |               | time:2.13s, loops:2, cop_task: {num: 21, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                      | data:ExchangeSender_65                                                                             | N/A    | N/A  |
| └─ExchangeSender_65                      | 100450000.00 | 20        | mpp[tiflash] |               | tiflash_task:{proc max:2.13s, min:2.08s, avg: 2.11s, p80:2.13s, p95:2.13s, iters:768, tasks:3, threads:60}                                                                                                                                                                                                                                    | ExchangeType: PassThrough                                                                          | N/A    | N/A  |
|   └─Projection_60                        | 100450000.00 | 20        | mpp[tiflash] |               | tiflash_task:{proc max:2.13s, min:2.08s, avg: 2.11s, p80:2.13s, p95:2.13s, iters:768, tasks:3, threads:60}                                                                                                                                                                                                                                    | Column#43                                                                                          | N/A    | N/A  |
|     └─HashAgg_58                         | 100450000.00 | 20        | mpp[tiflash] |               | tiflash_task:{proc max:2.13s, min:2.08s, avg: 2.11s, p80:2.13s, p95:2.13s, iters:768, tasks:3, threads:60}                                                                                                                                                                                                                                    | group by:test.t1.a, funcs:count(1)->Column#43                                                      | N/A    | N/A  |
|       └─HashJoin_41                      | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:1.77s, min:1.67s, avg: 1.73s, p80:1.77s, p95:1.77s, iters:10955, tasks:3, threads:60}                                                                                                                                                                                                                                  | inner join, equal:[eq(test.t1.a, test.t2.b)], stream_count: 20                                     | N/A    | N/A  |
|         ├─ExchangeReceiver_22(Build)     | 100450000.00 | 99980000  | mpp[tiflash] |               | tiflash_task:{proc max:171ms, min:127ms, avg: 154.5ms, p80:171ms, p95:171ms, iters:10440, tasks:3, threads:60}                                                                                                                                                                                                                                | stream_count: 20                                                                                   | N/A    | N/A  |
|         │ └─ExchangeSender_21            | 100450000.00 | 99980000  | mpp[tiflash] |               | tiflash_task:{proc max:498.7ms, min:0s, avg: 166.2ms, p80:498.7ms, p95:498.7ms, iters:1653, tasks:3, threads:60}                                                                                                                                                                                                                              | ExchangeType: HashPartition, Hash Cols: [name: test.t1.a, collate: binary], stream_count: 20       | N/A    | N/A  |
|         │   └─Selection_20               | 100450000.00 | 99980000  | mpp[tiflash] |               | tiflash_task:{proc max:83.7ms, min:0s, avg: 27.9ms, p80:83.7ms, p95:83.7ms, iters:1653, tasks:3, threads:60}                                                                                                                                                                                                                                  | not(isnull(test.t1.a))                                                                             | N/A    | N/A  |
|         │     └─TableFullScan_19         | 100450000.00 | 99980000  | mpp[tiflash] | table:t1      | tiflash_task:{proc max:41.7ms, min:0s, avg: 13.9ms, p80:41.7ms, p95:41.7ms, iters:1653, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:12336, total_skipped_packs:5759, total_scanned_rows:100083566, total_skipped_rows:46786721, total_rs_index_load_time: 5ms, total_read_time: 1105ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                   | N/A    | N/A  |
|         └─ExchangeReceiver_26(Probe)     | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:373ms, min:71ms, avg: 253.2ms, p80:373ms, p95:373ms, iters:28135, tasks:3, threads:60}                                                                                                                                                                                                                                 |                                                                                                    | N/A    | N/A  |
|           └─ExchangeSender_25            | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:2.07s, min:0s, avg: 689.6ms, p80:2.07s, p95:2.07s, iters:9388, tasks:3, threads:60}                                                                                                                                                                                                                                    | ExchangeType: HashPartition, Hash Cols: [name: test.t2.b, collate: binary]                         | N/A    | N/A  |
|             └─Selection_24               | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:146.9ms, min:0s, avg: 49ms, p80:146.9ms, p95:146.9ms, iters:9388, tasks:3, threads:60}                                                                                                                                                                                                                                 | not(isnull(test.t2.b))                                                                             | N/A    | N/A  |
|               └─TableFullScan_23         | 600000000.00 | 600000000 | mpp[tiflash] | table:t2      | tiflash_task:{proc max:63.8ms, min:0s, avg: 21.3ms, p80:63.8ms, p95:63.8ms, iters:9388, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73834, total_skipped_packs:439, total_scanned_rows:600007458, total_skipped_rows:3553131, total_rs_index_load_time: 12ms, total_read_time: 270ms}, total_create_snapshot_time: 0ms}   | keep order:false                                                                                   | N/A    | N/A  |
+------------------------------------------+--------------+-----------+--------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------+--------+------+
13 rows in set (2.15 sec)

Enable tidb_opt_agg_push_down:

mysql> set @@tidb_opt_agg_push_down = ON;
Query OK, 0 rows affected (0.00 sec)

After tidb_opt_agg_push_down is enabled:

mysql> explain analyze select count(*) from t1 join t2 where t1.a = t2.b group by t1.a;
+------------------------------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
| id                                                   | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                            | operator info                                                                                                                 | memory | disk |
+------------------------------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_85                                       | 20.00        | 20        | root         |               | time:432.4ms, loops:2, cop_task: {num: 19, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                | data:ExchangeSender_84                                                                                                        | N/A    | N/A  |
| └─ExchangeSender_84                                  | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:431.1ms, min:429.3ms, avg: 430.4ms, p80:431.1ms, p95:431.1ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                       | ExchangeType: PassThrough                                                                                                     | N/A    | N/A  |
|   └─Projection_83                                    | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:430.1ms, min:429.3ms, avg: 429.7ms, p80:430.1ms, p95:430.1ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                       | Column#43                                                                                                                     | N/A    | N/A  |
|     └─HashAgg_82                                     | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:430.1ms, min:429.3ms, avg: 429.7ms, p80:430.1ms, p95:430.1ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                       | group by:test.t1.a, funcs:count(Column#44)->Column#43, stream_count: 20                                                       | N/A    | N/A  |
|       └─ExchangeReceiver_78                          | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:430.1ms, min:429.3ms, avg: 429.7ms, p80:430.1ms, p95:430.1ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                       | stream_count: 20                                                                                                              | N/A    | N/A  |
|         └─ExchangeSender_77                          | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:427ms, min:0s, avg: 142.3ms, p80:427ms, p95:427ms, iters:60, tasks:3, threads:60}                                                                                                                                                                                                                                  | ExchangeType: HashPartition, Hash Cols: [name: test.t1.a, collate: binary], stream_count: 20                                  | N/A    | N/A  |
|           └─HashJoin_76                              | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:426ms, min:0s, avg: 142ms, p80:426ms, p95:426ms, iters:60, tasks:3, threads:60}                                                                                                                                                                                                                                    | inner join, equal:[eq(test.t1.a, test.t2.b)]                                                                                  | N/A    | N/A  |
|             ├─ExchangeReceiver_36(Build)             | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:374ms, min:0s, avg: 124.7ms, p80:374ms, p95:374ms, iters:46, tasks:3, threads:60}                                                                                                                                                                                                                                  |                                                                                                                               | N/A    | N/A  |
|             │ └─ExchangeSender_35                    | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:373.7ms, min:0s, avg: 124.6ms, p80:373.7ms, p95:373.7ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                            | ExchangeType: Broadcast                                                                                                       | N/A    | N/A  |
|             │   └─Projection_31                      | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:373.7ms, min:0s, avg: 124.6ms, p80:373.7ms, p95:373.7ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                            | Column#44, test.t2.b                                                                                                          | N/A    | N/A  |
|             │     └─HashAgg_32                       | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:373.7ms, min:0s, avg: 124.6ms, p80:373.7ms, p95:373.7ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                            | group by:test.t2.b, funcs:sum(Column#45)->Column#44, funcs:firstrow(test.t2.b)->test.t2.b, stream_count: 20                   | N/A    | N/A  |
|             │       └─ExchangeReceiver_34            | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:373.7ms, min:0s, avg: 124.6ms, p80:373.7ms, p95:373.7ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                            | stream_count: 20                                                                                                              | N/A    | N/A  |
|             │         └─ExchangeSender_33            | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:368.3ms, min:0s, avg: 122.8ms, p80:368.3ms, p95:368.3ms, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                             | ExchangeType: HashPartition, Hash Cols: [name: test.t2.b, collate: binary], stream_count: 20                                  | N/A    | N/A  |
|             │           └─HashAgg_21                 | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:367.3ms, min:0s, avg: 122.4ms, p80:367.3ms, p95:367.3ms, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                             | group by:test.t2.b, funcs:count(1)->Column#45                                                                                 | N/A    | N/A  |
|             │             └─Selection_30             | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:77.2ms, min:0s, avg: 25.7ms, p80:77.2ms, p95:77.2ms, iters:9389, tasks:3, threads:60}                                                                                                                                                                                                                              | not(isnull(test.t2.b))                                                                                                        | N/A    | N/A  |
|             │               └─TableFullScan_29       | 600000000.00 | 600000000 | mpp[tiflash] | table:t2      | tiflash_task:{proc max:37.2ms, min:0s, avg: 12.4ms, p80:37.2ms, p95:37.2ms, iters:9389, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73834, total_skipped_packs:412, total_scanned_rows:600002896, total_skipped_rows:3342145, total_rs_index_load_time: 0ms, total_read_time: 45ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                              | N/A    | N/A  |
|             └─Selection_19(Probe)                    | 100450000.00 | 99980000  | mpp[tiflash] |               | tiflash_task:{proc max:30ms, min:0s, avg: 10ms, p80:30ms, p95:30ms, iters:1645, tasks:3, threads:60}                                                                                                                                                                                                                                      | not(isnull(test.t1.a))                                                                                                        | N/A    | N/A  |
|               └─TableFullScan_18                     | 100450000.00 | 99980000  | mpp[tiflash] | table:t1      | tiflash_task:{proc max:22ms, min:0s, avg: 7.34ms, p80:22ms, p95:22ms, iters:1645, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:12326, total_skipped_packs:4762, total_scanned_rows:100045535, total_skipped_rows:38651216, total_rs_index_load_time: 4ms, total_read_time: 355ms}, total_create_snapshot_time: 0ms}    | keep order:false                                                                                                              | N/A    | N/A  |
+------------------------------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
18 rows in set (0.46 sec)

Enable Distinct optimization

TiFlash does not support some aggregate functions that accept the Distinct column, such as Sum. By default, the entire aggregate function is calculated in TiDB. By enabling the Distinct optimization, some operations can be pushed down to TiFlash, thereby improving query performance.

If the aggregate function with the distinct operation is slow in a query, you can enable the optimization operation of pushing down the aggregate function with Distinct (such as select sum(distinct a) from t) to Coprocessor by setting the value of the tidb_opt_distinct_agg_push_down variable to ON.

set @@tidb_opt_distinct_agg_push_down = ON;

The following example shows the query result before and after the tidb_opt_distinct_agg_push_down variable is enabled. Before this variable is enabled, TiDB needs to read all data from TiFlash and execute distinct in TiDB. After this variable is enabled, distinct a is pushed down to TiFlash, and a new group by column test.t.a is added in HashAgg_6. The two warnings in the query result indicate that the aggregate function cannot be fully pushed down to TiFlash.

Before tidb_opt_distinct_agg_push_down is enabled:

mysql> explain analyze select count(distinct a) from test.t;
+----------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+----------+---------+
| id                         | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                                   | operator info                                          | memory   | disk    |
+----------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+----------+---------+
| HashAgg_6                  | 1.00         | 1         | root         |               | time:2m23.2s, loops:2                                                                                                                                                                                                                                                                                                                            | funcs:sum(distinct Column#23)->Column#22               | 41.3 KB  | 0 Bytes |
| └─Projection_16            | 600000000.00 | 600000000 | root         |               | time:2.51s, loops:586548, Concurrency:5                                                                                                                                                                                                                                                                                                          | cast(test.t.a, decimal(10,0) BINARY)->Column#23        | 243.2 KB | N/A     |
|   └─TableReader_11         | 600000000.00 | 600000000 | root         |               | time:1.6s, loops:586548, cop_task: {num: 1173, max: 256.2ms, min: 25.1ms, avg: 46.9ms, p95: 63.5ms, rpc_num: 1173, rpc_time: 55s, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15}                                                                                                                                                           | data:TableFullScan_10                                  | 70.2 MB  | N/A     |
|     └─TableFullScan_10     | 600000000.00 | 600000000 | cop[tiflash] | table:t       | tiflash_task:{proc max:9ms, min:531µs, avg: 4.49ms, p80:5.55ms, p95:6.74ms, iters:9390, tasks:1173, threads:1173}, tiflash_scan:{dtfile:{total_scanned_packs:73834, total_skipped_packs:1231, total_scanned_rows:600010914, total_skipped_rows:9988978, total_rs_index_load_time: 0ms, total_read_time: 16ms}, total_create_snapshot_time: 0ms}  | keep order:false                                       | N/A      | N/A     |
+----------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+----------+---------+
4 rows in set, 2 warnings (2 min 23.21 sec)

Enable tidb_opt_distinct_agg_push_down:

mysql> set @@tidb_opt_distinct_agg_push_down = ON;
Query OK, 0 rows affected (0.00 sec)

After tidb_opt_distinct_agg_push_down is enabled:

mysql> explain analyze select count(distinct a) from test.t;
+-----------------------------+--------------+-----------+-------------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+-----------+---------+
| id                          | estRows      | actRows   | task              | access object | execution info                                                                                                                                                                                                                                                                                                                                | operator info                                          | memory    | disk    |
+-----------------------------+--------------+-----------+-------------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+-----------+---------+
| HashAgg_10                  | 1.00         | 1         | root              |               | time:233.8ms, loops:2                                                                                                                                                                                                                                                                                                                         | funcs:sum(distinct Column#23)->Column#22               | 2.42 KB   | 0 Bytes |
| └─Projection_12             | 1.00         | 3         | root              |               | time:233.7ms, loops:2, Concurrency:OFF                                                                                                                                                                                                                                                                                                        | cast(test.c.a, decimal(10,0) BINARY)->Column#23        | 380 Bytes | N/A     |
|   └─TableReader_11          | 1.00         | 3         | root              |               | time:233.7ms, loops:2, cop_task: {num: 6, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15}                                                                                                                                                                                                            | data:HashAgg_6                                         | 100 Bytes | N/A     |
|     └─HashAgg_6             | 1.00         | 3         | batchCop[tiflash] |               | tiflash_task:{proc max:225.8ms, min:210.7ms, avg: 216.9ms, p80:225.8ms, p95:225.8ms, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                            | group by:test.t.a,                                     | N/A       | N/A     |
|       └─TableFullScan_9     | 600000000.00 | 600000000 | batchCop[tiflash] | table:t       | tiflash_task:{proc max:50.3ms, min:33.7ms, avg: 44.6ms, p80:50.3ms, p95:50.3ms, iters:9387, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73833, total_skipped_packs:475, total_scanned_rows:600000000, total_skipped_rows:3852098, total_rs_index_load_time: 0ms, total_read_time: 84ms}, total_create_snapshot_time: 0ms} | keep order:false                                       | N/A       | N/A     |
+-----------------------------+--------------+-----------+-------------------+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--------------------------------------------------------+-----------+---------+
5 rows in set, 2 warnings (0.24 sec)

Compact data using the ALTER TABLE ... COMPACT statement

Executing the ALTER TABLE ... COMPACT statement can initiate compaction for a specific table or partition on a TiFlash node. During the compaction, the physical data on the node is rewritten, including cleaning up deleted rows and merging multiple versions of data caused by updates. This helps enhance access performance and reduce disk usage. The following are examples:

ALTER TABLE employees COMPACT TIFLASH REPLICA;
ALTER TABLE employees COMPACT PARTITION pNorth, pEast TIFLASH REPLICA;

Replace Shuffled Hash Join with Broadcast Hash Join

For Join operations with small tables, the Broadcast Hash Join algorithm can avoid transfering large tables, thereby improving the computing performance.

  • The tidb_broadcast_join_threshold_size variable controls whether to use the Broadcast Hash Join algorithm. If the table size (unit: byte) is smaller than the value of this variable, the Broadcast Hash Join algorithm is used. Otherwise, the Shuffled Hash Join algorithm is used.

    set @@tidb_broadcast_join_threshold_size = 2000000;
  • The tidb_broadcast_join_threshold_count variable also controls whether to use the Broadcast Hash Join algorithm. If the objects of the join operation belong to a subquery, the optimizer cannot estimate the size of the subquery result set. In this situation, the size is determined by the number of rows in the result set. If the estimated number of rows for the subquery is fewer than the value of this variable, the Broadcast Hash Join algorithm is used. Otherwise, the Shuffled Hash Join algorithm is used.

    set @@tidb_broadcast_join_threshold_count = 100000;

The following example shows the query result before and after tidb_broadcast_join_threshold_size is re-configured. Before the re-configuration, the ExchangeType of ExchangeSender_29 is HashPartition. After the value of this variable chages to 10000000, the ExchangeType of ExchangeSender_29 changes to Broadcast.

Before tidb_broadcast_join_threshold_size is re-configured:

mysql> explain analyze select max(l_shipdate), max(l_commitdate), max(l_receiptdate) from supplier,lineitem where s_suppkey = l_suppkey;
+------------------------------------------+--------------+-----------+--------------+----------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
| id                                       | estRows      | actRows   | task         | access object  | execution info                                                                                                                                                                                                                                                                                                                                               | operator info                                                                                                                                                   | memory  | disk |
+------------------------------------------+--------------+-----------+--------------+----------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
| HashAgg_45                               | 1.00         | 1         | root         |                | time:3.8s, loops:2, partial_worker:{wall_time:3.798284809s, concurrency:5, task_num:1, tot_wait:18.99079929s, tot_exec:8.193µs, tot_time:18.990819019s, max:3.798181723s, p95:3.798181723s}, final_worker:{wall_time:0s, concurrency:5, task_num:1, tot_wait:18.991291379s, tot_exec:197.45µs, tot_time:18.991494363s, max:3.798334972s, p95:3.798334972s}   | funcs:max(Column#28)->Column#25, funcs:max(Column#29)->Column#26, funcs:max(Column#30)->Column#27                                                               | 17.3 KB | N/A  |
| └─TableReader_47                         | 1.00         | 3         | root         |                | time:3.8s, loops:2, cop_task: {num: 6, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                                       | data:ExchangeSender_46                                                                                                                                          | N/A     | N/A  |
|   └─ExchangeSender_46                    | 1.00         | 3         | mpp[tiflash] |                | tiflash_task:{proc max:3.79s, min:3.78s, avg: 3.78s, p80:3.79s, p95:3.79s, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                                                     | ExchangeType: PassThrough                                                                                                                                       | N/A     | N/A  |
|     └─HashAgg_13                         | 1.00         | 3         | mpp[tiflash] |                | tiflash_task:{proc max:3.79s, min:3.78s, avg: 3.78s, p80:3.79s, p95:3.79s, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                                                     | funcs:max(tpch_100.lineitem.l_shipdate)->Column#28, funcs:max(tpch_100.lineitem.l_commitdate)->Column#29, funcs:max(tpch_100.lineitem.l_receiptdate)->Column#30 | N/A     | N/A  |
|       └─HashJoin_44                      | 600845438.27 | 600037902 | mpp[tiflash] |                | tiflash_task:{proc max:3.17s, min:3.12s, avg: 3.14s, p80:3.17s, p95:3.17s, iters:11143, tasks:3, threads:60}                                                                                                                                                                                                                                                 | inner join, equal:[eq(tpch_100.supplier.s_suppkey, tpch_100.lineitem.l_suppkey)], stream_count: 20                                                              | N/A     | N/A  |
|         ├─ExchangeReceiver_30(Build)     | 1000000.00   | 1000000   | mpp[tiflash] |                | tiflash_task:{proc max:23.3ms, min:21ms, avg: 22.5ms, p80:23.3ms, p95:23.3ms, iters:120, tasks:3, threads:60}                                                                                                                                                                                                                                                | stream_count: 20                                                                                                                                                | N/A     | N/A  |
|         │ └─ExchangeSender_29            | 1000000.00   | 1000000   | mpp[tiflash] |                | tiflash_task:{proc max:22.7ms, min:0s, avg: 7.57ms, p80:22.7ms, p95:22.7ms, iters:16, tasks:3, threads:2}                                                                                                                                                                                                                                                    | ExchangeType: HashPartition, Hash Cols: [name: tpch_100.supplier.s_suppkey, collate: binary], stream_count: 20                                                  | N/A     | N/A  |
|         │   └─TableFullScan_28           | 1000000.00   | 1000000   | mpp[tiflash] | table:supplier | tiflash_task:{proc max:9.71ms, min:0s, avg: 3.24ms, p80:9.71ms, p95:9.71ms, iters:16, tasks:3, threads:2}, tiflash_scan:{dtfile:{total_scanned_packs:123, total_skipped_packs:0, total_scanned_rows:1000000, total_skipped_rows:0, total_rs_index_load_time: 0ms, total_read_time: 2ms}, total_create_snapshot_time: 0ms}                                    | keep order:false                                                                                                                                                | N/A     | N/A  |
|         └─ExchangeReceiver_33(Probe)     | 600037902.00 | 600037902 | mpp[tiflash] |                | tiflash_task:{proc max:564.3ms, min:340ms, avg: 438.5ms, p80:564.3ms, p95:564.3ms, iters:27583, tasks:3, threads:60}                                                                                                                                                                                                                                         |                                                                                                                                                                 | N/A     | N/A  |
|           └─ExchangeSender_32            | 600037902.00 | 600037902 | mpp[tiflash] |                | tiflash_task:{proc max:3.69s, min:0s, avg: 1.23s, p80:3.69s, p95:3.69s, iters:9298, tasks:3, threads:60}                                                                                                                                                                                                                                                     | ExchangeType: HashPartition, Hash Cols: [name: tpch_100.lineitem.l_suppkey, collate: binary]                                                                    | N/A     | N/A  |
|             └─TableFullScan_31           | 600037902.00 | 600037902 | mpp[tiflash] | table:lineitem | tiflash_task:{proc max:62.8ms, min:0s, avg: 20.9ms, p80:62.8ms, p95:62.8ms, iters:9298, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73465, total_skipped_packs:13060, total_scanned_rows:600168663, total_skipped_rows:106699256, total_rs_index_load_time: 16ms, total_read_time: 27979ms}, total_create_snapshot_time: 0ms}            | keep order:false                                                                                                                                                | N/A     | N/A  |
+------------------------------------------+--------------+-----------+--------------+----------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
11 rows in set (3.83 sec)

Set tidb_broadcast_join_threshold_size to 10000000:

mysql> set @@tidb_broadcast_join_threshold_size = 10000000;
Query OK, 0 rows affected (0.00 sec)

After tidb_broadcast_join_threshold_size is set to 10000000:

mysql> explain analyze select max(l_shipdate), max(l_commitdate), max(l_receiptdate) from supplier,lineitem where s_suppkey = l_suppkey;
+------------------------------------------+--------------+-----------+--------------+----------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
| id                                       | estRows      | actRows   | task         | access object  | execution info                                                                                                                                                                                                                                                                                                                                                         | operator info                                                                                                                                                   | memory  | disk |
+------------------------------------------+--------------+-----------+--------------+----------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
| HashAgg_43                               | 1.00         | 1         | root         |                | time:2.75s, loops:2, partial_worker:{wall_time:2.748445779s, concurrency:5, task_num:1, tot_wait:13.74202679s, tot_exec:8.012µs, tot_time:13.742045721s, max:2.748414915s, p95:2.748414915s}, final_worker:{wall_time:2.74848039s, concurrency:5, task_num:1, tot_wait:13.742157526s, tot_exec:53.191µs, tot_time:13.742214417s, max:2.748462393s, p95:2.748462393s}   | funcs:max(Column#28)->Column#25, funcs:max(Column#29)->Column#26, funcs:max(Column#30)->Column#27                                                               | 17.3 KB | N/A  |
| └─TableReader_45                         | 1.00         | 3         | root         |                | time:2.75s, loops:2, cop_task: {num: 6, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                                                | data:ExchangeSender_44                                                                                                                                          | N/A     | N/A  |
|   └─ExchangeSender_44                    | 1.00         | 3         | mpp[tiflash] |                | tiflash_task:{proc max:2.74s, min:2.65s, avg: 2.7s, p80:2.74s, p95:2.74s, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                                                                | ExchangeType: PassThrough                                                                                                                                       | N/A     | N/A  |
|     └─HashAgg_13                         | 1.00         | 3         | mpp[tiflash] |                | tiflash_task:{proc max:2.74s, min:2.65s, avg: 2.7s, p80:2.74s, p95:2.74s, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                                                                | funcs:max(tpch_100.lineitem.l_shipdate)->Column#28, funcs:max(tpch_100.lineitem.l_commitdate)->Column#29, funcs:max(tpch_100.lineitem.l_receiptdate)->Column#30 | N/A     | N/A  |
|       └─HashJoin_42                      | 600845438.27 | 600037902 | mpp[tiflash] |                | tiflash_task:{proc max:2.15s, min:2.06s, avg: 2.11s, p80:2.15s, p95:2.15s, iters:9286, tasks:3, threads:60}                                                                                                                                                                                                                                                            | inner join, equal:[eq(tpch_100.supplier.s_suppkey, tpch_100.lineitem.l_suppkey)]                                                                                | N/A     | N/A  |
|         ├─ExchangeReceiver_30(Build)     | 1000000.00   | 3000000   | mpp[tiflash] |                | tiflash_task:{proc max:28.1ms, min:24.9ms, avg: 27ms, p80:28.1ms, p95:28.1ms, iters:48, tasks:3, threads:60}                                                                                                                                                                                                                                                           |                                                                                                                                                                 | N/A     | N/A  |
|         │ └─ExchangeSender_29            | 1000000.00   | 1000000   | mpp[tiflash] |                | tiflash_task:{proc max:14ms, min:0s, avg: 4.67ms, p80:14ms, p95:14ms, iters:16, tasks:3, threads:2}                                                                                                                                                                                                                                                                    | ExchangeType: Broadcast                                                                                                                                         | N/A     | N/A  |
|         │   └─TableFullScan_28           | 1000000.00   | 1000000   | mpp[tiflash] | table:supplier | tiflash_task:{proc max:9ms, min:0s, avg: 3ms, p80:9ms, p95:9ms, iters:16, tasks:3, threads:2}, tiflash_scan:{dtfile:{total_scanned_packs:123, total_skipped_packs:0, total_scanned_rows:1000000, total_skipped_rows:0, total_rs_index_load_time: 0ms, total_read_time: 2ms}, total_create_snapshot_time: 0ms}                                                          | keep order:false                                                                                                                                                | N/A     | N/A  |
|         └─TableFullScan_31(Probe)        | 600037902.00 | 600037902 | mpp[tiflash] | table:lineitem | tiflash_task:{proc max:57.9ms, min:42.9ms, avg: 51.3ms, p80:57.9ms, p95:57.9ms, iters:9297, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73464, total_skipped_packs:12985, total_scanned_rows:600169085, total_skipped_rows:106014866, total_rs_index_load_time: 23ms, total_read_time: 21667ms}, total_create_snapshot_time: 0ms}                  | keep order:false                                                                                                                                                | N/A     | N/A  |
+------------------------------------------+--------------+-----------+--------------+----------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+------+
9 rows in set (2.76 sec)

Set a greater execution concurrency

A greater execution concurrency allows TiFlash to occupy more CPU resources of the system, thereby improving query performance.

The tidb_max_tiflash_threads variable is used to set the maximum concurrency for TiFlash to execute a request. The unit is threads.

set @@tidb_max_tiflash_threads = 20;

The following example shows the query result before and after tidb_max_tiflash_threads is re-configured. Before the re-configuration, the execution concurrency of all TiFlash operators is 24. After the value of this variable changes to 20, the concurrency becomes 60.

Before tidb_max_tiflash_threads is re-configured:

mysql> explain analyze select a, count(*) from t group by a;
+--------------------------------------+--------------+-----------+--------------+---------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------+--------+------+
| id                                   | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                     | operator info                                                                                                                | memory | disk |
+--------------------------------------+--------------+-----------+--------------+---------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_44                       | 20.00        | 20        | root         |               | time:655.9ms, loops:2, cop_task: {num: 14, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                         | data:ExchangeSender_43                                                                                                       | N/A    | N/A  |
| └─ExchangeSender_43                  | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:654.6ms, min:652.4ms, avg: 653.8ms, p80:654.6ms, p95:654.6ms, iters:11, tasks:3, threads:24}                                                                                                                                                                                                                | ExchangeType: PassThrough                                                                                                    | N/A    | N/A  |
|   └─Projection_5                     | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:654.3ms, min:651.4ms, avg: 653.1ms, p80:654.3ms, p95:654.3ms, iters:11, tasks:3, threads:24}                                                                                                                                                                                                                | test.t.a, Column#22                                                                                                          | N/A    | N/A  |
|     └─Projection_39                  | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:654.3ms, min:651.4ms, avg: 653.1ms, p80:654.3ms, p95:654.3ms, iters:11, tasks:3, threads:24}                                                                                                                                                                                                                | Column#22, test.t.a                                                                                                          | N/A    | N/A  |
|       └─HashAgg_40                   | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:654.3ms, min:651.4ms, avg: 653.1ms, p80:654.3ms, p95:654.3ms, iters:11, tasks:3, threads:24}                                                                                                                                                                                                                | group by:test.t.a, funcs:sum(Column#29)->Column#22, funcs:firstrow(test.t.a)->test.t.a, stream_count: 8                      | N/A    | N/A  |
|         └─ExchangeReceiver_42        | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:654.3ms, min:651.4ms, avg: 653.1ms, p80:654.3ms, p95:654.3ms, iters:11, tasks:3, threads:24}                                                                                                                                                                                                                | stream_count: 8                                                                                                              | N/A    | N/A  |
|           └─ExchangeSender_41        | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:649ms, min:0s, avg: 216.3ms, p80:649ms, p95:649ms, iters:3, tasks:3, threads:24}                                                                                                                                                                                                                            | ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], stream_count: 8                                   | N/A    | N/A  |
|             └─HashAgg_37             | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:649ms, min:0s, avg: 216.3ms, p80:649ms, p95:649ms, iters:3, tasks:3, threads:24}                                                                                                                                                                                                                            | group by:test.t.a, funcs:count(1)->Column#29                                                                                 | N/A    | N/A  |
|               └─TableFullScan_26     | 600000000.00 | 600000000 | mpp[tiflash] | table:t       | tiflash_task:{proc max:40ms, min:0s, avg: 13.3ms, p80:40ms, p95:40ms, iters:9386, tasks:3, threads:24}, tiflash_scan:{dtfile:{total_scanned_packs:73833, total_skipped_packs:174, total_scanned_rows:600000000, total_skipped_rows:1402537, total_rs_index_load_time: 0ms, total_read_time: 4ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                             | N/A    | N/A  |
+--------------------------------------+--------------+-----------+--------------+---------------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------------------------------------------------------------------------------------------------------------------+--------+------+
9 rows in set (0.67 sec)

Set tidb_max_tiflash_threads to 20:

mysql> set @@tidb_max_tiflash_threads = 20;
Query OK, 0 rows affected (0.00 sec)

After tidb_max_tiflash_threads is set to 20:

mysql> explain analyze select a, count(*) from t group by a;
+--------------------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
| id                                   | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                             | operator info                                                                                                                 | memory | disk |
+--------------------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_44                       | 20.00        | 20        | root         |               | time:357.7ms, loops:2, cop_task: {num: 19, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                 | data:ExchangeSender_43                                                                                                        | N/A    | N/A  |
| └─ExchangeSender_43                  | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:355.3ms, min:354.6ms, avg: 355ms, p80:355.3ms, p95:355.3ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                          | ExchangeType: PassThrough                                                                                                     | N/A    | N/A  |
|   └─Projection_5                     | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:354.3ms, min:353.6ms, avg: 354ms, p80:354.3ms, p95:354.3ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                          | test.t.a, Column#22                                                                                                           | N/A    | N/A  |
|     └─Projection_39                  | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:354.3ms, min:353.6ms, avg: 354ms, p80:354.3ms, p95:354.3ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                          | Column#22, test.t.a                                                                                                           | N/A    | N/A  |
|       └─HashAgg_40                   | 20.00        | 20        | mpp[tiflash] |               | tiflash_task:{proc max:354.3ms, min:353.6ms, avg: 354ms, p80:354.3ms, p95:354.3ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                          | group by:test.t.a, funcs:sum(Column#29)->Column#22, funcs:firstrow(test.t.a)->test.t.a, stream_count: 20                      | N/A    | N/A  |
|         └─ExchangeReceiver_42        | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:354.3ms, min:353.6ms, avg: 354ms, p80:354.3ms, p95:354.3ms, iters:16, tasks:3, threads:60}                                                                                                                                                                                                                          | stream_count: 20                                                                                                              | N/A    | N/A  |
|           └─ExchangeSender_41        | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:349.6ms, min:0s, avg: 116.5ms, p80:349.6ms, p95:349.6ms, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                              | ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], stream_count: 20                                   | N/A    | N/A  |
|             └─HashAgg_37             | 20.00        | 60        | mpp[tiflash] |               | tiflash_task:{proc max:347.6ms, min:0s, avg: 115.9ms, p80:347.6ms, p95:347.6ms, iters:3, tasks:3, threads:60}                                                                                                                                                                                                                              | group by:test.t.a, funcs:count(1)->Column#29                                                                                  | N/A    | N/A  |
|               └─TableFullScan_26     | 600000000.00 | 600000000 | mpp[tiflash] | table:t       | tiflash_task:{proc max:36.6ms, min:0s, avg: 12.2ms, p80:36.6ms, p95:36.6ms, iters:9389, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73833, total_skipped_packs:418, total_scanned_rows:600001386, total_skipped_rows:3381854, total_rs_index_load_time: 10ms, total_read_time: 61ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                              | N/A    | N/A  |
+--------------------------------------+--------------+-----------+--------------+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------------------------------------------------------------------------------------------------------------+--------+------+
9 rows in set (0.37 sec)

Configure tiflash_fine_grained_shuffle_stream_count

You can increase the concurrency for executing window functions by configuring tiflash_fine_grained_shuffle_stream_count of the Fine Grained Shuffle feature. In this way, the execution of window functions can occupy more system resources, which improves query performance.

When a window function is pushed down to TiFlash for execution, you can use this variable to control the concurrency level of the window function execution. The unit is threads.

set @@tiflash_fine_grained_shuffle_stream_count = 20;

The following example shows the query result before and after the tiflash_fine_grained_shuffle_stream_count variable is re-configured. Before the re-configuration, the stream_count of [ExchangeSender_11, ExchangeReceiver_12, Sort_13, Window_22] is 8. After the re-configuration, the stream_count becomes 20.

Before tiflash_fine_grained_shuffle_stream_count is re-configured:

mysql> explain analyze select *, row_number() over (partition by a) from t;
+----------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------+------+
| id                               | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                                  | operator info                                                                                                        | memory | disk |
+----------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_24                   | 600000000.00 | 600000000 | root         |               | time:4m30.5s, loops:585941, cop_task: {num: 9163, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                               | data:ExchangeSender_23                                                                                               | N/A    | N/A  |
| └─ExchangeSender_23              | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:4m30.5s, min:3m4.8s, avg: 3m36.1s, p80:4m30.5s, p95:4m30.5s, iters:9160, tasks:3, threads:24}                                                                                                                                                                                                                            | ExchangeType: PassThrough                                                                                            | N/A    | N/A  |
|   └─Window_22                    | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:3m31.6s, min:2m26.2s, avg: 2m50.7s, p80:3m31.6s, p95:3m31.6s, iters:9160, tasks:3, threads:24}                                                                                                                                                                                                                           | row_number()->Column#23 over(partition by test.t.a rows between current row and current row), stream_count: 8        | N/A    | N/A  |
|     └─Sort_13                    | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:3m28.6s, min:2m24.2s, avg: 2m48.4s, p80:3m28.6s, p95:3m28.6s, iters:9160, tasks:3, threads:24}                                                                                                                                                                                                                           | test.t.a, stream_count: 8                                                                                            | N/A    | N/A  |
|       └─ExchangeReceiver_12      | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:32.4s, min:32s, avg: 32.1s, p80:32.4s, p95:32.4s, iters:49307, tasks:3, threads:24}                                                                                                                                                                                                                                      | stream_count: 8                                                                                                      | N/A    | N/A  |
|         └─ExchangeSender_11      | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:32s, min:0s, avg: 10.7s, p80:32s, p95:32s, iters:9386, tasks:3, threads:60}                                                                                                                                                                                                                                              | ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], stream_count: 8                           | N/A    | N/A  |
|           └─TableFullScan_10     | 600000000.00 | 600000000 | mpp[tiflash] | table:t       | tiflash_task:{proc max:113.9ms, min:0s, avg: 38ms, p80:113.9ms, p95:113.9ms, iters:9386, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73834, total_skipped_packs:190, total_scanned_rows:600000000, total_skipped_rows:1536382, total_rs_index_load_time: 16ms, total_read_time: 166324ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                     | N/A    | N/A  |
+----------------------------------+--------------+-----------+--------------+---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+----------------------------------------------------------------------------------------------------------------------+--------+------+
7 rows in set (4 min 30.59 sec)

Set tiflash_fine_grained_shuffle_stream_count to 20:

mysql> set @@tiflash_fine_grained_shuffle_stream_count = 20;
Query OK, 0 rows affected (0.00 sec)

After tiflash_fine_grained_shuffle_stream_count is set to 20:

mysql> explain analyze select *, row_number() over (partition by a) from t;
+----------------------------------+--------------+-----------+--------------+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------+--------+------+
| id                               | estRows      | actRows   | task         | access object | execution info                                                                                                                                                                                                                                                                                                                                    | operator info                                                                                                         | memory | disk |
+----------------------------------+--------------+-----------+--------------+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------+--------+------+
| TableReader_24                   | 600000000.00 | 600000000 | root         |               | time:2m55s, loops:585941, cop_task: {num: 9163, max: 0s, min: 0s, avg: 0s, p95: 0s, copr_cache_hit_ratio: 0.00}                                                                                                                                                                                                                                   | data:ExchangeSender_23                                                                                                | N/A    | N/A  |
| └─ExchangeSender_23              | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:2m55s, min:1m37s, avg: 2m28.7s, p80:2m55s, p95:2m55s, iters:9160, tasks:3, threads:60}                                                                                                                                                                                                                                     | ExchangeType: PassThrough                                                                                             | N/A    | N/A  |
|   └─Window_22                    | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:2m12.9s, min:1m17s, avg: 1m54.2s, p80:2m12.9s, p95:2m12.9s, iters:9160, tasks:3, threads:60}                                                                                                                                                                                                                               | row_number()->Column#23 over(partition by test.t.a rows between current row and current row), stream_count: 20        | N/A    | N/A  |
|     └─Sort_13                    | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:2m10.9s, min:1m16s, avg: 1m52.5s, p80:2m10.9s, p95:2m10.9s, iters:9160, tasks:3, threads:60}                                                                                                                                                                                                                               | test.t.a, stream_count: 20                                                                                            | N/A    | N/A  |
|       └─ExchangeReceiver_12      | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:27.2s, min:25.5s, avg: 26.6s, p80:27.2s, p95:27.2s, iters:49602, tasks:3, threads:60}                                                                                                                                                                                                                                      | stream_count: 20                                                                                                      | N/A    | N/A  |
|         └─ExchangeSender_11      | 600000000.00 | 600000000 | mpp[tiflash] |               | tiflash_task:{proc max:25.5s, min:0s, avg: 8.51s, p80:25.5s, p95:25.5s, iters:9388, tasks:3, threads:60}                                                                                                                                                                                                                                          | ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], stream_count: 20                           | N/A    | N/A  |
|           └─TableFullScan_10     | 600000000.00 | 600000000 | mpp[tiflash] | table:t       | tiflash_task:{proc max:167.3ms, min:0s, avg: 55.8ms, p80:167.3ms, p95:167.3ms, iters:9388, tasks:3, threads:60}, tiflash_scan:{dtfile:{total_scanned_packs:73834, total_skipped_packs:408, total_scanned_rows:600002896, total_skipped_rows:3307316, total_rs_index_load_time: 20ms, total_read_time: 179431ms}, total_create_snapshot_time: 0ms} | keep order:false                                                                                                      | N/A    | N/A  |
+----------------------------------+--------------+-----------+--------------+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-----------------------------------------------------------------------------------------------------------------------+--------+------+
7 rows in set (2 min 55.09 sec)