diff --git a/DEPS.bzl b/DEPS.bzl index ac0a348ad55fe..dbd1edbaf98d6 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -4467,8 +4467,8 @@ def go_deps(): name = "org_golang_x_time", build_file_proto_mode = "disable_global", importpath = "golang.org/x/time", - sum = "h1:52I/1L54xyEQAYdtcSuxtiT84KGYTBGXwayxmIpNJhE=", - version = "v0.2.0", + sum = "h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4=", + version = "v0.3.0", ) go_repository( name = "org_golang_x_tools", diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 907b6e94ccde0..59919e2b5ad85 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -533,6 +533,15 @@ func (h *BindHandle) SetBindRecordStatus(originalSQL string, binding *Binding, n return } +// SetBindRecordStatusByDigest set a BindRecord's status to the storage and bind cache. +func (h *BindHandle) SetBindRecordStatusByDigest(newStatus, sqlDigest string) (ok bool, err error) { + oldRecord, err := h.GetBindRecordBySQLDigest(sqlDigest) + if err != nil { + return false, err + } + return h.SetBindRecordStatus(oldRecord.OriginalSQL, nil, newStatus) +} + // GCBindRecord physically removes the deleted bind records in mysql.bind_info. func (h *BindHandle) GCBindRecord() (err error) { h.bindInfo.Lock() diff --git a/ddl/backfilling.go b/ddl/backfilling.go index d1035bad084bd..0f0910e1caf28 100644 --- a/ddl/backfilling.go +++ b/ddl/backfilling.go @@ -807,12 +807,7 @@ func (b *backfillScheduler) initCopReqSenderPool() { logutil.BgLogger().Warn("[ddl-ingest] cannot init cop request sender", zap.Error(err)) return } - ver, err := sessCtx.GetStore().CurrentVersion(kv.GlobalTxnScope) - if err != nil { - logutil.BgLogger().Warn("[ddl-ingest] cannot init cop request sender", zap.Error(err)) - return - } - b.copReqSenderPool = newCopReqSenderPool(b.ctx, copCtx, ver.Ver) + b.copReqSenderPool = newCopReqSenderPool(b.ctx, copCtx, sessCtx.GetStore()) } func (b *backfillScheduler) canSkipError(err error) bool { diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 429d3f13425b2..a5e89e4996d0a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1980,7 +1980,7 @@ func addIndexForForeignKey(ctx sessionctx.Context, tbInfo *model.TableInfo) erro if handleCol != nil && len(fk.Cols) == 1 && handleCol.Name.L == fk.Cols[0].L { continue } - if model.FindIndexByColumns(tbInfo, fk.Cols...) != nil { + if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) != nil { continue } idxName := fk.Name @@ -3264,6 +3264,14 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast return dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Alter Table") } } + // set name for anonymous foreign key. + maxForeignKeyID := tb.Meta().MaxForeignKeyID + for _, spec := range validSpecs { + if spec.Tp == ast.AlterTableAddConstraint && spec.Constraint.Tp == ast.ConstraintForeignKey && spec.Constraint.Name == "" { + maxForeignKeyID++ + spec.Constraint.Name = fmt.Sprintf("fk_%d", maxForeignKeyID) + } + } if len(validSpecs) > 1 { sctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() @@ -6570,7 +6578,7 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode if err != nil { return err } - if model.FindIndexByColumns(t.Meta(), fkInfo.Cols...) == nil { + if model.FindIndexByColumns(t.Meta(), t.Meta().Indices, fkInfo.Cols...) == nil { // Need to auto create index for fk cols if ctx.GetSessionVars().StmtCtx.MultiSchemaInfo == nil { ctx.GetSessionVars().StmtCtx.MultiSchemaInfo = model.NewMultiSchemaInfo() diff --git a/ddl/export_test.go b/ddl/export_test.go index 486390f9a6810..3ea26fb04290c 100644 --- a/ddl/export_test.go +++ b/ddl/export_test.go @@ -28,7 +28,7 @@ func SetBatchInsertDeleteRangeSize(i int) { var NewCopContext4Test = newCopContext -func FetchRowsFromCop4Test(copCtx *copContext, startKey, endKey kv.Key, startTS uint64, +func FetchRowsFromCop4Test(copCtx *copContext, startKey, endKey kv.Key, store kv.Storage, batchSize int) ([]*indexRecord, bool, error) { variable.SetDDLReorgBatchSize(int32(batchSize)) task := &reorgBackfillTask{ @@ -36,7 +36,7 @@ func FetchRowsFromCop4Test(copCtx *copContext, startKey, endKey kv.Key, startTS startKey: startKey, endKey: endKey, } - pool := newCopReqSenderPool(context.Background(), copCtx, startTS) + pool := newCopReqSenderPool(context.Background(), copCtx, store) pool.adjustSize(1) pool.tasksCh <- task idxRec, _, _, done, err := pool.fetchRowColValsFromCop(*task) diff --git a/ddl/fktest/foreign_key_test.go b/ddl/fktest/foreign_key_test.go index 13701ad5b610d..f64de90ca4955 100644 --- a/ddl/fktest/foreign_key_test.go +++ b/ddl/fktest/foreign_key_test.go @@ -1562,3 +1562,32 @@ func getLatestSchemaDiff(t *testing.T, tk *testkit.TestKit) *model.SchemaDiff { require.NoError(t, err) return diff } + +func TestTestMultiSchemaAddForeignKey(t *testing.T) { + store, _ := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@foreign_key_checks=1;") + tk.MustExec("use test") + tk.MustExec("create table t1 (id int key);") + tk.MustExec("create table t2 (a int, b int);") + tk.MustExec("alter table t2 add foreign key (a) references t1(id), add foreign key (b) references t1(id)") + tk.MustExec("alter table t2 add column c int, add column d int") + tk.MustExec("alter table t2 add foreign key (c) references t1(id), add foreign key (d) references t1(id), add index(c), add index(d)") + tk.MustExec("drop table t2") + tk.MustExec("create table t2 (a int, b int, index idx1(a), index idx2(b));") + tk.MustGetErrMsg("alter table t2 drop index idx1, drop index idx2, add foreign key (a) references t1(id), add foreign key (b) references t1(id)", + "[ddl:1553]Cannot drop index 'idx1': needed in a foreign key constraint") + tk.MustExec("alter table t2 drop index idx1, drop index idx2") + tk.MustExec("alter table t2 add foreign key (a) references t1(id), add foreign key (b) references t1(id)") + tk.MustQuery("show create table t2").Check(testkit.Rows("t2 CREATE TABLE `t2` (\n" + + " `a` int(11) DEFAULT NULL,\n" + + " `b` int(11) DEFAULT NULL,\n" + + " KEY `fk_1` (`a`),\n" + + " KEY `fk_2` (`b`),\n" + + " CONSTRAINT `fk_1` FOREIGN KEY (`a`) REFERENCES `test`.`t1` (`id`),\n" + + " CONSTRAINT `fk_2` FOREIGN KEY (`b`) REFERENCES `test`.`t1` (`id`)\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) + tk.MustExec("drop table t2") + tk.MustExec("create table t2 (a int, b int, index idx0(a,b), index idx1(a), index idx2(b));") + tk.MustExec("alter table t2 drop index idx1, add foreign key (a) references t1(id), add foreign key (b) references t1(id)") +} diff --git a/ddl/foreign_key.go b/ddl/foreign_key.go index af9570fbee837..cdf3695cdc017 100644 --- a/ddl/foreign_key.go +++ b/ddl/foreign_key.go @@ -291,7 +291,7 @@ func checkTableForeignKey(referTblInfo, tblInfo *model.TableInfo, fkInfo *model. } } // check refer columns should have index. - if model.FindIndexByColumns(referTblInfo, fkInfo.RefCols...) == nil { + if model.FindIndexByColumns(referTblInfo, referTblInfo.Indices, fkInfo.RefCols...) == nil { return infoschema.ErrForeignKeyNoIndexInParent.GenWithStackByArgs(fkInfo.Name, fkInfo.RefTable) } return nil @@ -660,7 +660,7 @@ func checkAddForeignKeyValidInOwner(d *ddlCtx, t *meta.Meta, schema string, tbIn return nil } } - if model.FindIndexByColumns(tbInfo, fk.Cols...) == nil { + if model.FindIndexByColumns(tbInfo, tbInfo.Indices, fk.Cols...) == nil { return errors.Errorf("Failed to add the foreign key constraint. Missing index for '%s' foreign key columns in the table '%s'", fk.Name, tbInfo.Name) } return nil diff --git a/ddl/index.go b/ddl/index.go index 273b89e041233..7c9fc1d083ff4 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -81,7 +81,12 @@ func buildIndexColumns(ctx sessionctx.Context, columns []*model.ColumnInfo, inde if err := checkIndexColumn(ctx, col, ip.Length); err != nil { return nil, false, err } - mvIndex = mvIndex || col.FieldType.IsArray() + if col.FieldType.IsArray() { + if mvIndex { + return nil, false, dbterror.ErrNotSupportedYet.GenWithStack("'more than one multi-valued key part per index'") + } + mvIndex = true + } indexColLen := ip.Length indexColumnLength, err := getIndexColumnLength(col, ip.Length) if err != nil { diff --git a/ddl/index_cop.go b/ddl/index_cop.go index 0a04ac63eb190..fab097727139b 100644 --- a/ddl/index_cop.go +++ b/ddl/index_cop.go @@ -103,9 +103,9 @@ type copReqSenderPool struct { resultsCh chan idxRecResult results generic.SyncMap[int, struct{}] - ctx context.Context - copCtx *copContext - startTS uint64 + ctx context.Context + copCtx *copContext + store kv.Storage senders []*copReqSender wg sync.WaitGroup @@ -139,7 +139,12 @@ func (c *copReqSender) run() { curTaskID = task.id logutil.BgLogger().Info("[ddl-ingest] start a cop-request task", zap.Int("id", task.id), zap.String("task", task.String())) - rs, err := p.copCtx.buildTableScan(p.ctx, p.startTS, task.startKey, task.excludedEndKey()) + ver, err := p.store.CurrentVersion(kv.GlobalTxnScope) + if err != nil { + p.resultsCh <- idxRecResult{id: task.id, err: err} + return + } + rs, err := p.copCtx.buildTableScan(p.ctx, ver.Ver, task.startKey, task.excludedEndKey()) if err != nil { p.resultsCh <- idxRecResult{id: task.id, err: err} return @@ -167,7 +172,7 @@ func (c *copReqSender) run() { } } -func newCopReqSenderPool(ctx context.Context, copCtx *copContext, startTS uint64) *copReqSenderPool { +func newCopReqSenderPool(ctx context.Context, copCtx *copContext, store kv.Storage) *copReqSenderPool { poolSize := copReadChunkPoolSize() idxBufPool := make(chan []*indexRecord, poolSize) srcChkPool := make(chan *chunk.Chunk, poolSize) @@ -181,7 +186,7 @@ func newCopReqSenderPool(ctx context.Context, copCtx *copContext, startTS uint64 results: generic.NewSyncMap[int, struct{}](10), ctx: ctx, copCtx: copCtx, - startTS: startTS, + store: store, senders: make([]*copReqSender, 0, variable.GetDDLReorgWorkerCounter()), wg: sync.WaitGroup{}, idxBufPool: idxBufPool, diff --git a/ddl/index_cop_test.go b/ddl/index_cop_test.go index 80e37f6a74121..38bced0b6678d 100644 --- a/ddl/index_cop_test.go +++ b/ddl/index_cop_test.go @@ -43,7 +43,7 @@ func TestAddIndexFetchRowsFromCoprocessor(t *testing.T) { endKey := startKey.PrefixNext() txn, err := store.Begin() require.NoError(t, err) - idxRec, done, err := ddl.FetchRowsFromCop4Test(copCtx, startKey, endKey, txn.StartTS(), 10) + idxRec, done, err := ddl.FetchRowsFromCop4Test(copCtx, startKey, endKey, store, 10) require.NoError(t, err) require.False(t, done) require.NoError(t, txn.Rollback()) diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index ab306fe546932..a21bc27965c77 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -261,7 +261,10 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error case model.ActionRebaseAutoID, model.ActionModifyTableComment, model.ActionModifyTableCharsetAndCollate: case model.ActionAddForeignKey: fkInfo := job.Args[0].(*model.FKInfo) - info.ForeignKeys = append(info.ForeignKeys, fkInfo.Name) + info.AddForeignKeys = append(info.AddForeignKeys, model.AddForeignKeyInfo{ + Name: fkInfo.Name, + Cols: fkInfo.Cols, + }) default: return dbterror.ErrRunMultiSchemaChanges.FastGenByArgs(job.Type.String()) } @@ -323,6 +326,32 @@ func checkOperateSameColAndIdx(info *model.MultiSchemaInfo) error { return checkIndexes(info.AlterIndexes, true) } +func checkOperateDropIndexUseByForeignKey(info *model.MultiSchemaInfo, t table.Table) error { + var remainIndexes, droppingIndexes []*model.IndexInfo + tbInfo := t.Meta() + for _, idx := range tbInfo.Indices { + dropping := false + for _, name := range info.DropIndexes { + if name.L == idx.Name.L { + dropping = true + break + } + } + if dropping { + droppingIndexes = append(droppingIndexes, idx) + } else { + remainIndexes = append(remainIndexes, idx) + } + } + + for _, fk := range info.AddForeignKeys { + if droppingIdx := model.FindIndexByColumns(tbInfo, droppingIndexes, fk.Cols...); droppingIdx != nil && model.FindIndexByColumns(tbInfo, remainIndexes, fk.Cols...) == nil { + return dbterror.ErrDropIndexNeededInForeignKey.GenWithStackByArgs(droppingIdx.Name) + } + } + return nil +} + func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { err := checkOperateSameColAndIdx(info) if err != nil { @@ -334,6 +363,11 @@ func checkMultiSchemaInfo(info *model.MultiSchemaInfo, t table.Table) error { return err } + err = checkOperateDropIndexUseByForeignKey(info, t) + if err != nil { + return err + } + return checkAddColumnTooManyColumns(len(t.Cols()) + len(info.AddColumns) - len(info.DropColumns)) } diff --git a/errors.toml b/errors.toml index 32b8d40e902b2..7d5be840c0432 100644 --- a/errors.toml +++ b/errors.toml @@ -766,6 +766,11 @@ error = ''' Incorrect usage of %s and %s ''' +["ddl:1235"] +error = ''' +This version of TiDB doesn't yet support '%s' +''' + ["ddl:1246"] error = ''' Converting column '%s' from %s to %s diff --git a/executor/admin_test.go b/executor/admin_test.go index 41b926cf2c377..0b2530e76d5a3 100644 --- a/executor/admin_test.go +++ b/executor/admin_test.go @@ -1095,9 +1095,7 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindInt64 1]\"", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindInt64 1]\"") hook.checkLogCount(t, 1) hook.logs[0].checkMsg(t, "admin check found data inconsistency") hook.logs[0].checkField(t, @@ -1119,9 +1117,7 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindString 10]\"", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"\" != record-values:\"handle: 1, values: [KindString 10]\"") hook.checkLogCount(t, 1) hook.logs[0].checkMsg(t, "admin check found data inconsistency") hook.logs[0].checkField(t, @@ -1143,9 +1139,8 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"handle: 1, values: [KindString 100 KindInt64 1]\" != record-values:\"\"", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", + "[admin:8223]data inconsistency in table: admin_test, index: k2, handle: 1, index-values:\"handle: 1, values: [KindString 100 KindInt64 1]\" != record-values:\"\"") hook.checkLogCount(t, 1) logEntry := hook.logs[0] logEntry.checkMsg(t, "admin check found data inconsistency") @@ -1188,9 +1183,8 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"handle: 1, values: [KindInt64 10 KindInt64 1]\" != record-values:\"\"", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", + "[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 1, index-values:\"handle: 1, values: [KindInt64 10 KindInt64 1]\" != record-values:\"\"") hook.checkLogCount(t, 1) logEntry := hook.logs[0] logEntry.checkMsg(t, "admin check found data inconsistency") @@ -1233,9 +1227,8 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: uk1, col: c2, handle: \"1\", index-values:\"KindInt64 20\" != record-values:\"KindInt64 10\", compare err:", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", + "[executor:8134]data inconsistency in table: admin_test, index: uk1, col: c2, handle: \"1\", index-values:\"KindInt64 20\" != record-values:\"KindInt64 10\", compare err:") hook.checkLogCount(t, 1) logEntry := hook.logs[0] logEntry.checkMsg(t, "admin check found data inconsistency") @@ -1261,9 +1254,8 @@ func TestCheckFailReport(t *testing.T) { require.NoError(t, err) require.NoError(t, txn.Commit(tk.ctx)) ctx, hook := withLogHook(tk.ctx, t, "inconsistency") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, "[executor:8134]data inconsistency in table: admin_test, index: k2, col: c3, handle: \"1\", index-values:\"KindString 200\" != record-values:\"KindString 100\", compare err:", err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", + "[executor:8134]data inconsistency in table: admin_test, index: k2, col: c3, handle: \"1\", index-values:\"KindString 200\" != record-values:\"KindString 100\", compare err:") hook.checkLogCount(t, 1) logEntry := hook.logs[0] logEntry.checkMsg(t, "admin check found data inconsistency") @@ -1301,12 +1293,10 @@ func TestCheckFailReport(t *testing.T) { // TODO(tiancaiamao): admin check doesn't support the chunk protocol. // Remove this after https://github.com/pingcap/tidb/issues/35156 - _, err = tk.Exec(ctx, "set @@tidb_enable_chunk_rpc = off") - require.NoError(t, err) + tk.MustExec(ctx, "set @@tidb_enable_chunk_rpc = off") - _, err = tk.Exec(ctx, "admin check table admin_test") - require.Error(t, err) - require.Equal(t, `[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 282574488403969, index-values:"handle: 282574488403969, values: [KindInt64 282578800083201 KindInt64 282574488403969]" != record-values:""`, err.Error()) + tk.MustGetErrMsg(ctx, "admin check table admin_test", + `[admin:8223]data inconsistency in table: admin_test, index: uk1, handle: 282574488403969, index-values:"handle: 282574488403969, values: [KindInt64 282578800083201 KindInt64 282574488403969]" != record-values:""`) hook.checkLogCount(t, 1) logEntry := hook.logs[0] logEntry.checkMsg(t, "admin check found data inconsistency") diff --git a/executor/bind.go b/executor/bind.go index c5f254121a9bf..90272e6878620 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -63,6 +63,8 @@ func (e *SQLBindExec) Next(ctx context.Context, req *chunk.Chunk) error { return e.reloadBindings() case plannercore.OpSetBindingStatus: return e.setBindingStatus() + case plannercore.OpSetBindingStatusByDigest: + return e.setBindingStatusByDigest() default: return errors.Errorf("unsupported SQL bind operation: %v", e.sqlBindOp) } @@ -119,6 +121,15 @@ func (e *SQLBindExec) setBindingStatus() error { return err } +func (e *SQLBindExec) setBindingStatusByDigest() error { + ok, err := domain.GetDomain(e.ctx).BindHandle().SetBindRecordStatusByDigest(e.newStatus, e.sqlDigest) + if err == nil && !ok { + warningMess := errors.New("There are no bindings can be set the status. Please check the SQL text") + e.ctx.GetSessionVars().StmtCtx.AppendWarning(warningMess) + } + return err +} + func (e *SQLBindExec) createSQLBind() error { // For audit log, SQLBindExec execute "explain" statement internally, save and recover stmtctx // is necessary to avoid 'create binding' been recorded as 'explain'. diff --git a/executor/executor.go b/executor/executor.go index 2e5b5c4a0280f..9f95e63aaed20 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1933,7 +1933,7 @@ func (e *UnionExec) Close() error { func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars := ctx.GetSessionVars() var sc *stmtctx.StatementContext - if vars.TxnCtx.CouldRetry { + if vars.TxnCtx.CouldRetry || mysql.HasCursorExistsFlag(vars.Status) { // Must construct new statement context object, the retry history need context for every statement. // TODO: Maybe one day we can get rid of transaction retry, then this logic can be deleted. sc = &stmtctx.StatementContext{} diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 11dae5e570767..5edbed52b4e13 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -420,14 +420,14 @@ func TestPreparedInsert(t *testing.T) { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(1), hit) + require.Equal(t, float64(0), hit) // insert-values-stmt cannot use the plan cache } tk.MustExec(`set @a=3,@b=3; execute stmt_insert using @a, @b;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(2), hit) + require.Equal(t, float64(0), hit) } result := tk.MustQuery("select id, c1 from prepare_test where id = ?", 1) @@ -443,21 +443,21 @@ func TestPreparedInsert(t *testing.T) { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(2), hit) + require.Equal(t, float64(0), hit) } tk.MustExec(`set @a=2; execute stmt_insert_select using @a;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(3), hit) + require.Equal(t, float64(1), hit) } tk.MustExec(`set @a=3; execute stmt_insert_select using @a;`) if flag { err = counter.Write(pb) require.NoError(t, err) hit := pb.GetCounter().GetValue() - require.Equal(t, float64(4), hit) + require.Equal(t, float64(2), hit) } result = tk.MustQuery("select id, c1 from prepare_test where id = ?", 101) diff --git a/expression/expression.go b/expression/expression.go index 6d7eb080b29fc..ee4ba67f1f854 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -998,7 +998,7 @@ func ColumnInfos2ColumnsAndNames(ctx sessionctx.Context, dbName, tblName model.C if err != nil { return nil, nil, errors.Trace(err) } - e, err := RewriteAstExpr(ctx, expr, mockSchema, names, false) + e, err := RewriteAstExpr(ctx, expr, mockSchema, names, true) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/expression/multi_valued_index_test.go b/expression/multi_valued_index_test.go index 058d955faa4fb..788dadddab02c 100644 --- a/expression/multi_valued_index_test.go +++ b/expression/multi_valued_index_test.go @@ -44,4 +44,15 @@ func TestMultiValuedIndexDDL(t *testing.T) { tk.MustGetErrCode("create table t(j json, gc json as (cast(j->'$[*]' as unsigned array)));", errno.ErrNotSupportedYet) tk.MustGetErrCode("create view v as select cast('[1,2,3]' as unsigned array);", errno.ErrNotSupportedYet) tk.MustExec("create table t(a json, index idx((cast(a as signed array))));") + + tk.MustExec("drop table t") + tk.MustGetErrCode("create table t(a json, b int, index idx(b, (cast(a as signed array)), (cast(a as signed array))));", errno.ErrNotSupportedYet) + tk.MustExec("create table t(a json, b int);") + tk.MustGetErrCode("create index idx on t (b, (cast(a as signed array)), (cast(a as signed array)))", errno.ErrNotSupportedYet) + tk.MustGetErrCode("alter table t add index idx(b, (cast(a as signed array)), (cast(a as signed array)))", errno.ErrNotSupportedYet) + tk.MustExec("create index idx1 on t (b, (cast(a as signed array)))") + tk.MustExec("alter table t add index idx2(b, (cast(a as signed array)))") + + tk.MustExec("drop table t") + tk.MustExec("create table t(a json, b int, index idx3(b, (cast(a as signed array))));") } diff --git a/go.mod b/go.mod index df18ccec64b18..f4d3fb93f9e98 100644 --- a/go.mod +++ b/go.mod @@ -116,7 +116,7 @@ require ( golang.org/x/sys v0.3.0 golang.org/x/term v0.3.0 golang.org/x/text v0.5.0 - golang.org/x/time v0.2.0 + golang.org/x/time v0.3.0 golang.org/x/tools v0.2.0 google.golang.org/api v0.74.0 google.golang.org/grpc v1.45.0 diff --git a/go.sum b/go.sum index b7de00223b8a3..8c21c0326306d 100644 --- a/go.sum +++ b/go.sum @@ -1338,8 +1338,8 @@ golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxb golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210220033141-f8bda1e9f3ba/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.2.0 h1:52I/1L54xyEQAYdtcSuxtiT84KGYTBGXwayxmIpNJhE= -golang.org/x/time v0.2.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= +golang.org/x/time v0.3.0 h1:rg5rLMjNzMS1RkNLzCG38eapWhnYLFYXDXj2gOlr8j4= +golang.org/x/time v0.3.0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/tools v0.0.0-20180221164845-07fd8470d635/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180525024113-a5b4c53f6e8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 00892c95cfe9c..73488028a26ec 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -1028,3 +1028,34 @@ func TestBindingFromHistoryWithTiFlashBindable(t *testing.T) { planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.statements_summary where query_sample_text = '%s'", sql)).Rows() tk.MustGetErrMsg(fmt.Sprintf("create binding from history using plan digest '%s'", planDigest[0][0]), "can't create binding for query with tiflash engine") } + +func TestSetBindingStatusBySQLDigest(t *testing.T) { + s := new(clusterTablesSuite) + s.store, s.dom = testkit.CreateMockStoreAndDomain(t) + s.rpcserver, s.listenAddr = s.setUpRPCService(t, "127.0.0.1:0", nil) + s.httpServer, s.mockAddr = s.setUpMockPDHTTPServer() + s.startTime = time.Now() + defer s.httpServer.Close() + defer s.rpcserver.Stop() + tk := s.newTestKitWithRoot(t) + require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil)) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int, a int, key(a))") + sql := "select /*+ ignore_index(t, a) */ * from t where t.a = 1" + tk.MustExec(sql) + planDigest := tk.MustQuery(fmt.Sprintf("select plan_digest from information_schema.cluster_statements_summary where query_sample_text = '%s'", sql)).Rows() + tk.MustExec(fmt.Sprintf("create global binding from history using plan digest '%s'", planDigest[0][0])) + sql = "select * from t where t.a = 1" + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + + sqlDigest := tk.MustQuery("show global bindings").Rows() + tk.MustExec(fmt.Sprintf("set binding disabled for sql digest '%s'", sqlDigest[0][9])) + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec(fmt.Sprintf("set binding enabled for sql digest '%s'", sqlDigest[0][9])) + tk.MustExec(sql) + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + tk.MustGetErrMsg("set binding enabled for sql digest '2'", "can't find any binding for '2'") +} diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 7677a718e156f..0e2ce93934f7c 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -17000,6 +17000,212 @@ "title": "TTL", "type": "row" }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 19 + }, + "id": 291, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 20 + }, + "hiddenSeries": false, + "id": 289, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_server_gogc{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "GOGC", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "exponential moving average of CPU Usage", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 20 + }, + "hiddenSeries": false, + "id": 293, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "tidb_rm_ema_cpu_usage{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EMA CPU Usage", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Resource Manager", + "type": "row" + }, { "aliasColors": {}, "bars": false, diff --git a/parser/ast/misc.go b/parser/ast/misc.go index 048b0d9c65b52..7a0e2fc7a1a50 100644 --- a/parser/ast/misc.go +++ b/parser/ast/misc.go @@ -3768,7 +3768,7 @@ func (n *TableOptimizerHint) Restore(ctx *format.RestoreCtx) error { } table.Restore(ctx) } - case "use_index", "ignore_index", "use_index_merge", "force_index": + case "use_index", "ignore_index", "use_index_merge", "force_index", "keep_order", "no_keep_order": n.Tables[0].Restore(ctx) ctx.WritePlain(" ") for i, index := range n.Indexes { diff --git a/parser/ast/misc_test.go b/parser/ast/misc_test.go index 8042a171874d0..36e8cb1ef002f 100644 --- a/parser/ast/misc_test.go +++ b/parser/ast/misc_test.go @@ -228,6 +228,18 @@ func TestTableOptimizerHintRestore(t *testing.T) { {"IGNORE_INDEX(@sel_1 t1 c1)", "IGNORE_INDEX(@`sel_1` `t1` `c1`)"}, {"IGNORE_INDEX(t1@sel_1 c1)", "IGNORE_INDEX(`t1`@`sel_1` `c1`)"}, {"IGNORE_INDEX(t1@sel_1 partition(p0, p1) c1)", "IGNORE_INDEX(`t1`@`sel_1` PARTITION(`p0`, `p1`) `c1`)"}, + {"KEEP_ORDER(t1 c1)", "KEEP_ORDER(`t1` `c1`)"}, + {"KEEP_ORDER(test.t1 c1)", "KEEP_ORDER(`test`.`t1` `c1`)"}, + {"KEEP_ORDER(@sel_1 t1 c1)", "KEEP_ORDER(@`sel_1` `t1` `c1`)"}, + {"KEEP_ORDER(t1@sel_1 c1)", "KEEP_ORDER(`t1`@`sel_1` `c1`)"}, + {"KEEP_ORDER(test.t1@sel_1 c1)", "KEEP_ORDER(`test`.`t1`@`sel_1` `c1`)"}, + {"KEEP_ORDER(test.t1@sel_1 partition(p0) c1)", "KEEP_ORDER(`test`.`t1`@`sel_1` PARTITION(`p0`) `c1`)"}, + {"NO_KEEP_ORDER(t1 c1)", "NO_KEEP_ORDER(`t1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1 c1)", "NO_KEEP_ORDER(`test`.`t1` `c1`)"}, + {"NO_KEEP_ORDER(@sel_1 t1 c1)", "NO_KEEP_ORDER(@`sel_1` `t1` `c1`)"}, + {"NO_KEEP_ORDER(t1@sel_1 c1)", "NO_KEEP_ORDER(`t1`@`sel_1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1@sel_1 c1)", "NO_KEEP_ORDER(`test`.`t1`@`sel_1` `c1`)"}, + {"NO_KEEP_ORDER(test.t1@sel_1 partition(p0) c1)", "NO_KEEP_ORDER(`test`.`t1`@`sel_1` PARTITION(`p0`) `c1`)"}, {"TIDB_SMJ(`t1`)", "TIDB_SMJ(`t1`)"}, {"TIDB_SMJ(t1)", "TIDB_SMJ(`t1`)"}, {"TIDB_SMJ(t1,t2)", "TIDB_SMJ(`t1`, `t2`)"}, diff --git a/parser/hintparser.go b/parser/hintparser.go index 1b14e7d292e1e..a619d2a1c74f2 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -41,18 +41,18 @@ type yyhintXError struct { } const ( - yyhintDefault = 57424 + yyhintDefault = 57426 yyhintEOFCode = 57344 yyhintErrCode = 57345 hintAggToCop = 57379 hintBCJoin = 57394 hintBKA = 57355 hintBNL = 57357 - hintDupsWeedOut = 57420 - hintFalse = 57416 - hintFirstMatch = 57421 - hintForceIndex = 57406 - hintGB = 57419 + hintDupsWeedOut = 57422 + hintFalse = 57418 + hintFirstMatch = 57423 + hintForceIndex = 57408 + hintGB = 57421 hintHashAgg = 57381 hintHashJoin = 57359 hintHashJoinBuild = 57360 @@ -70,12 +70,13 @@ const ( hintJoinOrder = 57352 hintJoinPrefix = 57353 hintJoinSuffix = 57354 - hintLeading = 57408 - hintLimitToCop = 57405 - hintLooseScan = 57422 - hintMB = 57418 + hintKeepOrder = 57400 + hintLeading = 57410 + hintLimitToCop = 57407 + hintLooseScan = 57424 + hintMB = 57420 hintMRR = 57367 - hintMaterialization = 57423 + hintMaterialization = 57425 hintMaxExecutionTime = 57375 hintMemoryQuota = 57388 hintMerge = 57363 @@ -83,175 +84,178 @@ const ( hintMpp2PhaseAgg = 57383 hintNoBKA = 57356 hintNoBNL = 57358 - hintNoDecorrelate = 57410 + hintNoDecorrelate = 57412 hintNoHashJoin = 57362 hintNoICP = 57369 hintNoIndexMerge = 57366 + hintNoKeepOrder = 57401 hintNoMRR = 57368 hintNoMerge = 57364 hintNoRangeOptimization = 57370 hintNoSemijoin = 57374 hintNoSkipScan = 57372 hintNoSwapJoinInputs = 57389 - hintNthPlan = 57404 - hintOLAP = 57411 - hintOLTP = 57412 - hintPartition = 57413 + hintNthPlan = 57406 + hintOLAP = 57413 + hintOLTP = 57414 + hintPartition = 57415 hintQBName = 57378 hintQueryType = 57390 hintReadConsistentReplica = 57391 hintReadFromStorage = 57392 hintResourceGroup = 57377 hintSMJoin = 57393 - hintSemiJoinRewrite = 57409 + hintSemiJoinRewrite = 57411 hintSemijoin = 57373 hintSetVar = 57376 hintShuffleJoin = 57395 hintSingleAtIdentifier = 57349 hintSkipScan = 57371 - hintStraightJoin = 57407 + hintStraightJoin = 57409 hintStreamAgg = 57396 hintStringLit = 57350 hintSwapJoinInputs = 57397 - hintTiFlash = 57415 - hintTiKV = 57414 - hintTimeRange = 57402 - hintTrue = 57417 - hintUseCascades = 57403 + hintTiFlash = 57417 + hintTiKV = 57416 + hintTimeRange = 57404 + hintTrue = 57419 + hintUseCascades = 57405 hintUseIndex = 57399 hintUseIndexMerge = 57398 - hintUsePlanCache = 57400 - hintUseToja = 57401 + hintUsePlanCache = 57402 + hintUseToja = 57403 yyhintMaxDepth = 200 - yyhintTabOfs = -193 + yyhintTabOfs = -197 ) var ( yyhintXLAT = map[int]int{ - 41: 0, // ')' (146x) - 57379: 1, // hintAggToCop (133x) - 57394: 2, // hintBCJoin (133x) - 57355: 3, // hintBKA (133x) - 57357: 4, // hintBNL (133x) - 57406: 5, // hintForceIndex (133x) - 57381: 6, // hintHashAgg (133x) - 57359: 7, // hintHashJoin (133x) - 57360: 8, // hintHashJoinBuild (133x) - 57361: 9, // hintHashJoinProbe (133x) - 57384: 10, // hintIgnoreIndex (133x) - 57380: 11, // hintIgnorePlanCache (133x) - 57365: 12, // hintIndexMerge (133x) - 57385: 13, // hintInlHashJoin (133x) - 57386: 14, // hintInlJoin (133x) - 57387: 15, // hintInlMergeJoin (133x) - 57351: 16, // hintJoinFixedOrder (133x) - 57352: 17, // hintJoinOrder (133x) - 57353: 18, // hintJoinPrefix (133x) - 57354: 19, // hintJoinSuffix (133x) - 57408: 20, // hintLeading (133x) - 57405: 21, // hintLimitToCop (133x) - 57375: 22, // hintMaxExecutionTime (133x) - 57388: 23, // hintMemoryQuota (133x) - 57363: 24, // hintMerge (133x) - 57382: 25, // hintMpp1PhaseAgg (133x) - 57383: 26, // hintMpp2PhaseAgg (133x) - 57367: 27, // hintMRR (133x) - 57356: 28, // hintNoBKA (133x) - 57358: 29, // hintNoBNL (133x) - 57410: 30, // hintNoDecorrelate (133x) - 57362: 31, // hintNoHashJoin (133x) - 57369: 32, // hintNoICP (133x) - 57366: 33, // hintNoIndexMerge (133x) - 57364: 34, // hintNoMerge (133x) - 57368: 35, // hintNoMRR (133x) - 57370: 36, // hintNoRangeOptimization (133x) - 57374: 37, // hintNoSemijoin (133x) - 57372: 38, // hintNoSkipScan (133x) - 57389: 39, // hintNoSwapJoinInputs (133x) - 57404: 40, // hintNthPlan (133x) - 57378: 41, // hintQBName (133x) - 57390: 42, // hintQueryType (133x) - 57391: 43, // hintReadConsistentReplica (133x) - 57392: 44, // hintReadFromStorage (133x) - 57377: 45, // hintResourceGroup (133x) - 57373: 46, // hintSemijoin (133x) - 57409: 47, // hintSemiJoinRewrite (133x) - 57376: 48, // hintSetVar (133x) - 57395: 49, // hintShuffleJoin (133x) - 57371: 50, // hintSkipScan (133x) - 57393: 51, // hintSMJoin (133x) - 57407: 52, // hintStraightJoin (133x) - 57396: 53, // hintStreamAgg (133x) - 57397: 54, // hintSwapJoinInputs (133x) - 57402: 55, // hintTimeRange (133x) - 57403: 56, // hintUseCascades (133x) - 57399: 57, // hintUseIndex (133x) - 57398: 58, // hintUseIndexMerge (133x) - 57400: 59, // hintUsePlanCache (133x) - 57401: 60, // hintUseToja (133x) - 44: 61, // ',' (130x) - 57420: 62, // hintDupsWeedOut (110x) - 57421: 63, // hintFirstMatch (110x) - 57422: 64, // hintLooseScan (110x) - 57423: 65, // hintMaterialization (110x) - 57415: 66, // hintTiFlash (110x) - 57414: 67, // hintTiKV (110x) - 57416: 68, // hintFalse (109x) - 57411: 69, // hintOLAP (109x) - 57412: 70, // hintOLTP (109x) - 57417: 71, // hintTrue (109x) - 57419: 72, // hintGB (108x) - 57418: 73, // hintMB (108x) - 57347: 74, // hintIdentifier (107x) - 57349: 75, // hintSingleAtIdentifier (93x) - 93: 76, // ']' (84x) - 46: 77, // '.' (83x) - 57413: 78, // hintPartition (78x) - 61: 79, // '=' (74x) - 40: 80, // '(' (69x) - 57344: 81, // $end (25x) - 57444: 82, // QueryBlockOpt (20x) - 57436: 83, // Identifier (15x) - 57346: 84, // hintIntLit (8x) - 57350: 85, // hintStringLit (5x) - 57426: 86, // CommaOpt (4x) - 57432: 87, // HintTable (4x) - 57433: 88, // HintTableList (4x) - 91: 89, // '[' (3x) - 57425: 90, // BooleanHintName (2x) - 57427: 91, // HintIndexList (2x) - 57429: 92, // HintStorageType (2x) - 57430: 93, // HintStorageTypeAndTable (2x) - 57434: 94, // HintTableListOpt (2x) - 57439: 95, // JoinOrderOptimizerHintName (2x) - 57440: 96, // NullaryHintName (2x) - 57443: 97, // PartitionListOpt (2x) - 57446: 98, // StorageOptimizerHintOpt (2x) - 57447: 99, // SubqueryOptimizerHintName (2x) - 57450: 100, // SubqueryStrategy (2x) - 57451: 101, // SupportedIndexLevelOptimizerHintName (2x) - 57452: 102, // SupportedTableLevelOptimizerHintName (2x) - 57453: 103, // TableOptimizerHintOpt (2x) - 57455: 104, // UnsupportedIndexLevelOptimizerHintName (2x) - 57456: 105, // UnsupportedTableLevelOptimizerHintName (2x) - 57458: 106, // ViewName (2x) - 57428: 107, // HintQueryType (1x) - 57431: 108, // HintStorageTypeAndTableList (1x) - 57435: 109, // HintTrueOrFalse (1x) - 57437: 110, // IndexNameList (1x) - 57438: 111, // IndexNameListOpt (1x) - 57441: 112, // OptimizerHintList (1x) - 57442: 113, // PartitionList (1x) - 57445: 114, // Start (1x) - 57448: 115, // SubqueryStrategies (1x) - 57449: 116, // SubqueryStrategiesOpt (1x) - 57454: 117, // UnitOfBytes (1x) - 57457: 118, // Value (1x) - 57459: 119, // ViewNameList (1x) - 57424: 120, // $default (0x) - 57345: 121, // error (0x) - 57348: 122, // hintInvalid (0x) + 41: 0, // ')' (148x) + 57379: 1, // hintAggToCop (135x) + 57394: 2, // hintBCJoin (135x) + 57355: 3, // hintBKA (135x) + 57357: 4, // hintBNL (135x) + 57408: 5, // hintForceIndex (135x) + 57381: 6, // hintHashAgg (135x) + 57359: 7, // hintHashJoin (135x) + 57360: 8, // hintHashJoinBuild (135x) + 57361: 9, // hintHashJoinProbe (135x) + 57384: 10, // hintIgnoreIndex (135x) + 57380: 11, // hintIgnorePlanCache (135x) + 57365: 12, // hintIndexMerge (135x) + 57385: 13, // hintInlHashJoin (135x) + 57386: 14, // hintInlJoin (135x) + 57387: 15, // hintInlMergeJoin (135x) + 57351: 16, // hintJoinFixedOrder (135x) + 57352: 17, // hintJoinOrder (135x) + 57353: 18, // hintJoinPrefix (135x) + 57354: 19, // hintJoinSuffix (135x) + 57400: 20, // hintKeepOrder (135x) + 57410: 21, // hintLeading (135x) + 57407: 22, // hintLimitToCop (135x) + 57375: 23, // hintMaxExecutionTime (135x) + 57388: 24, // hintMemoryQuota (135x) + 57363: 25, // hintMerge (135x) + 57382: 26, // hintMpp1PhaseAgg (135x) + 57383: 27, // hintMpp2PhaseAgg (135x) + 57367: 28, // hintMRR (135x) + 57356: 29, // hintNoBKA (135x) + 57358: 30, // hintNoBNL (135x) + 57412: 31, // hintNoDecorrelate (135x) + 57362: 32, // hintNoHashJoin (135x) + 57369: 33, // hintNoICP (135x) + 57366: 34, // hintNoIndexMerge (135x) + 57401: 35, // hintNoKeepOrder (135x) + 57364: 36, // hintNoMerge (135x) + 57368: 37, // hintNoMRR (135x) + 57370: 38, // hintNoRangeOptimization (135x) + 57374: 39, // hintNoSemijoin (135x) + 57372: 40, // hintNoSkipScan (135x) + 57389: 41, // hintNoSwapJoinInputs (135x) + 57406: 42, // hintNthPlan (135x) + 57378: 43, // hintQBName (135x) + 57390: 44, // hintQueryType (135x) + 57391: 45, // hintReadConsistentReplica (135x) + 57392: 46, // hintReadFromStorage (135x) + 57377: 47, // hintResourceGroup (135x) + 57373: 48, // hintSemijoin (135x) + 57411: 49, // hintSemiJoinRewrite (135x) + 57376: 50, // hintSetVar (135x) + 57395: 51, // hintShuffleJoin (135x) + 57371: 52, // hintSkipScan (135x) + 57393: 53, // hintSMJoin (135x) + 57409: 54, // hintStraightJoin (135x) + 57396: 55, // hintStreamAgg (135x) + 57397: 56, // hintSwapJoinInputs (135x) + 57404: 57, // hintTimeRange (135x) + 57405: 58, // hintUseCascades (135x) + 57399: 59, // hintUseIndex (135x) + 57398: 60, // hintUseIndexMerge (135x) + 57402: 61, // hintUsePlanCache (135x) + 57403: 62, // hintUseToja (135x) + 44: 63, // ',' (132x) + 57422: 64, // hintDupsWeedOut (112x) + 57423: 65, // hintFirstMatch (112x) + 57424: 66, // hintLooseScan (112x) + 57425: 67, // hintMaterialization (112x) + 57417: 68, // hintTiFlash (112x) + 57416: 69, // hintTiKV (112x) + 57418: 70, // hintFalse (111x) + 57413: 71, // hintOLAP (111x) + 57414: 72, // hintOLTP (111x) + 57419: 73, // hintTrue (111x) + 57421: 74, // hintGB (110x) + 57420: 75, // hintMB (110x) + 57347: 76, // hintIdentifier (109x) + 57349: 77, // hintSingleAtIdentifier (95x) + 93: 78, // ']' (86x) + 46: 79, // '.' (85x) + 57415: 80, // hintPartition (80x) + 61: 81, // '=' (76x) + 40: 82, // '(' (71x) + 57344: 83, // $end (25x) + 57446: 84, // QueryBlockOpt (20x) + 57438: 85, // Identifier (15x) + 57346: 86, // hintIntLit (8x) + 57350: 87, // hintStringLit (5x) + 57428: 88, // CommaOpt (4x) + 57434: 89, // HintTable (4x) + 57435: 90, // HintTableList (4x) + 91: 91, // '[' (3x) + 57427: 92, // BooleanHintName (2x) + 57429: 93, // HintIndexList (2x) + 57431: 94, // HintStorageType (2x) + 57432: 95, // HintStorageTypeAndTable (2x) + 57436: 96, // HintTableListOpt (2x) + 57441: 97, // JoinOrderOptimizerHintName (2x) + 57442: 98, // NullaryHintName (2x) + 57445: 99, // PartitionListOpt (2x) + 57448: 100, // StorageOptimizerHintOpt (2x) + 57449: 101, // SubqueryOptimizerHintName (2x) + 57452: 102, // SubqueryStrategy (2x) + 57453: 103, // SupportedIndexLevelOptimizerHintName (2x) + 57454: 104, // SupportedTableLevelOptimizerHintName (2x) + 57455: 105, // TableOptimizerHintOpt (2x) + 57457: 106, // UnsupportedIndexLevelOptimizerHintName (2x) + 57458: 107, // UnsupportedTableLevelOptimizerHintName (2x) + 57460: 108, // ViewName (2x) + 57430: 109, // HintQueryType (1x) + 57433: 110, // HintStorageTypeAndTableList (1x) + 57437: 111, // HintTrueOrFalse (1x) + 57439: 112, // IndexNameList (1x) + 57440: 113, // IndexNameListOpt (1x) + 57443: 114, // OptimizerHintList (1x) + 57444: 115, // PartitionList (1x) + 57447: 116, // Start (1x) + 57450: 117, // SubqueryStrategies (1x) + 57451: 118, // SubqueryStrategiesOpt (1x) + 57456: 119, // UnitOfBytes (1x) + 57459: 120, // Value (1x) + 57461: 121, // ViewNameList (1x) + 57426: 122, // $default (0x) + 57345: 123, // error (0x) + 57348: 124, // hintInvalid (0x) } yyhintSymNames = []string{ @@ -275,6 +279,7 @@ var ( "hintJoinOrder", "hintJoinPrefix", "hintJoinSuffix", + "hintKeepOrder", "hintLeading", "hintLimitToCop", "hintMaxExecutionTime", @@ -289,6 +294,7 @@ var ( "hintNoHashJoin", "hintNoICP", "hintNoIndexMerge", + "hintNoKeepOrder", "hintNoMerge", "hintNoMRR", "hintNoRangeOptimization", @@ -382,89 +388,76 @@ var ( yyhintReductions = []struct{ xsym, components int }{ {0, 1}, + {116, 1}, {114, 1}, - {112, 1}, - {112, 3}, - {112, 1}, - {112, 3}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 4}, - {103, 5}, - {103, 5}, - {103, 5}, - {103, 6}, - {103, 4}, - {103, 4}, - {103, 6}, - {103, 6}, - {103, 6}, - {103, 5}, - {103, 4}, - {103, 5}, - {98, 5}, - {108, 1}, - {108, 3}, - {93, 4}, - {82, 0}, - {82, 1}, - {86, 0}, - {86, 1}, - {97, 0}, - {97, 4}, - {113, 1}, - {113, 3}, - {94, 1}, - {94, 1}, - {88, 2}, - {88, 3}, - {87, 3}, - {87, 5}, - {119, 3}, - {119, 1}, - {106, 2}, - {106, 1}, - {91, 4}, - {111, 0}, - {111, 1}, + {114, 3}, + {114, 1}, + {114, 3}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 4}, + {105, 5}, + {105, 5}, + {105, 5}, + {105, 6}, + {105, 4}, + {105, 4}, + {105, 6}, + {105, 6}, + {105, 6}, + {105, 5}, + {105, 4}, + {105, 5}, + {100, 5}, {110, 1}, {110, 3}, - {116, 0}, - {116, 1}, + {95, 4}, + {84, 0}, + {84, 1}, + {88, 0}, + {88, 1}, + {99, 0}, + {99, 4}, {115, 1}, {115, 3}, + {96, 1}, + {96, 1}, + {90, 2}, + {90, 3}, + {89, 3}, + {89, 5}, + {121, 3}, + {121, 1}, + {108, 2}, + {108, 1}, + {93, 4}, + {113, 0}, + {113, 1}, + {112, 1}, + {112, 3}, + {118, 0}, {118, 1}, - {118, 1}, - {118, 1}, - {117, 1}, {117, 1}, - {109, 1}, - {109, 1}, - {95, 1}, - {95, 1}, - {95, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {105, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, - {102, 1}, + {117, 3}, + {120, 1}, + {120, 1}, + {120, 1}, + {119, 1}, + {119, 1}, + {111, 1}, + {111, 1}, + {97, 1}, + {97, 1}, + {97, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, + {107, 1}, {104, 1}, {104, 1}, {104, 1}, @@ -472,449 +465,471 @@ var ( {104, 1}, {104, 1}, {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {104, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {106, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, + {103, 1}, {101, 1}, {101, 1}, - {101, 1}, - {101, 1}, - {99, 1}, - {99, 1}, - {100, 1}, - {100, 1}, - {100, 1}, - {100, 1}, - {90, 1}, - {90, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {96, 1}, - {107, 1}, - {107, 1}, + {102, 1}, + {102, 1}, + {102, 1}, + {102, 1}, {92, 1}, {92, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, - {83, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {98, 1}, + {109, 1}, + {109, 1}, + {94, 1}, + {94, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, + {85, 1}, } yyhintXErrors = map[yyhintXError]string{} - yyhintParseTab = [280][]uint16{ + yyhintParseTab = [284][]uint16{ // 0 - {1: 258, 226, 219, 221, 248, 254, 234, 235, 236, 246, 262, 238, 230, 228, 233, 198, 216, 217, 218, 237, 259, 205, 210, 229, 255, 256, 239, 220, 222, 265, 223, 241, 260, 224, 240, 242, 250, 244, 232, 206, 209, 214, 261, 215, 208, 249, 264, 207, 227, 243, 225, 263, 257, 231, 211, 252, 245, 247, 253, 251, 90: 212, 95: 199, 213, 98: 197, 204, 101: 203, 201, 196, 202, 200, 112: 195, 114: 194}, - {81: 193}, - {1: 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 81: 192, 86: 470}, - {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 81: 191}, - {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 81: 189}, + {1: 264, 230, 223, 225, 252, 260, 238, 239, 240, 250, 268, 242, 234, 232, 237, 202, 220, 221, 222, 253, 241, 265, 209, 214, 233, 261, 262, 243, 224, 226, 271, 227, 245, 266, 254, 228, 244, 246, 256, 248, 236, 210, 213, 218, 267, 219, 212, 255, 270, 211, 231, 247, 229, 269, 263, 235, 215, 258, 249, 251, 259, 257, 92: 216, 97: 203, 217, 100: 201, 208, 103: 207, 205, 200, 206, 204, 114: 199, 116: 198}, + {83: 197}, + {1: 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 83: 196, 88: 478}, + {1: 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 195, 83: 195}, + {1: 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 193, 83: 193}, // 5 - {80: 467}, - {80: 464}, - {80: 461}, - {80: 456}, - {80: 453}, + {82: 475}, + {82: 472}, + {82: 469}, + {82: 464}, + {82: 461}, // 10 - {80: 442}, - {80: 430}, - {80: 426}, - {80: 422}, - {80: 414}, + {82: 450}, + {82: 438}, + {82: 434}, + {82: 430}, + {82: 422}, // 15 - {80: 411}, - {80: 399}, - {80: 392}, - {80: 387}, - {80: 381}, + {82: 419}, + {82: 407}, + {82: 400}, + {82: 395}, + {82: 389}, // 20 - {80: 378}, - {80: 372}, - {80: 266}, - {80: 131}, - {80: 130}, + {82: 386}, + {82: 380}, + {82: 272}, + {82: 135}, + {82: 134}, // 25 - {80: 129}, - {80: 128}, - {80: 127}, - {80: 126}, - {80: 125}, + {82: 133}, + {82: 132}, + {82: 131}, + {82: 130}, + {82: 129}, // 30 - {80: 124}, - {80: 123}, - {80: 122}, - {80: 121}, - {80: 120}, + {82: 128}, + {82: 127}, + {82: 126}, + {82: 125}, + {82: 124}, // 35 - {80: 119}, - {80: 118}, - {80: 117}, - {80: 116}, - {80: 115}, + {82: 123}, + {82: 122}, + {82: 121}, + {82: 120}, + {82: 119}, // 40 - {80: 114}, - {80: 113}, - {80: 112}, - {80: 111}, - {80: 110}, + {82: 118}, + {82: 117}, + {82: 116}, + {82: 115}, + {82: 114}, // 45 - {80: 109}, - {80: 108}, - {80: 107}, - {80: 106}, - {80: 105}, + {82: 113}, + {82: 112}, + {82: 111}, + {82: 110}, + {82: 109}, // 50 - {80: 104}, - {80: 103}, - {80: 102}, - {80: 101}, - {80: 100}, + {82: 108}, + {82: 107}, + {82: 106}, + {82: 105}, + {82: 104}, // 55 - {80: 99}, - {80: 98}, - {80: 97}, - {80: 92}, - {80: 91}, + {82: 103}, + {82: 102}, + {82: 101}, + {82: 100}, + {82: 99}, // 60 - {80: 90}, - {80: 89}, - {80: 88}, - {80: 87}, - {80: 86}, + {82: 94}, + {82: 93}, + {82: 92}, + {82: 91}, + {82: 90}, // 65 - {80: 85}, - {80: 84}, - {80: 83}, - {80: 82}, - {80: 81}, + {82: 89}, + {82: 88}, + {82: 87}, + {82: 86}, + {82: 85}, // 70 - {80: 80}, - {80: 79}, - {80: 78}, - {66: 165, 165, 75: 268, 82: 267}, - {66: 273, 272, 92: 271, 270, 108: 269}, + {82: 84}, + {82: 83}, + {82: 82}, + {82: 81}, + {82: 80}, // 75 - {164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 76: 164, 164, 164, 84: 164}, - {369, 61: 370}, - {168, 61: 168}, - {89: 274}, - {89: 75}, + {68: 169, 169, 77: 274, 84: 273}, + {68: 279, 278, 94: 277, 276, 110: 275}, + {168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 168, 78: 168, 168, 168, 86: 168}, + {377, 63: 378}, + {172, 63: 172}, // 80 - {89: 74}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 276, 88: 275}, - {61: 367, 76: 366}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 277}, - {155, 61: 155, 76: 155}, + {91: 280}, + {91: 77}, + {91: 76}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 282, 90: 281}, + {63: 375, 78: 374}, // 85 - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 165, 353, 165, 82: 352}, - {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, - {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, - {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, - {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 283}, + {159, 63: 159, 78: 159}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 169, 361, 169, 84: 360}, + {75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75, 75}, + {74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, 74}, // 90 - {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, - {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, - {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, - {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, - {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, + {73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73, 73}, + {72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72, 72}, + {71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71, 71}, + {70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70, 70}, + {69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69, 69}, // 95 - {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, - {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, - {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, - {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, - {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, + {68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68, 68}, + {67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67, 67}, + {66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66, 66}, + {65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65, 65}, + {64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64, 64}, // 100 - {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, - {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, - {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, - {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, - {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, + {63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63, 63}, + {62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, 62}, + {61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61, 61}, + {60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, 60}, + {59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59, 59}, // 105 - {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, - {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, - {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, - {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, - {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, + {58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58, 58}, + {57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57, 57}, + {56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56, 56}, + {55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55, 55}, + {54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54, 54}, // 110 - {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, - {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, - {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, - {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, - {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, + {53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53, 53}, + {52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52, 52}, + {51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51, 51}, + {50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50, 50}, + {49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49, 49}, // 115 - {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, - {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, - {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, - {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, - {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, + {48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48, 48}, + {47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47, 47}, + {46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46, 46}, + {45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45, 45}, + {44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44, 44}, // 120 - {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, - {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, - {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, - {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, - {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, + {43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43, 43}, + {42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42, 42}, + {41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41, 41}, + {40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40, 40}, + {39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39, 39}, // 125 - {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, - {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, - {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, - {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, - {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, + {38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38, 38}, + {37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37, 37}, + {36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36, 36}, + {35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35, 35}, + {34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34, 34}, // 130 - {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, - {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, - {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, - {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, - {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, + {33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33, 33}, + {32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32, 32}, + {31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31, 31}, + {30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30}, + {29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29, 29}, // 135 - {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, - {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, - {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, - {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, - {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, + {28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28, 28}, + {27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27, 27}, + {26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26, 26}, + {25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25, 25}, + {24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24, 24}, // 140 - {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, - {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, - {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, - {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, - {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, + {23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23, 23}, + {22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22, 22}, + {21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21, 21}, + {20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20, 20}, + {19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19, 19}, // 145 - {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, - {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, - {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, - {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, - {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, + {18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18, 18}, + {17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17, 17}, + {16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16, 16}, + {15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15, 15}, + {14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14, 14}, // 150 - {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, - {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, - {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, - {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, - {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, + {13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13, 13}, + {12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12, 12}, + {11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11, 11}, + {10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10, 10}, + {9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9, 9}, // 155 - {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, - {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, - {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, - {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, - {161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 76: 161, 78: 356, 97: 365}, + {8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8, 8}, + {7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7}, + {6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6, 6}, + {5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5}, + {4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4}, // 160 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 354}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 165, 78: 165, 82: 355}, - {161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 161, 76: 161, 78: 356, 97: 357}, - {80: 358}, - {152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 152, 76: 152}, + {3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3}, + {2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2}, + {1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1}, + {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 78: 165, 80: 364, 99: 373}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 362}, // 165 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 360, 113: 359}, - {362, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 86: 363}, - {159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159, 159}, - {162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 62: 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 85: 162}, - {160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 160, 76: 160}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 169, 80: 169, 84: 363}, + {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 78: 165, 80: 364, 99: 365}, + {82: 366}, + {156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 156, 78: 156}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 368, 115: 367}, // 170 - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 364}, - {158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158, 158}, - {153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 153, 76: 153}, - {166, 61: 166}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 368}, + {370, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 88: 371}, + {163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163}, + {166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 64: 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 166, 87: 166}, + {164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 164, 78: 164}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 372}, // 175 - {154, 61: 154, 76: 154}, - {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 81: 169}, - {66: 273, 272, 92: 271, 371}, - {167, 61: 167}, - {69: 165, 165, 75: 268, 82: 373}, + {162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162, 162}, + {157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 157, 78: 157}, + {170, 63: 170}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 376}, + {158, 63: 158, 78: 158}, // 180 - {69: 375, 376, 107: 374}, - {377}, - {77}, - {76}, - {1: 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 170, 81: 170}, + {1: 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 83: 173}, + {68: 279, 278, 94: 277, 379}, + {171, 63: 171}, + {71: 169, 169, 77: 274, 84: 381}, + {71: 383, 384, 109: 382}, // 185 - {165, 75: 268, 82: 379}, - {380}, - {1: 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 171, 81: 171}, - {68: 165, 71: 165, 75: 268, 82: 382}, - {68: 385, 71: 384, 109: 383}, + {385}, + {79}, + {78}, + {1: 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 83: 174}, + {169, 77: 274, 84: 387}, // 190 - {386}, - {133}, - {132}, - {1: 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 172, 81: 172}, - {85: 388}, + {388}, + {1: 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 83: 175}, + {70: 169, 73: 169, 77: 274, 84: 390}, + {70: 393, 73: 392, 111: 391}, + {394}, // 195 - {61: 361, 85: 163, 389}, - {85: 390}, - {391}, - {1: 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 173, 81: 173}, - {75: 268, 82: 393, 84: 165}, + {137}, + {136}, + {1: 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 83: 176}, + {87: 396}, + {63: 369, 87: 167, 397}, // 200 - {84: 394}, - {72: 397, 396, 117: 395}, - {398}, - {135}, - {134}, + {87: 398}, + {399}, + {1: 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 83: 177}, + {77: 274, 84: 401, 86: 169}, + {86: 402}, // 205 - {1: 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 174, 81: 174}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 400}, - {401, 61: 402}, - {1: 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 176, 81: 176}, - {165, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 268, 77: 165, 82: 406, 405, 106: 404, 119: 403}, + {74: 405, 404, 119: 403}, + {406}, + {139}, + {138}, + {1: 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 83: 178}, // 210 - {408, 77: 409}, - {150, 77: 150}, - {165, 75: 268, 77: 165, 82: 407}, - {148, 77: 148}, - {149, 77: 149}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 408}, + {409, 63: 410}, + {1: 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 83: 180}, + {169, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 274, 79: 169, 84: 414, 413, 108: 412, 121: 411}, + {416, 79: 417}, // 215 - {1: 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 175, 81: 175}, - {165, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 268, 77: 165, 82: 406, 405, 106: 410}, - {151, 77: 151}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 412}, - {413}, + {154, 79: 154}, + {169, 77: 274, 79: 169, 84: 415}, + {152, 79: 152}, + {153, 79: 153}, + {1: 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 83: 179}, // 220 - {1: 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 177, 81: 177}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 415}, - {79: 416}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 419, 420, 418, 118: 417}, + {169, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 274, 79: 169, 84: 414, 413, 108: 418}, + {155, 79: 155}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 420}, {421}, + {1: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 83: 181}, // 225 - {138}, - {137}, - {136}, - {1: 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 178, 81: 178}, - {75: 268, 82: 423, 84: 165}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 423}, + {81: 424}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 427, 428, 426, 120: 425}, + {429}, + {142}, // 230 - {84: 424}, - {425}, - {1: 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 179, 81: 179}, - {75: 268, 82: 427, 84: 165}, - {84: 428}, + {141}, + {140}, + {1: 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 83: 182}, + {77: 274, 84: 431, 86: 169}, + {86: 432}, // 235 - {429}, - {1: 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 180, 81: 180}, - {165, 62: 165, 165, 165, 165, 75: 268, 82: 431}, - {142, 62: 435, 436, 437, 438, 100: 434, 115: 433, 432}, - {441}, + {433}, + {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 83: 183}, + {77: 274, 84: 435, 86: 169}, + {86: 436}, + {437}, // 240 - {141, 61: 439}, - {140, 61: 140}, - {96, 61: 96}, - {95, 61: 95}, - {94, 61: 94}, + {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 83: 184}, + {169, 64: 169, 169, 169, 169, 77: 274, 84: 439}, + {146, 64: 443, 444, 445, 446, 102: 442, 117: 441, 440}, + {449}, + {145, 63: 447}, // 245 - {93, 61: 93}, - {62: 435, 436, 437, 438, 100: 440}, - {139, 61: 139}, - {1: 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 181, 81: 181}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 444, 91: 443}, + {144, 63: 144}, + {98, 63: 98}, + {97, 63: 97}, + {96, 63: 96}, + {95, 63: 95}, // 250 - {452}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 445}, - {163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 361, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 163, 86: 446}, - {146, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 449, 110: 448, 447}, - {147}, + {64: 443, 444, 445, 446, 102: 448}, + {143, 63: 143}, + {1: 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 83: 185}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 452, 93: 451}, + {460}, // 255 - {145, 61: 450}, - {144, 61: 144}, - {1: 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 451}, - {143, 61: 143}, - {1: 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 182, 81: 182}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 453}, + {167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 369, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 167, 88: 454}, + {150, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 457, 112: 456, 455}, + {151}, + {149, 63: 458}, // 260 - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 444, 91: 454}, - {455}, - {1: 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 183, 81: 183}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 459, 88: 458, 94: 457}, - {460}, + {148, 63: 148}, + {1: 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 459}, + {147, 63: 147}, + {1: 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 83: 186}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 452, 93: 462}, // 265 - {157, 61: 367}, - {156, 308, 324, 284, 286, 335, 311, 288, 289, 290, 314, 310, 294, 315, 316, 317, 280, 281, 282, 283, 337, 309, 304, 318, 292, 312, 313, 296, 285, 287, 339, 291, 298, 295, 293, 297, 299, 303, 301, 319, 334, 307, 320, 321, 322, 306, 302, 338, 305, 325, 300, 323, 336, 326, 327, 332, 333, 329, 328, 330, 331, 62: 348, 349, 350, 351, 343, 342, 344, 340, 341, 345, 347, 346, 279, 83: 278, 87: 277}, - {1: 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 184, 81: 184}, - {165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 459, 88: 458, 94: 462}, {463}, + {1: 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 83: 187}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 467, 90: 466, 96: 465}, + {468}, + {161, 63: 375}, // 270 - {1: 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 185, 81: 185}, - {1: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 62: 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 165, 268, 82: 276, 88: 465}, - {466, 61: 367}, - {1: 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 186, 81: 186}, - {165, 75: 268, 82: 468}, + {160, 314, 330, 290, 292, 343, 317, 294, 295, 296, 320, 316, 300, 321, 322, 323, 286, 287, 288, 289, 336, 345, 315, 310, 324, 298, 318, 319, 302, 291, 293, 347, 297, 304, 301, 337, 299, 303, 305, 309, 307, 325, 342, 313, 326, 327, 328, 312, 308, 346, 311, 331, 306, 329, 344, 332, 333, 340, 341, 335, 334, 338, 339, 64: 356, 357, 358, 359, 351, 350, 352, 348, 349, 353, 355, 354, 285, 85: 284, 89: 283}, + {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 83: 188}, + {169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 467, 90: 466, 96: 470}, + {471}, + {1: 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 189, 83: 189}, // 275 - {469}, - {1: 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 187, 81: 187}, - {1: 258, 226, 219, 221, 248, 254, 234, 235, 236, 246, 262, 238, 230, 228, 233, 198, 216, 217, 218, 237, 259, 205, 210, 229, 255, 256, 239, 220, 222, 265, 223, 241, 260, 224, 240, 242, 250, 244, 232, 206, 209, 214, 261, 215, 208, 249, 264, 207, 227, 243, 225, 263, 257, 231, 211, 252, 245, 247, 253, 251, 90: 212, 95: 199, 213, 98: 472, 204, 101: 203, 201, 471, 202, 200}, - {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 81: 190}, - {1: 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 188, 81: 188}, + {1: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 64: 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 169, 274, 84: 282, 90: 473}, + {474, 63: 375}, + {1: 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 190, 83: 190}, + {169, 77: 274, 84: 476}, + {477}, + // 280 + {1: 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 191, 83: 191}, + {1: 264, 230, 223, 225, 252, 260, 238, 239, 240, 250, 268, 242, 234, 232, 237, 202, 220, 221, 222, 253, 241, 265, 209, 214, 233, 261, 262, 243, 224, 226, 271, 227, 245, 266, 254, 228, 244, 246, 256, 248, 236, 210, 213, 218, 267, 219, 212, 255, 270, 211, 231, 247, 229, 269, 263, 235, 215, 258, 249, 251, 259, 257, 92: 216, 97: 203, 217, 100: 480, 208, 103: 207, 205, 479, 206, 204}, + {1: 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 194, 83: 194}, + {1: 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 192, 83: 192}, } ) @@ -954,7 +969,7 @@ func yyhintlex1(yylex yyhintLexer, lval *yyhintSymType) (n int) { } func yyhintParse(yylex yyhintLexer, parser *hintParser) int { - const yyError = 121 + const yyError = 123 yyEx, _ := yylex.(yyhintLexerEx) var yyn int diff --git a/parser/hintparser.y b/parser/hintparser.y index a2ccd21d2d401..0d92905bb86f3 100644 --- a/parser/hintparser.y +++ b/parser/hintparser.y @@ -103,6 +103,8 @@ import ( hintSwapJoinInputs "SWAP_JOIN_INPUTS" hintUseIndexMerge "USE_INDEX_MERGE" hintUseIndex "USE_INDEX" + hintKeepOrder "KEEP_ORDER" + hintNoKeepOrder "NO_KEEP_ORDER" hintUsePlanCache "USE_PLAN_CACHE" hintUseToja "USE_TOJA" hintTimeRange "TIME_RANGE" @@ -607,6 +609,8 @@ SupportedIndexLevelOptimizerHintName: | "IGNORE_INDEX" | "USE_INDEX_MERGE" | "FORCE_INDEX" +| "KEEP_ORDER" +| "NO_KEEP_ORDER" SubqueryOptimizerHintName: "SEMIJOIN" @@ -699,6 +703,8 @@ Identifier: | "SWAP_JOIN_INPUTS" | "USE_INDEX_MERGE" | "USE_INDEX" +| "KEEP_ORDER" +| "NO_KEEP_ORDER" | "USE_PLAN_CACHE" | "USE_TOJA" | "TIME_RANGE" diff --git a/parser/misc.go b/parser/misc.go index 3c859c30283b1..c5bda32024409 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -956,6 +956,8 @@ var hintTokenMap = map[string]int{ "SWAP_JOIN_INPUTS": hintSwapJoinInputs, "USE_INDEX_MERGE": hintUseIndexMerge, "USE_INDEX": hintUseIndex, + "KEEP_ORDER": hintKeepOrder, + "NO_KEEP_ORDER": hintNoKeepOrder, "USE_PLAN_CACHE": hintUsePlanCache, "USE_TOJA": hintUseToja, "TIME_RANGE": hintTimeRange, diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 7bb9eaef01e6f..c278ffb56167d 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -316,12 +316,19 @@ type MultiSchemaInfo struct { AddIndexes []CIStr `json:"-"` DropIndexes []CIStr `json:"-"` AlterIndexes []CIStr `json:"-"` - ForeignKeys []CIStr `json:"-"` + + AddForeignKeys []AddForeignKeyInfo `json:"-"` RelativeColumns []CIStr `json:"-"` PositionColumns []CIStr `json:"-"` } +// AddForeignKeyInfo contains foreign key information. +type AddForeignKeyInfo struct { + Name CIStr + Cols []CIStr +} + // NewMultiSchemaInfo new a MultiSchemaInfo. func NewMultiSchemaInfo() *MultiSchemaInfo { return &MultiSchemaInfo{ diff --git a/parser/model/model.go b/parser/model/model.go index 19aabf4a06572..411db3fcf1d20 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -381,8 +381,8 @@ func FindFKInfoByName(fks []*FKInfo, name string) *FKInfo { } // FindIndexByColumns find IndexInfo in indices which is cover the specified columns. -func FindIndexByColumns(tbInfo *TableInfo, cols ...CIStr) *IndexInfo { - for _, index := range tbInfo.Indices { +func FindIndexByColumns(tbInfo *TableInfo, indices []*IndexInfo, cols ...CIStr) *IndexInfo { + for _, index := range indices { if IsIndexPrefixCovered(tbInfo, index, cols...) { return index } diff --git a/parser/parser_test.go b/parser/parser_test.go index 7b72117f69d16..98f90a0104a5c 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -3891,6 +3891,44 @@ func TestOptimizerHints(t *testing.T) { require.Len(t, hints[1].Indexes, 1) require.Equal(t, "t4", hints[1].Indexes[0].L) + // Test KEEP_ORDER + stmt, _, err = p.Parse("select /*+ KEEP_ORDER(T1,T2), keep_order(t3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") + require.NoError(t, err) + selectStmt = stmt[0].(*ast.SelectStmt) + + hints = selectStmt.TableHints + require.Len(t, hints, 2) + require.Equal(t, "keep_order", hints[0].HintName.L) + require.Len(t, hints[0].Tables, 1) + require.Equal(t, "t1", hints[0].Tables[0].TableName.L) + require.Len(t, hints[0].Indexes, 1) + require.Equal(t, "t2", hints[0].Indexes[0].L) + + require.Equal(t, "keep_order", hints[1].HintName.L) + require.Len(t, hints[1].Tables, 1) + require.Equal(t, "t3", hints[1].Tables[0].TableName.L) + require.Len(t, hints[1].Indexes, 1) + require.Equal(t, "t4", hints[1].Indexes[0].L) + + // Test NO_KEEP_ORDER + stmt, _, err = p.Parse("select /*+ NO_KEEP_ORDER(T1,T2), no_keep_order(t3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") + require.NoError(t, err) + selectStmt = stmt[0].(*ast.SelectStmt) + + hints = selectStmt.TableHints + require.Len(t, hints, 2) + require.Equal(t, "no_keep_order", hints[0].HintName.L) + require.Len(t, hints[0].Tables, 1) + require.Equal(t, "t1", hints[0].Tables[0].TableName.L) + require.Len(t, hints[0].Indexes, 1) + require.Equal(t, "t2", hints[0].Indexes[0].L) + + require.Equal(t, "no_keep_order", hints[1].HintName.L) + require.Len(t, hints[1].Tables, 1) + require.Equal(t, "t3", hints[1].Tables[0].TableName.L) + require.Len(t, hints[1].Indexes, 1) + require.Equal(t, "t4", hints[1].Indexes[0].L) + // Test TIDB_SMJ stmt, _, err = p.Parse("select /*+ TIDB_SMJ(T1,t2), tidb_smj(T3,t4) */ c1, c2 from t1, t2 where t1.c1 = t2.c1", "", "") require.NoError(t, err) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 8feb357745853..53aa657d2f9c0 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -252,6 +252,8 @@ const ( OpSetBindingStatus // OpSQLBindDropByDigest is used to drop SQL binds by digest OpSQLBindDropByDigest + // OpSetBindingStatusByDigest represents the operation to set SQL binding status by sql digest. + OpSetBindingStatusByDigest ) // SQLBindPlan represents a plan for SQL bind. diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 2817f370ffcec..dfd8d9572db81 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -1928,7 +1928,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } }) - if (p.preferJoinType&preferBCJoin) == 0 && p.preferJoinType > 0 { + if (p.preferJoinType&preferBCJoin) == 0 && (p.preferJoinType&preferShuffleJoin) == 0 && p.preferJoinType > 0 { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have used hint to specify a join algorithm which is not supported by mpp now.") if prop.IsFlashProp() { return nil, false, nil @@ -1957,6 +1957,21 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P mppJoins := p.tryToGetMppHashJoin(prop, false) joins = append(joins, mppJoins...) } + } else { + hasMppHints := false + var errMsg string + if (p.preferJoinType & preferShuffleJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the shuffle_join() hint is invalid" + hasMppHints = true + } + if (p.preferJoinType & preferBCJoin) > 0 { + errMsg = "The join can not push down to the MPP side, the broadcast_join() hint is invalid" + hasMppHints = true + } + if hasMppHints { + warning := ErrInternal.GenWithStack(errMsg) + p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } } if prop.IsFlashProp() { return joins, true, nil @@ -2858,6 +2873,21 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy } if canPushDownToMPP { taskTypes = append(taskTypes, property.MppTaskType) + } else { + hasMppHints := false + var errMsg string + if la.aggHints.preferAggType&preferMPP1PhaseAgg > 0 { + errMsg = "The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid" + hasMppHints = true + } + if la.aggHints.preferAggType&preferMPP2PhaseAgg > 0 { + errMsg = "The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid" + hasMppHints = true + } + if hasMppHints { + warning := ErrInternal.GenWithStack(errMsg) + la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } } if prop.IsFlashProp() { taskTypes = []property.TaskType{prop.TaskTp} diff --git a/planner/core/foreign_key.go b/planner/core/foreign_key.go index 00b87e11c01f8..d63a5f489b7ac 100644 --- a/planner/core/foreign_key.go +++ b/planner/core/foreign_key.go @@ -414,7 +414,7 @@ func buildFKCheck(ctx sessionctx.Context, tbl table.Table, cols []model.CIStr, f } } - referTbIdxInfo := model.FindIndexByColumns(tblInfo, cols...) + referTbIdxInfo := model.FindIndexByColumns(tblInfo, tblInfo.Indices, cols...) if referTbIdxInfo == nil { return nil, failedErr } @@ -460,7 +460,7 @@ func buildFKCascade(ctx sessionctx.Context, tp FKCascadeType, referredFK *model. return fkCascade, nil } } - indexForFK := model.FindIndexByColumns(childTable.Meta(), fk.Cols...) + indexForFK := model.FindIndexByColumns(childTable.Meta(), childTable.Meta().Indices, fk.Cols...) if indexForFK == nil { return nil, errors.Errorf("Missing index for '%s' foreign key columns in the table '%s'", fk.Name, childTable.Meta().Name) } diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 7e5b13b588e16..640c0f04630c1 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -813,9 +813,11 @@ func TestMPPHints(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustExec("set tidb_cost_model_version=2") - tk.MustExec("create table t (a int, b int, c int)") + tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") tk.MustExec("alter table t set tiflash replica 1") tk.MustExec("set @@session.tidb_allow_mpp=ON") + tk.MustExec("create definer='root'@'localhost' view v as select a, sum(b) from t group by a, c;") + tk.MustExec("create definer='root'@'localhost' view v1 as select t1.a from t t1, t t2 where t1.a=t2.a;") tb := external.GetTableByName(t, tk, "test", "t") err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) require.NoError(t, err) @@ -824,6 +826,7 @@ func TestMPPHints(t *testing.T) { var output []struct { SQL string Plan []string + Warn []string } planSuiteData := core.GetPlanSuiteData() @@ -833,11 +836,101 @@ func TestMPPHints(t *testing.T) { testdata.OnRecord(func() { output[i].SQL = ts output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + ts).Rows()) + output[i].Warn = testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings()) }) tk.MustQuery("explain format = 'brief' " + ts).Check(testkit.Rows(output[i].Plan...)) + require.Equal(t, output[i].Warn, testdata.ConvertSQLWarnToStrings(tk.Session().GetSessionVars().StmtCtx.GetWarnings())) } } +func TestMPPHintsScope(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t (a int, b int, c int, index idx_a(a), index idx_b(b))") + tk.MustExec("select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid")) + tk.MustExec("select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid")) + tk.MustExec("select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the shuffle_join() hint is invalid")) + tk.MustExec("select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the broadcast_join() hint is invalid")) + tk.MustExec("alter table t set tiflash replica 1") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + tk.MustExec("set @@session.tidb_allow_mpp=true") + tk.MustExec("explain select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows()) + tk.MustExec("explain select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows()) + + tk.MustExec("set @@session.tidb_allow_mpp=false") + tk.MustExec("explain select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid")) + tk.MustExec("explain select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid")) + tk.MustExec("explain select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the shuffle_join() hint is invalid")) + tk.MustExec("explain select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 The join can not push down to the MPP side, the broadcast_join() hint is invalid")) +} + +func TestMPPHintsWithBinding(t *testing.T) { + store := testkit.CreateMockStore(t, withMockTiFlash(2)) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") + tk.MustExec("create table t (a int, b int, c int)") + tk.MustExec("alter table t set tiflash replica 1") + tk.MustExec("set @@session.tidb_allow_mpp=ON") + tb := external.GetTableByName(t, tk, "test", "t") + err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true) + require.NoError(t, err) + + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select a, sum(b) from t group by a, c using select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c;") + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res := tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select `a` , sum ( `b` ) from `test` . `t` group by `a` , `c`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t`]) MPP_1PHASE_AGG()*/ `a`,sum(`b`) FROM `test`.`t` GROUP BY `a`,`c`") + tk.MustExec("create global binding for select a, sum(b) from t group by a, c using select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c;") + tk.MustExec("explain select a, sum(b) from t group by a, c") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select `a` , sum ( `b` ) from `test` . `t` group by `a` , `c`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t`]) MPP_2PHASE_AGG()*/ `a`,sum(`b`) FROM `test`.`t` GROUP BY `a`,`c`") + tk.MustExec("drop global binding for select a, sum(b) from t group by a, c;") + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) + + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("0")) + tk.MustExec("create global binding for select * from t t1, t t2 where t1.a=t2.a using select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a") + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t` as `t1` ) join `test` . `t` as `t2` where `t1` . `a` = `t2` . `a`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t1`, `t2`]) shuffle_join(`t1`, `t2`)*/ * FROM (`test`.`t` AS `t1`) JOIN `test`.`t` AS `t2` WHERE `t1`.`a` = `t2`.`a`") + tk.MustExec("create global binding for select * from t t1, t t2 where t1.a=t2.a using select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a;") + tk.MustExec("explain select * from t t1, t t2 where t1.a=t2.a") + tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1")) + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, res[0][0], "select * from ( `test` . `t` as `t1` ) join `test` . `t` as `t2` where `t1` . `a` = `t2` . `a`") + require.Equal(t, res[0][1], "SELECT /*+ read_from_storage(tiflash[`t1`, `t2`]) broadcast_join(`t1`, `t2`)*/ * FROM (`test`.`t` AS `t1`) JOIN `test`.`t` AS `t2` WHERE `t1`.`a` = `t2`.`a`") + tk.MustExec("drop global binding for select * from t t1, t t2 where t1.a=t2.a;") + res = tk.MustQuery("show global bindings").Rows() + require.Equal(t, len(res), 0) +} + func TestHintScope(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 7a4ac860d8593..a480c583d0434 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -221,6 +221,29 @@ func TestIssue38533(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } +func TestIgnoreInsertStmt(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + + // do not cache native insert-stmt + tk.MustExec("prepare st from 'insert into t values (1)'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) + + // ignore-hint in insert-stmt can work + tk.MustExec("prepare st from 'insert into t select * from t'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("1")) + tk.MustExec("prepare st from 'insert /*+ ignore_plan_cache() */ into t select * from t'") + tk.MustExec("execute st") + tk.MustExec("execute st") + tk.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) +} + func TestIssue38710(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index 3412ace379271..041509d224792 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -94,6 +94,21 @@ func (checker *cacheableChecker) Enter(in ast.Node) (out ast.Node, skipChildren return in, true } } + case *ast.InsertStmt: + if node.Select == nil { + // do not cache insert-values-stmt like 'insert into t values (...)' since + // no performance benefit and to save memory. + checker.cacheable = false + checker.reason = "ignore insert-values-stmt" + return in, true + } + for _, hints := range node.TableHints { + if hints.HintName.L == HintIgnorePlanCache { + checker.cacheable = false + checker.reason = "ignore plan cache by hint" + return in, true + } + } case *ast.VariableExpr, *ast.ExistsSubqueryExpr, *ast.SubqueryExpr: checker.cacheable = false checker.reason = "query has sub-queries is un-cacheable" diff --git a/planner/core/plan_cacheable_checker_test.go b/planner/core/plan_cacheable_checker_test.go index 658537bde4188..e87a08592eb16 100644 --- a/planner/core/plan_cacheable_checker_test.go +++ b/planner/core/plan_cacheable_checker_test.go @@ -53,7 +53,9 @@ func TestCacheable(t *testing.T) { tableRefsClause := &ast.TableRefsClause{TableRefs: &ast.Join{Left: &ast.TableSource{Source: tbl}}} // test InsertStmt - stmt = &ast.InsertStmt{Table: tableRefsClause} + stmt = &ast.InsertStmt{Table: tableRefsClause} // insert-values-stmt + require.False(t, core.Cacheable(stmt, is)) + stmt = &ast.InsertStmt{Table: tableRefsClause, Select: &ast.SelectStmt{}} // insert-select-stmt require.True(t, core.Cacheable(stmt, is)) // test DeleteStmt diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 26508814523d2..238e2a687d758 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1014,10 +1014,20 @@ func (b *PlanBuilder) buildDropBindPlan(v *ast.DropBindingStmt) (Plan, error) { } func (b *PlanBuilder) buildSetBindingStatusPlan(v *ast.SetBindingStmt) (Plan, error) { - p := &SQLBindPlan{ - SQLBindOp: OpSetBindingStatus, - NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())), - Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB), + var p *SQLBindPlan + if v.OriginNode != nil { + p = &SQLBindPlan{ + SQLBindOp: OpSetBindingStatus, + NormdOrigSQL: parser.Normalize(utilparser.RestoreWithDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB, v.OriginNode.Text())), + Db: utilparser.GetDefaultDB(v.OriginNode, b.ctx.GetSessionVars().CurrentDB), + } + } else if v.SQLDigest != "" { + p = &SQLBindPlan{ + SQLBindOp: OpSetBindingStatusByDigest, + SQLDigest: v.SQLDigest, + } + } else { + return nil, errors.New("sql digest is empty") } switch v.BindingStatusType { case ast.BindingStatusTypeEnabled: diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index d61124de927bd..c1dbae406a922 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -6229,9 +6229,9 @@ { "SQL": "explain format = 'brief' select /*+ qb_name(qb, v12) read_from_storage(tiflash[t1@qb, t@qb]), shuffle_join(t1@qb, t@qb) */ * from v12;", "Plan": [ - "Projection 12500.00 root test.t.a, test.t.b", - "└─TableReader 12500.00 root data:ExchangeSender", - " └─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + "TableReader 12500.00 root data:ExchangeSender", + "└─ExchangeSender 12500.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12500.00 mpp[tiflash] test.t.a, test.t.b", " └─HashJoin 12500.00 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", " ├─ExchangeReceiver(Build) 10000.00 mpp[tiflash] ", " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index c38250c802454..d433f5dd88dbe 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -2,10 +2,70 @@ { "name": "TestMPPHints", "cases": [ + "select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", + "select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", + "select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // READ_FROM_STORAGE hint "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", - "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a" + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // Join hint + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_build(t1) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_build(t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_probe(t1) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_probe(t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + + // AGG hint + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), hash_agg() */ a, sum(b) from t group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), stream_agg() */ a, sum(b) from t group by a, c", + + // Index hint + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), use_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), ignore_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), force_index(t, idx_b) */ a, sum(b) from t where b < 2 group by a, c", + "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), index_merge(t, idx_b, idx_a) */ a, sum(b) from t where b < 2 or a > 2 group by a, c", + + // Join Order hint + "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), straight_join() */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), leading(t3, t1) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), straight_join() */ * from t t2, t t1, t t3 where t1.a=t2.a and t2.b=t3.b", + "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), leading(t2, t3) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + + // View Hint + "select /*+ qb_name(qb, v), MPP_1PHASE_AGG(@qb) */ * from v", + "select /*+ qb_name(qb, v), MPP_2PHASE_AGG(@qb) */ * from v", + "select /*+ qb_name(qb, v1), shuffle_join(t1@qb, t2@qb) */ * from v1", + "select /*+ qb_name(qb, v1), broadcast_join(t1@qb, t2@qb) */ * from v1", + + // Subquery hint + "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "select * from t t1 where t1.a < (select /*+ MPP_2PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + + // CTE + "WITH CTE AS (SELECT /*+ MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "WITH CTE AS (SELECT /*+ MERGE(), broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index b3a7664b2b2fd..31964823e95f2 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -2,6 +2,69 @@ { "Name": "TestMPPHints", "Cases": [ + { + "SQL": "select /*+ MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#10)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, { "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG() */ a, sum(b) from t group by a, c", "Plan": [ @@ -14,7 +77,8 @@ " └─ExchangeReceiver 10000.00 mpp[tiflash] ", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG() */ a, sum(b) from t group by a, c", @@ -29,7 +93,8 @@ " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", @@ -45,7 +110,8 @@ " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" - ] + ], + "Warn": null }, { "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", @@ -59,6 +125,732 @@ " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_build(t1) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_build(t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), hash_join_probe(t1) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), hash_join_probe(t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), merge_join(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), shuffle_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2]), broadcast_join(t1, t2), INL_JOIN(t1, t2) */ * from t t1, t t2 where t1.a=t2.a", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]Join hints are conflict, you can only specify one type of join" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), hash_agg() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), stream_agg() */ a, sum(b) from t group by a, c", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#6)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#9, funcs:sum(Column#8)->Column#6", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), use_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "Plan": [ + "Projection 2666.67 root test.t.a, Column#5", + "└─HashAgg 2666.67 root group by:test.t.a, test.t.c, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─IndexLookUp 2666.67 root ", + " ├─IndexRangeScan(Build) 3333.33 cop[tikv] table:t, index:idx_a(a) range:(1,+inf], keep order:false, stats:pseudo", + " └─HashAgg(Probe) 2666.67 cop[tikv] group by:test.t.a, test.t.c, funcs:sum(test.t.b)->Column#7", + " └─TableRowIDScan 3333.33 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{} 1:{} 2:{}])", + "[planner:1815]The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_1PHASE_AGG(), ignore_index(t, idx_a) */ a, sum(b) from t where a > 1 group by a, c", + "Plan": [ + "TableReader 2666.67 root data:ExchangeSender", + "└─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 2666.67 mpp[tiflash] test.t.a, Column#5", + " └─Projection 2666.67 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 2666.67 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 3333.33 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 3333.33 mpp[tiflash] ", + " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─Selection 3333.33 mpp[tiflash] gt(test.t.a, 1)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), force_index(t, idx_b) */ a, sum(b) from t where b < 2 group by a, c", + "Plan": [ + "Projection 2658.67 root test.t.a, Column#5", + "└─HashAgg 2658.67 root group by:test.t.a, test.t.c, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─IndexLookUp 2658.67 root ", + " ├─IndexRangeScan(Build) 3323.33 cop[tikv] table:t, index:idx_b(b) range:[-inf,2), keep order:false, stats:pseudo", + " └─HashAgg(Probe) 2658.67 cop[tikv] group by:test.t.a, test.t.c, funcs:sum(test.t.b)->Column#7", + " └─TableRowIDScan 3323.33 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]No available path for table test.t with the store type tiflash of the hint /*+ read_from_storage */, please check the status of the table replica and variable value of tidb_isolation_read_engines(map[0:{} 1:{} 2:{}])", + "[planner:1815]The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t]), MPP_2PHASE_AGG(), index_merge(t, idx_b, idx_a) */ a, sum(b) from t where b < 2 or a > 2 group by a, c", + "Plan": [ + "TableReader 4439.11 root data:ExchangeSender", + "└─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 4439.11 mpp[tiflash] test.t.a, Column#5", + " └─Projection 4439.11 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 4439.11 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#13)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 4439.11 mpp[tiflash] ", + " └─ExchangeSender 4439.11 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 4439.11 mpp[tiflash] group by:Column#16, Column#17, funcs:sum(Column#15)->Column#13", + " └─Projection 5548.89 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#15, test.t.a, test.t.c", + " └─Selection 5548.89 mpp[tiflash] or(lt(test.t.b, 2), gt(test.t.a, 2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[parser:8061]Optimizer hint index_merge is not supported by TiDB and is ignored" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), straight_join() */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 mpp[tiflash] ", + " └─ExchangeSender 12475.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashJoin 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), shuffle_join(t1, t2, t3), leading(t3, t1) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 12475.01 mpp[tiflash] ", + " └─ExchangeSender 12475.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashJoin 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint", + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), straight_join() */ * from t t2, t t1, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ read_from_storage(tiflash[t1, t2, t3]), broadcast_join(t1, t2, t3), leading(t2, t3) */ * from t t1, t t2, t t3 where t1.a=t2.a and t2.b=t3.b", + "Plan": [ + "TableReader 15593.77 root data:ExchangeSender", + "└─ExchangeSender 15593.77 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 15593.77 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t3 keep order:false, stats:pseudo", + " └─HashJoin(Probe) 12475.01 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9980.01 mpp[tiflash] ", + " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.a)), not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint", + "[planner:1815]leading hint is inapplicable, check the join type or the join algorithm hint" + ] + }, + { + "SQL": "select /*+ qb_name(qb, v), MPP_1PHASE_AGG(@qb) */ * from v", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#10, Column#11, funcs:sum(Column#8)->Column#5, funcs:firstrow(Column#9)->test.t.a", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#8, test.t.a, test.t.a, test.t.c", + " └─ExchangeReceiver 10000.00 mpp[tiflash] ", + " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v), MPP_2PHASE_AGG(@qb) */ * from v", + "Plan": [ + "TableReader 8000.00 root data:ExchangeSender", + "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 8000.00 mpp[tiflash] test.t.a, Column#5", + " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.a", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.a, test.t.c, funcs:sum(Column#10)->Column#5, funcs:firstrow(test.t.a)->test.t.a", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary], [name: test.t.c, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:Column#13, Column#14, funcs:sum(Column#12)->Column#10", + " └─Projection 10000.00 mpp[tiflash] cast(test.t.b, decimal(10,0) BINARY)->Column#12, test.t.a, test.t.c", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v1), shuffle_join(t1@qb, t2@qb) */ * from v1", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.t.a", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select /*+ qb_name(qb, v1), broadcast_join(t1@qb, t2@qb) */ * from v1", + "Plan": [ + "TableReader 12487.50 root data:ExchangeSender", + "└─ExchangeSender 12487.50 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 12487.50 mpp[tiflash] test.t.a", + " └─HashJoin 12487.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "SELECT /*+ shuffle_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ SHUFFLE_JOIN(t) */ or /*+ SHUFFLE_JOIN(t) */. Maybe you can use the table alias name", + "[parser:1064]Optimizer hint syntax error at line 1 column 109 near \"\" " + ] + }, + { + "SQL": "SELECT /*+ broadcast_join(t) */ * FROM t WHERE EXISTS (SELECT /*+ SEMI_JOIN_REWRITE */ 1 FROM t t1 WHERE t1.b = t.b);", + "Plan": [ + "TableReader 7992.00 root data:ExchangeSender", + "└─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 7992.00 mpp[tiflash] semi join, equal:[eq(test.t.b, test.t.b)]", + " ├─ExchangeReceiver(Build) 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t) in optimizer hint /*+ BROADCAST_JOIN(t) */ or /*+ TIDB_BCJ(t) */. Maybe you can use the table alias name", + "[parser:1064]Optimizer hint syntax error at line 1 column 111 near \"\" " + ] + }, + { + "SQL": "select * from t t1 where t1.a < (select /*+ MPP_1PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)], other cond:lt(cast(test.t.a, decimal(10,0) BINARY), Column#9)", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] Column#9, test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#26, funcs:sum(Column#24)->Column#9, funcs:firstrow(Column#25)->test.t.b", + " │ └─Projection 9990.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#24, test.t.b, test.t.b", + " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", + " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "select * from t t1 where t1.a < (select /*+ MPP_2PHASE_AGG() */ sum(t2.a) from t t2 where t2.b = t1.b);", + "Plan": [ + "TableReader 9990.00 root data:ExchangeSender", + "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 9990.00 mpp[tiflash] test.t.a, test.t.b, test.t.c", + " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.b, test.t.b)], other cond:lt(cast(test.t.a, decimal(10,0) BINARY), Column#9)", + " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 7992.00 mpp[tiflash] Column#9, test.t.b", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.b, funcs:sum(Column#13)->Column#9, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#29, funcs:sum(Column#28)->Column#13", + " │ └─Projection 9990.00 mpp[tiflash] cast(test.t.a, decimal(10,0) BINARY)->Column#28, test.t.b", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t2 keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 3403.09 root group by:Column#10, Column#11, funcs:firstrow(Column#10)->Column#10, funcs:firstrow(Column#11)->Column#11", + "└─Union 3403.09 root ", + " ├─Selection 1701.55 root lt(Column#6, 18)", + " │ └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + " └─Selection 1701.55 root gt(test.t.b, 1)", + " └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + "CTE_0 2126.93 root Non-Recursive CTE", + "└─TableReader(Seed Part) 2126.93 root data:ExchangeSender", + " └─ExchangeSender 2126.93 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 2126.93 mpp[tiflash] or(lt(Column#5, 18), gt(test.t.b, 1))", + " └─Projection 2658.67 mpp[tiflash] Column#5, test.t.b", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#5, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 3323.33 mpp[tiflash] ", + " └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 3403.09 root group by:Column#10, Column#11, funcs:firstrow(Column#10)->Column#10, funcs:firstrow(Column#11)->Column#11", + "└─Union 3403.09 root ", + " ├─Selection 1701.55 root lt(Column#6, 18)", + " │ └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + " └─Selection 1701.55 root gt(test.t.b, 1)", + " └─CTEFullScan 2126.93 root CTE:cte data:CTE_0", + "CTE_0 2126.93 root Non-Recursive CTE", + "└─TableReader(Seed Part) 2126.93 root data:ExchangeSender", + " └─ExchangeSender 2126.93 mpp[tiflash] ExchangeType: PassThrough", + " └─Selection 2126.93 mpp[tiflash] or(lt(Column#5, 18), gt(test.t.b, 1))", + " └─Projection 2658.67 mpp[tiflash] Column#5, test.t.b", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:sum(Column#22)->Column#5, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 2658.67 mpp[tiflash] ", + " └─ExchangeSender 2658.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#22", + " └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 7095.48 root group by:Column#13, Column#14, funcs:firstrow(Column#13)->Column#13, funcs:firstrow(Column#14)->Column#14", + "└─Union 11086.68 root ", + " ├─Selection 5543.34 root lt(test.t.a, 18)", + " │ └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + " └─Selection 5543.34 root gt(test.t.b, 1)", + " └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + "CTE_0 6929.18 root Non-Recursive CTE", + "└─TableReader(Seed Part) 6929.18 root data:ExchangeSender", + " └─ExchangeSender 6929.18 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6929.18 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 6929.18 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)], other cond:or(lt(test.t.a, 18), gt(test.t.b, 1))", + " ├─ExchangeReceiver(Build) 5543.34 mpp[tiflash] ", + " │ └─ExchangeSender 5543.34 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 5543.34 mpp[tiflash] not(isnull(test.t.a)), or(lt(test.t.a, 18), gt(test.t.b, 1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "HashAgg 7095.48 root group by:Column#13, Column#14, funcs:firstrow(Column#13)->Column#13, funcs:firstrow(Column#14)->Column#14", + "└─Union 11086.68 root ", + " ├─Selection 5543.34 root lt(test.t.a, 18)", + " │ └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + " └─Selection 5543.34 root gt(test.t.b, 1)", + " └─CTEFullScan 6929.18 root CTE:cte data:CTE_0", + "CTE_0 6929.18 root Non-Recursive CTE", + "└─TableReader(Seed Part) 6929.18 root data:ExchangeSender", + " └─ExchangeSender 6929.18 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 6929.18 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 6929.18 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)], other cond:or(lt(test.t.a, 18), gt(test.t.b, 1))", + " ├─ExchangeReceiver(Build) 5543.34 mpp[tiflash] ", + " │ └─ExchangeSender 5543.34 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 5543.34 mpp[tiflash] not(isnull(test.t.a)), or(lt(test.t.a, 18), gt(test.t.b, 1))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), MPP_1PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 3013.16 root data:ExchangeSender", + "└─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3013.16 mpp[tiflash] Column#20, Column#21", + " └─HashAgg 3013.16 mpp[tiflash] group by:Column#20, Column#21, funcs:firstrow(Column#20)->Column#20, funcs:firstrow(Column#21)->Column#21", + " └─ExchangeReceiver 3013.16 mpp[tiflash] ", + " └─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary], [name: Column#21, collate: binary]", + " └─Union 3013.16 mpp[tiflash] ", + " ├─Projection 2126.93 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#20, test.t.b", + " │ └─Selection 2126.93 mpp[tiflash] lt(Column#12, 18)", + " │ └─Projection 2658.67 mpp[tiflash] Column#12, test.t.b", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#12, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 3323.33 mpp[tiflash] ", + " │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 886.22 mpp[tiflash] cast(Column#20, bigint(21) BINARY)->Column#20, Column#21", + " └─Projection 886.22 mpp[tiflash] Column#19, test.t.b", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#19, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 1107.78 mpp[tiflash] ", + " └─ExchangeSender 1107.78 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─Selection 1107.78 mpp[tiflash] gt(test.t.b, 1), lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), MPP_2PHASE_AGG() */ count(*) as a, b FROM t WHERE t.a < 60 group by b) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 3013.16 root data:ExchangeSender", + "└─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 3013.16 mpp[tiflash] Column#20, Column#21", + " └─HashAgg 3013.16 mpp[tiflash] group by:Column#20, Column#21, funcs:firstrow(Column#20)->Column#20, funcs:firstrow(Column#21)->Column#21", + " └─ExchangeReceiver 3013.16 mpp[tiflash] ", + " └─ExchangeSender 3013.16 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#20, collate: binary], [name: Column#21, collate: binary]", + " └─Union 3013.16 mpp[tiflash] ", + " ├─Projection 2126.93 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#20, test.t.b", + " │ └─Selection 2126.93 mpp[tiflash] lt(Column#12, 18)", + " │ └─Projection 2658.67 mpp[tiflash] Column#12, test.t.b", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:sum(Column#32)->Column#12, funcs:firstrow(test.t.b)->test.t.b", + " │ └─ExchangeReceiver 2658.67 mpp[tiflash] ", + " │ └─ExchangeSender 2658.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " │ └─HashAgg 2658.67 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#32", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 60)", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Projection 886.22 mpp[tiflash] cast(Column#20, bigint(21) BINARY)->Column#20, Column#21", + " └─Projection 886.22 mpp[tiflash] Column#19, test.t.b", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:sum(Column#46)->Column#19, funcs:firstrow(test.t.b)->test.t.b", + " └─ExchangeReceiver 886.22 mpp[tiflash] ", + " └─ExchangeSender 886.22 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.b, collate: binary]", + " └─HashAgg 886.22 mpp[tiflash] group by:test.t.b, funcs:count(1)->Column#46", + " └─Selection 1107.78 mpp[tiflash] gt(test.t.b, 1), lt(test.t.a, 60)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + ], + "Warn": null + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), shuffle_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 5322.67 root data:ExchangeSender", + "└─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 5322.67 mpp[tiflash] Column#29, Column#30", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, funcs:firstrow(Column#29)->Column#29, funcs:firstrow(Column#30)->Column#30", + " └─ExchangeReceiver 5322.67 mpp[tiflash] ", + " └─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#29, collate: binary], [name: Column#30, collate: binary]", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, ", + " └─Union 8316.67 mpp[tiflash] ", + " ├─Projection 4154.17 mpp[tiflash] test.t.a, test.t.b", + " │ └─HashJoin 4154.17 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 3323.33 mpp[tiflash] ", + " │ │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver(Probe) 3323.33 mpp[tiflash] ", + " │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Projection 4162.50 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 4162.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 3330.00 mpp[tiflash] ", + " │ └─ExchangeSender 3330.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " │ └─Selection 3330.00 mpp[tiflash] gt(test.t.b, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", + " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.a, collate: binary]", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t1, t) in optimizer hint /*+ SHUFFLE_JOIN(t1, t, t1, t) */ or /*+ SHUFFLE_JOIN(t1, t, t1, t) */. Maybe you can use the table alias name", + "[planner:1815]There are no matching table names for (t1, t, t1, t) in optimizer hint /*+ SHUFFLE_JOIN(t1, t, t1, t, t1, t) */ or /*+ SHUFFLE_JOIN(t1, t, t1, t, t1, t) */. Maybe you can use the table alias name" + ] + }, + { + "SQL": "WITH CTE AS (SELECT /*+ MERGE(), broadcast_join(t1, t) */ t.a, t.b FROM t join t t1 where t.a = t1.a) SELECT * FROM CTE WHERE CTE.a <18 union select * from cte where cte.b > 1;", + "Plan": [ + "TableReader 5322.67 root data:ExchangeSender", + "└─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 5322.67 mpp[tiflash] Column#29, Column#30", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, funcs:firstrow(Column#29)->Column#29, funcs:firstrow(Column#30)->Column#30", + " └─ExchangeReceiver 5322.67 mpp[tiflash] ", + " └─ExchangeSender 5322.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: Column#29, collate: binary], [name: Column#30, collate: binary]", + " └─HashAgg 5322.67 mpp[tiflash] group by:Column#29, Column#30, ", + " └─Union 8316.67 mpp[tiflash] ", + " ├─Projection 4154.17 mpp[tiflash] test.t.a, test.t.b", + " │ └─HashJoin 4154.17 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " │ ├─ExchangeReceiver(Build) 3323.33 mpp[tiflash] ", + " │ │ └─ExchangeSender 3323.33 mpp[tiflash] ExchangeType: Broadcast", + " │ │ └─Selection 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─Selection(Probe) 3323.33 mpp[tiflash] lt(test.t.a, 18), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo", + " └─Projection 4162.50 mpp[tiflash] test.t.a, test.t.b", + " └─HashJoin 4162.50 mpp[tiflash] inner join, equal:[eq(test.t.a, test.t.a)]", + " ├─ExchangeReceiver(Build) 3330.00 mpp[tiflash] ", + " │ └─ExchangeSender 3330.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 3330.00 mpp[tiflash] gt(test.t.b, 1), not(isnull(test.t.a))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + ], + "Warn": [ + "[planner:1815]There are no matching table names for (t1, t) in optimizer hint /*+ BROADCAST_JOIN(t1, t, t1, t) */ or /*+ TIDB_BCJ(t1, t, t1, t) */. Maybe you can use the table alias name", + "[planner:1815]There are no matching table names for (t1, t, t1, t) in optimizer hint /*+ BROADCAST_JOIN(t1, t, t1, t, t1, t) */ or /*+ TIDB_BCJ(t1, t, t1, t, t1, t) */. Maybe you can use the table alias name" ] } ] diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 43a2253fff477..acb12ae660b94 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -158,7 +158,10 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e return mysql.NewErrf(mysql.ErrUnknown, "unsupported flag: CursorTypeScrollable", nil) } - if !useCursor { + if useCursor { + cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, true) + defer cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, false) + } else { // not using streaming ,can reuse chunk cc.ctx.GetSessionVars().SetAlloc(cc.chunkAlloc) } @@ -251,7 +254,8 @@ func (cc *clientConn) executePlanCacheStmt(ctx context.Context, stmt interface{} // The first return value indicates whether the call of executePreparedStmtAndWriteResult has no side effect and can be retried. // Currently the first return value is used to fallback to TiKV when TiFlash is down. func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stmt PreparedStatement, args []expression.Expression, useCursor bool) (bool, error) { - prepStmt, err := (&cc.ctx).GetSessionVars().GetPreparedStmtByID(uint32(stmt.ID())) + vars := (&cc.ctx).GetSessionVars() + prepStmt, err := vars.GetPreparedStmtByID(uint32(stmt.ID())) if err != nil { return true, errors.Annotate(err, cc.preparedStmt2String(uint32(stmt.ID()))) } @@ -264,6 +268,9 @@ func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stm return true, errors.Annotate(err, cc.preparedStmt2String(uint32(stmt.ID()))) } if rs == nil { + if useCursor { + vars.SetStatusFlag(mysql.ServerStatusCursorExists, false) + } return false, cc.writeOK(ctx) } // since there are multiple implementations of ResultSet (the rs might be wrapped), we have to unwrap the rs before @@ -294,7 +301,7 @@ func (cc *clientConn) executePreparedStmtAndWriteResult(ctx context.Context, stm cl.OnFetchReturned() } // explicitly flush columnInfo to client. - err = cc.writeEOF(ctx, cc.ctx.Status()|mysql.ServerStatusCursorExists) + err = cc.writeEOF(ctx, cc.ctx.Status()) if err != nil { return false, err } @@ -316,6 +323,8 @@ const ( func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err error) { cc.ctx.GetSessionVars().StartTime = time.Now() cc.ctx.GetSessionVars().ClearAlloc(nil, false) + cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, true) + defer cc.ctx.GetSessionVars().SetStatusFlag(mysql.ServerStatusCursorExists, false) stmtID, fetchSize, err := parseStmtFetchCmd(data) if err != nil { @@ -344,7 +353,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err strconv.FormatUint(uint64(stmtID), 10), "stmt_fetch_rs"), cc.preparedStmt2String(stmtID)) } - _, err = cc.writeResultset(ctx, rs, true, cc.ctx.Status()|mysql.ServerStatusCursorExists, int(fetchSize)) + _, err = cc.writeResultset(ctx, rs, true, cc.ctx.Status(), int(fetchSize)) if err != nil { return errors.Annotate(err, cc.preparedStmt2String(stmtID)) } diff --git a/server/conn_stmt_test.go b/server/conn_stmt_test.go index 2e60fc1085332..366e5c54ac222 100644 --- a/server/conn_stmt_test.go +++ b/server/conn_stmt_test.go @@ -15,6 +15,7 @@ package server import ( + "bytes" "context" "encoding/binary" "testing" @@ -340,3 +341,59 @@ func TestCursorReadHoldTS(t *testing.T) { require.Zero(t, tk.Session().ShowProcess().GetMinStartTS(0)) require.Zero(t, srv.GetMinStartTS(0)) } + +func TestCursorExistsFlag(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + srv := CreateMockServer(t, store) + srv.SetDomain(dom) + defer srv.Close() + + appendUint32 := binary.LittleEndian.AppendUint32 + ctx := context.Background() + c := CreateMockConn(t, srv).(*mockConn) + out := new(bytes.Buffer) + c.pkt.bufWriter.Reset(out) + c.capability |= mysql.ClientDeprecateEOF | mysql.ClientProtocol41 + tk := testkit.NewTestKitWithSession(t, store, c.Context().Session) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key)") + tk.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8)") + tk.MustQuery("select count(*) from t").Check(testkit.Rows("8")) + + getLastStatus := func() uint16 { + raw := out.Bytes() + return binary.LittleEndian.Uint16(raw[len(raw)-4 : len(raw)-2]) + } + + stmt, _, _, err := c.Context().Prepare("select * from t") + require.NoError(t, err) + + require.NoError(t, c.Dispatch(ctx, append( + appendUint32([]byte{mysql.ComStmtExecute}, uint32(stmt.ID())), + mysql.CursorTypeReadOnly, 0x1, 0x0, 0x0, 0x0, + ))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // fetch first 5 + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // COM_QUERY during fetch + require.NoError(t, c.Dispatch(ctx, append([]byte{mysql.ComQuery}, "select * from t"...))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // fetch last 3 + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.True(t, mysql.HasCursorExistsFlag(getLastStatus())) + + // final fetch with no row retured + // (tidb doesn't unset cursor-exists flag in the previous response like mysql, one more fetch is needed) + require.NoError(t, c.Dispatch(ctx, appendUint32(appendUint32([]byte{mysql.ComStmtFetch}, uint32(stmt.ID())), 5))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) + require.True(t, getLastStatus()&mysql.ServerStatusLastRowSend > 0) + + // COM_QUERY after fetch + require.NoError(t, c.Dispatch(ctx, append([]byte{mysql.ComQuery}, "select * from t"...))) + require.False(t, mysql.HasCursorExistsFlag(getLastStatus())) +} diff --git a/sessionctx/sessionstates/session_states_test.go b/sessionctx/sessionstates/session_states_test.go index 5910d6b18e071..1eef55ed20bcf 100644 --- a/sessionctx/sessionstates/session_states_test.go +++ b/sessionctx/sessionstates/session_states_test.go @@ -133,7 +133,7 @@ func TestSystemVars(t *testing.T) { { // sem invisible variable inSessionStates: false, - varName: variable.TiDBAllowRemoveAutoInc, + varName: variable.TiDBConfig, }, { // noop variables diff --git a/sessionctx/stmtctx/BUILD.bazel b/sessionctx/stmtctx/BUILD.bazel index b42939f7ec324..a2e12d1e860e1 100644 --- a/sessionctx/stmtctx/BUILD.bazel +++ b/sessionctx/stmtctx/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/pingcap/tidb/sessionctx/stmtctx", visibility = ["//visibility:public"], deps = [ + "//errno", "//parser", "//parser/ast", "//parser/model", diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 47159dc8f8a60..9a49c24851c08 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -24,6 +24,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" @@ -865,6 +866,21 @@ func (sc *StatementContext) HandleTruncate(err error) error { if err == nil { return nil } + + err = errors.Cause(err) + if e, ok := err.(*errors.Error); !ok || + (e.Code() != errno.ErrTruncatedWrongValue && + e.Code() != errno.ErrDataTooLong && + e.Code() != errno.ErrTruncatedWrongValueForField && + e.Code() != errno.ErrWarnDataOutOfRange && + e.Code() != errno.ErrDataOutOfRange && + e.Code() != errno.ErrBadNumber && + e.Code() != errno.ErrWrongValueForType && + e.Code() != errno.ErrDatetimeFunctionOverflow && + e.Code() != errno.WarnDataTruncated) { + return err + } + if sc.IgnoreTruncate { return nil } diff --git a/sessiontxn/txn_rc_tso_optimize_test.go b/sessiontxn/txn_rc_tso_optimize_test.go index f321d40340496..3e945778ca107 100644 --- a/sessiontxn/txn_rc_tso_optimize_test.go +++ b/sessiontxn/txn_rc_tso_optimize_test.go @@ -105,7 +105,7 @@ func TestRcTSOCmdCountForPrepareExecuteNormal(t *testing.T) { tk.MustExec("commit") } countTsoRequest, countTsoUseConstant, countWaitTsoOracle := getAllTsoCounter(sctx) - require.Equal(t, uint64(398), countTsoRequest.(uint64)) + require.Equal(t, uint64(496), countTsoRequest.(uint64)) require.Equal(t, uint64(594), countTsoUseConstant.(uint64)) require.Equal(t, uint64(198), countWaitTsoOracle.(uint64)) @@ -137,7 +137,7 @@ func TestRcTSOCmdCountForPrepareExecuteNormal(t *testing.T) { tk.MustExec("commit") } count := sctx.Value(sessiontxn.TsoRequestCount) - require.Equal(t, uint64(594), count) + require.Equal(t, uint64(693), count) } func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { @@ -234,7 +234,7 @@ func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { tk.MustExec("commit") } countTsoRequest, countTsoUseConstant, countWaitTsoOracle = getAllTsoCounter(sctx) - require.Equal(t, uint64(16), countTsoRequest.(uint64)) + require.Equal(t, uint64(20), countTsoRequest.(uint64)) require.Equal(t, uint64(5), countTsoUseConstant.(uint64)) require.Equal(t, uint64(5), countWaitTsoOracle.(uint64)) @@ -412,7 +412,7 @@ func TestRcTSOCmdCountForPrepareExecuteExtra(t *testing.T) { require.Nil(t, stmt) tk.MustExec("commit") countTsoRequest, countTsoUseConstant, countWaitTsoOracle = getAllTsoCounter(sctx) - require.Equal(t, uint64(3), countTsoRequest.(uint64)) + require.Equal(t, uint64(4), countTsoRequest.(uint64)) require.Equal(t, uint64(2), countTsoUseConstant.(uint64)) require.Equal(t, 0, countWaitTsoOracle.(int)) tk.MustQuery("SELECT * FROM t1 WHERE id1 = 1").Check(testkit.Rows("1 1 1")) diff --git a/table/column.go b/table/column.go index 0a80ddae4e8d4..c63febe75b0ba 100644 --- a/table/column.go +++ b/table/column.go @@ -323,6 +323,7 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo, r } err = sc.HandleTruncate(err) + err = sc.HandleOverflow(err, err) if forceIgnoreTruncate { err = nil diff --git a/testkit/asynctestkit.go b/testkit/asynctestkit.go index aa0f3fcadf8ef..a875088c82abf 100644 --- a/testkit/asynctestkit.go +++ b/testkit/asynctestkit.go @@ -183,6 +183,21 @@ func (tk *AsyncTestKit) MustExec(ctx context.Context, sql string, args ...interf } } +// MustGetErrMsg executes a sql statement and assert its error message. +func (tk *AsyncTestKit) MustGetErrMsg(ctx context.Context, sql string, errStr string) { + err := tk.ExecToErr(ctx, sql) + tk.require.EqualError(err, errStr) +} + +// ExecToErr executes a sql statement and discard results. +func (tk *AsyncTestKit) ExecToErr(ctx context.Context, sql string, args ...interface{}) error { + res, err := tk.Exec(ctx, sql, args...) + if res != nil { + tk.require.NoError(res.Close()) + } + return err +} + // MustQuery query the statements and returns result rows. // If expected result is set it asserts the query result equals expected result. func (tk *AsyncTestKit) MustQuery(ctx context.Context, sql string, args ...interface{}) *Result { diff --git a/util/dbterror/ddl_terror.go b/util/dbterror/ddl_terror.go index 6c80d60ad6ec2..5db0d90c848e1 100644 --- a/util/dbterror/ddl_terror.go +++ b/util/dbterror/ddl_terror.go @@ -429,4 +429,7 @@ var ( ErrTempTableNotAllowedWithTTL = ClassDDL.NewStd(mysql.ErrTempTableNotAllowedWithTTL) // ErrUnsupportedTTLReferencedByFK returns when the TTL config is set for a table referenced by foreign key ErrUnsupportedTTLReferencedByFK = ClassDDL.NewStd(mysql.ErrUnsupportedTTLReferencedByFK) + + // ErrNotSupportedYet returns when tidb does not support this feature. + ErrNotSupportedYet = ClassDDL.NewStd(mysql.ErrNotSupportedYet) ) diff --git a/util/sem/sem.go b/util/sem/sem.go index 161334c880a4a..e2364893b05b2 100644 --- a/util/sem/sem.go +++ b/util/sem/sem.go @@ -136,7 +136,6 @@ func IsInvisibleStatusVar(varName string) bool { func IsInvisibleSysVar(varNameInLower string) bool { switch varNameInLower { case variable.TiDBDDLSlowOprThreshold, // ddl_slow_threshold - variable.TiDBAllowRemoveAutoInc, variable.TiDBCheckMb4ValueInUTF8, variable.TiDBConfig, variable.TiDBEnableSlowLog, diff --git a/util/sem/sem_test.go b/util/sem/sem_test.go index cb0c47ad9225d..d19c2af27adcb 100644 --- a/util/sem/sem_test.go +++ b/util/sem/sem_test.go @@ -81,8 +81,8 @@ func TestIsInvisibleSysVar(t *testing.T) { assert.False(IsInvisibleSysVar(variable.Hostname)) // changes the value to default, but is not invisible assert.False(IsInvisibleSysVar(variable.TiDBEnableEnhancedSecurity)) // should be able to see the mode is on. + assert.False(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc)) - assert.True(IsInvisibleSysVar(variable.TiDBAllowRemoveAutoInc)) assert.True(IsInvisibleSysVar(variable.TiDBCheckMb4ValueInUTF8)) assert.True(IsInvisibleSysVar(variable.TiDBConfig)) assert.True(IsInvisibleSysVar(variable.TiDBEnableSlowLog))