855 lines
18 KiB
Plaintext
855 lines
18 KiB
Plaintext
%{
|
|
// Copyright 2020 PingCAP, Inc.
|
|
//
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
// you may not use this file except in compliance with the License.
|
|
// You may obtain a copy of the License at
|
|
//
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
//
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package parser
|
|
|
|
import (
|
|
"math"
|
|
"strconv"
|
|
|
|
"github.com/pingcap/tidb/pkg/parser/ast"
|
|
)
|
|
|
|
%}
|
|
|
|
%union {
|
|
offset int
|
|
ident string
|
|
number uint64
|
|
hint *ast.TableOptimizerHint
|
|
hints []*ast.TableOptimizerHint
|
|
table ast.HintTable
|
|
modelIdents []ast.CIStr
|
|
leadingList *ast.LeadingList
|
|
leadingElement interface{} // Modified: Represents either *ast.HintTable or *ast.LeadingList
|
|
}
|
|
|
|
%token <number>
|
|
|
|
/*yy:token "%d" */
|
|
hintIntLit "a 64-bit unsigned integer"
|
|
|
|
%token <ident>
|
|
|
|
/*yy:token "%c" */
|
|
hintIdentifier
|
|
hintInvalid "a special token never used by parser, used by lexer to indicate error"
|
|
|
|
/*yy:token "@%c" */
|
|
hintSingleAtIdentifier "identifier with single leading at"
|
|
|
|
/*yy:token "'%c'" */
|
|
hintStringLit
|
|
|
|
/* MySQL 8.0 hint names */
|
|
hintJoinFixedOrder "JOIN_FIXED_ORDER"
|
|
hintJoinOrder "JOIN_ORDER"
|
|
hintJoinPrefix "JOIN_PREFIX"
|
|
hintJoinSuffix "JOIN_SUFFIX"
|
|
hintBKA "BKA"
|
|
hintNoBKA "NO_BKA"
|
|
hintBNL "BNL"
|
|
hintNoBNL "NO_BNL"
|
|
hintHashJoin "HASH_JOIN"
|
|
hintHashJoinBuild "HASH_JOIN_BUILD"
|
|
hintHashJoinProbe "HASH_JOIN_PROBE"
|
|
hintNoHashJoin "NO_HASH_JOIN"
|
|
hintMerge "MERGE"
|
|
hintNoMerge "NO_MERGE"
|
|
hintIndexMerge "INDEX_MERGE"
|
|
hintNoIndexMerge "NO_INDEX_MERGE"
|
|
hintMRR "MRR"
|
|
hintNoMRR "NO_MRR"
|
|
hintNoICP "NO_ICP"
|
|
hintNoRangeOptimization "NO_RANGE_OPTIMIZATION"
|
|
hintSkipScan "SKIP_SCAN"
|
|
hintNoSkipScan "NO_SKIP_SCAN"
|
|
hintSemijoin "SEMIJOIN"
|
|
hintNoSemijoin "NO_SEMIJOIN"
|
|
hintMaxExecutionTime "MAX_EXECUTION_TIME"
|
|
hintSetVar "SET_VAR"
|
|
hintResourceGroup "RESOURCE_GROUP"
|
|
hintQBName "QB_NAME"
|
|
hintHypoIndex "HYPO_INDEX"
|
|
|
|
/* TiDB hint names */
|
|
hintAggToCop "AGG_TO_COP"
|
|
hintIgnorePlanCache "IGNORE_PLAN_CACHE"
|
|
hintWriteSlowLog "WRITE_SLOW_LOG"
|
|
hintHashAgg "HASH_AGG"
|
|
hintMpp1PhaseAgg "MPP_1PHASE_AGG"
|
|
hintMpp2PhaseAgg "MPP_2PHASE_AGG"
|
|
hintIgnoreIndex "IGNORE_INDEX"
|
|
hintIndexJoin "INDEX_JOIN"
|
|
hintNoIndexJoin "NO_INDEX_JOIN"
|
|
hintInlHashJoin "INL_HASH_JOIN"
|
|
hintIndexHashJoin "INDEX_HASH_JOIN"
|
|
hintNoIndexHashJoin "NO_INDEX_HASH_JOIN"
|
|
hintInlJoin "INL_JOIN"
|
|
hintInlMergeJoin "INL_MERGE_JOIN"
|
|
hintIndexMergeJoin "INDEX_MERGE_JOIN"
|
|
hintNoIndexMergeJoin "NO_INDEX_MERGE_JOIN"
|
|
hintMemoryQuota "MEMORY_QUOTA"
|
|
hintNoSwapJoinInputs "NO_SWAP_JOIN_INPUTS"
|
|
hintQueryType "QUERY_TYPE"
|
|
hintReadConsistentReplica "READ_CONSISTENT_REPLICA"
|
|
hintReadFromStorage "READ_FROM_STORAGE"
|
|
hintSMJoin "MERGE_JOIN"
|
|
hintNoSMJoin "NO_MERGE_JOIN"
|
|
hintBCJoin "BROADCAST_JOIN"
|
|
hintShuffleJoin "SHUFFLE_JOIN"
|
|
hintStreamAgg "STREAM_AGG"
|
|
hintSwapJoinInputs "SWAP_JOIN_INPUTS"
|
|
hintUseIndexMerge "USE_INDEX_MERGE"
|
|
hintUseIndex "USE_INDEX"
|
|
hintOrderIndex "ORDER_INDEX"
|
|
hintNoOrderIndex "NO_ORDER_INDEX"
|
|
hintIndexLookUpPushDown "INDEX_LOOKUP_PUSHDOWN"
|
|
hintUsePlanCache "USE_PLAN_CACHE"
|
|
hintUseToja "USE_TOJA"
|
|
hintTimeRange "TIME_RANGE"
|
|
hintUseCascades "USE_CASCADES"
|
|
hintNthPlan "NTH_PLAN"
|
|
hintLimitToCop "LIMIT_TO_COP"
|
|
hintForceIndex "FORCE_INDEX"
|
|
hintStraightJoin "STRAIGHT_JOIN"
|
|
hintLeading "LEADING"
|
|
hintSemiJoinRewrite "SEMI_JOIN_REWRITE"
|
|
hintNoDecorrelate "NO_DECORRELATE"
|
|
|
|
/* Other keywords */
|
|
hintOLAP "OLAP"
|
|
hintOLTP "OLTP"
|
|
hintPartition "PARTITION"
|
|
hintTiKV "TIKV"
|
|
hintTiFlash "TIFLASH"
|
|
hintFalse "FALSE"
|
|
hintTrue "TRUE"
|
|
hintMB "MB"
|
|
hintGB "GB"
|
|
hintDupsWeedOut "DUPSWEEDOUT"
|
|
hintFirstMatch "FIRSTMATCH"
|
|
hintLooseScan "LOOSESCAN"
|
|
hintMaterialization "MATERIALIZATION"
|
|
|
|
%type <ident>
|
|
Identifier "identifier (including keywords)"
|
|
QueryBlockOpt "Query block identifier optional"
|
|
JoinOrderOptimizerHintName
|
|
UnsupportedTableLevelOptimizerHintName
|
|
SupportedTableLevelOptimizerHintName
|
|
UnsupportedIndexLevelOptimizerHintName
|
|
SupportedIndexLevelOptimizerHintName
|
|
SubqueryOptimizerHintName
|
|
BooleanHintName "name of hints which take a boolean input"
|
|
NullaryHintName "name of hints which take no input"
|
|
SubqueryStrategy
|
|
Value "the value in the SET_VAR() hint"
|
|
HintQueryType "query type in optimizer hint (OLAP or OLTP)"
|
|
HintStorageType "storage type in optimizer hint (TiKV or TiFlash)"
|
|
|
|
%type <number>
|
|
UnitOfBytes "unit of bytes (MB or GB)"
|
|
CommaOpt "optional ','"
|
|
|
|
%type <hints>
|
|
OptimizerHintList "optimizer hint list"
|
|
StorageOptimizerHintOpt "storage level optimizer hint"
|
|
HintStorageTypeAndTableList "storage type and tables list in optimizer hint"
|
|
|
|
%type <hint>
|
|
TableOptimizerHintOpt "optimizer hint"
|
|
HintTableList "table list in optimizer hint"
|
|
HintTableListOpt "optional table list in optimizer hint"
|
|
HintIndexList "table name with index list in optimizer hint"
|
|
IndexNameList "index list in optimizer hint"
|
|
IndexNameListOpt "optional index list in optimizer hint"
|
|
ViewNameList "view name list in optimizer hint"
|
|
SubqueryStrategies "subquery strategies"
|
|
SubqueryStrategiesOpt "optional subquery strategies"
|
|
HintTrueOrFalse "true or false in optimizer hint"
|
|
HintStorageTypeAndTable "storage type and tables in optimizer hint"
|
|
|
|
%type <table>
|
|
HintTable "Table in optimizer hint"
|
|
ViewName "View name in optimizer hint"
|
|
|
|
%type <modelIdents>
|
|
PartitionList "partition name list in optimizer hint"
|
|
PartitionListOpt "optional partition name list in optimizer hint"
|
|
|
|
%type <leadingList>
|
|
LeadingTableList "leading table list"
|
|
|
|
%type <leadingElement>
|
|
LeadingTableElement "leading element (table or list)"
|
|
|
|
|
|
%start Start
|
|
|
|
%%
|
|
|
|
Start:
|
|
OptimizerHintList
|
|
{
|
|
parser.result = $1
|
|
}
|
|
|
|
OptimizerHintList:
|
|
TableOptimizerHintOpt
|
|
{
|
|
if $1 != nil {
|
|
$$ = []*ast.TableOptimizerHint{$1}
|
|
}
|
|
}
|
|
| OptimizerHintList CommaOpt TableOptimizerHintOpt
|
|
{
|
|
if $3 != nil {
|
|
$$ = append($1, $3)
|
|
} else {
|
|
$$ = $1
|
|
}
|
|
}
|
|
| StorageOptimizerHintOpt
|
|
{
|
|
$$ = $1
|
|
}
|
|
| OptimizerHintList CommaOpt StorageOptimizerHintOpt
|
|
{
|
|
$$ = append($1, $3...)
|
|
}
|
|
|
|
TableOptimizerHintOpt:
|
|
"JOIN_FIXED_ORDER" '(' QueryBlockOpt ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| JoinOrderOptimizerHintName '(' HintTableList ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| UnsupportedTableLevelOptimizerHintName '(' HintTableListOpt ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| SupportedTableLevelOptimizerHintName '(' HintTableListOpt ')'
|
|
{
|
|
h := $3
|
|
h.HintName = ast.NewCIStr($1)
|
|
$$ = h
|
|
}
|
|
| "LEADING" '(' QueryBlockOpt LeadingTableList ')'
|
|
{
|
|
h := &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
HintData: $4,
|
|
}
|
|
// For LEADING hints we need to maintain two views of the tables:
|
|
// h.HintData:
|
|
// - Stores the structured AST node (LeadingList).
|
|
// - Preserves the nesting and order information of LEADING(...),
|
|
//
|
|
// h.Tables:
|
|
// - Stores a flat slice of all HintTable elements inside the LeadingList.
|
|
// - Only used for initialization.
|
|
if leadingList, ok := h.HintData.(*ast.LeadingList); ok {
|
|
// be compatible with the prior flatten writing style
|
|
h.Tables = ast.FlattenLeadingList(leadingList)
|
|
}
|
|
$$ = h
|
|
}
|
|
| UnsupportedIndexLevelOptimizerHintName '(' HintIndexList ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| SupportedIndexLevelOptimizerHintName '(' HintIndexList ')'
|
|
{
|
|
h := $3
|
|
h.HintName = ast.NewCIStr($1)
|
|
$$ = h
|
|
}
|
|
| SubqueryOptimizerHintName '(' QueryBlockOpt SubqueryStrategiesOpt ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| "MAX_EXECUTION_TIME" '(' QueryBlockOpt hintIntLit ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
HintData: $4,
|
|
}
|
|
}
|
|
| "NTH_PLAN" '(' QueryBlockOpt hintIntLit ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
HintData: int64($4),
|
|
}
|
|
}
|
|
| "SET_VAR" '(' Identifier '=' Value ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
HintData: ast.HintSetVar{
|
|
VarName: $3,
|
|
Value: $5,
|
|
},
|
|
}
|
|
}
|
|
| "RESOURCE_GROUP" '(' Identifier ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
HintData: $3,
|
|
}
|
|
}
|
|
| "QB_NAME" '(' Identifier ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
}
|
|
}
|
|
| "QB_NAME" '(' Identifier ',' ViewNameList ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
Tables: $5.Tables,
|
|
}
|
|
}
|
|
| "MEMORY_QUOTA" '(' QueryBlockOpt hintIntLit UnitOfBytes ')'
|
|
{
|
|
maxValue := uint64(math.MaxInt64) / $5
|
|
if $4 <= maxValue {
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
HintData: int64($4 * $5),
|
|
QBName: ast.NewCIStr($3),
|
|
}
|
|
} else {
|
|
yylex.AppendError(ErrWarnMemoryQuotaOverflow.GenWithStackByArgs(math.MaxInt))
|
|
parser.lastErrorAsWarn()
|
|
$$ = nil
|
|
}
|
|
}
|
|
| "TIME_RANGE" '(' hintStringLit CommaOpt hintStringLit ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
HintData: ast.HintTimeRange{
|
|
From: $3,
|
|
To: $5,
|
|
},
|
|
}
|
|
}
|
|
| BooleanHintName '(' QueryBlockOpt HintTrueOrFalse ')'
|
|
{
|
|
h := $4
|
|
h.HintName = ast.NewCIStr($1)
|
|
h.QBName = ast.NewCIStr($3)
|
|
$$ = h
|
|
}
|
|
| NullaryHintName '(' QueryBlockOpt ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
}
|
|
}
|
|
| "WRITE_SLOW_LOG"
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
}
|
|
}
|
|
| "QUERY_TYPE" '(' QueryBlockOpt HintQueryType ')'
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
HintName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($3),
|
|
HintData: ast.NewCIStr($4),
|
|
}
|
|
}
|
|
| hintIdentifier '(' QueryBlockOpt hintIntLit ')'
|
|
/* The hints below are pseudo hint. They are unsupported hints */
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| hintIdentifier '(' PartitionList ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| hintIdentifier '(' PartitionList CommaOpt hintIntLit ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
| hintIdentifier '(' Identifier '=' Value ')'
|
|
{
|
|
parser.warnUnsupportedHint($1)
|
|
$$ = nil
|
|
}
|
|
|
|
StorageOptimizerHintOpt:
|
|
"READ_FROM_STORAGE" '(' QueryBlockOpt HintStorageTypeAndTableList ')'
|
|
{
|
|
hs := $4
|
|
name := ast.NewCIStr($1)
|
|
qb := ast.NewCIStr($3)
|
|
for _, h := range hs {
|
|
h.HintName = name
|
|
h.QBName = qb
|
|
}
|
|
$$ = hs
|
|
}
|
|
|
|
HintStorageTypeAndTableList:
|
|
HintStorageTypeAndTable
|
|
{
|
|
$$ = []*ast.TableOptimizerHint{$1}
|
|
}
|
|
| HintStorageTypeAndTableList ',' HintStorageTypeAndTable
|
|
{
|
|
$$ = append($1, $3)
|
|
}
|
|
|
|
HintStorageTypeAndTable:
|
|
HintStorageType '[' HintTableList ']'
|
|
{
|
|
h := $3
|
|
h.HintData = ast.NewCIStr($1)
|
|
$$ = h
|
|
}
|
|
|
|
LeadingTableList:
|
|
LeadingTableElement
|
|
{
|
|
$$ = &ast.LeadingList{Items: []interface{}{$1}}
|
|
}
|
|
| LeadingTableList ',' LeadingTableElement
|
|
{
|
|
$$ = $1
|
|
$$.Items = append($$.Items, $3)
|
|
}
|
|
|
|
LeadingTableElement:
|
|
HintTable
|
|
{
|
|
tmp := $1
|
|
$$ = &tmp
|
|
}
|
|
| '(' LeadingTableList ')'
|
|
{
|
|
$$ = $2
|
|
}
|
|
|
|
QueryBlockOpt:
|
|
/* empty */
|
|
{
|
|
$$ = ""
|
|
}
|
|
| hintSingleAtIdentifier
|
|
|
|
CommaOpt:
|
|
/*empty*/
|
|
{}
|
|
| ','
|
|
{}
|
|
|
|
PartitionListOpt:
|
|
/* empty */
|
|
{
|
|
$$ = nil
|
|
}
|
|
| "PARTITION" '(' PartitionList ')'
|
|
{
|
|
$$ = $3
|
|
}
|
|
|
|
PartitionList:
|
|
Identifier
|
|
{
|
|
$$ = []ast.CIStr{ast.NewCIStr($1)}
|
|
}
|
|
| PartitionList CommaOpt Identifier
|
|
{
|
|
$$ = append($1, ast.NewCIStr($3))
|
|
}
|
|
|
|
/**
|
|
* HintTableListOpt:
|
|
*
|
|
* [@query_block_name] [tbl_name [, tbl_name] ...]
|
|
* [tbl_name@query_block_name [, tbl_name@query_block_name] ...]
|
|
*
|
|
*/
|
|
HintTableListOpt:
|
|
HintTableList
|
|
| QueryBlockOpt
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
QBName: ast.NewCIStr($1),
|
|
}
|
|
}
|
|
|
|
HintTableList:
|
|
QueryBlockOpt HintTable
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
Tables: []ast.HintTable{$2},
|
|
QBName: ast.NewCIStr($1),
|
|
}
|
|
}
|
|
| HintTableList ',' HintTable
|
|
{
|
|
h := $1
|
|
h.Tables = append(h.Tables, $3)
|
|
$$ = h
|
|
}
|
|
|
|
HintTable:
|
|
Identifier QueryBlockOpt PartitionListOpt
|
|
{
|
|
$$ = ast.HintTable{
|
|
TableName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($2),
|
|
PartitionList: $3,
|
|
}
|
|
}
|
|
| Identifier '.' Identifier QueryBlockOpt PartitionListOpt
|
|
{
|
|
$$ = ast.HintTable{
|
|
DBName: ast.NewCIStr($1),
|
|
TableName: ast.NewCIStr($3),
|
|
QBName: ast.NewCIStr($4),
|
|
PartitionList: $5,
|
|
}
|
|
}
|
|
|
|
ViewNameList:
|
|
ViewNameList '.' ViewName
|
|
{
|
|
h := $1
|
|
h.Tables = append(h.Tables, $3)
|
|
$$ = h
|
|
}
|
|
| ViewName
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
Tables: []ast.HintTable{$1},
|
|
}
|
|
}
|
|
|
|
ViewName:
|
|
Identifier QueryBlockOpt
|
|
{
|
|
$$ = ast.HintTable{
|
|
TableName: ast.NewCIStr($1),
|
|
QBName: ast.NewCIStr($2),
|
|
}
|
|
}
|
|
| QueryBlockOpt
|
|
{
|
|
$$ = ast.HintTable{
|
|
QBName: ast.NewCIStr($1),
|
|
}
|
|
}
|
|
|
|
/**
|
|
* HintIndexList:
|
|
*
|
|
* [@query_block_name] tbl_name [index_name [, index_name] ...]
|
|
* tbl_name@query_block_name [index_name [, index_name] ...]
|
|
*/
|
|
HintIndexList:
|
|
QueryBlockOpt HintTable CommaOpt IndexNameListOpt
|
|
{
|
|
h := $4
|
|
h.Tables = []ast.HintTable{$2}
|
|
h.QBName = ast.NewCIStr($1)
|
|
$$ = h
|
|
}
|
|
|
|
IndexNameListOpt:
|
|
/* empty */
|
|
{
|
|
$$ = &ast.TableOptimizerHint{}
|
|
}
|
|
| IndexNameList
|
|
|
|
IndexNameList:
|
|
Identifier
|
|
{
|
|
$$ = &ast.TableOptimizerHint{
|
|
Indexes: []ast.CIStr{ast.NewCIStr($1)},
|
|
}
|
|
}
|
|
| IndexNameList ',' Identifier
|
|
{
|
|
h := $1
|
|
h.Indexes = append(h.Indexes, ast.NewCIStr($3))
|
|
$$ = h
|
|
}
|
|
|
|
/**
|
|
* Miscellaneous rules
|
|
*/
|
|
SubqueryStrategiesOpt:
|
|
/* empty */
|
|
{}
|
|
| SubqueryStrategies
|
|
|
|
SubqueryStrategies:
|
|
SubqueryStrategy
|
|
{}
|
|
| SubqueryStrategies ',' SubqueryStrategy
|
|
|
|
Value:
|
|
hintStringLit
|
|
| Identifier
|
|
| hintIntLit
|
|
{
|
|
$$ = strconv.FormatUint($1, 10)
|
|
}
|
|
| '+' hintIntLit
|
|
{
|
|
$$ = strconv.FormatUint($2, 10)
|
|
}
|
|
| '-' hintIntLit
|
|
{
|
|
if $2 > 9223372036854775808 {
|
|
yylex.AppendError(yylex.Errorf("the Signed Value should be at the range of [-9223372036854775808, 9223372036854775807]."))
|
|
return 1
|
|
} else if $2 == 9223372036854775808 {
|
|
signed_one := int64(1)
|
|
$$ = strconv.FormatInt(signed_one<<63, 10)
|
|
} else {
|
|
$$ = strconv.FormatInt(-int64($2), 10)
|
|
}
|
|
}
|
|
|
|
UnitOfBytes:
|
|
"MB"
|
|
{
|
|
$$ = 1024 * 1024
|
|
}
|
|
| "GB"
|
|
{
|
|
$$ = 1024 * 1024 * 1024
|
|
}
|
|
|
|
HintTrueOrFalse:
|
|
"TRUE"
|
|
{
|
|
$$ = &ast.TableOptimizerHint{HintData: true}
|
|
}
|
|
| "FALSE"
|
|
{
|
|
$$ = &ast.TableOptimizerHint{HintData: false}
|
|
}
|
|
|
|
JoinOrderOptimizerHintName:
|
|
"JOIN_ORDER"
|
|
| "JOIN_PREFIX"
|
|
| "JOIN_SUFFIX"
|
|
|
|
UnsupportedTableLevelOptimizerHintName:
|
|
"BKA"
|
|
| "NO_BKA"
|
|
| "BNL"
|
|
| "NO_BNL"
|
|
/* HASH_JOIN is supported by TiDB */
|
|
| "NO_MERGE"
|
|
|
|
SupportedTableLevelOptimizerHintName:
|
|
"MERGE_JOIN"
|
|
| "NO_MERGE_JOIN"
|
|
| "BROADCAST_JOIN"
|
|
| "SHUFFLE_JOIN"
|
|
| "INL_JOIN"
|
|
| "INDEX_JOIN"
|
|
| "NO_INDEX_JOIN"
|
|
| "MERGE"
|
|
| "INL_HASH_JOIN"
|
|
| "INDEX_HASH_JOIN"
|
|
| "NO_INDEX_HASH_JOIN"
|
|
| "SWAP_JOIN_INPUTS"
|
|
| "NO_SWAP_JOIN_INPUTS"
|
|
| "INL_MERGE_JOIN"
|
|
| "INDEX_MERGE_JOIN"
|
|
| "NO_INDEX_MERGE_JOIN"
|
|
| "HASH_JOIN"
|
|
| "NO_HASH_JOIN"
|
|
| "HASH_JOIN_BUILD"
|
|
| "HASH_JOIN_PROBE"
|
|
| "HYPO_INDEX"
|
|
|
|
UnsupportedIndexLevelOptimizerHintName:
|
|
"INDEX_MERGE"
|
|
/* NO_INDEX_MERGE is currently a nullary hint in TiDB */
|
|
| "MRR"
|
|
| "NO_MRR"
|
|
| "NO_ICP"
|
|
| "NO_RANGE_OPTIMIZATION"
|
|
| "SKIP_SCAN"
|
|
| "NO_SKIP_SCAN"
|
|
|
|
SupportedIndexLevelOptimizerHintName:
|
|
"USE_INDEX"
|
|
| "IGNORE_INDEX"
|
|
| "USE_INDEX_MERGE"
|
|
| "FORCE_INDEX"
|
|
| "ORDER_INDEX"
|
|
| "NO_ORDER_INDEX"
|
|
| "INDEX_LOOKUP_PUSHDOWN"
|
|
|
|
SubqueryOptimizerHintName:
|
|
"SEMIJOIN"
|
|
| "NO_SEMIJOIN"
|
|
|
|
SubqueryStrategy:
|
|
"DUPSWEEDOUT"
|
|
| "FIRSTMATCH"
|
|
| "LOOSESCAN"
|
|
| "MATERIALIZATION"
|
|
|
|
BooleanHintName:
|
|
"USE_TOJA"
|
|
| "USE_CASCADES"
|
|
|
|
NullaryHintName:
|
|
"USE_PLAN_CACHE"
|
|
| "HASH_AGG"
|
|
| "MPP_1PHASE_AGG"
|
|
| "MPP_2PHASE_AGG"
|
|
| "STREAM_AGG"
|
|
| "AGG_TO_COP"
|
|
| "LIMIT_TO_COP"
|
|
| "NO_INDEX_MERGE"
|
|
| "READ_CONSISTENT_REPLICA"
|
|
| "IGNORE_PLAN_CACHE"
|
|
| "STRAIGHT_JOIN"
|
|
| "SEMI_JOIN_REWRITE"
|
|
| "NO_DECORRELATE"
|
|
|
|
HintQueryType:
|
|
"OLAP"
|
|
| "OLTP"
|
|
|
|
HintStorageType:
|
|
"TIKV"
|
|
| "TIFLASH"
|
|
|
|
Identifier:
|
|
hintIdentifier
|
|
/* MySQL 8.0 hint names */
|
|
| "JOIN_FIXED_ORDER"
|
|
| "JOIN_ORDER"
|
|
| "JOIN_PREFIX"
|
|
| "JOIN_SUFFIX"
|
|
| "BKA"
|
|
| "NO_BKA"
|
|
| "BNL"
|
|
| "NO_BNL"
|
|
| "HASH_JOIN"
|
|
| "HASH_JOIN_BUILD"
|
|
| "HASH_JOIN_PROBE"
|
|
| "NO_HASH_JOIN"
|
|
| "MERGE"
|
|
| "NO_MERGE"
|
|
| "INDEX_MERGE"
|
|
| "NO_INDEX_MERGE"
|
|
| "MRR"
|
|
| "NO_MRR"
|
|
| "NO_ICP"
|
|
| "NO_RANGE_OPTIMIZATION"
|
|
| "SKIP_SCAN"
|
|
| "NO_SKIP_SCAN"
|
|
| "SEMIJOIN"
|
|
| "NO_SEMIJOIN"
|
|
| "MAX_EXECUTION_TIME"
|
|
| "SET_VAR"
|
|
| "RESOURCE_GROUP"
|
|
| "QB_NAME"
|
|
| "HYPO_INDEX"
|
|
/* TiDB hint names */
|
|
| "AGG_TO_COP"
|
|
| "LIMIT_TO_COP"
|
|
| "IGNORE_PLAN_CACHE"
|
|
| "WRITE_SLOW_LOG"
|
|
| "HASH_AGG"
|
|
| "MPP_1PHASE_AGG"
|
|
| "MPP_2PHASE_AGG"
|
|
| "IGNORE_INDEX"
|
|
| "INL_HASH_JOIN"
|
|
| "INDEX_HASH_JOIN"
|
|
| "NO_INDEX_HASH_JOIN"
|
|
| "INL_JOIN"
|
|
| "INDEX_JOIN"
|
|
| "NO_INDEX_JOIN"
|
|
| "INL_MERGE_JOIN"
|
|
| "INDEX_MERGE_JOIN"
|
|
| "NO_INDEX_MERGE_JOIN"
|
|
| "MEMORY_QUOTA"
|
|
| "NO_SWAP_JOIN_INPUTS"
|
|
| "QUERY_TYPE"
|
|
| "READ_CONSISTENT_REPLICA"
|
|
| "READ_FROM_STORAGE"
|
|
| "MERGE_JOIN"
|
|
| "NO_MERGE_JOIN"
|
|
| "BROADCAST_JOIN"
|
|
| "SHUFFLE_JOIN"
|
|
| "STREAM_AGG"
|
|
| "SWAP_JOIN_INPUTS"
|
|
| "USE_INDEX_MERGE"
|
|
| "USE_INDEX"
|
|
| "ORDER_INDEX"
|
|
| "NO_ORDER_INDEX"
|
|
| "INDEX_LOOKUP_PUSHDOWN"
|
|
| "USE_PLAN_CACHE"
|
|
| "USE_TOJA"
|
|
| "TIME_RANGE"
|
|
| "USE_CASCADES"
|
|
| "NTH_PLAN"
|
|
| "FORCE_INDEX"
|
|
| "STRAIGHT_JOIN"
|
|
| "LEADING"
|
|
| "SEMI_JOIN_REWRITE"
|
|
| "NO_DECORRELATE"
|
|
/* other keywords */
|
|
| "OLAP"
|
|
| "OLTP"
|
|
| "TIKV"
|
|
| "TIFLASH"
|
|
| "FALSE"
|
|
| "TRUE"
|
|
| "MB"
|
|
| "GB"
|
|
| "DUPSWEEDOUT"
|
|
| "FIRSTMATCH"
|
|
| "LOOSESCAN"
|
|
| "MATERIALIZATION"
|
|
%%
|