Sign InTry Free

Optimizer Hints

TiDB supports optimizer hints, which are based on the comment-like syntax introduced in MySQL 5.7. For example, one of the common syntaxes is /*+ HINT_NAME([t1_name [, t2_name] ...]) */. Use of optimizer hints is recommended in cases where the TiDB optimizer selects a less optimal query plan.

Syntax

Optimizer hints are case insensitive and specified within /*+ ... */ comments following the SELECT, UPDATE or DELETE keyword in a SQL statement. Optimizer hints are not currently supported for INSERT statements.

Multiple hints can be specified by separating with commas. For example, the following query uses three different hints:

SELECT /*+ USE_INDEX(t1, idx1), HASH_AGG(), HASH_JOIN(t1) */ count(*) FROM t t1, t t2 WHERE t1.a = t2.b;

How optimizer hints affect query execution plans can be observed in the output of EXPLAIN and EXPLAIN ANALYZE.

An incorrect or incomplete hint will not result in a statement error. This is because hints are intended to have only a hint (suggestion) semantic to query execution. Similarly, TiDB will at most return a warning if a hint is not applicable.

Currently, TiDB supports two categories of hints, which are different in scope. The first category of hints takes effect in the scope of query blocks, such as /*+ HASH_AGG() */; the second category of hints takes effect in the whole query, such as /*+ MEMORY_QUOTA(1024 MB)*/.

Each query or sub-query in a statement corresponds to a different query block, and each query block has its own name. For example:

SELECT * FROM (SELECT * FROM t) t1, (SELECT * FROM t) t2;

The above query statement has three query blocks: the outermost SELECT corresponds to the first query block, whose name is sel_1; the two SELECT sub-queries correspond to the second and the third query block, whose names are sel_2 and sel_3, respectively. The sequence of the numbers is based on the appearance of SELECT from left to right. If you replace the first SELECT with DELETE or UPDATE, then the corresponding query block names are del_1 or upd_1.

Hints that take effect in query blocks

This category of hints can follow behind any SELECT, UPDATE or DELETE keywords. To control the effective scope of the hint, use the name of the query block in the hint. You can make the hint parameters clear by accurately identifying each table in the query (in case of duplicated table names or aliases). If no query block is specified in the hint, the hint takes effect in the current block by default.

For example:

SELECT /*+ HASH_JOIN(@sel_1 t1@sel_1, t3) */ * FROM (SELECT t1.a, t1.b FROM t t1, t t2 WHERE t1.a = t2.a) t1, t t3 WHERE t1.b = t3.b;

This hint takes effect in the sel_1 query block, and its parameters are the t1 and t3 tables in sel_1 (sel_2 also contains a t1 table).

As described above, you can specify the name of the query block in the hint in the following ways:

  • Set the query block name as the first parameter of the hint, and separate it from other parameters with a space. In addition to QB_NAME, all the hints listed in this section also have another optional hidden parameter @QB_NAME. By using this parameter, you can specify the effective scope of this hint.
  • Append @QB_NAME to a table name in the parameter to explicitly specify which query block this table belongs to.

QB_NAME

If the query statement is a complicated statement that includes multiple nested queries, the ID and name of a certain query block might be mistakenly identified. The hint QB_NAME can help us in this regard.

QB_NAME means Query Block Name. You can specify a new name to a query block. The specified QB_NAME and the previous default name are both valid. For example:

SELECT /*+ QB_NAME(QB1) */ * FROM (SELECT * FROM t) t1, (SELECT * FROM t) t2;

This hint specifies the outer SELECT query block's name to QB1, which makes QB1 and the default name sel_1 both valid for the query block.

MERGE_JOIN(t1_name [, tl_name ...])

The MERGE_JOIN(t1_name [, tl_name ...]) hint tells the optimizer to use the sort-merge join algorithm for the given table(s). Generally, this algorithm consumes less memory but takes longer processing time. If there is a very large data volume or insufficient system memory, it is recommended to use this hint. For example:

select /*+ MERGE_JOIN(t1, t2) */ * from t1, t2 where t1.id = t2.id;

INL_JOIN(t1_name [, tl_name ...])

The INL_JOIN(t1_name [, tl_name ...]) hint tells the optimizer to use the index nested loop join algorithm for the given table(s). This algorithm might consume less system resources and take shorter processing time in some scenarios and might produce an opposite result in other scenarios. If the result set is less than 10,000 rows after the outer table is filtered by the WHERE condition, it is recommended to use this hint. For example:

