diff --git a/Makefile b/Makefile index 107e73629bbf7..36291f5d6ca40 100644 --- a/Makefile +++ b/Makefile @@ -147,7 +147,7 @@ gotest_in_verify_ci_part_2: failpoint-enable tools/bin/gotestsum tools/bin/gocov race: failpoint-enable @export log_level=debug; \ - $(GOTEST) -timeout 20m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; } + $(GOTEST) -timeout 25m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; } @$(FAILPOINT_DISABLE) leak: failpoint-enable diff --git a/executor/executor_test.go b/executor/executor_test.go index c821901d5a18d..f503504d739b0 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -9487,3 +9487,17 @@ func (s *testSerialSuite) TestIssue28650(c *C) { }() } } + +func (s *testSerialSuite) TestIssue30289(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + fpName := "github.com/pingcap/tidb/executor/issue30289" + c.Assert(failpoint.Enable(fpName, `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable(fpName), IsNil) + }() + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int)") + err := tk.QueryToErr("select /*+ hash_join(t1) */ * from t t1 join t t2 on t1.a=t2.a") + c.Assert(err.Error(), Matches, "issue30289 build return error") +} diff --git a/executor/join.go b/executor/join.go index 2b97c0b1f93cf..c28d1e107932e 100644 --- a/executor/join.go +++ b/executor/join.go @@ -214,9 +214,13 @@ func (e *HashJoinExec) fetchProbeSideChunks(ctx context.Context) { return } if !hasWaitedForBuild { + failpoint.Inject("issue30289", func(val failpoint.Value) { + if val.(bool) { + probeSideResult.Reset() + } + }) if probeSideResult.NumRows() == 0 && !e.useOuterToBuild { e.finished.Store(true) - return } emptyBuild, buildErr := e.wait4BuildSide() if buildErr != nil { @@ -258,6 +262,13 @@ func (e *HashJoinExec) wait4BuildSide() (emptyBuild bool, err error) { func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chunk.Chunk, doneCh <-chan struct{}) { defer close(chkCh) var err error + failpoint.Inject("issue30289", func(val failpoint.Value) { + if val.(bool) { + err = errors.Errorf("issue30289 build return error") + e.buildFinished <- errors.Trace(err) + return + } + }) for { if e.finished.Load().(bool) { return diff --git a/executor/shuffle.go b/executor/shuffle.go index a71e388c02bac..9143afd032f10 100644 --- a/executor/shuffle.go +++ b/executor/shuffle.go @@ -142,17 +142,29 @@ func (e *ShuffleExec) Close() error { if !e.prepared { for _, w := range e.workers { for _, r := range w.receivers { - close(r.inputHolderCh) - close(r.inputCh) + if r.inputHolderCh != nil { + close(r.inputHolderCh) + } + if r.inputCh != nil { + close(r.inputCh) + } } - close(w.outputHolderCh) + if w.outputHolderCh != nil { + close(w.outputHolderCh) + } + } + if e.outputCh != nil { + close(e.outputCh) } - close(e.outputCh) } - close(e.finishCh) + if e.finishCh != nil { + close(e.finishCh) + } for _, w := range e.workers { for _, r := range w.receivers { - for range r.inputCh { + if r.inputCh != nil { + for range r.inputCh { + } } } // close child executor of each worker @@ -160,7 +172,9 @@ func (e *ShuffleExec) Close() error { firstErr = err } } - for range e.outputCh { // workers exit before `e.outputCh` is closed. + if e.outputCh != nil { + for range e.outputCh { // workers exit before `e.outputCh` is closed. + } } e.executed = false diff --git a/expression/builtin_time.go b/expression/builtin_time.go index c6b3cfa669ef9..0c87fe8ec299c 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -4856,7 +4856,7 @@ func (b *builtinUnixTimestampIntSig) evalIntWithCtx(ctx sessionctx.Context, row } tz := ctx.GetSessionVars().Location() - t, err := val.GoTime(tz) + t, err := val.AdjustedGoTime(tz) if err != nil { return 0, false, nil } diff --git a/expression/builtin_time_vec.go b/expression/builtin_time_vec.go index 90fc1323117f9..13a0135d267b5 100644 --- a/expression/builtin_time_vec.go +++ b/expression/builtin_time_vec.go @@ -2390,7 +2390,7 @@ func (b *builtinUnixTimestampIntSig) vecEvalInt(input *chunk.Chunk, result *chun continue } - t, err := buf.GetTime(i).GoTime(getTimeZone(b.ctx)) + t, err := buf.GetTime(i).AdjustedGoTime(getTimeZone(b.ctx)) if err != nil { i64s[i] = 0 continue diff --git a/expression/integration_test.go b/expression/integration_test.go index 8f23a4ae37f9c..bbf0961e63509 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -3978,6 +3978,8 @@ func (s *testIntegrationSuite) TestCompareBuiltin(c *C) { result.Check(testkit.Rows("1 1 1")) result = tk.MustQuery(`select INTERVAL(100, NULL, NULL, NULL, NULL, NULL, 100);`) result.Check(testkit.Rows("6")) + result = tk.MustQuery(`SELECT INTERVAL(0,(1*5)/2) + INTERVAL(5,4,3);`) + result.Check(testkit.Rows("2")) // for greatest result = tk.MustQuery(`select greatest(1, 2, 3), greatest("a", "b", "c"), greatest(1.1, 1.2, 1.3), greatest("123a", 1, 2)`) @@ -9365,6 +9367,25 @@ func (s *testIntegrationSuite) TestIssue30101(c *C) { tk.MustQuery("select greatest(c1, c2) from t1;").Sort().Check(testkit.Rows("9223372036854775809")) } +func (s *testIntegrationSuite) TestIssue28739(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec(`USE test`) + tk.MustExec("SET time_zone = 'Europe/Vilnius'") + tk.MustQuery("SELECT UNIX_TIMESTAMP('2020-03-29 03:45:00')").Check(testkit.Rows("1585443600")) + tk.MustQuery("SELECT FROM_UNIXTIME(UNIX_TIMESTAMP('2020-03-29 03:45:00'))").Check(testkit.Rows("2020-03-29 04:00:00")) + tk.MustExec(`DROP TABLE IF EXISTS t`) + tk.MustExec(`CREATE TABLE t (dt DATETIME NULL)`) + defer tk.MustExec(`DROP TABLE t`) + // Test the vector implememtation + tk.MustExec(`INSERT INTO t VALUES ('2021-10-31 02:30:00'), ('2021-03-28 02:30:00'), ('2020-10-04 02:15:00'), ('2020-03-29 03:45:00'), (NULL)`) + tk.MustQuery(`SELECT dt, UNIX_TIMESTAMP(dt) FROM t`).Sort().Check(testkit.Rows( + "2020-03-29 03:45:00 1585443600", + "2020-10-04 02:15:00 1601766900", + "2021-03-28 02:30:00 1616891400", + "2021-10-31 02:30:00 1635636600", + " ")) +} + func (s *testIntegrationSuite) TestIssue30326(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/parser/parser.go b/parser/parser.go index 01b072fc0bf2a..99a2267eb4aa6 100644 --- a/parser/parser.go +++ b/parser/parser.go @@ -54,7 +54,7 @@ type yyXError struct { } const ( - yyDefault = 58101 + yyDefault = 58100 yyEOFCode = 57344 account = 57573 action = 57574 @@ -195,7 +195,7 @@ const ( correlation = 57996 cpu = 57636 create = 57383 - createTableSelect = 58085 + createTableSelect = 58084 cross = 57384 csvBackslashEscape = 57637 csvDelimiter = 57638 @@ -325,8 +325,8 @@ const ( help = 57693 hexLit = 58059 highPriority = 57430 - higherThanComma = 58100 - higherThanParenthese = 58094 + higherThanComma = 58099 + higherThanParenthese = 58093 hintComment = 57353 histogram = 57694 history = 57695 @@ -352,7 +352,7 @@ const ( inplace = 57934 insert = 57446 insertMethod = 57705 - insertValues = 58083 + insertValues = 58082 instance = 57706 instant = 57935 int1Type = 57448 @@ -422,25 +422,24 @@ const ( longblobType = 57470 longtextType = 57471 lowPriority = 57472 - lowerThanCharsetKwd = 58086 - lowerThanComma = 58099 - lowerThanCreateTableSelect = 58084 - lowerThanEq = 58096 - lowerThanFunction = 58091 - lowerThanInsertValues = 58082 - lowerThanIntervalKeyword = 58077 - lowerThanKey = 58087 - lowerThanLocal = 58088 - lowerThanNot = 58098 - lowerThanOn = 58095 - lowerThanParenthese = 58093 - lowerThanRemove = 58089 + lowerThanCharsetKwd = 58085 + lowerThanComma = 58098 + lowerThanCreateTableSelect = 58083 + lowerThanEq = 58095 + lowerThanFunction = 58090 + lowerThanInsertValues = 58081 + lowerThanKey = 58086 + lowerThanLocal = 58087 + lowerThanNot = 58097 + lowerThanOn = 58094 + lowerThanParenthese = 58092 + lowerThanRemove = 58088 lowerThanSelectOpt = 58076 - lowerThanSelectStmt = 58081 - lowerThanSetKeyword = 58080 - lowerThanStringLitToken = 58079 - lowerThanValueKeyword = 58078 - lowerThenOrder = 58090 + lowerThanSelectStmt = 58080 + lowerThanSetKeyword = 58079 + lowerThanStringLitToken = 58078 + lowerThanValueKeyword = 58077 + lowerThenOrder = 58089 lsh = 58068 master = 57727 match = 57473 @@ -474,7 +473,7 @@ const ( national = 57746 natural = 57572 ncharType = 57747 - neg = 58097 + neg = 58096 neq = 58069 neqSynonym = 58070 never = 57748 @@ -716,7 +715,7 @@ const ( systemTime = 57873 tableChecksum = 57874 tableKwd = 57534 - tableRefPriority = 58092 + tableRefPriority = 58091 tableSample = 57535 tables = 57875 tablespace = 57876 @@ -1467,10 +1466,10 @@ var ( 57464: 645, // lines (490x) 57371: 646, // by (487x) 58062: 647, // assignmentEq (485x) - 58320: 648, // Identifier (483x) - 58395: 649, // NotKeywordToken (483x) - 58616: 650, // TiDBKeyword (483x) - 58626: 651, // UnReservedKeyword (483x) + 58319: 648, // Identifier (483x) + 58394: 649, // NotKeywordToken (483x) + 58615: 650, // TiDBKeyword (483x) + 58625: 651, // UnReservedKeyword (483x) 57512: 652, // require (482x) 57361: 653, // alter (481x) 64: 654, // '@' (477x) @@ -1515,672 +1514,671 @@ var ( 57539: 693, // tinyblobType (461x) 57540: 694, // tinyIntType (461x) 57541: 695, // tinytextType (461x) - 58581: 696, // SubSelect (208x) - 58635: 697, // UserVariable (171x) - 58557: 698, // SimpleIdent (170x) - 58372: 699, // Literal (168x) - 58571: 700, // StringLiteral (168x) - 58393: 701, // NextValueForSequence (167x) - 58297: 702, // FunctionCallGeneric (166x) - 58298: 703, // FunctionCallKeyword (166x) - 58299: 704, // FunctionCallNonKeyword (166x) - 58300: 705, // FunctionNameConflict (166x) - 58301: 706, // FunctionNameDateArith (166x) - 58302: 707, // FunctionNameDateArithMultiForms (166x) - 58303: 708, // FunctionNameDatetimePrecision (166x) - 58304: 709, // FunctionNameOptionalBraces (166x) - 58305: 710, // FunctionNameSequence (166x) - 58556: 711, // SimpleExpr (166x) - 58582: 712, // SumExpr (166x) - 58584: 713, // SystemVariable (166x) - 58646: 714, // Variable (166x) - 58669: 715, // WindowFuncCall (166x) - 58149: 716, // BitExpr (153x) - 58466: 717, // PredicateExpr (130x) - 58152: 718, // BoolPri (127x) - 58264: 719, // Expression (127x) - 58684: 720, // logAnd (96x) - 58685: 721, // logOr (96x) - 58391: 722, // NUM (96x) - 58254: 723, // EqOpt (86x) - 58594: 724, // TableName (75x) - 58572: 725, // StringName (56x) + 58580: 696, // SubSelect (208x) + 58634: 697, // UserVariable (171x) + 58556: 698, // SimpleIdent (170x) + 58371: 699, // Literal (168x) + 58570: 700, // StringLiteral (168x) + 58392: 701, // NextValueForSequence (167x) + 58296: 702, // FunctionCallGeneric (166x) + 58297: 703, // FunctionCallKeyword (166x) + 58298: 704, // FunctionCallNonKeyword (166x) + 58299: 705, // FunctionNameConflict (166x) + 58300: 706, // FunctionNameDateArith (166x) + 58301: 707, // FunctionNameDateArithMultiForms (166x) + 58302: 708, // FunctionNameDatetimePrecision (166x) + 58303: 709, // FunctionNameOptionalBraces (166x) + 58304: 710, // FunctionNameSequence (166x) + 58555: 711, // SimpleExpr (166x) + 58581: 712, // SumExpr (166x) + 58583: 713, // SystemVariable (166x) + 58645: 714, // Variable (166x) + 58668: 715, // WindowFuncCall (166x) + 58148: 716, // BitExpr (153x) + 58465: 717, // PredicateExpr (130x) + 58151: 718, // BoolPri (127x) + 58263: 719, // Expression (127x) + 58683: 720, // logAnd (96x) + 58684: 721, // logOr (96x) + 58390: 722, // NUM (96x) + 58253: 723, // EqOpt (86x) + 58593: 724, // TableName (75x) + 58571: 725, // StringName (56x) 57549: 726, // unsigned (47x) 57495: 727, // over (45x) 57571: 728, // zerofill (45x) - 58174: 729, // ColumnName (40x) + 58173: 729, // ColumnName (40x) 57400: 730, // deleteKwd (40x) - 58363: 731, // LengthNum (40x) + 58362: 731, // LengthNum (40x) 57404: 732, // distinct (36x) 57405: 733, // distinctRow (36x) - 58674: 734, // WindowingClause (35x) + 58673: 734, // WindowingClause (35x) 57399: 735, // delayed (33x) 57430: 736, // highPriority (33x) 57472: 737, // lowPriority (33x) - 58512: 738, // SelectStmt (29x) - 58513: 739, // SelectStmtBasic (29x) - 58515: 740, // SelectStmtFromDualTable (29x) - 58516: 741, // SelectStmtFromTable (29x) - 58532: 742, // SetOprClause (29x) - 58533: 743, // SetOprClauseList (28x) - 58536: 744, // SetOprStmtWithLimitOrderBy (28x) - 58537: 745, // SetOprStmtWoutLimitOrderBy (28x) + 58511: 738, // SelectStmt (29x) + 58512: 739, // SelectStmtBasic (29x) + 58514: 740, // SelectStmtFromDualTable (29x) + 58515: 741, // SelectStmtFromTable (29x) + 58531: 742, // SetOprClause (29x) + 58532: 743, // SetOprClauseList (28x) + 58535: 744, // SetOprStmtWithLimitOrderBy (28x) + 58536: 745, // SetOprStmtWoutLimitOrderBy (28x) 57353: 746, // hintComment (27x) - 58275: 747, // FieldLen (26x) - 58352: 748, // Int64Num (26x) - 58525: 749, // SelectStmtWithClause (25x) - 58535: 750, // SetOprStmt (25x) - 58675: 751, // WithClause (25x) - 58432: 752, // OptWindowingClause (24x) - 58437: 753, // OrderBy (23x) - 58519: 754, // SelectStmtLimit (23x) + 58274: 747, // FieldLen (26x) + 58351: 748, // Int64Num (26x) + 58524: 749, // SelectStmtWithClause (25x) + 58534: 750, // SetOprStmt (25x) + 58674: 751, // WithClause (25x) + 58431: 752, // OptWindowingClause (24x) + 58436: 753, // OrderBy (23x) + 58518: 754, // SelectStmtLimit (23x) 57527: 755, // sqlBigResult (23x) 57528: 756, // sqlCalcFoundRows (23x) 57529: 757, // sqlSmallResult (23x) - 58231: 758, // DirectPlacementOption (21x) - 58162: 759, // CharsetKw (20x) - 58637: 760, // Username (20x) - 58265: 761, // ExpressionList (17x) - 58461: 762, // PlacementPolicyOption (17x) - 58629: 763, // UpdateStmtNoWith (17x) - 58230: 764, // DeleteWithoutUsingStmt (16x) - 58321: 765, // IfExists (16x) - 58459: 766, // PlacementOption (16x) + 58230: 758, // DirectPlacementOption (21x) + 58161: 759, // CharsetKw (20x) + 58636: 760, // Username (20x) + 58264: 761, // ExpressionList (17x) + 58460: 762, // PlacementPolicyOption (17x) + 58628: 763, // UpdateStmtNoWith (17x) + 58229: 764, // DeleteWithoutUsingStmt (16x) + 58320: 765, // IfExists (16x) + 58458: 766, // PlacementOption (16x) 57537: 767, // terminated (16x) - 58232: 768, // DistinctKwd (15x) - 58322: 769, // IfNotExists (15x) - 58349: 770, // InsertIntoStmt (15x) - 58417: 771, // OptFieldLen (15x) - 58487: 772, // ReplaceIntoStmt (15x) - 58628: 773, // UpdateStmt (15x) - 58233: 774, // DistinctOpt (14x) + 58231: 768, // DistinctKwd (15x) + 58321: 769, // IfNotExists (15x) + 58348: 770, // InsertIntoStmt (15x) + 58416: 771, // OptFieldLen (15x) + 58486: 772, // ReplaceIntoStmt (15x) + 58627: 773, // UpdateStmt (15x) + 58232: 774, // DistinctOpt (14x) 57411: 775, // enclosed (14x) - 58448: 776, // PartitionNameList (14x) - 58659: 777, // WhereClause (14x) - 58660: 778, // WhereClauseOptional (14x) - 58225: 779, // DefaultKwdOpt (13x) + 58447: 776, // PartitionNameList (14x) + 58658: 777, // WhereClause (14x) + 58659: 778, // WhereClauseOptional (14x) + 58224: 779, // DefaultKwdOpt (13x) 57412: 780, // escaped (13x) 57491: 781, // optionally (13x) - 58595: 782, // TableNameList (13x) - 58229: 783, // DeleteWithUsingStmt (12x) - 58263: 784, // ExprOrDefault (12x) - 58357: 785, // JoinTable (12x) - 58411: 786, // OptBinary (12x) - 58503: 787, // RolenameComposed (12x) - 58591: 788, // TableFactor (12x) - 58604: 789, // TableRef (12x) - 58124: 790, // AnalyzeOptionListOpt (11x) - 58228: 791, // DeleteFromStmt (11x) - 58292: 792, // FromOrIn (11x) - 58618: 793, // TimestampUnit (11x) - 58163: 794, // CharsetName (10x) - 58175: 795, // ColumnNameList (10x) - 58396: 796, // NotSym (10x) - 58438: 797, // OrderByOptional (10x) - 58440: 798, // PartDefOption (10x) - 58555: 799, // SignedNum (10x) - 58155: 800, // BuggyDefaultFalseDistinctOpt (9x) - 58215: 801, // DBName (9x) - 58224: 802, // DefaultFalseDistinctOpt (9x) - 58358: 803, // JoinType (9x) + 58594: 782, // TableNameList (13x) + 58228: 783, // DeleteWithUsingStmt (12x) + 58262: 784, // ExprOrDefault (12x) + 58356: 785, // JoinTable (12x) + 58410: 786, // OptBinary (12x) + 58502: 787, // RolenameComposed (12x) + 58590: 788, // TableFactor (12x) + 58603: 789, // TableRef (12x) + 58123: 790, // AnalyzeOptionListOpt (11x) + 58227: 791, // DeleteFromStmt (11x) + 58291: 792, // FromOrIn (11x) + 58617: 793, // TimestampUnit (11x) + 58162: 794, // CharsetName (10x) + 58174: 795, // ColumnNameList (10x) + 58395: 796, // NotSym (10x) + 58437: 797, // OrderByOptional (10x) + 58439: 798, // PartDefOption (10x) + 58554: 799, // SignedNum (10x) + 58154: 800, // BuggyDefaultFalseDistinctOpt (9x) + 58214: 801, // DBName (9x) + 58223: 802, // DefaultFalseDistinctOpt (9x) + 58357: 803, // JoinType (9x) 57466: 804, // load (9x) 57482: 805, // noWriteToBinLog (9x) - 58401: 806, // NumLiteral (9x) - 58502: 807, // Rolename (9x) - 58497: 808, // RoleNameString (9x) - 58120: 809, // AlterTableStmt (8x) - 58214: 810, // CrossOpt (8x) - 58255: 811, // EqOrAssignmentEq (8x) - 58266: 812, // ExpressionListOpt (8x) - 58343: 813, // IndexPartSpecification (8x) - 58359: 814, // KeyOrIndex (8x) - 58520: 815, // SelectStmtLimitOpt (8x) - 58617: 816, // TimeUnit (8x) - 58649: 817, // VariableName (8x) - 58106: 818, // AllOrPartitionNameList (7x) - 58198: 819, // ConstraintKeywordOpt (7x) - 58281: 820, // FieldsOrColumns (7x) - 58290: 821, // ForceOpt (7x) - 58344: 822, // IndexPartSpecificationList (7x) - 58394: 823, // NoWriteToBinLogAliasOpt (7x) - 58470: 824, // Priority (7x) - 58507: 825, // RowFormat (7x) - 58510: 826, // RowValue (7x) - 58530: 827, // SetExpr (7x) - 58541: 828, // ShowDatabaseNameOpt (7x) - 58601: 829, // TableOption (7x) + 58400: 806, // NumLiteral (9x) + 58501: 807, // Rolename (9x) + 58496: 808, // RoleNameString (9x) + 58119: 809, // AlterTableStmt (8x) + 58213: 810, // CrossOpt (8x) + 58254: 811, // EqOrAssignmentEq (8x) + 58265: 812, // ExpressionListOpt (8x) + 58342: 813, // IndexPartSpecification (8x) + 58358: 814, // KeyOrIndex (8x) + 58519: 815, // SelectStmtLimitOpt (8x) + 58616: 816, // TimeUnit (8x) + 58648: 817, // VariableName (8x) + 58105: 818, // AllOrPartitionNameList (7x) + 58197: 819, // ConstraintKeywordOpt (7x) + 58280: 820, // FieldsOrColumns (7x) + 58289: 821, // ForceOpt (7x) + 58343: 822, // IndexPartSpecificationList (7x) + 58393: 823, // NoWriteToBinLogAliasOpt (7x) + 58469: 824, // Priority (7x) + 58506: 825, // RowFormat (7x) + 58509: 826, // RowValue (7x) + 58529: 827, // SetExpr (7x) + 58540: 828, // ShowDatabaseNameOpt (7x) + 58600: 829, // TableOption (7x) 57562: 830, // varying (7x) 57380: 831, // column (6x) - 58169: 832, // ColumnDef (6x) - 58217: 833, // DatabaseOption (6x) - 58220: 834, // DatabaseSym (6x) - 58257: 835, // EscapedTableRef (6x) - 58262: 836, // ExplainableStmt (6x) - 58279: 837, // FieldTerminator (6x) + 58168: 832, // ColumnDef (6x) + 58216: 833, // DatabaseOption (6x) + 58219: 834, // DatabaseSym (6x) + 58256: 835, // EscapedTableRef (6x) + 58261: 836, // ExplainableStmt (6x) + 58278: 837, // FieldTerminator (6x) 57426: 838, // grant (6x) - 58326: 839, // IgnoreOptional (6x) - 58335: 840, // IndexInvisible (6x) - 58340: 841, // IndexNameList (6x) - 58346: 842, // IndexType (6x) - 58449: 843, // PartitionNameListOpt (6x) + 58325: 839, // IgnoreOptional (6x) + 58334: 840, // IndexInvisible (6x) + 58339: 841, // IndexNameList (6x) + 58345: 842, // IndexType (6x) + 58448: 843, // PartitionNameListOpt (6x) 57508: 844, // release (6x) - 58504: 845, // RolenameList (6x) + 58503: 845, // RolenameList (6x) 57523: 846, // show (6x) - 58599: 847, // TableOptimizerHints (6x) - 58638: 848, // UsernameList (6x) - 58676: 849, // WithClustered (6x) - 58104: 850, // AlgorithmClause (5x) - 58145: 851, // BeginTransactionStmt (5x) - 58156: 852, // ByItem (5x) - 58168: 853, // CollationName (5x) - 58172: 854, // ColumnKeywordOpt (5x) - 58188: 855, // CommitStmt (5x) - 58277: 856, // FieldOpt (5x) - 58278: 857, // FieldOpts (5x) - 58318: 858, // IdentList (5x) - 58338: 859, // IndexName (5x) - 58341: 860, // IndexOption (5x) - 58342: 861, // IndexOptionList (5x) + 58598: 847, // TableOptimizerHints (6x) + 58637: 848, // UsernameList (6x) + 58675: 849, // WithClustered (6x) + 58103: 850, // AlgorithmClause (5x) + 58144: 851, // BeginTransactionStmt (5x) + 58155: 852, // ByItem (5x) + 58167: 853, // CollationName (5x) + 58171: 854, // ColumnKeywordOpt (5x) + 58187: 855, // CommitStmt (5x) + 58276: 856, // FieldOpt (5x) + 58277: 857, // FieldOpts (5x) + 58317: 858, // IdentList (5x) + 58337: 859, // IndexName (5x) + 58340: 860, // IndexOption (5x) + 58341: 861, // IndexOptionList (5x) 57438: 862, // infile (5x) - 58368: 863, // LimitOption (5x) - 58376: 864, // LoadDataStmt (5x) - 58380: 865, // LockClause (5x) - 58413: 866, // OptCharsetWithOptBinary (5x) - 58424: 867, // OptNullTreatment (5x) - 58464: 868, // PolicyName (5x) - 58471: 869, // PriorityOpt (5x) - 58506: 870, // RollbackStmt (5x) - 58511: 871, // SelectLockOpt (5x) - 58518: 872, // SelectStmtIntoOption (5x) - 58540: 873, // SetStmt (5x) - 58605: 874, // TableRefs (5x) - 58631: 875, // UserSpec (5x) - 58130: 876, // Assignment (4x) - 58136: 877, // AuthString (4x) - 58147: 878, // BindableStmt (4x) - 58137: 879, // BRIEBooleanOptionName (4x) - 58138: 880, // BRIEIntegerOptionName (4x) - 58139: 881, // BRIEKeywordOptionName (4x) - 58140: 882, // BRIEOption (4x) - 58141: 883, // BRIEOptions (4x) - 58143: 884, // BRIEStringOptionName (4x) - 58157: 885, // ByList (4x) - 58161: 886, // Char (4x) - 58192: 887, // ConfigItemName (4x) - 58196: 888, // Constraint (4x) - 58286: 889, // FloatOpt (4x) - 58347: 890, // IndexTypeName (4x) + 58367: 863, // LimitOption (5x) + 58375: 864, // LoadDataStmt (5x) + 58379: 865, // LockClause (5x) + 58412: 866, // OptCharsetWithOptBinary (5x) + 58423: 867, // OptNullTreatment (5x) + 58463: 868, // PolicyName (5x) + 58470: 869, // PriorityOpt (5x) + 58505: 870, // RollbackStmt (5x) + 58510: 871, // SelectLockOpt (5x) + 58517: 872, // SelectStmtIntoOption (5x) + 58539: 873, // SetStmt (5x) + 58604: 874, // TableRefs (5x) + 58630: 875, // UserSpec (5x) + 58129: 876, // Assignment (4x) + 58135: 877, // AuthString (4x) + 58146: 878, // BindableStmt (4x) + 58136: 879, // BRIEBooleanOptionName (4x) + 58137: 880, // BRIEIntegerOptionName (4x) + 58138: 881, // BRIEKeywordOptionName (4x) + 58139: 882, // BRIEOption (4x) + 58140: 883, // BRIEOptions (4x) + 58142: 884, // BRIEStringOptionName (4x) + 58156: 885, // ByList (4x) + 58160: 886, // Char (4x) + 58191: 887, // ConfigItemName (4x) + 58195: 888, // Constraint (4x) + 58285: 889, // FloatOpt (4x) + 58346: 890, // IndexTypeName (4x) 57490: 891, // option (4x) - 58429: 892, // OptWild (4x) + 58428: 892, // OptWild (4x) 57494: 893, // outer (4x) - 58465: 894, // Precision (4x) - 58479: 895, // ReferDef (4x) - 58493: 896, // RestrictOrCascadeOpt (4x) - 58509: 897, // RowStmt (4x) - 58526: 898, // SequenceOption (4x) + 58464: 894, // Precision (4x) + 58478: 895, // ReferDef (4x) + 58492: 896, // RestrictOrCascadeOpt (4x) + 58508: 897, // RowStmt (4x) + 58525: 898, // SequenceOption (4x) 57532: 899, // statsExtended (4x) - 58586: 900, // TableAsName (4x) - 58587: 901, // TableAsNameOpt (4x) - 58598: 902, // TableNameOptWild (4x) - 58600: 903, // TableOptimizerHintsOpt (4x) - 58602: 904, // TableOptionList (4x) - 58621: 905, // TransactionChar (4x) - 58632: 906, // UserSpecList (4x) - 58670: 907, // WindowName (4x) - 58127: 908, // AsOfClause (3x) - 58131: 909, // AssignmentList (3x) - 58133: 910, // AttributesOpt (3x) - 58153: 911, // Boolean (3x) - 58181: 912, // ColumnOption (3x) - 58184: 913, // ColumnPosition (3x) - 58189: 914, // CommonTableExpr (3x) - 58210: 915, // CreateTableStmt (3x) - 58218: 916, // DatabaseOptionList (3x) - 58226: 917, // DefaultTrueDistinctOpt (3x) - 58251: 918, // EnforcedOrNot (3x) + 58585: 900, // TableAsName (4x) + 58586: 901, // TableAsNameOpt (4x) + 58597: 902, // TableNameOptWild (4x) + 58599: 903, // TableOptimizerHintsOpt (4x) + 58601: 904, // TableOptionList (4x) + 58620: 905, // TransactionChar (4x) + 58631: 906, // UserSpecList (4x) + 58669: 907, // WindowName (4x) + 58126: 908, // AsOfClause (3x) + 58130: 909, // AssignmentList (3x) + 58132: 910, // AttributesOpt (3x) + 58152: 911, // Boolean (3x) + 58180: 912, // ColumnOption (3x) + 58183: 913, // ColumnPosition (3x) + 58188: 914, // CommonTableExpr (3x) + 58209: 915, // CreateTableStmt (3x) + 58217: 916, // DatabaseOptionList (3x) + 58225: 917, // DefaultTrueDistinctOpt (3x) + 58250: 918, // EnforcedOrNot (3x) 57414: 919, // explain (3x) - 58268: 920, // ExtendedPriv (3x) - 58306: 921, // GeneratedAlways (3x) - 58308: 922, // GlobalScope (3x) - 58312: 923, // GroupByClause (3x) - 58330: 924, // IndexHint (3x) - 58334: 925, // IndexHintType (3x) - 58339: 926, // IndexNameAndTypeOpt (3x) + 58267: 920, // ExtendedPriv (3x) + 58305: 921, // GeneratedAlways (3x) + 58307: 922, // GlobalScope (3x) + 58311: 923, // GroupByClause (3x) + 58329: 924, // IndexHint (3x) + 58333: 925, // IndexHintType (3x) + 58338: 926, // IndexNameAndTypeOpt (3x) 57455: 927, // keys (3x) - 58370: 928, // Lines (3x) - 58388: 929, // MaxValueOrExpression (3x) - 58425: 930, // OptOrder (3x) - 58428: 931, // OptTemporary (3x) - 58441: 932, // PartDefOptionList (3x) - 58443: 933, // PartitionDefinition (3x) - 58452: 934, // PasswordExpire (3x) - 58454: 935, // PasswordOrLockOption (3x) - 58463: 936, // PluginNameList (3x) - 58469: 937, // PrimaryOpt (3x) - 58472: 938, // PrivElem (3x) - 58474: 939, // PrivType (3x) + 58369: 928, // Lines (3x) + 58387: 929, // MaxValueOrExpression (3x) + 58424: 930, // OptOrder (3x) + 58427: 931, // OptTemporary (3x) + 58440: 932, // PartDefOptionList (3x) + 58442: 933, // PartitionDefinition (3x) + 58451: 934, // PasswordExpire (3x) + 58453: 935, // PasswordOrLockOption (3x) + 58462: 936, // PluginNameList (3x) + 58468: 937, // PrimaryOpt (3x) + 58471: 938, // PrivElem (3x) + 58473: 939, // PrivType (3x) 57500: 940, // procedure (3x) - 58488: 941, // RequireClause (3x) - 58489: 942, // RequireClauseOpt (3x) - 58491: 943, // RequireListElement (3x) - 58505: 944, // RolenameWithoutIdent (3x) - 58498: 945, // RoleOrPrivElem (3x) - 58517: 946, // SelectStmtGroup (3x) - 58534: 947, // SetOprOpt (3x) - 58585: 948, // TableAliasRefList (3x) - 58588: 949, // TableElement (3x) - 58597: 950, // TableNameListOpt2 (3x) - 58613: 951, // TextString (3x) - 58620: 952, // TraceableStmt (3x) - 58622: 953, // TransactionChars (3x) + 58487: 941, // RequireClause (3x) + 58488: 942, // RequireClauseOpt (3x) + 58490: 943, // RequireListElement (3x) + 58504: 944, // RolenameWithoutIdent (3x) + 58497: 945, // RoleOrPrivElem (3x) + 58516: 946, // SelectStmtGroup (3x) + 58533: 947, // SetOprOpt (3x) + 58584: 948, // TableAliasRefList (3x) + 58587: 949, // TableElement (3x) + 58596: 950, // TableNameListOpt2 (3x) + 58612: 951, // TextString (3x) + 58619: 952, // TraceableStmt (3x) + 58621: 953, // TransactionChars (3x) 57544: 954, // trigger (3x) 57548: 955, // unlock (3x) 57551: 956, // usage (3x) - 58642: 957, // ValuesList (3x) - 58644: 958, // ValuesStmtList (3x) - 58640: 959, // ValueSym (3x) - 58647: 960, // VariableAssignment (3x) - 58667: 961, // WindowFrameStart (3x) - 58103: 962, // AdminStmt (2x) - 58105: 963, // AllColumnsOrPredicateColumnsOpt (2x) - 58107: 964, // AlterDatabaseStmt (2x) - 58108: 965, // AlterImportStmt (2x) - 58109: 966, // AlterInstanceStmt (2x) - 58110: 967, // AlterOrderItem (2x) - 58112: 968, // AlterPolicyStmt (2x) - 58113: 969, // AlterSequenceOption (2x) - 58115: 970, // AlterSequenceStmt (2x) - 58117: 971, // AlterTableSpec (2x) - 58121: 972, // AlterUserStmt (2x) - 58122: 973, // AnalyzeOption (2x) - 58125: 974, // AnalyzeTableStmt (2x) - 58148: 975, // BinlogStmt (2x) - 58142: 976, // BRIEStmt (2x) - 58144: 977, // BRIETables (2x) + 58641: 957, // ValuesList (3x) + 58643: 958, // ValuesStmtList (3x) + 58639: 959, // ValueSym (3x) + 58646: 960, // VariableAssignment (3x) + 58666: 961, // WindowFrameStart (3x) + 58102: 962, // AdminStmt (2x) + 58104: 963, // AllColumnsOrPredicateColumnsOpt (2x) + 58106: 964, // AlterDatabaseStmt (2x) + 58107: 965, // AlterImportStmt (2x) + 58108: 966, // AlterInstanceStmt (2x) + 58109: 967, // AlterOrderItem (2x) + 58111: 968, // AlterPolicyStmt (2x) + 58112: 969, // AlterSequenceOption (2x) + 58114: 970, // AlterSequenceStmt (2x) + 58116: 971, // AlterTableSpec (2x) + 58120: 972, // AlterUserStmt (2x) + 58121: 973, // AnalyzeOption (2x) + 58124: 974, // AnalyzeTableStmt (2x) + 58147: 975, // BinlogStmt (2x) + 58141: 976, // BRIEStmt (2x) + 58143: 977, // BRIETables (2x) 57372: 978, // call (2x) - 58158: 979, // CallStmt (2x) - 58159: 980, // CastType (2x) - 58160: 981, // ChangeStmt (2x) - 58166: 982, // CheckConstraintKeyword (2x) - 58176: 983, // ColumnNameListOpt (2x) - 58179: 984, // ColumnNameOrUserVariable (2x) - 58182: 985, // ColumnOptionList (2x) - 58183: 986, // ColumnOptionListOpt (2x) - 58185: 987, // ColumnSetValue (2x) - 58191: 988, // CompletionTypeWithinTransaction (2x) - 58193: 989, // ConnectionOption (2x) - 58195: 990, // ConnectionOptions (2x) - 58199: 991, // CreateBindingStmt (2x) - 58200: 992, // CreateDatabaseStmt (2x) - 58201: 993, // CreateImportStmt (2x) - 58202: 994, // CreateIndexStmt (2x) - 58203: 995, // CreatePolicyStmt (2x) - 58204: 996, // CreateRoleStmt (2x) - 58206: 997, // CreateSequenceStmt (2x) - 58207: 998, // CreateStatisticsStmt (2x) - 58208: 999, // CreateTableOptionListOpt (2x) - 58211: 1000, // CreateUserStmt (2x) - 58213: 1001, // CreateViewStmt (2x) + 58157: 979, // CallStmt (2x) + 58158: 980, // CastType (2x) + 58159: 981, // ChangeStmt (2x) + 58165: 982, // CheckConstraintKeyword (2x) + 58175: 983, // ColumnNameListOpt (2x) + 58178: 984, // ColumnNameOrUserVariable (2x) + 58181: 985, // ColumnOptionList (2x) + 58182: 986, // ColumnOptionListOpt (2x) + 58184: 987, // ColumnSetValue (2x) + 58190: 988, // CompletionTypeWithinTransaction (2x) + 58192: 989, // ConnectionOption (2x) + 58194: 990, // ConnectionOptions (2x) + 58198: 991, // CreateBindingStmt (2x) + 58199: 992, // CreateDatabaseStmt (2x) + 58200: 993, // CreateImportStmt (2x) + 58201: 994, // CreateIndexStmt (2x) + 58202: 995, // CreatePolicyStmt (2x) + 58203: 996, // CreateRoleStmt (2x) + 58205: 997, // CreateSequenceStmt (2x) + 58206: 998, // CreateStatisticsStmt (2x) + 58207: 999, // CreateTableOptionListOpt (2x) + 58210: 1000, // CreateUserStmt (2x) + 58212: 1001, // CreateViewStmt (2x) 57392: 1002, // databases (2x) - 58222: 1003, // DeallocateStmt (2x) - 58223: 1004, // DeallocateSym (2x) + 58221: 1003, // DeallocateStmt (2x) + 58222: 1004, // DeallocateSym (2x) 57403: 1005, // describe (2x) - 58234: 1006, // DoStmt (2x) - 58235: 1007, // DropBindingStmt (2x) - 58236: 1008, // DropDatabaseStmt (2x) - 58237: 1009, // DropImportStmt (2x) - 58238: 1010, // DropIndexStmt (2x) - 58239: 1011, // DropPolicyStmt (2x) - 58240: 1012, // DropRoleStmt (2x) - 58241: 1013, // DropSequenceStmt (2x) - 58242: 1014, // DropStatisticsStmt (2x) - 58243: 1015, // DropStatsStmt (2x) - 58244: 1016, // DropTableStmt (2x) - 58245: 1017, // DropUserStmt (2x) - 58246: 1018, // DropViewStmt (2x) - 58247: 1019, // DuplicateOpt (2x) - 58249: 1020, // EmptyStmt (2x) - 58250: 1021, // EncryptionOpt (2x) - 58252: 1022, // EnforcedOrNotOpt (2x) - 58256: 1023, // ErrorHandling (2x) - 58258: 1024, // ExecuteStmt (2x) - 58260: 1025, // ExplainStmt (2x) - 58261: 1026, // ExplainSym (2x) - 58270: 1027, // Field (2x) - 58273: 1028, // FieldItem (2x) - 58280: 1029, // Fields (2x) - 58284: 1030, // FlashbackTableStmt (2x) - 58289: 1031, // FlushStmt (2x) - 58295: 1032, // FuncDatetimePrecList (2x) - 58296: 1033, // FuncDatetimePrecListOpt (2x) - 58309: 1034, // GrantProxyStmt (2x) - 58310: 1035, // GrantRoleStmt (2x) - 58311: 1036, // GrantStmt (2x) - 58313: 1037, // HandleRange (2x) - 58315: 1038, // HashString (2x) - 58317: 1039, // HelpStmt (2x) - 58329: 1040, // IndexAdviseStmt (2x) - 58331: 1041, // IndexHintList (2x) - 58332: 1042, // IndexHintListOpt (2x) - 58337: 1043, // IndexLockAndAlgorithmOpt (2x) - 58350: 1044, // InsertValues (2x) - 58354: 1045, // IntoOpt (2x) - 58360: 1046, // KeyOrIndexOpt (2x) + 58233: 1006, // DoStmt (2x) + 58234: 1007, // DropBindingStmt (2x) + 58235: 1008, // DropDatabaseStmt (2x) + 58236: 1009, // DropImportStmt (2x) + 58237: 1010, // DropIndexStmt (2x) + 58238: 1011, // DropPolicyStmt (2x) + 58239: 1012, // DropRoleStmt (2x) + 58240: 1013, // DropSequenceStmt (2x) + 58241: 1014, // DropStatisticsStmt (2x) + 58242: 1015, // DropStatsStmt (2x) + 58243: 1016, // DropTableStmt (2x) + 58244: 1017, // DropUserStmt (2x) + 58245: 1018, // DropViewStmt (2x) + 58246: 1019, // DuplicateOpt (2x) + 58248: 1020, // EmptyStmt (2x) + 58249: 1021, // EncryptionOpt (2x) + 58251: 1022, // EnforcedOrNotOpt (2x) + 58255: 1023, // ErrorHandling (2x) + 58257: 1024, // ExecuteStmt (2x) + 58259: 1025, // ExplainStmt (2x) + 58260: 1026, // ExplainSym (2x) + 58269: 1027, // Field (2x) + 58272: 1028, // FieldItem (2x) + 58279: 1029, // Fields (2x) + 58283: 1030, // FlashbackTableStmt (2x) + 58288: 1031, // FlushStmt (2x) + 58294: 1032, // FuncDatetimePrecList (2x) + 58295: 1033, // FuncDatetimePrecListOpt (2x) + 58308: 1034, // GrantProxyStmt (2x) + 58309: 1035, // GrantRoleStmt (2x) + 58310: 1036, // GrantStmt (2x) + 58312: 1037, // HandleRange (2x) + 58314: 1038, // HashString (2x) + 58316: 1039, // HelpStmt (2x) + 58328: 1040, // IndexAdviseStmt (2x) + 58330: 1041, // IndexHintList (2x) + 58331: 1042, // IndexHintListOpt (2x) + 58336: 1043, // IndexLockAndAlgorithmOpt (2x) + 58349: 1044, // InsertValues (2x) + 58353: 1045, // IntoOpt (2x) + 58359: 1046, // KeyOrIndexOpt (2x) 57456: 1047, // kill (2x) - 58361: 1048, // KillOrKillTiDB (2x) - 58362: 1049, // KillStmt (2x) - 58367: 1050, // LimitClause (2x) + 58360: 1048, // KillOrKillTiDB (2x) + 58361: 1049, // KillStmt (2x) + 58366: 1050, // LimitClause (2x) 57465: 1051, // linear (2x) - 58369: 1052, // LinearOpt (2x) - 58373: 1053, // LoadDataSetItem (2x) - 58377: 1054, // LoadStatsStmt (2x) - 58378: 1055, // LocalOpt (2x) - 58381: 1056, // LockTablesStmt (2x) - 58389: 1057, // MaxValueOrExpressionList (2x) - 58397: 1058, // NowSym (2x) - 58398: 1059, // NowSymFunc (2x) - 58399: 1060, // NowSymOptionFraction (2x) - 58400: 1061, // NumList (2x) - 58403: 1062, // ObjectType (2x) + 58368: 1052, // LinearOpt (2x) + 58372: 1053, // LoadDataSetItem (2x) + 58376: 1054, // LoadStatsStmt (2x) + 58377: 1055, // LocalOpt (2x) + 58380: 1056, // LockTablesStmt (2x) + 58388: 1057, // MaxValueOrExpressionList (2x) + 58396: 1058, // NowSym (2x) + 58397: 1059, // NowSymFunc (2x) + 58398: 1060, // NowSymOptionFraction (2x) + 58399: 1061, // NumList (2x) + 58402: 1062, // ObjectType (2x) 57487: 1063, // of (2x) - 58404: 1064, // OfTablesOpt (2x) - 58405: 1065, // OnCommitOpt (2x) - 58406: 1066, // OnDelete (2x) - 58409: 1067, // OnUpdate (2x) - 58414: 1068, // OptCollate (2x) - 58419: 1069, // OptFull (2x) - 58421: 1070, // OptInteger (2x) - 58434: 1071, // OptionalBraces (2x) - 58433: 1072, // OptionLevel (2x) - 58423: 1073, // OptLeadLagInfo (2x) - 58422: 1074, // OptLLDefault (2x) - 58439: 1075, // OuterOpt (2x) - 58444: 1076, // PartitionDefinitionList (2x) - 58445: 1077, // PartitionDefinitionListOpt (2x) - 58451: 1078, // PartitionOpt (2x) - 58453: 1079, // PasswordOpt (2x) - 58455: 1080, // PasswordOrLockOptionList (2x) - 58456: 1081, // PasswordOrLockOptions (2x) - 58460: 1082, // PlacementOptionList (2x) - 58462: 1083, // PlanReplayerStmt (2x) - 58468: 1084, // PreparedStmt (2x) - 58473: 1085, // PrivLevel (2x) - 58476: 1086, // PurgeImportStmt (2x) - 58477: 1087, // QuickOptional (2x) - 58478: 1088, // RecoverTableStmt (2x) - 58480: 1089, // ReferOpt (2x) - 58482: 1090, // RegexpSym (2x) - 58483: 1091, // RenameTableStmt (2x) - 58484: 1092, // RenameUserStmt (2x) - 58486: 1093, // RepeatableOpt (2x) - 58492: 1094, // RestartStmt (2x) - 58494: 1095, // ResumeImportStmt (2x) + 58403: 1064, // OfTablesOpt (2x) + 58404: 1065, // OnCommitOpt (2x) + 58405: 1066, // OnDelete (2x) + 58408: 1067, // OnUpdate (2x) + 58413: 1068, // OptCollate (2x) + 58418: 1069, // OptFull (2x) + 58420: 1070, // OptInteger (2x) + 58433: 1071, // OptionalBraces (2x) + 58432: 1072, // OptionLevel (2x) + 58422: 1073, // OptLeadLagInfo (2x) + 58421: 1074, // OptLLDefault (2x) + 58438: 1075, // OuterOpt (2x) + 58443: 1076, // PartitionDefinitionList (2x) + 58444: 1077, // PartitionDefinitionListOpt (2x) + 58450: 1078, // PartitionOpt (2x) + 58452: 1079, // PasswordOpt (2x) + 58454: 1080, // PasswordOrLockOptionList (2x) + 58455: 1081, // PasswordOrLockOptions (2x) + 58459: 1082, // PlacementOptionList (2x) + 58461: 1083, // PlanReplayerStmt (2x) + 58467: 1084, // PreparedStmt (2x) + 58472: 1085, // PrivLevel (2x) + 58475: 1086, // PurgeImportStmt (2x) + 58476: 1087, // QuickOptional (2x) + 58477: 1088, // RecoverTableStmt (2x) + 58479: 1089, // ReferOpt (2x) + 58481: 1090, // RegexpSym (2x) + 58482: 1091, // RenameTableStmt (2x) + 58483: 1092, // RenameUserStmt (2x) + 58485: 1093, // RepeatableOpt (2x) + 58491: 1094, // RestartStmt (2x) + 58493: 1095, // ResumeImportStmt (2x) 57514: 1096, // revoke (2x) - 58495: 1097, // RevokeRoleStmt (2x) - 58496: 1098, // RevokeStmt (2x) - 58499: 1099, // RoleOrPrivElemList (2x) - 58500: 1100, // RoleSpec (2x) - 58521: 1101, // SelectStmtOpt (2x) - 58524: 1102, // SelectStmtSQLCache (2x) - 58528: 1103, // SetDefaultRoleOpt (2x) - 58529: 1104, // SetDefaultRoleStmt (2x) - 58539: 1105, // SetRoleStmt (2x) - 58542: 1106, // ShowImportStmt (2x) - 58547: 1107, // ShowProfileType (2x) - 58550: 1108, // ShowStmt (2x) - 58551: 1109, // ShowTableAliasOpt (2x) - 58553: 1110, // ShutdownStmt (2x) - 58554: 1111, // SignedLiteral (2x) - 58558: 1112, // SplitOption (2x) - 58559: 1113, // SplitRegionStmt (2x) - 58563: 1114, // Statement (2x) - 58565: 1115, // StatsOptionsOpt (2x) - 58566: 1116, // StatsPersistentVal (2x) - 58567: 1117, // StatsType (2x) - 58568: 1118, // StopImportStmt (2x) - 58575: 1119, // SubPartDefinition (2x) - 58578: 1120, // SubPartitionMethod (2x) - 58583: 1121, // Symbol (2x) - 58589: 1122, // TableElementList (2x) - 58592: 1123, // TableLock (2x) - 58596: 1124, // TableNameListOpt (2x) - 58603: 1125, // TableOrTables (2x) - 58612: 1126, // TablesTerminalSym (2x) - 58610: 1127, // TableToTable (2x) - 58614: 1128, // TextStringList (2x) - 58619: 1129, // TraceStmt (2x) - 58624: 1130, // TruncateTableStmt (2x) - 58627: 1131, // UnlockTablesStmt (2x) - 58633: 1132, // UserToUser (2x) - 58630: 1133, // UseStmt (2x) - 58645: 1134, // Varchar (2x) - 58648: 1135, // VariableAssignmentList (2x) - 58657: 1136, // WhenClause (2x) - 58662: 1137, // WindowDefinition (2x) - 58665: 1138, // WindowFrameBound (2x) - 58672: 1139, // WindowSpec (2x) - 58677: 1140, // WithGrantOptionOpt (2x) - 58678: 1141, // WithList (2x) - 58682: 1142, // Writeable (2x) - 58102: 1143, // AdminShowSlow (1x) - 58111: 1144, // AlterOrderList (1x) - 58114: 1145, // AlterSequenceOptionList (1x) - 58116: 1146, // AlterTablePartitionOpt (1x) - 58118: 1147, // AlterTableSpecList (1x) - 58119: 1148, // AlterTableSpecListOpt (1x) - 58123: 1149, // AnalyzeOptionList (1x) - 58126: 1150, // AnyOrAll (1x) - 58128: 1151, // AsOfClauseOpt (1x) - 58129: 1152, // AsOpt (1x) - 58134: 1153, // AuthOption (1x) - 58135: 1154, // AuthPlugin (1x) - 58146: 1155, // BetweenOrNotOp (1x) - 58150: 1156, // BitValueType (1x) - 58151: 1157, // BlobType (1x) - 58154: 1158, // BooleanType (1x) + 58494: 1097, // RevokeRoleStmt (2x) + 58495: 1098, // RevokeStmt (2x) + 58498: 1099, // RoleOrPrivElemList (2x) + 58499: 1100, // RoleSpec (2x) + 58520: 1101, // SelectStmtOpt (2x) + 58523: 1102, // SelectStmtSQLCache (2x) + 58527: 1103, // SetDefaultRoleOpt (2x) + 58528: 1104, // SetDefaultRoleStmt (2x) + 58538: 1105, // SetRoleStmt (2x) + 58541: 1106, // ShowImportStmt (2x) + 58546: 1107, // ShowProfileType (2x) + 58549: 1108, // ShowStmt (2x) + 58550: 1109, // ShowTableAliasOpt (2x) + 58552: 1110, // ShutdownStmt (2x) + 58553: 1111, // SignedLiteral (2x) + 58557: 1112, // SplitOption (2x) + 58558: 1113, // SplitRegionStmt (2x) + 58562: 1114, // Statement (2x) + 58564: 1115, // StatsOptionsOpt (2x) + 58565: 1116, // StatsPersistentVal (2x) + 58566: 1117, // StatsType (2x) + 58567: 1118, // StopImportStmt (2x) + 58574: 1119, // SubPartDefinition (2x) + 58577: 1120, // SubPartitionMethod (2x) + 58582: 1121, // Symbol (2x) + 58588: 1122, // TableElementList (2x) + 58591: 1123, // TableLock (2x) + 58595: 1124, // TableNameListOpt (2x) + 58602: 1125, // TableOrTables (2x) + 58611: 1126, // TablesTerminalSym (2x) + 58609: 1127, // TableToTable (2x) + 58613: 1128, // TextStringList (2x) + 58618: 1129, // TraceStmt (2x) + 58623: 1130, // TruncateTableStmt (2x) + 58626: 1131, // UnlockTablesStmt (2x) + 58632: 1132, // UserToUser (2x) + 58629: 1133, // UseStmt (2x) + 58644: 1134, // Varchar (2x) + 58647: 1135, // VariableAssignmentList (2x) + 58656: 1136, // WhenClause (2x) + 58661: 1137, // WindowDefinition (2x) + 58664: 1138, // WindowFrameBound (2x) + 58671: 1139, // WindowSpec (2x) + 58676: 1140, // WithGrantOptionOpt (2x) + 58677: 1141, // WithList (2x) + 58681: 1142, // Writeable (2x) + 58101: 1143, // AdminShowSlow (1x) + 58110: 1144, // AlterOrderList (1x) + 58113: 1145, // AlterSequenceOptionList (1x) + 58115: 1146, // AlterTablePartitionOpt (1x) + 58117: 1147, // AlterTableSpecList (1x) + 58118: 1148, // AlterTableSpecListOpt (1x) + 58122: 1149, // AnalyzeOptionList (1x) + 58125: 1150, // AnyOrAll (1x) + 58127: 1151, // AsOfClauseOpt (1x) + 58128: 1152, // AsOpt (1x) + 58133: 1153, // AuthOption (1x) + 58134: 1154, // AuthPlugin (1x) + 58145: 1155, // BetweenOrNotOp (1x) + 58149: 1156, // BitValueType (1x) + 58150: 1157, // BlobType (1x) + 58153: 1158, // BooleanType (1x) 57370: 1159, // both (1x) - 58164: 1160, // CharsetNameOrDefault (1x) - 58165: 1161, // CharsetOpt (1x) - 58167: 1162, // ClearPasswordExpireOptions (1x) - 58171: 1163, // ColumnFormat (1x) - 58173: 1164, // ColumnList (1x) - 58180: 1165, // ColumnNameOrUserVariableList (1x) - 58177: 1166, // ColumnNameOrUserVarListOpt (1x) - 58178: 1167, // ColumnNameOrUserVarListOptWithBrackets (1x) - 58186: 1168, // ColumnSetValueList (1x) - 58190: 1169, // CompareOp (1x) - 58194: 1170, // ConnectionOptionList (1x) - 58197: 1171, // ConstraintElem (1x) - 58205: 1172, // CreateSequenceOptionListOpt (1x) - 58209: 1173, // CreateTableSelectOpt (1x) - 58212: 1174, // CreateViewSelectOpt (1x) - 58219: 1175, // DatabaseOptionListOpt (1x) - 58221: 1176, // DateAndTimeType (1x) - 58216: 1177, // DBNameList (1x) - 58227: 1178, // DefaultValueExpr (1x) + 58163: 1160, // CharsetNameOrDefault (1x) + 58164: 1161, // CharsetOpt (1x) + 58166: 1162, // ClearPasswordExpireOptions (1x) + 58170: 1163, // ColumnFormat (1x) + 58172: 1164, // ColumnList (1x) + 58179: 1165, // ColumnNameOrUserVariableList (1x) + 58176: 1166, // ColumnNameOrUserVarListOpt (1x) + 58177: 1167, // ColumnNameOrUserVarListOptWithBrackets (1x) + 58185: 1168, // ColumnSetValueList (1x) + 58189: 1169, // CompareOp (1x) + 58193: 1170, // ConnectionOptionList (1x) + 58196: 1171, // ConstraintElem (1x) + 58204: 1172, // CreateSequenceOptionListOpt (1x) + 58208: 1173, // CreateTableSelectOpt (1x) + 58211: 1174, // CreateViewSelectOpt (1x) + 58218: 1175, // DatabaseOptionListOpt (1x) + 58220: 1176, // DateAndTimeType (1x) + 58215: 1177, // DBNameList (1x) + 58226: 1178, // DefaultValueExpr (1x) 57409: 1179, // dual (1x) - 58248: 1180, // ElseOpt (1x) - 58253: 1181, // EnforcedOrNotOrNotNullOpt (1x) - 58259: 1182, // ExplainFormatType (1x) - 58267: 1183, // ExpressionOpt (1x) - 58269: 1184, // FetchFirstOpt (1x) - 58271: 1185, // FieldAsName (1x) - 58272: 1186, // FieldAsNameOpt (1x) - 58274: 1187, // FieldItemList (1x) - 58276: 1188, // FieldList (1x) - 58282: 1189, // FirstOrNext (1x) - 58283: 1190, // FixedPointType (1x) - 58285: 1191, // FlashbackToNewName (1x) - 58287: 1192, // FloatingPointType (1x) - 58288: 1193, // FlushOption (1x) - 58291: 1194, // FromDual (1x) - 58293: 1195, // FulltextSearchModifierOpt (1x) - 58294: 1196, // FuncDatetimePrec (1x) - 58307: 1197, // GetFormatSelector (1x) - 58314: 1198, // HandleRangeList (1x) - 58316: 1199, // HavingClause (1x) - 58319: 1200, // IdentListWithParenOpt (1x) - 58323: 1201, // IfNotRunning (1x) - 58324: 1202, // IfRunning (1x) - 58325: 1203, // IgnoreLines (1x) - 58327: 1204, // ImportTruncate (1x) - 58333: 1205, // IndexHintScope (1x) - 58336: 1206, // IndexKeyTypeOpt (1x) - 58345: 1207, // IndexPartSpecificationListOpt (1x) - 58348: 1208, // IndexTypeOpt (1x) - 58328: 1209, // InOrNotOp (1x) - 58351: 1210, // InstanceOption (1x) - 58353: 1211, // IntegerType (1x) - 58356: 1212, // IsolationLevel (1x) - 58355: 1213, // IsOrNotOp (1x) + 58247: 1180, // ElseOpt (1x) + 58252: 1181, // EnforcedOrNotOrNotNullOpt (1x) + 58258: 1182, // ExplainFormatType (1x) + 58266: 1183, // ExpressionOpt (1x) + 58268: 1184, // FetchFirstOpt (1x) + 58270: 1185, // FieldAsName (1x) + 58271: 1186, // FieldAsNameOpt (1x) + 58273: 1187, // FieldItemList (1x) + 58275: 1188, // FieldList (1x) + 58281: 1189, // FirstOrNext (1x) + 58282: 1190, // FixedPointType (1x) + 58284: 1191, // FlashbackToNewName (1x) + 58286: 1192, // FloatingPointType (1x) + 58287: 1193, // FlushOption (1x) + 58290: 1194, // FromDual (1x) + 58292: 1195, // FulltextSearchModifierOpt (1x) + 58293: 1196, // FuncDatetimePrec (1x) + 58306: 1197, // GetFormatSelector (1x) + 58313: 1198, // HandleRangeList (1x) + 58315: 1199, // HavingClause (1x) + 58318: 1200, // IdentListWithParenOpt (1x) + 58322: 1201, // IfNotRunning (1x) + 58323: 1202, // IfRunning (1x) + 58324: 1203, // IgnoreLines (1x) + 58326: 1204, // ImportTruncate (1x) + 58332: 1205, // IndexHintScope (1x) + 58335: 1206, // IndexKeyTypeOpt (1x) + 58344: 1207, // IndexPartSpecificationListOpt (1x) + 58347: 1208, // IndexTypeOpt (1x) + 58327: 1209, // InOrNotOp (1x) + 58350: 1210, // InstanceOption (1x) + 58352: 1211, // IntegerType (1x) + 58355: 1212, // IsolationLevel (1x) + 58354: 1213, // IsOrNotOp (1x) 57460: 1214, // leading (1x) - 58364: 1215, // LikeEscapeOpt (1x) - 58365: 1216, // LikeOrNotOp (1x) - 58366: 1217, // LikeTableWithOrWithoutParen (1x) - 58371: 1218, // LinesTerminated (1x) - 58374: 1219, // LoadDataSetList (1x) - 58375: 1220, // LoadDataSetSpecOpt (1x) - 58379: 1221, // LocationLabelList (1x) - 58382: 1222, // LockType (1x) - 58383: 1223, // LogTypeOpt (1x) - 58384: 1224, // Match (1x) - 58385: 1225, // MatchOpt (1x) - 58386: 1226, // MaxIndexNumOpt (1x) - 58387: 1227, // MaxMinutesOpt (1x) - 58390: 1228, // NChar (1x) - 58402: 1229, // NumericType (1x) - 58392: 1230, // NVarchar (1x) - 58407: 1231, // OnDeleteUpdateOpt (1x) - 58408: 1232, // OnDuplicateKeyUpdate (1x) - 58410: 1233, // OptBinMod (1x) - 58412: 1234, // OptCharset (1x) - 58415: 1235, // OptErrors (1x) - 58416: 1236, // OptExistingWindowName (1x) - 58418: 1237, // OptFromFirstLast (1x) - 58420: 1238, // OptGConcatSeparator (1x) - 58426: 1239, // OptPartitionClause (1x) - 58427: 1240, // OptTable (1x) - 58430: 1241, // OptWindowFrameClause (1x) - 58431: 1242, // OptWindowOrderByClause (1x) - 58436: 1243, // Order (1x) - 58435: 1244, // OrReplace (1x) + 58363: 1215, // LikeEscapeOpt (1x) + 58364: 1216, // LikeOrNotOp (1x) + 58365: 1217, // LikeTableWithOrWithoutParen (1x) + 58370: 1218, // LinesTerminated (1x) + 58373: 1219, // LoadDataSetList (1x) + 58374: 1220, // LoadDataSetSpecOpt (1x) + 58378: 1221, // LocationLabelList (1x) + 58381: 1222, // LockType (1x) + 58382: 1223, // LogTypeOpt (1x) + 58383: 1224, // Match (1x) + 58384: 1225, // MatchOpt (1x) + 58385: 1226, // MaxIndexNumOpt (1x) + 58386: 1227, // MaxMinutesOpt (1x) + 58389: 1228, // NChar (1x) + 58401: 1229, // NumericType (1x) + 58391: 1230, // NVarchar (1x) + 58406: 1231, // OnDeleteUpdateOpt (1x) + 58407: 1232, // OnDuplicateKeyUpdate (1x) + 58409: 1233, // OptBinMod (1x) + 58411: 1234, // OptCharset (1x) + 58414: 1235, // OptErrors (1x) + 58415: 1236, // OptExistingWindowName (1x) + 58417: 1237, // OptFromFirstLast (1x) + 58419: 1238, // OptGConcatSeparator (1x) + 58425: 1239, // OptPartitionClause (1x) + 58426: 1240, // OptTable (1x) + 58429: 1241, // OptWindowFrameClause (1x) + 58430: 1242, // OptWindowOrderByClause (1x) + 58435: 1243, // Order (1x) + 58434: 1244, // OrReplace (1x) 57444: 1245, // outfile (1x) - 58442: 1246, // PartDefValuesOpt (1x) - 58446: 1247, // PartitionKeyAlgorithmOpt (1x) - 58447: 1248, // PartitionMethod (1x) - 58450: 1249, // PartitionNumOpt (1x) - 58457: 1250, // PerDB (1x) - 58458: 1251, // PerTable (1x) + 58441: 1246, // PartDefValuesOpt (1x) + 58445: 1247, // PartitionKeyAlgorithmOpt (1x) + 58446: 1248, // PartitionMethod (1x) + 58449: 1249, // PartitionNumOpt (1x) + 58456: 1250, // PerDB (1x) + 58457: 1251, // PerTable (1x) 57498: 1252, // precisionType (1x) - 58467: 1253, // PrepareSQL (1x) - 58475: 1254, // ProcedureCall (1x) + 58466: 1253, // PrepareSQL (1x) + 58474: 1254, // ProcedureCall (1x) 57505: 1255, // recursive (1x) - 58481: 1256, // RegexpOrNotOp (1x) - 58485: 1257, // ReorganizePartitionRuleOpt (1x) - 58490: 1258, // RequireList (1x) - 58501: 1259, // RoleSpecList (1x) - 58508: 1260, // RowOrRows (1x) - 58514: 1261, // SelectStmtFieldList (1x) - 58522: 1262, // SelectStmtOpts (1x) - 58523: 1263, // SelectStmtOptsList (1x) - 58527: 1264, // SequenceOptionList (1x) - 58531: 1265, // SetOpr (1x) - 58538: 1266, // SetRoleOpt (1x) - 58543: 1267, // ShowIndexKwd (1x) - 58544: 1268, // ShowLikeOrWhereOpt (1x) - 58545: 1269, // ShowPlacementTarget (1x) - 58546: 1270, // ShowProfileArgsOpt (1x) - 58548: 1271, // ShowProfileTypes (1x) - 58549: 1272, // ShowProfileTypesOpt (1x) - 58552: 1273, // ShowTargetFilterable (1x) + 58480: 1256, // RegexpOrNotOp (1x) + 58484: 1257, // ReorganizePartitionRuleOpt (1x) + 58489: 1258, // RequireList (1x) + 58500: 1259, // RoleSpecList (1x) + 58507: 1260, // RowOrRows (1x) + 58513: 1261, // SelectStmtFieldList (1x) + 58521: 1262, // SelectStmtOpts (1x) + 58522: 1263, // SelectStmtOptsList (1x) + 58526: 1264, // SequenceOptionList (1x) + 58530: 1265, // SetOpr (1x) + 58537: 1266, // SetRoleOpt (1x) + 58542: 1267, // ShowIndexKwd (1x) + 58543: 1268, // ShowLikeOrWhereOpt (1x) + 58544: 1269, // ShowPlacementTarget (1x) + 58545: 1270, // ShowProfileArgsOpt (1x) + 58547: 1271, // ShowProfileTypes (1x) + 58548: 1272, // ShowProfileTypesOpt (1x) + 58551: 1273, // ShowTargetFilterable (1x) 57525: 1274, // spatial (1x) - 58560: 1275, // SplitSyntaxOption (1x) + 58559: 1275, // SplitSyntaxOption (1x) 57530: 1276, // ssl (1x) - 58561: 1277, // Start (1x) - 58562: 1278, // Starting (1x) + 58560: 1277, // Start (1x) + 58561: 1278, // Starting (1x) 57531: 1279, // starting (1x) - 58564: 1280, // StatementList (1x) - 58569: 1281, // StorageMedia (1x) + 58563: 1280, // StatementList (1x) + 58568: 1281, // StorageMedia (1x) 57536: 1282, // stored (1x) - 58570: 1283, // StringList (1x) - 58573: 1284, // StringNameOrBRIEOptionKeyword (1x) - 58574: 1285, // StringType (1x) - 58576: 1286, // SubPartDefinitionList (1x) - 58577: 1287, // SubPartDefinitionListOpt (1x) - 58579: 1288, // SubPartitionNumOpt (1x) - 58580: 1289, // SubPartitionOpt (1x) - 58590: 1290, // TableElementListOpt (1x) - 58593: 1291, // TableLockList (1x) - 58606: 1292, // TableRefsClause (1x) - 58607: 1293, // TableSampleMethodOpt (1x) - 58608: 1294, // TableSampleOpt (1x) - 58609: 1295, // TableSampleUnitOpt (1x) - 58611: 1296, // TableToTableList (1x) - 58615: 1297, // TextType (1x) + 58569: 1283, // StringList (1x) + 58572: 1284, // StringNameOrBRIEOptionKeyword (1x) + 58573: 1285, // StringType (1x) + 58575: 1286, // SubPartDefinitionList (1x) + 58576: 1287, // SubPartDefinitionListOpt (1x) + 58578: 1288, // SubPartitionNumOpt (1x) + 58579: 1289, // SubPartitionOpt (1x) + 58589: 1290, // TableElementListOpt (1x) + 58592: 1291, // TableLockList (1x) + 58605: 1292, // TableRefsClause (1x) + 58606: 1293, // TableSampleMethodOpt (1x) + 58607: 1294, // TableSampleOpt (1x) + 58608: 1295, // TableSampleUnitOpt (1x) + 58610: 1296, // TableToTableList (1x) + 58614: 1297, // TextType (1x) 57543: 1298, // trailing (1x) - 58623: 1299, // TrimDirection (1x) - 58625: 1300, // Type (1x) - 58634: 1301, // UserToUserList (1x) - 58636: 1302, // UserVariableList (1x) - 58639: 1303, // UsingRoles (1x) - 58641: 1304, // Values (1x) - 58643: 1305, // ValuesOpt (1x) - 58650: 1306, // ViewAlgorithm (1x) - 58651: 1307, // ViewCheckOption (1x) - 58652: 1308, // ViewDefiner (1x) - 58653: 1309, // ViewFieldList (1x) - 58654: 1310, // ViewName (1x) - 58655: 1311, // ViewSQLSecurity (1x) + 58622: 1299, // TrimDirection (1x) + 58624: 1300, // Type (1x) + 58633: 1301, // UserToUserList (1x) + 58635: 1302, // UserVariableList (1x) + 58638: 1303, // UsingRoles (1x) + 58640: 1304, // Values (1x) + 58642: 1305, // ValuesOpt (1x) + 58649: 1306, // ViewAlgorithm (1x) + 58650: 1307, // ViewCheckOption (1x) + 58651: 1308, // ViewDefiner (1x) + 58652: 1309, // ViewFieldList (1x) + 58653: 1310, // ViewName (1x) + 58654: 1311, // ViewSQLSecurity (1x) 57563: 1312, // virtual (1x) - 58656: 1313, // VirtualOrStored (1x) - 58658: 1314, // WhenClauseList (1x) - 58661: 1315, // WindowClauseOptional (1x) - 58663: 1316, // WindowDefinitionList (1x) - 58664: 1317, // WindowFrameBetween (1x) - 58666: 1318, // WindowFrameExtent (1x) - 58668: 1319, // WindowFrameUnits (1x) - 58671: 1320, // WindowNameOrSpec (1x) - 58673: 1321, // WindowSpecDetails (1x) - 58679: 1322, // WithReadLockOpt (1x) - 58680: 1323, // WithValidation (1x) - 58681: 1324, // WithValidationOpt (1x) - 58683: 1325, // Year (1x) - 58101: 1326, // $default (0x) + 58655: 1313, // VirtualOrStored (1x) + 58657: 1314, // WhenClauseList (1x) + 58660: 1315, // WindowClauseOptional (1x) + 58662: 1316, // WindowDefinitionList (1x) + 58663: 1317, // WindowFrameBetween (1x) + 58665: 1318, // WindowFrameExtent (1x) + 58667: 1319, // WindowFrameUnits (1x) + 58670: 1320, // WindowNameOrSpec (1x) + 58672: 1321, // WindowSpecDetails (1x) + 58678: 1322, // WithReadLockOpt (1x) + 58679: 1323, // WithValidation (1x) + 58680: 1324, // WithValidationOpt (1x) + 58682: 1325, // Year (1x) + 58100: 1326, // $default (0x) 58061: 1327, // andnot (0x) - 58132: 1328, // AssignmentListOpt (0x) - 58170: 1329, // ColumnDefList (0x) - 58187: 1330, // CommaOpt (0x) - 58085: 1331, // createTableSelect (0x) + 58131: 1328, // AssignmentListOpt (0x) + 58169: 1329, // ColumnDefList (0x) + 58186: 1330, // CommaOpt (0x) + 58084: 1331, // createTableSelect (0x) 58075: 1332, // empty (0x) 57345: 1333, // error (0x) - 58100: 1334, // higherThanComma (0x) - 58094: 1335, // higherThanParenthese (0x) - 58083: 1336, // insertValues (0x) + 58099: 1334, // higherThanComma (0x) + 58093: 1335, // higherThanParenthese (0x) + 58082: 1336, // insertValues (0x) 57352: 1337, // invalid (0x) - 58086: 1338, // lowerThanCharsetKwd (0x) - 58099: 1339, // lowerThanComma (0x) - 58084: 1340, // lowerThanCreateTableSelect (0x) - 58096: 1341, // lowerThanEq (0x) - 58091: 1342, // lowerThanFunction (0x) - 58082: 1343, // lowerThanInsertValues (0x) - 58077: 1344, // lowerThanIntervalKeyword (0x) - 58087: 1345, // lowerThanKey (0x) - 58088: 1346, // lowerThanLocal (0x) - 58098: 1347, // lowerThanNot (0x) - 58095: 1348, // lowerThanOn (0x) - 58093: 1349, // lowerThanParenthese (0x) - 58089: 1350, // lowerThanRemove (0x) - 58076: 1351, // lowerThanSelectOpt (0x) - 58081: 1352, // lowerThanSelectStmt (0x) - 58080: 1353, // lowerThanSetKeyword (0x) - 58079: 1354, // lowerThanStringLitToken (0x) - 58078: 1355, // lowerThanValueKeyword (0x) - 58090: 1356, // lowerThenOrder (0x) - 58097: 1357, // neg (0x) - 57356: 1358, // odbcDateType (0x) - 57358: 1359, // odbcTimestampType (0x) - 57357: 1360, // odbcTimeType (0x) - 58092: 1361, // tableRefPriority (0x) + 58085: 1338, // lowerThanCharsetKwd (0x) + 58098: 1339, // lowerThanComma (0x) + 58083: 1340, // lowerThanCreateTableSelect (0x) + 58095: 1341, // lowerThanEq (0x) + 58090: 1342, // lowerThanFunction (0x) + 58081: 1343, // lowerThanInsertValues (0x) + 58086: 1344, // lowerThanKey (0x) + 58087: 1345, // lowerThanLocal (0x) + 58097: 1346, // lowerThanNot (0x) + 58094: 1347, // lowerThanOn (0x) + 58092: 1348, // lowerThanParenthese (0x) + 58088: 1349, // lowerThanRemove (0x) + 58076: 1350, // lowerThanSelectOpt (0x) + 58080: 1351, // lowerThanSelectStmt (0x) + 58079: 1352, // lowerThanSetKeyword (0x) + 58078: 1353, // lowerThanStringLitToken (0x) + 58077: 1354, // lowerThanValueKeyword (0x) + 58089: 1355, // lowerThenOrder (0x) + 58096: 1356, // neg (0x) + 57356: 1357, // odbcDateType (0x) + 57358: 1358, // odbcTimestampType (0x) + 57357: 1359, // odbcTimeType (0x) + 58091: 1360, // tableRefPriority (0x) } yySymNames = []string{ @@ -3528,7 +3526,6 @@ var ( "lowerThanEq", "lowerThanFunction", "lowerThanInsertValues", - "lowerThanIntervalKeyword", "lowerThanKey", "lowerThanLocal", "lowerThanNot", @@ -7375,12 +7372,12 @@ var ( // 1140 {1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 460: 1249, 1249, 1249, 1249, 465: 1249, 1249, 468: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 476: 1249, 1249, 479: 1249, 1249, 1249, 1249, 1249, 485: 1249, 487: 1249, 1249, 1249, 1249, 1249, 494: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 530: 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 1249, 3581, 1249, 1249, 1249, 1249, 1249, 1249}, {1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 460: 1252, 1252, 1252, 1252, 465: 1252, 1252, 468: 1252, 1252, 1252, 1252, 1252, 3580, 1252, 476: 1252, 1252, 479: 1252, 1252, 1252, 1252, 1252, 485: 1252, 487: 1252, 1252, 1252, 1252, 1252, 494: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3576, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 530: 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 1252, 3577, 3578, 1252, 3581, 1252, 3579, 1252, 1252, 1252, 1252}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3593}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3593}, {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3594}, {1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 460: 1250, 1250, 1250, 1250, 465: 1250, 1250, 468: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 476: 1250, 1250, 479: 1250, 1250, 1250, 1250, 1250, 485: 1250, 487: 1250, 1250, 1250, 1250, 1250, 494: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 530: 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250, 1250}, // 1145 {1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 460: 1253, 1253, 1253, 1253, 465: 1253, 1253, 468: 1253, 1253, 1253, 1253, 1253, 3580, 1253, 476: 1253, 1253, 479: 1253, 1253, 1253, 1253, 1253, 485: 1253, 487: 1253, 1253, 1253, 1253, 1253, 494: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3576, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 530: 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 1253, 3577, 3578, 1253, 3581, 1253, 3579, 1253, 1253, 1253, 1253}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3597}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3597}, {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3598}, {1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 460: 1251, 1251, 1251, 1251, 465: 1251, 1251, 468: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 476: 1251, 1251, 479: 1251, 1251, 1251, 1251, 1251, 485: 1251, 487: 1251, 1251, 1251, 1251, 1251, 494: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 530: 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251, 1251}, {1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 460: 1254, 1254, 1254, 1254, 465: 1254, 1254, 468: 1254, 1254, 1254, 3575, 3574, 3580, 1254, 476: 1254, 1254, 479: 1254, 1254, 1254, 1254, 1254, 485: 1254, 487: 1254, 1254, 1254, 1254, 1254, 494: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3576, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 530: 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 1254, 3577, 3578, 1254, 3581, 1254, 3579, 1254, 1254, 1254, 1254}, @@ -7407,7 +7404,7 @@ var ( {7: 3617, 491: 3236, 494: 3234, 3235, 3233, 3231, 720: 3232, 3230}, {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3619, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3618}, {57: 3623, 491: 3236, 494: 3234, 3235, 3233, 3231, 720: 3232, 3230}, - {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 3149, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3620}, + {2: 2902, 2750, 2786, 2904, 2677, 8: 2723, 2678, 2809, 2921, 2914, 2691, 2743, 3035, 3064, 3112, 3116, 3105, 3115, 3117, 3108, 3113, 3114, 3118, 3111, 2789, 2709, 2791, 2765, 2712, 2701, 2734, 2793, 2794, 2898, 2788, 2922, 3024, 3023, 2676, 2787, 2790, 2801, 2741, 2745, 2797, 2907, 2756, 2835, 2674, 2675, 2834, 2906, 2673, 2919, 58: 2879, 2990, 2755, 2758, 2973, 2970, 2962, 2974, 2977, 2978, 2975, 2979, 2980, 2976, 2969, 2981, 2964, 2965, 2968, 2971, 2972, 2982, 2772, 2821, 2759, 2949, 2948, 2950, 2945, 2944, 2951, 2946, 2947, 2751, 2864, 2934, 2997, 2932, 2998, 2933, 2692, 2824, 2763, 2670, 2686, 2829, 2920, 2777, 2704, 2721, 2848, 2931, 2764, 2733, 2842, 2843, 2838, 2798, 2923, 2924, 2925, 2926, 2927, 2928, 2930, 2779, 2849, 2760, 2853, 2854, 2855, 2856, 2845, 2873, 2916, 2875, 2694, 2874, 2736, 2995, 2826, 2865, 2731, 2784, 2940, 2846, 2805, 2700, 2711, 2726, 2935, 2808, 2775, 2825, 2695, 2710, 3093, 2984, 3067, 2753, 2861, 2773, 2783, 2681, 2730, 2664, 2740, 2744, 2752, 2774, 2985, 2685, 2703, 2702, 2724, 2802, 2803, 2954, 2882, 2991, 2992, 2956, 2820, 2993, 2912, 3063, 3018, 2952, 2852, 2768, 2910, 2812, 2671, 2817, 2707, 2708, 2818, 2715, 2725, 2728, 2716, 2938, 2963, 2778, 2877, 3065, 2844, 2815, 2872, 2915, 2804, 2754, 3019, 2762, 3028, 2769, 2911, 3000, 2960, 2822, 2883, 2684, 3001, 3004, 2690, 2986, 3005, 2837, 2696, 2697, 2885, 3046, 3007, 2881, 2705, 3009, 2894, 2918, 2905, 2706, 3011, 2913, 2719, 2943, 3100, 2729, 2732, 2895, 2941, 3055, 3056, 2889, 3013, 3012, 2939, 2996, 2827, 2655, 3014, 3015, 2831, 2887, 3016, 2994, 2748, 2749, 2860, 2966, 2862, 3068, 3017, 2908, 2909, 2850, 2757, 2891, 3031, 3020, 2672, 3077, 2890, 3083, 3084, 3085, 3086, 3088, 3087, 3089, 3090, 3030, 2770, 2668, 2669, 2942, 2959, 2679, 2961, 2987, 2682, 2683, 3044, 3002, 3003, 2687, 2871, 2688, 2689, 2858, 2785, 3006, 2806, 2693, 2698, 2699, 3008, 3010, 3050, 3051, 2713, 2714, 2828, 2718, 2878, 3094, 2720, 2888, 2727, 2823, 2799, 2896, 2917, 2880, 2814, 2936, 3057, 2866, 2884, 2929, 2737, 2735, 2811, 2897, 2792, 2953, 2867, 2795, 2796, 2656, 2830, 2739, 2761, 3032, 3095, 2742, 2900, 2903, 2955, 2989, 3033, 2999, 2840, 2841, 2847, 3061, 3036, 3062, 2937, 3037, 2967, 2870, 2810, 2901, 2859, 3025, 3022, 3021, 3069, 2886, 2988, 2899, 3027, 2868, 2766, 2767, 3029, 3103, 3091, 2892, 2771, 2800, 2807, 2869, 3109, 2776, 3034, 2876, 3038, 2781, 3039, 3040, 2680, 3041, 3042, 3043, 3096, 3045, 3047, 3048, 3049, 2717, 2863, 3097, 2833, 3052, 2722, 3104, 3053, 3054, 3102, 3101, 2957, 3106, 3107, 3059, 3058, 2738, 3060, 3066, 2839, 2746, 2747, 2983, 2857, 2819, 2836, 2958, 2851, 2782, 2893, 2813, 2816, 3098, 3073, 3074, 3075, 3076, 3099, 3070, 3071, 3072, 2832, 3026, 3081, 3082, 3092, 3078, 3079, 3080, 3110, 2780, 459: 1189, 461: 3129, 3147, 2659, 3157, 469: 3162, 3166, 3145, 3146, 3184, 478: 3120, 484: 3158, 486: 3182, 492: 3165, 3124, 529: 3153, 552: 3160, 554: 2657, 3183, 3167, 3119, 3121, 3123, 3122, 3150, 3127, 564: 3140, 3152, 3128, 3161, 569: 3159, 3151, 572: 3156, 574: 3227, 3163, 3172, 3173, 3174, 3126, 3143, 3144, 3197, 3200, 3201, 3202, 3203, 3204, 3154, 3205, 3180, 3185, 3195, 3196, 3189, 3206, 3207, 3208, 3190, 3210, 3211, 3198, 3191, 3209, 3186, 3194, 3192, 3178, 3212, 3213, 3155, 3217, 3168, 3169, 3171, 3216, 3222, 3221, 3223, 3220, 3224, 3219, 3218, 3215, 3164, 3214, 3170, 3175, 3176, 636: 2660, 648: 3133, 2666, 2667, 2665, 696: 3148, 3226, 3134, 3139, 3125, 3199, 3137, 3135, 3136, 3177, 3188, 3187, 3181, 3179, 3193, 3132, 3142, 3225, 3141, 3138, 2663, 2662, 2661, 3620}, // 1170 {105: 3352, 3348, 108: 3345, 3360, 111: 3347, 3344, 3346, 3350, 3351, 3356, 3355, 3354, 3358, 3359, 3353, 3357, 3349, 491: 3236, 494: 3234, 3235, 3233, 3231, 517: 3342, 3339, 3341, 3340, 3336, 3338, 3337, 3334, 3335, 3333, 3343, 720: 3232, 3230, 793: 3332, 816: 3621}, {57: 3622}, diff --git a/parser/parser.y b/parser/parser.y index 8c83474cb584f..56798d26b78b1 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -1399,8 +1399,6 @@ import ( %precedence sqlBigResult %precedence sqlSmallResult %precedence sqlCache sqlNoCache -%precedence lowerThanIntervalKeyword -%precedence interval %precedence next %precedence lowerThanValueKeyword %precedence value @@ -1453,6 +1451,7 @@ import ( %precedence lowerThanNot %right not not2 %right collate +%left interval %right encryption %left labels %precedence quick @@ -6920,7 +6919,7 @@ FunctionNameConflict: | "DAY" | "HOUR" | "IF" -| "INTERVAL" %prec lowerThanIntervalKeyword +| "INTERVAL" | "FORMAT" | "LEFT" | "MICROSECOND" diff --git a/parser/parser_test.go b/parser/parser_test.go index 134a9c79f5a5b..68e185775e18d 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -1494,6 +1494,8 @@ func TestBuiltin(t *testing.T) { {"SELECT LEAST(), LEAST(1, 2, 3);", true, "SELECT LEAST(),LEAST(1, 2, 3)"}, {"SELECT INTERVAL(1, 0, 1, 2)", true, "SELECT INTERVAL(1, 0, 1, 2)"}, + {"SELECT (INTERVAL(1, 0, 1, 2)+5)*7+INTERVAL(1, 0, 1, 2)/2", true, "SELECT (INTERVAL(1, 0, 1, 2)+5)*7+INTERVAL(1, 0, 1, 2)/2"}, + {"SELECT INTERVAL(0, (1*5)/2)+INTERVAL(5, 4, 3)", true, "SELECT INTERVAL(0, (1*5)/2)+INTERVAL(5, 4, 3)"}, {"SELECT DATE_ADD('2008-01-02', INTERVAL INTERVAL(1, 0, 1) DAY);", true, "SELECT DATE_ADD(_UTF8MB4'2008-01-02', INTERVAL INTERVAL(1, 0, 1) DAY)"}, // information functions diff --git a/server/conn.go b/server/conn.go index 113efd1d0a8e1..0209ab77171a5 100644 --- a/server/conn.go +++ b/server/conn.go @@ -2190,10 +2190,15 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool // fetchSize, the desired number of rows to be fetched each time when client uses cursor. func (cc *clientConn) writeChunksWithFetchSize(ctx context.Context, rs ResultSet, serverStatus uint16, fetchSize int) error { fetchedRows := rs.GetFetchedRows() + // if fetchedRows is not enough, getting data from recordSet. + req := rs.NewChunk(nil) for len(fetchedRows) < fetchSize { - // if fetchedRows is not enough, getting data from recordSet. - req := rs.NewChunk(cc.chunkAlloc) + // NOTE: chunk should not be allocated from the allocator + // the allocator will reset every statement + // but it maybe stored in the result set among statements + // ref https://github.com/pingcap/tidb/blob/7fc6ebbda4ddf84c0ba801ca7ebb636b934168cf/server/conn_stmt.go#L233-L239 // Here server.tidbResultSet implements Next method. + req.Reset() if err := rs.Next(ctx, req); err != nil { return err } @@ -2205,7 +2210,6 @@ func (cc *clientConn) writeChunksWithFetchSize(ctx context.Context, rs ResultSet for i := 0; i < rowCount; i++ { fetchedRows = append(fetchedRows, req.GetRow(i)) } - req = chunk.Renew(req, cc.ctx.GetSessionVars().MaxChunkSize) } // tell the client COM_STMT_FETCH has finished by setting proper serverStatus, diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 8c5f1d08a4013..23b2de4333af8 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -906,13 +906,39 @@ func (s *testSerialStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.T } // nolint:unused -func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, t string, p string, topn, buckets int) { +func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, db, t, p string, topn, buckets int) { + tbl, err := s.do.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(t)) + c.Assert(err, IsNil) + + tblInfo := tbl.Meta() + physicalID := tblInfo.ID + if p != "global" { + for _, def := range tbl.Meta().GetPartitionInfo().Definitions { + if def.Name.L == p { + physicalID = def.ID + } + } + } + tblStats, err := s.do.StatsHandle().TableStatsFromStorage(tblInfo, physicalID, true, 0) + c.Assert(err, IsNil) + delta := buckets/2 + 10 - for _, isIdx := range []int{0, 1} { - c.Assert(len(tk.MustQuery(fmt.Sprintf("show stats_topn where table_name='%v' and partition_name='%v' and is_index=%v", t, p, isIdx)).Rows()), Equals, topn) - numBuckets := len(tk.MustQuery(fmt.Sprintf("show stats_buckets where table_name='%v' and partition_name='%v' and is_index=%v", t, p, isIdx)).Rows()) + for _, idxStats := range tblStats.Indices { + numTopN := idxStats.TopN.Num() + numBuckets := len(idxStats.Buckets) // since the hist-building algorithm doesn't stipulate the final bucket number to be equal to the expected number exactly, // we have to check the results by a range here. + c.Assert(numTopN, Equals, topn) + c.Assert(numBuckets, GreaterEqual, buckets-delta) + c.Assert(numBuckets, LessEqual, buckets+delta) + } + for _, colStats := range tblStats.Columns { + if len(colStats.Buckets) == 0 { + continue // it's not loaded + } + numTopN := colStats.TopN.Num() + numBuckets := len(colStats.Buckets) + c.Assert(numTopN, Equals, topn) c.Assert(numBuckets, GreaterEqual, buckets-delta) c.Assert(numBuckets, LessEqual, buckets+delta) } @@ -947,9 +973,9 @@ func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { sql := fmt.Sprintf("analyze table test_gstats_opt with %v topn, %v buckets", ca.topn, ca.buckets) if !ca.err { tk.MustExec(sql) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "global", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "p0", ca.topn, ca.buckets) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "p1", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "global", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p0", ca.topn, ca.buckets) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt", "test_gstats_opt", "p1", ca.topn, ca.buckets) } else { err := tk.ExecToErr(sql) c.Assert(err, NotNil) @@ -966,25 +992,25 @@ func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { s.prepareForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2") tk.MustExec("analyze table test_gstats_opt2 with 20 topn, 50 buckets, 1000 samples") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 2, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 1, 50) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 2, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) // analyze a partition to let its options be different with others' tk.MustExec("analyze table test_gstats_opt2 partition p0 with 10 topn, 20 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 10, 20) // use new options - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 1, 50) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 10, 20) // use new options + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 1, 50) tk.MustExec("analyze table test_gstats_opt2 partition p1 with 100 topn, 200 buckets") - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 100, 200) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 10, 20) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 10, 20) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) tk.MustExec("analyze table test_gstats_opt2 partition p0 with 20 topn") // change back to 20 topn - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "global", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p0", 20, 256) - s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "p1", 100, 200) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "global", 20, 256) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p0", 20, 256) + s.checkForGlobalStatsWithOpts(c, tk, "test_gstats_opt2", "test_gstats_opt2", "p1", 100, 200) } func (s *testStatsSuite) TestGlobalStatsHealthy(c *C) { diff --git a/types/core_time.go b/types/core_time.go index 161180dd3b166..64820f98aaf79 100644 --- a/types/core_time.go +++ b/types/core_time.go @@ -184,6 +184,59 @@ func (t CoreTime) GoTime(loc *gotime.Location) (gotime.Time, error) { return tm, nil } +// FindZoneTransition check for one Time Zone transition within +/- 4h +// Currently the needed functions are not exported, if gotime.Location.lookup would be exported +// then it would be easy to use that directly +func FindZoneTransition(tIn gotime.Time) (gotime.Time, error) { + // Check most common case first, DST transition on full hour. + // round truncates away from zero! + t2 := tIn.Round(gotime.Hour).Add(-1 * gotime.Hour) + t1 := t2.Add(-1 * gotime.Second) + _, offset1 := t1.Zone() + _, offset2 := t2.Zone() + if offset1 != offset2 { + return t2, nil + } + + // Check if any offset change? + t1 = tIn.Add(-4 * gotime.Hour) + t2 = tIn.Add(4 * gotime.Hour) + _, offset1 = t1.Zone() + _, offset2 = t2.Zone() + if offset1 == offset2 { + return tIn, errors.Trace(ErrWrongValue.GenWithStackByArgs(TimeStr, tIn)) + } + + // Check generic case, like for 'Australia/Lord_Howe' + for t2.After(t1.Add(gotime.Second)) { + t := t1.Add(t2.Sub(t1) / 2).Round(gotime.Second) + _, offset := t.Zone() + if offset == offset1 { + t1 = t + } else { + t2 = t + } + } + return t2, nil +} + +// AdjustedGoTime converts Time to GoTime and adjust for invalid DST times +// like during the DST change with increased offset, +// normally moving to Daylight Saving Time. +// see https://github.com/pingcap/tidb/issues/28739 +func (t CoreTime) AdjustedGoTime(loc *gotime.Location) (gotime.Time, error) { + tm, err := t.GoTime(loc) + if err == nil { + return tm, nil + } + + tAdj, err2 := FindZoneTransition(tm) + if err2 == nil { + return tAdj, nil + } + return tm, err +} + // IsLeapYear returns if it's leap year. func (t CoreTime) IsLeapYear() bool { return isLeapYear(t.getYear()) diff --git a/types/core_time_test.go b/types/core_time_test.go index f34e38ab7f8a2..07e4d040a5c2c 100644 --- a/types/core_time_test.go +++ b/types/core_time_test.go @@ -316,3 +316,81 @@ func TestWeekday(t *testing.T) { require.Equal(t, tt.Expect, weekday.String()) } } + +func TestFindZoneTransition(t *testing.T) { + t.Parallel() + tests := []struct { + TZ string + dt string + Expect string + Success bool + }{ + {"Australia/Lord_Howe", "2020-06-29 03:45:00", "", false}, + {"Australia/Lord_Howe", "2020-10-04 02:15:00", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:29:59", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:29:59.99", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:00.0001", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:00", "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", "2020-10-04 02:30:01", "2020-10-04 02:30:00 +11 +1100", true}, + {"Europe/Vilnius", "2020-03-29 03:45:00", "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", "2020-10-25 03:45:00", "2020-10-25 03:00:00 EET +0200", true}, + {"Europe/Vilnius", "2020-06-29 03:45:00", "", false}, + {"Europe/Amsterdam", "2020-03-29 02:45:00", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-10-25 02:35:00", "2020-10-25 02:00:00 CET +0100", true}, + {"Europe/Amsterdam", "2020-03-29 02:59:59", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-03-29 02:59:59.999999999", "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", "2020-03-29 03:00:00.000000001", "2020-03-29 03:00:00 CEST +0200", true}, + } + + for _, tt := range tests { + loc, err := time.LoadLocation(tt.TZ) + require.NoError(t, err) + tm, err := time.ParseInLocation("2006-01-02 15:04:05", tt.dt, loc) + require.NoError(t, err) + tp, err := FindZoneTransition(tm) + if !tt.Success { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) + } + } +} + +func TestAdjustedGoTime(t *testing.T) { + t.Parallel() + tests := []struct { + TZ string + dt CoreTime + Expect string + Success bool + }{ + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 01, 59, 59, 997), "2020-10-04 01:59:59.000997 +1030 +1030", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 00, 00, 0), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 15, 00, 0), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 29, 59, 999999), "2020-10-04 02:30:00 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 10, 04, 02, 30, 00, 1), "2020-10-04 02:30:00.000001 +11 +1100", true}, + {"Australia/Lord_Howe", FromDate(2020, 06, 29, 03, 45, 00, 0), "2020-06-29 03:45:00 +1030 +1030", true}, + {"Australia/Lord_Howe", FromDate(2020, 04, 04, 01, 45, 00, 0), "2020-04-04 01:45:00 +11 +1100", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 03, 45, 00, 0), "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 03, 59, 59, 456789), "2020-03-29 04:00:00 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 03, 29, 04, 00, 01, 130000), "2020-03-29 04:00:01.13 EEST +0300", true}, + {"Europe/Vilnius", FromDate(2020, 10, 25, 03, 45, 00, 0), "2020-10-25 03:45:00 EET +0200", true}, + {"Europe/Vilnius", FromDate(2020, 06, 29, 03, 45, 00, 0), "2020-06-29 03:45:00 EEST +0300", true}, + {"Europe/Amsterdam", FromDate(2020, 03, 29, 02, 45, 00, 0), "2020-03-29 03:00:00 CEST +0200", true}, + {"Europe/Amsterdam", FromDate(2020, 10, 25, 02, 35, 00, 0), "2020-10-25 02:35:00 CET +0100", true}, + {"UTC", FromDate(2020, 2, 31, 02, 35, 00, 0), "", false}, + } + + for _, tt := range tests { + loc, err := time.LoadLocation(tt.TZ) + require.NoError(t, err) + tp, err := tt.dt.AdjustedGoTime(loc) + if !tt.Success { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, tt.Expect, tp.Format("2006-01-02 15:04:05.999999999 MST -0700")) + } + } +}