select /*+ INL_JOIN(t1, t2) */ * from t1, t2 where t1.id = t2.id;

The parameter(s) given in INL_JOIN() is the candidate table for the inner table when you create the query plan. For example, INL_JOIN(t1) means that TiDB only considers using t1 as the inner table to create a query plan. If the candidate table has an alias, you must use the alias as the parameter in INL_JOIN(); if it does not has an alias, use the table's original name as the parameter. For example, in the select /*+ INL_JOIN(t1) */ * from t t1, t t2 where t1.a = t2.b; query, you must use the t table's alias t1 or t2 rather than t as INL_JOIN()'s parameter.

INL_HASH_JOIN

The INL_HASH_JOIN(t1_name [, tl_name]) hint tells the optimizer to use the index nested loop hash join algorithm. The conditions for using this algorithm are the same with the conditions for using the index nested loop join algorithm. The difference between the two algorithms is that INL_JOIN creates a hash table on the joined inner table, but INL_HASH_JOIN creates a hash table on the joined outer table. INL_HASH_JOIN has a fixed limit on memory usage, while the memory used by INL_JOIN depends on the number of rows matched in the inner table.

HASH_JOIN(t1_name [, tl_name ...])

The HASH_JOIN(t1_name [, tl_name ...]) hint tells the optimizer to use the hash join algorithm for the given table(s). This algorithm allows the query to be executed concurrently with multiple threads, which achieves a higher processing speed but consumes more memory. For example:

select /*+ HASH_JOIN(t1, t2) */ * from t1, t2 where t1.id = t2.id;

HASH_JOIN_BUILD(t1_name [, tl_name ...])

The HASH_JOIN_BUILD(t1_name [, tl_name ...]) hint tells the optimizer to use the hash join algorithm on specified tables with these tables working as the build side. In this way, you can build hash tables using specific tables. For example:

SELECT /*+ HASH_JOIN_BUILD(t1) */ * FROM t1, t2 WHERE t1.id = t2.id;

HASH_JOIN_PROBE(t1_name [, tl_name ...])

The HASH_JOIN_PROBE(t1_name [, tl_name ...]) hint tells the optimizer to use the hash join algorithm on specified tables with these tables working as the probe side. In this way, you can execute the hash join algorithm with specific tables as the probe side. For example:

SELECT /*+ HASH_JOIN_PROBE(t2) */ * FROM t1, t2 WHERE t1.id = t2.id;

SEMI_JOIN_REWRITE()

The SEMI_JOIN_REWRITE() hint tells the optimizer to rewrite the semi-join query to an ordinary join query. Currently, this hint only works for EXISTS subqueries.

If this hint is not used to rewrite the query, when the hash join is selected in the execution plan, the semi-join query can only use the subquery to build a hash table. In this case, when the result of the subquery is bigger than that of the outer query, the execution speed might be slower than expected.

Similarly, when the index join is selected in the execution plan, the semi-join query can only use the outer query as the driving table. In this case, when the result of the subquery is smaller than that of the outer query, the execution speed might be slower than expected.

When SEMI_JOIN_REWRITE() is used to rewrite the query, the optimizer can extend the selection range to select a better execution plan.

-- Does not use SEMI_JOIN_REWRITE() to rewrite the query. EXPLAIN SELECT * FROM t WHERE EXISTS (SELECT 1 FROM t1 WHERE t1.a = t.a);
+-----------------------------+---------+-----------+------------------------+---------------------------------------------------+ | id | estRows | task | access object | operator info | +-----------------------------+---------+-----------+------------------------+---------------------------------------------------+ | MergeJoin_9 | 7992.00 | root | | semi join, left key:test.t.a, right key:test.t1.a | | ├─IndexReader_25(Build) | 9990.00 | root | | index:IndexFullScan_24 | | │ └─IndexFullScan_24 | 9990.00 | cop[tikv] | table:t1, index:idx(a) | keep order:true, stats:pseudo | | └─IndexReader_23(Probe) | 9990.00 | root | | index:IndexFullScan_22 | | └─IndexFullScan_22 | 9990.00 | cop[tikv] | table:t, index:idx(a) | keep order:true, stats:pseudo | +-----------------------------+---------+-----------+------------------------+---------------------------------------------------+
-- Uses SEMI_JOIN_REWRITE() to rewrite the query. EXPLAIN SELECT * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE() */ 1 FROM t1 WHERE t1.a = t.a);
+------------------------------+---------+-----------+------------------------+---------------------------------------------------------------------------------------------------------------+ | id | estRows | task | access object | operator info | +------------------------------+---------+-----------+------------------------+---------------------------------------------------------------------------------------------------------------+ | IndexJoin_16 | 1.25 | root | | inner join, inner:IndexReader_15, outer key:test.t1.a, inner key:test.t.a, equal cond:eq(test.t1.a, test.t.a) | | ├─StreamAgg_39(Build) | 1.00 | root | | group by:test.t1.a, funcs:firstrow(test.t1.a)->test.t1.a | | │ └─IndexReader_34 | 1.00 | root | | index:IndexFullScan_33 | | │ └─IndexFullScan_33 | 1.00 | cop[tikv] | table:t1, index:idx(a) | keep order:true | | └─IndexReader_15(Probe) | 1.25 | root | | index:Selection_14 | | └─Selection_14 | 1.25 | cop[tikv] | | not(isnull(test.t.a)) | | └─IndexRangeScan_13 | 1.25 | cop[tikv] | table:t, index:idx(a) | range: decided by [eq(test.t.a, test.t1.a)], keep order:false, stats:pseudo | +------------------------------+---------+-----------+------------------------+---------------------------------------------------------------------------------------------------------------+

From the preceding example, you can see that when using the SEMI_JOIN_REWRITE() hint, TiDB can select the execution method of IndexJoin based on the driving table t1.

NO_DECORRELATE()

The NO_DECORRELATE() hint tells the optimizer not to try to perform decorrelation for the correlated subquery in the specified query block. This hint is applicable to the EXISTS, IN, ANY, ALL, SOME subqueries and scalar subqueries that contain correlated columns (that is, correlated subqueries).

When this hint is used in a query block, the optimizer will not try to perform decorrelation for the correlated columns between the subquery and its outer query block, but always use the Apply operator to execute the query.

By default, TiDB tries to perform decorrelation for correlated subqueries to achieve higher execution efficiency. However, in some scenarios, decorrelation might actually reduce the execution efficiency. In this case, you can use this hint to manually tell the optimizer not to perform decorrelation. For example:

create table t1(a int, b int); create table t2(a int, b int, index idx(b));
-- Not using NO_DECORRELATE(). explain select * from t1 where t1.a < (select sum(t2.a) from t2 where t2.b = t1.b);
+----------------------------------+----------+-----------+---------------+--------------------------------------------------------------------------------------------------------------+ | id | estRows | task | access object | operator info | +----------------------------------+----------+-----------+---------------+--------------------------------------------------------------------------------------------------------------+ | HashJoin_11 | 9990.00 | root | | inner join, equal:[eq(test.t1.b, test.t2.b)], other cond:lt(cast(test.t1.a, decimal(10,0) BINARY), Column#7) | | ├─HashAgg_23(Build) | 7992.00 | root | | group by:test.t2.b, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t2.b)->test.t2.b | | │ └─TableReader_24 | 7992.00 | root | | data:HashAgg_16 | | │ └─HashAgg_16 | 7992.00 | cop[tikv] | | group by:test.t2.b, funcs:sum(test.t2.a)->Column#8 | | │ └─Selection_22 | 9990.00 | cop[tikv] | | not(isnull(test.t2.b)) | | │ └─TableFullScan_21 | 10000.00 | cop[tikv] | table:t2 | keep order:false, stats:pseudo | | └─TableReader_15(Probe) | 9990.00 | root | | data:Selection_14 | | └─Selection_14 | 9990.00 | cop[tikv] | | not(isnull(test.t1.b)) | | └─TableFullScan_13 | 10000.00 | cop[tikv] | table:t1 | keep order:false, stats:pseudo | +----------------------------------+----------+-----------+---------------+--------------------------------------------------------------------------------------------------------------+

From the preceding execution plan, you can see that the optimizer has automatically performed decorrelation. The decorrelated execution plan does not have the Apply operator. Instead, the plan has join operations between the subquery and the outer query block. The original filter condition (t2.b = t1.b) with the correlated column becomes a regular join condition.

-- Using NO_DECORRELATE(). explain select * from t1 where t1.a < (select /*+ NO_DECORRELATE() */ sum(t2.a) from t2 where t2.b = t1.b);
+------------------------------------------+-----------+-----------+------------------------+--------------------------------------------------------------------------------------+ | id | estRows | task | access object | operator info | +------------------------------------------+-----------+-----------+------------------------+--------------------------------------------------------------------------------------+ | Projection_10 | 10000.00 | root | | test.t1.a, test.t1.b | | └─Apply_12 | 10000.00 | root | | CARTESIAN inner join, other cond:lt(cast(test.t1.a, decimal(10,0) BINARY), Column#7) | | ├─TableReader_14(Build) | 10000.00 | root | | data:TableFullScan_13 | | │ └─TableFullScan_13 | 10000.00 | cop[tikv] | table:t1 | keep order:false, stats:pseudo | | └─MaxOneRow_15(Probe) | 10000.00 | root | | | | └─StreamAgg_20 | 10000.00 | root | | funcs:sum(Column#14)->Column#7 | | └─Projection_45 | 100000.00 | root | | cast(test.t2.a, decimal(10,0) BINARY)->Column#14 | | └─IndexLookUp_44 | 100000.00 | root | | | | ├─IndexRangeScan_42(Build) | 100000.00 | cop[tikv] | table:t2, index:idx(b) | range: decided by [eq(test.t2.b, test.t1.b)], keep order:false, stats:pseudo | | └─TableRowIDScan_43(Probe) | 100000.00 | cop[tikv] | table:t2 | keep order:false, stats:pseudo | +------------------------------------------+-----------+-----------+------------------------+--------------------------------------------------------------------------------------+

From the preceding execution plan, you can see that the optimizer does not perform decorrelation. The execution plan still contains the Apply operator. The filter condition (t2.b = t1.b) with the correlated column is still the filter condition when accessing the t2 table.

HASH_AGG()

The HASH_AGG() hint tells the optimizer to use the hash aggregation algorithm in all the aggregate functions in the specified query block. This algorithm allows the query to be executed concurrently with multiple threads, which achieves a higher processing speed but consumes more memory. For example:

select /*+ HASH_AGG() */ count(*) from t1, t2 where t1.a > 10 group by t1.id;

STREAM_AGG()

The STREAM_AGG() hint tells the optimizer to use the stream aggregation algorithm in all the aggregate functions in the specified query block. Generally, this algorithm consumes less memory but takes longer processing time. If there is a very large data volume or insufficient system memory, it is recommended to use this hint. For example:

select /*+ STREAM_AGG() */ count(*) from t1, t2 where t1.a > 10 group by t1.id;

USE_INDEX(t1_name, idx1_name [, idx2_name ...])

The USE_INDEX(t1_name, idx1_name [, idx2_name ...]) hint tells the optimizer to use only the given index(es) for a specified t1_name table. For example, applying the following hint has the same effect as executing the select * from t t1 use index(idx1, idx2); statement.

SELECT /*+ USE_INDEX(t1, idx1, idx2) */ * FROM t1;

FORCE_INDEX(t1_name, idx1_name [, idx2_name ...])

The FORCE_INDEX(t1_name, idx1_name [, idx2_name ...]) hint tells the optimizer to use only the given index(es).

The usage and effect of FORCE_INDEX(t1_name, idx1_name [, idx2_name ...]) are the same as the usage and effect of USE_INDEX(t1_name, idx1_name [, idx2_name ...]).

The following 4 queries have the same effect:

SELECT /*+ USE_INDEX(t, idx1) */ * FROM t; SELECT /*+ FORCE_INDEX(t, idx1) */ * FROM t; SELECT * FROM t use index(idx1); SELECT * FROM t force index(idx1);

IGNORE_INDEX(t1_name, idx1_name [, idx2_name ...])

The IGNORE_INDEX(t1_name, idx1_name [, idx2_name ...]) hint tells the optimizer to ignore the given index(es) for a specified t1_name table. For example, applying the following hint has the same effect as executing the select * from t t1 ignore index(idx1, idx2); statement.

select /*+ IGNORE_INDEX(t1, idx1, idx2) */ * from t t1;

AGG_TO_COP()

The AGG_TO_COP() hint tells the optimizer to push down the aggregate operation in the specified query block to the coprocessor. If the optimizer does not push down some aggregate function that is suitable for pushdown, then it is recommended to use this hint. For example:

select /*+ AGG_TO_COP() */ sum(t1.a) from t t1;

LIMIT_TO_COP()

The LIMIT_TO_COP() hint tells the optimizer to push down the Limit and TopN operators in the specified query block to the coprocessor. If the optimizer does not perform such an operation, it is recommended to use this hint. For example:

SELECT /*+ LIMIT_TO_COP() */ * FROM t WHERE a = 1 AND b > 10 ORDER BY c LIMIT 1;

READ_FROM_STORAGE(TIFLASH[t1_name [, tl_name ...]], TIKV[t2_name [, tl_name ...]])

The READ_FROM_STORAGE(TIFLASH[t1_name [, tl_name ...]], TIKV[t2_name [, tl_name ...]]) hint tells the optimizer to read specific table(s) from specific storage engine(s). Currently, this hint supports two storage engine parameters - TIKV and TIFLASH. If a table has an alias, use the alias as the parameter of READ_FROM_STORAGE(); if the table does not has an alias, use the table's original name as the parameter. For example:

select /*+ READ_FROM_STORAGE(TIFLASH[t1], TIKV[t2]) */ t1.a from t t1, t t2 where t1.a = t2.a;

USE_INDEX_MERGE(t1_name, idx1_name [, idx2_name ...])

The USE_INDEX_MERGE(t1_name, idx1_name [, idx2_name ...]) hint tells the optimizer to access a specific table with the index merge method. The given list of indexes are optional parameters. If you explicitly specify the list, TiDB selects indexes from the list to build index merge; if you do not give the list of indexes, TiDB selects indexes from all available indexes to build index merge. For example:

SELECT /*+ USE_INDEX_MERGE(t1, idx_a, idx_b, idx_c) */ * FROM t1 WHERE t1.a > 10 OR t1.b > 10;

When multiple USE_INDEX_MERGE hints are made to the same table, the optimizer tries to select the index from the union of the index sets specified by these hints.

This hint takes effect on strict conditions, including:

  • If the query can select a single index scan in addition to full table scan, the optimizer does not select index merge.

LEADING(t1_name [, tl_name ...])

The LEADING(t1_name [, tl_name ...]) hint reminds the optimizer that, when generating the execution plan, to determine the order of multi-table joins according to the order of table names specified in the hint. For example:

SELECT /*+ LEADING(t1, t2) */ * FROM t1, t2, t3 WHERE t1.id = t2.id and t2.id = t3.id;

In the above query with multi-table joins, the order of joins is determined by the order of table names specified in the LEADING() hint. The optimizer will first join t1 and t2 and then join the result with t3. This hint is more general than STRAIGHT_JOIN.

The LEADING hint does not take effect in the following situations:

  • Multiple LEADING hints are specified.
  • The table name specified in the LEADING hint does not exist.
  • A duplicated table name is specified in the LEADING hint.
  • The optimizer cannot perform join operations according to the order as specified by the LEADING hint.
  • The straight_join() hint already exists.
  • The query contains an outer join together with the Cartesian product.
  • Any of the MERGE_JOIN, INL_JOIN, INL_HASH_JOIN, and HASH_JOIN hints is used at the same time.

In the above situations, a warning is generated.

-- Multiple `LEADING` hints are specified. SELECT /*+ LEADING(t1, t2) LEADING(t3) */ * FROM t1, t2, t3 WHERE t1.id = t2.id and t2.id = t3.id; -- To learn why the `LEADING` hint fails to take effect, execute `show warnings`. SHOW WARNINGS;
+---------+------+-------------------------------------------------------------------------------------------------------------------+ | Level | Code | Message | +---------+------+-------------------------------------------------------------------------------------------------------------------+ | Warning | 1815 | We can only use one leading hint at most, when multiple leading hints are used, all leading hints will be invalid | +---------+------+-------------------------------------------------------------------------------------------------------------------+

MERGE()

Using the MERGE() hint in queries with common table expressions (CTE) can disable the materialization of the subqueries and expand the subquery inlines into CTE. This hint is only applicable to non-recursive CTE. In some scenarios, using MERGE() brings higher execution efficiency than the default behavior of allocating a temporary space. For example, pushing down query conditions or in nesting CTE queries:

-- Uses the hint to push down the predicate of the outer query. WITH CTE AS (SELECT /*+ MERGE() */ * FROM tc WHERE tc.a < 60) SELECT * FROM CTE WHERE CTE.a < 18; -- Uses the hint in a nested CTE query to expand a CTE inline into the outer query. WITH CTE1 AS (SELECT * FROM t1), CTE2 AS (WITH CTE3 AS (SELECT /*+ MERGE() */ * FROM t2), CTE4 AS (SELECT * FROM t3) SELECT * FROM CTE3, CTE4) SELECT * FROM CTE1, CTE2;

Hints that take effect in the whole query

This category of hints can only follow behind the first SELECT, UPDATE or DELETE keyword, which is equivalent to modifying the value of the specified system variable when this query is executed. The priority of the hint is higher than that of existing system variables.

NO_INDEX_MERGE()

The NO_INDEX_MERGE() hint disables the index merge feature of the optimizer.

For example, the following query will not use index merge:

select /*+ NO_INDEX_MERGE() */ * from t where t.a > 0 or t.b > 0;

In addition to this hint, setting the tidb_enable_index_merge system variable also controls whether to enable this feature.

USE_TOJA(boolean_value)

The boolean_value parameter can be TRUE or FALSE. The USE_TOJA(TRUE) hint enables the optimizer to convert an in condition (containing a sub-query) to join and aggregation operations. Comparatively, the USE_TOJA(FALSE) hint disables this feature.

For example, the following query will convert in (select t2.a from t2) subq to corresponding join and aggregation operations:

select /*+ USE_TOJA(TRUE) */ t1.a, t1.b from t1 where t1.a in (select t2.a from t2) subq;

In addition to this hint, setting the tidb_opt_insubq_to_join_and_agg system variable also controls whether to enable this feature.

MAX_EXECUTION_TIME(N)

The MAX_EXECUTION_TIME(N) hint places a limit N (a timeout value in milliseconds) on how long a statement is permitted to execute before the server terminates it. In the following hint, MAX_EXECUTION_TIME(1000) means that the timeout is 1000 milliseconds (that is, 1 second):

select /*+ MAX_EXECUTION_TIME(1000) */ * from t1 inner join t2 where t1.id = t2.id;

In addition to this hint, the global.max_execution_time system variable can also limit the execution time of a statement.

MEMORY_QUOTA(N)

The MEMORY_QUOTA(N) hint places a limit N (a threshold value in MB or GB) on how much memory a statement is permitted to use. When a statement's memory usage exceeds this limit, TiDB produces a log message based on the statement's over-limit behavior or just terminates it.

In the following hint, MEMORY_QUOTA(1024 MB) means that the memory usage is limited to 1024 MB:

select /*+ MEMORY_QUOTA(1024 MB) */ * from t;

In addition to this hint, the tidb_mem_quota_query system variable can also limit the memory usage of a statement.

READ_CONSISTENT_REPLICA()

The READ_CONSISTENT_REPLICA() hint enables the feature of reading consistent data from the TiKV follower node. For example:

select /*+ READ_CONSISTENT_REPLICA() */ * from t;

In addition to this hint, setting the tidb_replica_read environment variable to 'follower' or 'leader' also controls whether to enable this feature.

IGNORE_PLAN_CACHE()

The IGNORE_PLAN_CACHE() hint reminds the optimizer not to use the Plan Cache when handling the current prepare statement.

This hint is used to temporarily disable the Plan Cache for a certain type of queries when prepare-plan-cache is enabled.

In the following example, the Plan Cache is forcibly disabled when executing the prepare statement.

prepare stmt from 'select /*+ IGNORE_PLAN_CACHE() */ * from t where t.id = ?';

STRAIGHT_JOIN()

The STRAIGHT_JOIN() hint reminds the optimizer to join tables in the order of table names in the FROM clause when generating the join plan.

SELECT /*+ STRAIGHT_JOIN() */ * FROM t t1, t t2 WHERE t1.a = t2.a;

NTH_PLAN(N)

The NTH_PLAN(N) hint reminds the optimizer to select the Nth physical plan found during the physical optimization. N must be a positive integer.

If the specified N is beyond the search range of the physical optimization, TiDB will return a warning and select the optimal physical plan based on the strategy that ignores this hint.

This hint does not take effect when the cascades planner is enabled.

In the following example, the optimizer is forced to select the third physical plan found during the physical optimization:

SELECT /*+ NTH_PLAN(3) */ count(*) from t where a > 5;
Download PDF
Playground
New
One-stop & interactive experience of TiDB's capabilities WITHOUT registration.
Products
TiDB
TiDB Dedicated
TiDB Serverless
Pricing
Get Demo
Get Started
© 2024 PingCAP. All Rights Reserved.
Privacy Policy.