From 099c736ff30af1f05351f3bd037ab7921335bce8 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 27 Jul 2022 18:21:11 +0800 Subject: [PATCH 01/42] util: use tls1.0 as min tls version (#35011) (#36620) close pingcap/tidb#35012 --- util/security.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/util/security.go b/util/security.go index d0e33b9b57837..b12c53914bcf1 100644 --- a/util/security.go +++ b/util/security.go @@ -93,9 +93,9 @@ func ToTLSConfigWithVerify(caPath, certPath, keyPath string, verifyCN []string) if !certPool.AppendCertsFromPEM(ca) { return nil, errors.New("failed to append ca certs") } - + /* #nosec G402 */ tlsCfg := &tls.Config{ - MinVersion: tls.VersionTLS12, + MinVersion: tls.VersionTLS10, Certificates: certificates, RootCAs: certPool, ClientCAs: certPool, @@ -125,9 +125,9 @@ func ToTLSConfigWithVerifyByRawbytes(caData, certData, keyData []byte, verifyCN if !certPool.AppendCertsFromPEM(caData) { return nil, errors.New("failed to append ca certs") } - + /* #nosec G402 */ tlsCfg := &tls.Config{ - MinVersion: tls.VersionTLS12, + MinVersion: tls.VersionTLS10, Certificates: certificates, RootCAs: certPool, ClientCAs: certPool, From c5987a8a569762dfddfdfc2b41c954230053d7af Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 2 Aug 2022 16:12:06 +0800 Subject: [PATCH 02/42] planner: check virtual column for tiflash (#36771) (#36785) close pingcap/tiflash#5513, close pingcap/tidb#36773 --- expression/integration_test.go | 37 ++++++++++++++++++++++++++++++++++ planner/core/find_best_task.go | 13 ++++++++++++ 2 files changed, 50 insertions(+) diff --git a/expression/integration_test.go b/expression/integration_test.go index 31de453b0563b..31a7e1178a2b6 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -3688,6 +3688,43 @@ func TestIssue16973(t *testing.T) { "AND t1.status IN (2,6,10) AND timestampdiff(month, t2.begin_time, date'2020-05-06') = 0;").Check(testkit.Rows("1")) } +func TestShardIndexOnTiFlash(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(id int primary key clustered, a int, b int, unique key uk_expr((tidb_shard(a)),a))") + + // Create virtual tiflash replica info. + dom := domain.GetDomain(tk.Session()) + is := dom.InfoSchema() + db, exists := is.SchemaByName(model.NewCIStr("test")) + require.True(t, exists) + for _, tblInfo := range db.Tables { + if tblInfo.Name.L == "t" { + tblInfo.TiFlashReplica = &model.TiFlashReplicaInfo{ + Count: 1, + Available: true, + } + } + } + tk.MustExec("set @@session.tidb_enforce_mpp = 1") + rows := tk.MustQuery("explain select max(b) from t").Rows() + for _, row := range rows { + line := fmt.Sprintf("%v", row) + require.NotContains(t, line, "tiflash") + } + tk.MustExec("set @@session.tidb_enforce_mpp = 0") + tk.MustExec("set @@session.tidb_allow_mpp = 0") + rows = tk.MustQuery("explain select max(b) from t").Rows() + for _, row := range rows { + line := fmt.Sprintf("%v", row) + require.NotContains(t, line, "tiflash") + } +} + func TestExprPushdownBlacklist(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index c720879d15ed4..249cfdc1f69a6 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1873,6 +1873,19 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid if ts.KeepOrder && ts.Desc && ts.StoreType == kv.TiFlash { return invalidTask, nil } + if ts.StoreType == kv.TiFlash { + for _, col := range ts.schema.Columns { + // In theory, TiFlash does not support virtual expr, but in non-mpp mode, if the cop request only contain table scan, then + // TiDB will fill the virtual column after decoding the cop response(executor.FillVirtualColumnValue), that is to say, the virtual + // columns in Cop request is just a placeholder, so TiFlash can support virtual column in cop request mode. However, virtual column + // with TiDBShard is special, it can be added using create index statement, TiFlash's ddl does not handle create index statement, so + // there is a chance that the TiDBShard's virtual column is not seen by TiFlash, in this case, TiFlash will throw column not found error + if ds.containExprPrefixUk && expression.GcColumnExprIsTidbShard(col.VirtualExpr) { + ds.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because column `" + col.OrigName + "` is a virtual column which is not supported now.") + return invalidTask, nil + } + } + } if prop.TaskTp == property.MppTaskType { if ts.KeepOrder { return invalidTask, nil From 99dfc62fdd36d81c31175c07b5322fdf2e839f14 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 2 Aug 2022 16:24:05 +0800 Subject: [PATCH 03/42] executor: fix panic during update stmt (#36230) (#36528) close pingcap/tidb#32311 --- planner/core/logical_plan_builder.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index b905f973d39fe..b1c6c6f23d74d 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5179,6 +5179,11 @@ func CheckUpdateList(assignFlags []int, updt *Update, newTblID2Table map[int64]t } for i, col := range tbl.WritableCols() { + // schema may be changed between building plan and building executor + // If i >= len(flags), it means the target table has been added columns, then we directly skip the check + if i >= len(flags) { + continue + } if flags[i] >= 0 && col.State != model.StatePublic { return ErrUnknownColumn.GenWithStackByArgs(col.Name, clauseMsg[fieldList]) } From 392a844b0999f9bf1451d7f604619cc7d84a50fd Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 2 Aug 2022 18:12:05 +0800 Subject: [PATCH 04/42] executor: prevent sending cop request for show columns (#36613) (#36629) close pingcap/tidb#36426, ref pingcap/tidb#36496 --- executor/infoschema_reader_test.go | 19 +++++++++++++++++++ executor/show.go | 3 ++- planner/core/expression_rewriter.go | 4 ++-- planner/core/planbuilder.go | 24 ++++++++++++++++++++++-- 4 files changed, 45 insertions(+), 5 deletions(-) diff --git a/executor/infoschema_reader_test.go b/executor/infoschema_reader_test.go index 31fd99e92b60c..fcf16c9d5e60c 100644 --- a/executor/infoschema_reader_test.go +++ b/executor/infoschema_reader_test.go @@ -710,3 +710,22 @@ SELECT `).Check(testkit.Rows("t a b")) } } + +// https://github.com/pingcap/tidb/issues/36426. +func TestShowColumnsWithSubQueryView(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("CREATE TABLE added (`id` int(11), `name` text, `some_date` timestamp);") + tk.MustExec("CREATE TABLE incremental (`id` int(11), `name`text, `some_date` timestamp);") + tk.MustExec("create view temp_view as (select * from `added` where id > (select max(id) from `incremental`));") + // Show columns should not send coprocessor request to the storage. + require.NoError(t, failpoint.Enable("tikvclient/tikvStoreSendReqResult", `return("timeout")`)) + tk.MustQuery("show columns from temp_view;").Check(testkit.Rows( + "id int(11) YES ", + "name text YES ", + "some_date timestamp YES ")) + require.NoError(t, failpoint.Disable("tikvclient/tikvStoreSendReqResult")) +} diff --git a/executor/show.go b/executor/show.go index 8ee1ba9b84607..544cac2e3d2df 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1903,7 +1903,8 @@ func tryFillViewColumnType(ctx context.Context, sctx sessionctx.Context, is info // Take joining system table as an example, `fetchBuildSideRows` and `fetchProbeSideChunks` can be run concurrently. return runWithSystemSession(sctx, func(s sessionctx.Context) error { // Retrieve view columns info. - planBuilder, _ := plannercore.NewPlanBuilder().Init(s, is, &hint.BlockHintProcessor{}) + planBuilder, _ := plannercore.NewPlanBuilder( + plannercore.PlanBuilderOptNoExecution{}).Init(s, is, &hint.BlockHintProcessor{}) if viewLogicalPlan, err := planBuilder.BuildDataSourceFromView(ctx, dbName, tbl); err == nil { viewSchema := viewLogicalPlan.Schema() viewOutputNames := viewLogicalPlan.OutputNames() diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 76ca82e3b1b6b..3c20b60438a4d 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -805,7 +805,7 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex return v, true } np = er.popExistsSubPlan(np) - if len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { + if er.b.disableSubQueryPreprocessing || len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, v.Not) if er.err != nil || !er.asScalar { return v, true @@ -980,7 +980,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S return v, true } np = er.b.buildMaxOneRow(np) - if len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { + if er.b.disableSubQueryPreprocessing || len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { er.p = er.b.buildApplyWithJoinType(er.p, np, LeftOuterJoin) if np.Schema().Len() > 1 { newCols := make([]expression.Expression, 0, np.Schema().Len()) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 4c83651d02bc4..1c937cc672745 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -501,6 +501,9 @@ type PlanBuilder struct { allocIDForCTEStorage int buildingRecursivePartForCTE bool buildingCTE bool + + // disableSubQueryPreprocessing indicates whether to pre-process uncorrelated sub-queries in rewriting stage. + disableSubQueryPreprocessing bool } type handleColHelper struct { @@ -602,14 +605,31 @@ func (b *PlanBuilder) popSelectOffset() { b.selectOffset = b.selectOffset[:len(b.selectOffset)-1] } +// PlanBuilderOpt is used to adjust the plan builder. +type PlanBuilderOpt interface { + Apply(builder *PlanBuilder) +} + +// PlanBuilderOptNoExecution means the plan builder should not run any executor during Build(). +type PlanBuilderOptNoExecution struct{} + +// Apply implements the interface PlanBuilderOpt. +func (p PlanBuilderOptNoExecution) Apply(builder *PlanBuilder) { + builder.disableSubQueryPreprocessing = true +} + // NewPlanBuilder creates a new PlanBuilder. -func NewPlanBuilder() *PlanBuilder { - return &PlanBuilder{ +func NewPlanBuilder(opts ...PlanBuilderOpt) *PlanBuilder { + builder := &PlanBuilder{ outerCTEs: make([]*cteInfo, 0), colMapper: make(map[*ast.ColumnNameExpr]int), handleHelper: &handleColHelper{id2HandleMapStack: make([]map[int64][]HandleCols, 0)}, correlatedAggMapper: make(map[*ast.AggregateFuncExpr]*expression.CorrelatedColumn), } + for _, opt := range opts { + opt.Apply(builder) + } + return builder } // Init initialize a PlanBuilder. From dc157fc3a7133a4d29ea3dcc7bf7a177d279e13e Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 3 Aug 2022 15:40:06 +0800 Subject: [PATCH 05/42] util: let TypeFloat should be decoded as Float32 in Chunk (#35978) (#36096) close pingcap/tidb#35910 --- .github/licenserc.yml | 1 + executor/analyze_test.go | 21 + executor/testdata/analyze_test_data.sql | 726 ++++++++++++++++++++++++ statistics/cmsketch.go | 2 + statistics/histogram.go | 2 + types/datum.go | 6 + types/etc.go | 5 + util/codec/codec.go | 18 + 8 files changed, 781 insertions(+) create mode 100644 executor/testdata/analyze_test_data.sql diff --git a/.github/licenserc.yml b/.github/licenserc.yml index 9f4890a3a1329..68cb310420edc 100644 --- a/.github/licenserc.yml +++ b/.github/licenserc.yml @@ -31,4 +31,5 @@ header: - 'dumpling/' - 'tidb-binlog/driver/example' - 'tidb-binlog/proto/go-binlog/secondary_binlog.pb.go' + - '**/*.sql' comment: on-failure diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 2aff291c5a818..666ee4e921f63 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -18,6 +18,7 @@ import ( "context" "encoding/json" "fmt" + "io/ioutil" "strconv" "strings" "sync" @@ -3389,3 +3390,23 @@ PARTITION BY RANGE ( a ) ( tbl := h.GetTableStats(tableInfo) require.Equal(t, int64(6), tbl.Columns[tableInfo.Columns[0].ID].Histogram.NDV) } + +func TestAnalyzePartitionTableForFloat(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_partition_prune_mode='dynamic'") + tk.MustExec("use test") + tk.MustExec("CREATE TABLE t1 ( id bigint(20) unsigned NOT NULL AUTO_INCREMENT, num float(9,8) DEFAULT NULL, PRIMARY KEY (id) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin PARTITION BY HASH (id) PARTITIONS 128;") + // To reproduce the error we meet in https://github.com/pingcap/tidb/issues/35910, we should use the data provided in this issue + b, err := ioutil.ReadFile("testdata/analyze_test_data.sql") + require.NoError(t, err) + sqls := strings.Split(string(b), ";") + for _, sql := range sqls { + if len(sql) < 1 { + continue + } + tk.MustExec(sql) + } + tk.MustExec("analyze table t1") +} diff --git a/executor/testdata/analyze_test_data.sql b/executor/testdata/analyze_test_data.sql new file mode 100644 index 0000000000000..3f468ffdd2a63 --- /dev/null +++ b/executor/testdata/analyze_test_data.sql @@ -0,0 +1,726 @@ +INSERT INTO t1 (id,num) VALUES + (6404,0.44816685), + (6532,0.7986926), + (6660,0.00085072), + (6788,0.00002967), + (6916,0.0), + (7044,0.03088332), + (7172,0.00046703), + (7300,0.00568179), + (7428,0.33018935), + (7556,0.00455456); +INSERT INTO t1 (id,num) VALUES + (7684,0.36683157), + (7812,0.6140698), + (7940,0.10177323), + (8068,0.00434905), + (8196,0.07348831), + (8324,0.00006405), + (8452,0.02007892), + (8580,0.01050341), + (8708,0.00030031), + (8836,0.02070964); +INSERT INTO t1 (id,num) VALUES + (8964,0.02303072), + (9092,0.44560316), + (9220,0.00069178), + (9348,0.06521516), + (9476,NULL), + (9604,0.01747504), + (9732,0.6579575), + (9860,0.0025045), + (9988,0.03484399), + (10116,0.00193867); +INSERT INTO t1 (id,num) VALUES + (10244,0.0), + (10372,0.0007868), + (10500,0.00031922), + (10628,0.09769934), + (10756,0.00715676), + (10884,NULL), + (11012,0.30375117), + (11140,0.7572566), + (11268,0.00018916), + (11396,0.00969914); +INSERT INTO t1 (id,num) VALUES + (11524,0.05069033), + (11652,0.25621885), + (11780,0.00468599), + (11908,NULL), + (12036,0.00011619), + (12164,0.05855278), + (12292,0.00367099), + (12420,0.9929275), + (12548,0.15242451), + (12676,NULL); +INSERT INTO t1 (id,num) VALUES + (12804,0.0246472), + (12932,0.00472382), + (13060,NULL), + (13188,0.93716323), + (13316,NULL), + (13444,0.01234986), + (13572,0.00076421), + (13700,0.0), + (13828,NULL), + (13956,0.12757756); +INSERT INTO t1 (id,num) VALUES + (14084,0.00484138), + (14212,0.02226818), + (14340,0.00265866), + (14468,0.0016064), + (14596,0.8862497), + (14724,0.00004634), + (14852,0.00835311), + (14980,0.55768555), + (15108,0.04359115), + (15236,0.00277844); +INSERT INTO t1 (id,num) VALUES + (15364,NULL), + (15492,NULL), + (15620,0.01908805), + (15748,0.12949005), + (15876,0.00305156), + (16004,0.00148819), + (16132,0.5249692), + (16260,0.14020711), + (16388,0.16946065), + (16516,NULL); +INSERT INTO t1 (id,num) VALUES + (16644,0.00692773), + (16772,0.00527069), + (16900,NULL), + (17028,0.00114955), + (17156,0.00134839), + (17284,0.02818464), + (17412,NULL), + (17540,0.0), + (17668,0.00351416), + (17796,0.00437542); +INSERT INTO t1 (id,num) VALUES + (17924,0.0012496), + (18052,0.00040084), + (18180,0.35637453), + (18308,0.0220884), + (18436,0.0020587), + (18564,0.00297463), + (18692,0.00487483), + (18820,0.00246549), + (18948,NULL), + (19076,0.9188499); +INSERT INTO t1 (id,num) VALUES + (19204,0.0168393), + (19332,0.00016338), + (19460,0.0), + (19588,0.28465533), + (19716,0.00008774), + (19844,0.0), + (19972,0.46484554), + (20100,0.00113875), + (20228,0.02503506), + (20356,0.01585903); +INSERT INTO t1 (id,num) VALUES + (20484,0.84865177), + (20612,0.00004633), + (20740,0.00002771), + (20868,0.17606646), + (20996,NULL), + (21124,NULL), + (21252,0.06745174), + (21380,NULL), + (21508,0.06476934), + (21636,NULL); +INSERT INTO t1 (id,num) VALUES + (21764,0.05436462), + (21892,0.0), + (22020,0.01013111), + (22148,0.72076213), + (22276,0.00758889), + (22404,NULL), + (22532,0.01502812), + (22660,0.03836808), + (22788,0.01743244), + (22916,0.90398115); +INSERT INTO t1 (id,num) VALUES + (23044,0.39467624), + (23172,0.9271143), + (23300,0.0), + (23428,NULL), + (23556,0.01379231), + (23684,0.0557244), + (23812,NULL), + (23940,0.00256223), + (24068,0.05586718), + (24196,0.00675368); +INSERT INTO t1 (id,num) VALUES + (24324,0.00235754), + (24452,0.00048123), + (24580,0.00684315), + (24708,0.00022108), + (24836,0.8159851), + (24964,0.00049652), + (25092,0.11392157), + (25220,0.21413554), + (25348,0.00061148), + (25476,0.00622709); +INSERT INTO t1 (id,num) VALUES + (25604,0.00213495), + (25732,NULL), + (25860,NULL), + (25988,0.0), + (26116,0.02214408), + (26244,0.03450824), + (26372,0.85039985), + (26500,0.00969678), + (26628,0.01092109), + (26756,0.07614598); +INSERT INTO t1 (id,num) VALUES + (26884,0.02030184), + (27012,0.0), + (27140,0.7304159), + (27268,0.05062966), + (27396,0.07090286), + (27524,0.00589924), + (27652,0.00982369), + (27780,0.01139362), + (27908,0.12942761), + (28036,NULL); +INSERT INTO t1 (id,num) VALUES + (28164,0.973725), + (28292,0.0), + (28420,0.9658361), + (28548,0.7608606), + (28676,0.00033382), + (28804,0.0006193), + (28932,0.01344005), + (29060,0.00213988), + (29188,0.20606396), + (29316,NULL); +INSERT INTO t1 (id,num) VALUES + (29444,0.00029995), + (29572,0.11128831), + (29700,0.00022049), + (29828,0.09755096), + (29956,0.0), + (30084,0.15368505), + (30212,0.00385346), + (30340,NULL), + (30468,0.03264421), + (30596,NULL); +INSERT INTO t1 (id,num) VALUES + (30724,0.02093143), + (30852,0.00251696), + (30980,NULL), + (31108,0.5892975), + (31236,0.15591888), + (31364,NULL), + (31492,0.13643512), + (31620,0.0), + (31748,0.0124521), + (31876,0.10001199); +INSERT INTO t1 (id,num) VALUES + (32004,0.9572708), + (32132,0.2702167), + (32260,0.0), + (32388,NULL), + (32516,0.00197053), + (32644,NULL), + (32772,0.00294003), + (32900,0.0001796), + (33028,0.00109745), + (33156,0.00084148); +INSERT INTO t1 (id,num) VALUES + (33284,0.02335174), + (33412,NULL), + (33540,NULL), + (33668,0.00069347), + (33796,0.00317112), + (33924,0.00514138), + (34052,0.0), + (34180,0.20418067), + (34308,0.21699235), + (34436,0.00151851); +INSERT INTO t1 (id,num) VALUES + (34564,0.00260781), + (34692,0.00042542), + (34820,NULL), + (34948,0.0001932), + (35076,0.0), + (35204,0.00023503), + (35332,0.27567366), + (35460,0.00029132), + (35588,0.0038553), + (35716,0.00028413); +INSERT INTO t1 (id,num) VALUES + (35844,0.0001824), + (35972,0.8047172), + (36100,0.9117202), + (36228,0.00090282), + (36356,0.00003453), + (36484,0.00177934), + (36612,0.02046586), + (36740,0.00146691), + (36868,0.00613281), + (36996,0.02881232); +INSERT INTO t1 (id,num) VALUES + (37124,0.0), + (37252,0.5468523), + (37380,0.0), + (37508,NULL), + (37636,NULL), + (37764,0.08866176), + (37892,0.01583531), + (38020,0.00258847), + (38148,0.0), + (38276,0.0013263); +INSERT INTO t1 (id,num) VALUES + (38404,0.00610365), + (38532,0.0022153), + (38660,0.00080778), + (38788,0.19915293), + (38916,NULL), + (39044,0.0266057), + (39172,0.02324441), + (39300,0.00193156), + (39428,0.00059531), + (39556,0.10964896); +INSERT INTO t1 (id,num) VALUES + (39684,0.02697241), + (39812,0.00929319), + (39940,0.00053213), + (40068,0.0), + (40196,0.00294272), + (40324,NULL), + (40452,NULL), + (40580,NULL), + (40708,0.08479243), + (40836,NULL); +INSERT INTO t1 (id,num) VALUES + (40964,NULL), + (41092,0.2487916), + (41220,NULL), + (41348,0.00043917), + (41476,0.0), + (41604,0.00049506), + (41732,0.00250602), + (41860,0.00011041), + (41988,0.01406988), + (42116,0.00199188); +INSERT INTO t1 (id,num) VALUES + (42244,0.2924709), + (42372,NULL), + (42500,0.01702607), + (42628,NULL), + (42756,0.15324448), + (42884,0.13135524), + (43012,0.01943198), + (43140,0.00188494), + (43268,0.00071816), + (43396,0.00012918); +INSERT INTO t1 (id,num) VALUES + (43524,0.00498049), + (43652,0.13445085), + (43780,0.14880875), + (43908,0.00077732), + (44036,0.54969823), + (44164,0.02765146), + (44292,0.00238847), + (44420,0.93815935), + (44548,0.00112602), + (44676,0.00970578); +INSERT INTO t1 (id,num) VALUES + (44804,0.00539565), + (44932,0.00198959), + (45060,NULL), + (45188,0.0004845), + (45316,0.11085703), + (45444,0.00036987), + (45572,0.00451057), + (45700,0.00005143), + (45828,NULL), + (45956,0.02388487); +INSERT INTO t1 (id,num) VALUES + (46084,0.00037795), + (46212,NULL), + (46340,0.05443271), + (46468,0.02153216), + (46596,NULL), + (46724,0.0), + (46852,0.38851526), + (46980,0.7427731), + (47108,0.03969904), + (47236,0.00078965); +INSERT INTO t1 (id,num) VALUES + (47364,0.03562958), + (47492,0.00100583), + (47620,0.03697728), + (47748,0.08408318), + (47876,0.00057393), + (48004,0.00710408), + (48132,0.01589785), + (48260,NULL), + (48388,0.0029398), + (48516,0.0168582); +INSERT INTO t1 (id,num) VALUES + (48644,0.15535928), + (48772,0.04729551), + (48900,0.00045675), + (49028,0.111338), + (49156,0.05104741), + (49284,0.00957311), + (49412,0.00032368), + (49540,0.0011836), + (49668,0.59423715), + (49796,0.07930596); +INSERT INTO t1 (id,num) VALUES + (49924,0.00970574), + (50052,0.1389579), + (50180,0.01344621), + (50308,0.02605027), + (50436,0.81112456), + (50564,0.08266076), + (50692,0.00064055), + (50820,0.00506184), + (50948,0.07310118), + (51076,0.08932291); +INSERT INTO t1 (id,num) VALUES + (51204,0.00027035), + (51332,0.00340637), + (51460,0.03478277), + (51588,0.00016421), + (51716,0.01174176), + (51844,0.28394622), + (51972,0.09893544), + (52100,0.9233999), + (52228,0.00086335), + (52356,0.0101455); +INSERT INTO t1 (id,num) VALUES + (52484,0.0), + (52612,0.04069425), + (52740,0.00043829), + (52868,0.00023583), + (52996,0.00259821), + (53124,NULL), + (53252,0.13227282), + (53380,NULL), + (53508,0.00010061), + (53636,0.00038794); +INSERT INTO t1 (id,num) VALUES + (53764,0.34211513), + (53892,0.02905787), + (54020,0.06862675), + (54148,NULL), + (54276,0.0), + (54404,0.0), + (54532,0.00019574), + (54660,0.03211332), + (54788,0.61193645), + (54916,0.00076543); +INSERT INTO t1 (id,num) VALUES + (55044,0.0), + (55172,0.00331305), + (55300,0.28308892), + (55428,0.34937865), + (55556,0.0749846), + (55684,0.00009996), + (55812,0.0188099), + (55940,0.0161851), + (56068,NULL), + (56196,0.4822152); +INSERT INTO t1 (id,num) VALUES + (56324,0.00293867), + (56452,0.00180932), + (56580,0.00598398), + (56708,0.9666414), + (56836,0.01628674), + (56964,0.97586924), + (57092,0.000337), + (57220,0.01060274), + (57348,0.0002598), + (57476,0.00377207); +INSERT INTO t1 (id,num) VALUES + (57604,0.00066885), + (57732,0.12196585), + (57860,0.00030946), + (57988,0.20773576), + (58116,0.00061379), + (58244,0.01043255), + (58372,0.00191419), + (58500,0.001862), + (58628,NULL), + (58756,NULL); +INSERT INTO t1 (id,num) VALUES + (58884,NULL), + (59012,0.0), + (59140,0.00029226), + (59268,0.0), + (59396,0.00449778), + (59524,0.02558722), + (59652,NULL), + (59780,0.01153614), + (59908,0.0), + (60036,0.2763834); +INSERT INTO t1 (id,num) VALUES + (60164,NULL), + (60292,NULL), + (60420,0.00149), + (60548,0.9590044), + (60676,0.0), + (60804,0.02398982), + (60932,0.05412427), + (61060,0.00005242), + (61188,0.01232633), + (61316,0.0005189); +INSERT INTO t1 (id,num) VALUES + (61444,0.0), + (61572,0.00196546), + (61700,0.00716419), + (61828,0.0), + (61956,0.0), + (62084,0.00122224), + (62212,0.00988189), + (62340,NULL), + (62468,0.00006262), + (62596,0.00334187); +INSERT INTO t1 (id,num) VALUES + (62724,NULL), + (62852,0.11244337), + (62980,NULL), + (63108,0.4137224), + (63236,0.00094869), + (63364,0.00037691), + (63492,0.09065232), + (63620,0.70078176), + (63748,0.2610763), + (63876,0.58452475); +INSERT INTO t1 (id,num) VALUES + (64004,0.00037725), + (64132,0.01363691), + (64260,0.03295722), + (64388,0.00775894), + (64516,0.00597924), + (64644,0.00098757), + (64772,0.0), + (64900,0.02986313), + (65028,0.00005731), + (65156,0.259131); +INSERT INTO t1 (id,num) VALUES + (65284,0.00433746), + (65412,0.03780772), + (65540,0.00255001), + (65668,0.33036566), + (65796,0.00004995), + (65924,0.00010118), + (66052,0.0), + (66180,0.00053285), + (66308,NULL), + (66436,NULL); +INSERT INTO t1 (id,num) VALUES + (66564,0.00215946), + (66692,0.00012339), + (66820,0.20895952), + (66948,0.01364484), + (67076,0.00161306), + (67204,0.01784489), + (67332,0.51345026), + (67460,0.00022264), + (67588,0.5755771), + (67716,0.07973811); +INSERT INTO t1 (id,num) VALUES + (67844,0.00075325), + (67972,0.02588654), + (68100,0.0066424), + (68228,0.0), + (68356,NULL), + (68484,0.00452684), + (68612,0.0), + (68740,NULL), + (68868,0.00148165), + (68996,0.00069906); +INSERT INTO t1 (id,num) VALUES + (69124,0.36696395), + (69252,0.00176318), + (69380,0.0), + (69508,0.00322113), + (69636,0.66234267), + (69764,NULL), + (69892,0.17466402), + (70020,NULL), + (70148,0.3849655), + (70276,0.00634581); +INSERT INTO t1 (id,num) VALUES + (70404,0.00898295), + (70532,0.10214821), + (70660,0.03422715), + (70788,0.01116229), + (70916,0.40757605), + (71044,0.76234627), + (71172,0.00059572), + (71300,0.00323015), + (71428,0.00015896), + (71556,0.04674813); +INSERT INTO t1 (id,num) VALUES + (71684,0.00002449), + (71812,0.5066135), + (71940,0.94456), + (72068,0.00022794), + (72196,0.00155764), + (72324,0.00060991), + (72452,NULL), + (72580,0.0), + (72708,0.23283394), + (72836,0.04149162); +INSERT INTO t1 (id,num) VALUES + (72964,0.00059429), + (73092,0.0), + (73220,0.0), + (73348,0.19649409), + (73476,0.0017861), + (73604,0.0), + (73732,NULL), + (73860,0.02864335), + (73988,0.9), + (74116,0.01258155); +INSERT INTO t1 (id,num) VALUES + (74244,0.00925873), + (74372,0.00323526), + (74500,0.00258794), + (74628,0.09800948), + (74756,0.00680334), + (74884,NULL), + (75012,0.02857617), + (75140,0.06355022), + (75268,0.0), + (75396,0.01389665); +INSERT INTO t1 (id,num) VALUES + (75524,0.49072826), + (75652,0.00250901), + (75780,0.00081248), + (75908,0.02927421), + (76036,NULL), + (76164,0.01670666), + (76292,0.01635653), + (76420,NULL), + (76548,0.1040872), + (76676,0.10566042); +INSERT INTO t1 (id,num) VALUES + (76804,NULL), + (76932,0.00663577), + (77060,0.21661888), + (77188,0.45707023), + (77316,0.00272538), + (77444,0.00419203), + (77572,NULL), + (77700,0.00237827), + (77828,0.00146376), + (77956,0.03254667); +INSERT INTO t1 (id,num) VALUES + (78084,0.00237725), + (78212,NULL), + (78340,0.02337047), + (78468,0.6066429), + (78596,NULL), + (78724,0.99989676), + (78852,0.02890464), + (78980,0.07082515), + (79108,NULL), + (79236,0.0019473); +INSERT INTO t1 (id,num) VALUES + (79364,0.00358919), + (79492,NULL), + (79620,0.28481358), + (79748,0.00222982), + (79876,NULL), + (80004,0.00547454), + (80132,NULL), + (80260,0.0002941), + (80388,0.00061776), + (80516,0.00002488); +INSERT INTO t1 (id,num) VALUES + (80644,0.09067673), + (80772,0.97947466), + (80900,NULL), + (81028,0.00611359), + (81156,0.0001545), + (81284,NULL), + (81412,0.02183841), + (81540,0.01013091), + (81668,0.00044435), + (81796,0.00101722); +INSERT INTO t1 (id,num) VALUES + (81924,0.0), + (82052,0.01291962), + (82180,0.00089332), + (82308,NULL), + (82436,0.38697731), + (82564,0.0), + (82692,0.06748839), + (82820,0.00505329), + (82948,0.00197535), + (83076,NULL); +INSERT INTO t1 (id,num) VALUES + (83204,NULL), + (83332,0.02577116), + (83460,0.00040706), + (83588,0.00387944), + (83716,0.00060442), + (83844,0.00140539), + (83972,0.02212065), + (84100,0.0009958), + (84228,0.21305683), + (84356,0.01075263); +INSERT INTO t1 (id,num) VALUES + (84484,NULL), + (84612,0.04209423), + (84740,0.00016298), + (84868,0.00030807), + (84996,0.0033516), + (85124,0.01264506), + (85252,NULL), + (85380,0.00797018), + (85508,0.03014666), + (85636,0.0); +INSERT INTO t1 (id,num) VALUES + (85764,0.0198979), + (85892,NULL), + (86020,NULL), + (86148,0.00323176), + (86276,0.56539536), + (86404,0.04482054), + (86532,0.07097953), + (86660,0.00034572), + (86788,0.0), + (86916,0.00194455); +INSERT INTO t1 (id,num) VALUES + (87044,0.0), + (87172,0.0), + (87300,0.00335881), + (87428,0.6016782), + (87556,0.02347282), + (87684,0.0), + (87812,0.00998413), + (87940,0.10825046), + (88068,0.11218246), + (88196,NULL); +INSERT INTO t1 (id,num) VALUES + (88324,0.16741024), + (88452,0.00222767), + (88580,0.00215809), + (3,0.05589541), + (131,0.0), + (259,0.00747037), + (387,0.00063459), + (515,0.28516522), + (643,0.07687332), + (771,0.00015758); +INSERT INTO t1 (id,num) VALUES + (6019,0.36197448), + (6147,0.00694031), + (6275,NULL), + (6403,0.05160758), + (6531,0.98084956), + (6659,0.0), + (6787,0.00084533), + (6915,0.05876909), + (7043,NULL), + (7171,NULL); diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 65569963aaed9..c39999e922320 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -767,6 +767,8 @@ func MergePartTopN2GlobalTopN(sc *stmtctx.StatementContext, version int, topNs [ if types.IsTypeTime(hists[0].Tp.GetType()) { // handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne. _, d, err = codec.DecodeAsDateTime(val.Encoded, hists[0].Tp.GetType(), sc.TimeZone) + } else if types.IsTypeFloat(hists[0].Tp.GetType()) { + _, d, err = codec.DecodeAsFloat32(val.Encoded, hists[0].Tp.GetType()) } else { _, d, err = codec.DecodeOne(val.Encoded) } diff --git a/statistics/histogram.go b/statistics/histogram.go index 59571fbfbcc63..0c4764b8a90e6 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -2120,6 +2120,8 @@ func MergePartitionHist2GlobalHist(sc *stmtctx.StatementContext, hists []*Histog if types.IsTypeTime(hists[0].Tp.GetType()) { // handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne. _, d, err = codec.DecodeAsDateTime(meta.Encoded, hists[0].Tp.GetType(), sc.TimeZone) + } else if types.IsTypeFloat(hists[0].Tp.GetType()) { + _, d, err = codec.DecodeAsFloat32(meta.Encoded, hists[0].Tp.GetType()) } else { _, d, err = codec.DecodeOne(meta.Encoded) } diff --git a/types/datum.go b/types/datum.go index 8bc8fddafe870..dd3a8f14e9a85 100644 --- a/types/datum.go +++ b/types/datum.go @@ -183,6 +183,12 @@ func (d *Datum) SetFloat32(f float32) { d.i = int64(math.Float64bits(float64(f))) } +// SetFloat32FromF64 sets float32 values from f64 +func (d *Datum) SetFloat32FromF64(f float64) { + d.k = KindFloat32 + d.i = int64(math.Float64bits(f)) +} + // GetString gets string value. func (d *Datum) GetString() string { return string(hack.String(d.b)) diff --git a/types/etc.go b/types/etc.go index c1cfb1aa0877f..4aa5576cfec25 100644 --- a/types/etc.go +++ b/types/etc.go @@ -66,6 +66,11 @@ func IsTypeTime(tp byte) bool { return tp == mysql.TypeDatetime || tp == mysql.TypeDate || tp == mysql.TypeTimestamp } +// IsTypeFloat indicates whether the type is TypeFloat +func IsTypeFloat(tp byte) bool { + return tp == mysql.TypeFloat +} + // IsTypeInteger returns a boolean indicating whether the tp is integer type. func IsTypeInteger(tp byte) bool { switch tp { diff --git a/util/codec/codec.go b/util/codec/codec.go index 7d6104ff592bc..9c5099a4c4ae9 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -762,6 +762,8 @@ func DecodeRange(b []byte, size int, idxColumnTypes []byte, loc *time.Location) if types.IsTypeTime(idxColumnTypes[i]) { // handle datetime values specially since they are encoded to int and we'll get int values if using DecodeOne. b, d, err = DecodeAsDateTime(b, idxColumnTypes[i], loc) + } else if types.IsTypeFloat(idxColumnTypes[i]) { + b, d, err = DecodeAsFloat32(b, idxColumnTypes[i]) } else { b, d, err = DecodeOne(b) } @@ -899,6 +901,22 @@ func DecodeAsDateTime(b []byte, tp byte, loc *time.Location) (remain []byte, d t return b, d, nil } +// DecodeAsFloat32 decodes value for mysql.TypeFloat +func DecodeAsFloat32(b []byte, tp byte) (remain []byte, d types.Datum, err error) { + if len(b) < 1 || tp != mysql.TypeFloat { + return nil, d, errors.New("invalid encoded key") + } + flag := b[0] + b = b[1:] + if flag != floatFlag { + return b, d, errors.Errorf("invalid encoded key flag %v for DecodeAsFloat32", flag) + } + var v float64 + b, v, err = DecodeFloat(b) + d.SetFloat32FromF64(v) + return b, d, nil +} + // CutOne cuts the first encoded value from b. // It will return the first encoded item and the remains as byte slice. func CutOne(b []byte) (data []byte, remain []byte, err error) { From 7fc6661a3415117c3bd0ed8ff7f49ea004216b37 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 3 Aug 2022 15:56:06 +0800 Subject: [PATCH 06/42] ddl: invalid multiple MAXVALUE partitions (#36329) (#36345) (#36447) close pingcap/tidb#36329 --- ddl/db_partition_test.go | 8 ++++++++ ddl/ddl_api.go | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index c17c9a648790f..16d2f22214925 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -550,6 +550,14 @@ create table log_message_1 ( "partition p1 values less than ('A'));", dbterror.ErrRangeNotIncreasing, }, + { + "create table t(d datetime)" + + "partition by range columns (d) (" + + "partition p0 values less than ('2022-01-01')," + + "partition p1 values less than (MAXVALUE), " + + "partition p2 values less than (MAXVALUE));", + dbterror.ErrRangeNotIncreasing, + }, { "CREATE TABLE t1(c0 INT) PARTITION BY HASH((NOT c0)) PARTITIONS 2;", dbterror.ErrPartitionFunctionIsNotAllowed, diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 27e509439d652..7c867b9b0b3d6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2704,7 +2704,7 @@ func checkTwoRangeColumns(ctx sessionctx.Context, curr, prev *model.PartitionDef } for i := 0; i < len(pi.Columns); i++ { // Special handling for MAXVALUE. - if strings.EqualFold(curr.LessThan[i], partitionMaxValue) { + if strings.EqualFold(curr.LessThan[i], partitionMaxValue) && !strings.EqualFold(prev.LessThan[i], partitionMaxValue) { // If current is maxvalue, it certainly >= previous. return true, nil } From 9dfcd40048b3966aff9044e20d09e4664a68b7a5 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 4 Aug 2022 14:56:06 +0800 Subject: [PATCH 07/42] lightning: add ReadIndexNotReady as retryable ingest error (#36574) (#36874) close pingcap/tidb#36566 --- br/pkg/lightning/backend/local/local.go | 12 ++++++++++++ br/pkg/lightning/backend/local/local_test.go | 9 +++++++++ br/pkg/lightning/common/errors.go | 15 ++++++++------- errors.toml | 5 +++++ 4 files changed, 34 insertions(+), 7 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 0d19c6887c4f8..b05091ae00bcd 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1808,6 +1808,18 @@ func (local *local) isIngestRetryable( return retryNone, nil, common.ErrKVServerIsBusy.GenWithStack(errPb.GetMessage()) case errPb.RegionNotFound != nil: return retryNone, nil, common.ErrKVRegionNotFound.GenWithStack(errPb.GetMessage()) + case errPb.ReadIndexNotReady != nil: + // this error happens when this region is splitting, the error might be: + // read index not ready, reason can not read index due to split, region 64037 + // we have paused schedule, but it's temporary, + // if next request takes a long time, there's chance schedule is enabled again + // or on key range border, another engine sharing this region tries to split this + // region may cause this error too. + newRegion, err = getRegion() + if err != nil { + return retryNone, nil, errors.Trace(err) + } + return retryWrite, newRegion, common.ErrKVReadIndexNotReady.GenWithStack(errPb.GetMessage()) } return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) } diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 0711bfb1fc463..5ee3c31ebe13c 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -509,6 +509,15 @@ func TestIsIngestRetryable(t *testing.T) { retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) require.Equal(t, retryNone, retryType) require.EqualError(t, err, "non-retryable error: unknown error") + + resp.Error = &errorpb.Error{ + ReadIndexNotReady: &errorpb.ReadIndexNotReady{ + Reason: "test", + }, + } + retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) + require.Equal(t, retryWrite, retryType) + require.Error(t, err) } type testIngester struct{} diff --git a/br/pkg/lightning/common/errors.go b/br/pkg/lightning/common/errors.go index ccb465ec4b86a..4f1c598a84535 100644 --- a/br/pkg/lightning/common/errors.go +++ b/br/pkg/lightning/common/errors.go @@ -70,13 +70,14 @@ var ( ErrCreatePDClient = errors.Normalize("create pd client error", errors.RFCCodeText("Lightning:PD:ErrCreatePDClient")) ErrPauseGC = errors.Normalize("pause gc error", errors.RFCCodeText("Lightning:PD:ErrPauseGC")) - ErrCheckKVVersion = errors.Normalize("check tikv version error", errors.RFCCodeText("Lightning:KV:ErrCheckKVVersion")) - ErrCreateKVClient = errors.Normalize("create kv client error", errors.RFCCodeText("Lightning:KV:ErrCreateKVClient")) - ErrCheckMultiIngest = errors.Normalize("check multi-ingest support error", errors.RFCCodeText("Lightning:KV:ErrCheckMultiIngest")) - ErrKVEpochNotMatch = errors.Normalize("epoch not match", errors.RFCCodeText("Lightning:KV:EpochNotMatch")) - ErrKVNotLeader = errors.Normalize("not leader", errors.RFCCodeText("Lightning:KV:NotLeader")) - ErrKVServerIsBusy = errors.Normalize("server is busy", errors.RFCCodeText("Lightning:KV:ServerIsBusy")) - ErrKVRegionNotFound = errors.Normalize("region not found", errors.RFCCodeText("Lightning:KV:RegionNotFound")) + ErrCheckKVVersion = errors.Normalize("check tikv version error", errors.RFCCodeText("Lightning:KV:ErrCheckKVVersion")) + ErrCreateKVClient = errors.Normalize("create kv client error", errors.RFCCodeText("Lightning:KV:ErrCreateKVClient")) + ErrCheckMultiIngest = errors.Normalize("check multi-ingest support error", errors.RFCCodeText("Lightning:KV:ErrCheckMultiIngest")) + ErrKVEpochNotMatch = errors.Normalize("epoch not match", errors.RFCCodeText("Lightning:KV:EpochNotMatch")) + ErrKVNotLeader = errors.Normalize("not leader", errors.RFCCodeText("Lightning:KV:NotLeader")) + ErrKVServerIsBusy = errors.Normalize("server is busy", errors.RFCCodeText("Lightning:KV:ServerIsBusy")) + ErrKVRegionNotFound = errors.Normalize("region not found", errors.RFCCodeText("Lightning:KV:RegionNotFound")) + ErrKVReadIndexNotReady = errors.Normalize("read index not ready", errors.RFCCodeText("Lightning:KV:ReadIndexNotReady")) ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend")) ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile")) diff --git a/errors.toml b/errors.toml index d8112d297e6fe..c9e3aa6c9da9b 100755 --- a/errors.toml +++ b/errors.toml @@ -371,6 +371,11 @@ error = ''' not leader ''' +["Lightning:KV:ReadIndexNotReady"] +error = ''' +read index not ready +''' + ["Lightning:KV:RegionNotFound"] error = ''' region not found From 341e65cd333c40b6ffb6d6f116ba24d3ae38236e Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 4 Aug 2022 16:30:06 +0800 Subject: [PATCH 08/42] lightning: add more retryable err (#36673) (#36719) close pingcap/tidb#36674 --- br/pkg/lightning/backend/local/local.go | 16 +- br/pkg/lightning/backend/local/local_test.go | 7 + br/pkg/lightning/common/retry.go | 27 ++- br/pkg/lightning/common/retry_test.go | 3 + br/pkg/lightning/restore/meta_manager.go | 225 ++++++++++-------- br/pkg/lightning/restore/meta_manager_test.go | 63 ++++- 6 files changed, 231 insertions(+), 110 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index b05091ae00bcd..958be0434ad21 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "fmt" + "io" "math" "os" "path/filepath" @@ -1133,6 +1134,19 @@ const ( retryIngest ) +func (local *local) isRetryableTiKVWriteError(err error) bool { + err = errors.Cause(err) + // io.EOF is not retryable in normal case + // but on TiKV restart, if we're writing to TiKV(through GRPC) + // it might return io.EOF(it's GRPC Unavailable in most case), + // we need to retry on this error. + // see SendMsg in https://pkg.go.dev/google.golang.org/grpc#ClientStream + if err == io.EOF { + return true + } + return common.IsRetryableError(err) +} + func (local *local) writeAndIngestPairs( ctx context.Context, engine *Engine, @@ -1150,7 +1164,7 @@ loopWrite: var rangeStats rangeStats metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engine, region, start, end, regionSplitSize, regionSplitKeys) if err != nil { - if !common.IsRetryableError(err) { + if !local.isRetryableTiKVWriteError(err) { return err } diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 5ee3c31ebe13c..e5931808438a8 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -18,6 +18,7 @@ import ( "bytes" "context" "encoding/binary" + "io" "math" "math/rand" "os" @@ -1244,3 +1245,9 @@ func TestGetRegionSplitSizeKeys(t *testing.T) { require.Equal(t, int64(1), splitSize) require.Equal(t, int64(2), splitKeys) } + +func TestLocalIsRetryableTiKVWriteError(t *testing.T) { + l := local{} + require.True(t, l.isRetryableTiKVWriteError(io.EOF)) + require.True(t, l.isRetryableTiKVWriteError(errors.Trace(io.EOF))) +} diff --git a/br/pkg/lightning/common/retry.go b/br/pkg/lightning/common/retry.go index accf7423414b4..7cd71dd1f1625 100644 --- a/br/pkg/lightning/common/retry.go +++ b/br/pkg/lightning/common/retry.go @@ -20,6 +20,7 @@ import ( "io" "net" "os" + "strings" "syscall" "github.com/go-sql-driver/mysql" @@ -30,6 +31,28 @@ import ( "google.golang.org/grpc/status" ) +// some component doesn't have an accurate named error or transform a named error into string, +// so we need to check by error message, +// such as distsql.Checksum which transforms tikv other-error into its own error +var retryableErrorMsgList = []string{ + "is not fully replicated", + // for cluster >= 4.x, lightning calls distsql.Checksum to do checksum + // this error happens on when distsql.Checksum calls TiKV + // see https://github.com/pingcap/tidb/blob/2c3d4f1ae418881a95686e8b93d4237f2e76eec6/store/copr/coprocessor.go#L941 + "coprocessor task terminated due to exceeding the deadline", +} + +func isRetryableFromErrorMessage(err error) bool { + msg := err.Error() + msgLower := strings.ToLower(msg) + for _, errStr := range retryableErrorMsgList { + if strings.Contains(msgLower, errStr) { + return true + } + } + return false +} + // IsRetryableError returns whether the error is transient (e.g. network // connection dropped) or irrecoverable (e.g. user pressing Ctrl+C). This // function returns `false` (irrecoverable) if `err == nil`. @@ -91,10 +114,8 @@ func isSingleRetryableError(err error) bool { switch status.Code(err) { case codes.DeadlineExceeded, codes.NotFound, codes.AlreadyExists, codes.PermissionDenied, codes.ResourceExhausted, codes.Aborted, codes.OutOfRange, codes.Unavailable, codes.DataLoss: return true - case codes.Unknown: - return false default: - return false + return isRetryableFromErrorMessage(err) } } } diff --git a/br/pkg/lightning/common/retry_test.go b/br/pkg/lightning/common/retry_test.go index 670004260f5a1..004cce85622de 100644 --- a/br/pkg/lightning/common/retry_test.go +++ b/br/pkg/lightning/common/retry_test.go @@ -94,4 +94,7 @@ func TestIsRetryableError(t *testing.T) { require.False(t, IsRetryableError(multierr.Combine(context.Canceled, context.Canceled))) require.True(t, IsRetryableError(multierr.Combine(&net.DNSError{IsTimeout: true}, &net.DNSError{IsTimeout: true}))) require.False(t, IsRetryableError(multierr.Combine(context.Canceled, &net.DNSError{IsTimeout: true}))) + + require.True(t, IsRetryableError(errors.Errorf("region %d is not fully replicated", 1234))) + require.True(t, IsRetryableError(errors.New("other error: Coprocessor task terminated due to exceeding the deadline"))) } diff --git a/br/pkg/lightning/restore/meta_manager.go b/br/pkg/lightning/restore/meta_manager.go index 8eace8c5f979d..29689d42da1c7 100644 --- a/br/pkg/lightning/restore/meta_manager.go +++ b/br/pkg/lightning/restore/meta_manager.go @@ -8,6 +8,7 @@ import ( "encoding/json" "fmt" "strings" + "time" "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/backend/tidb" @@ -21,6 +22,11 @@ import ( "go.uber.org/zap" ) +const ( + maxRetryOnStatusConflict = 30 + maxBackoffTime = 30 * time.Second +) + type metaMgrBuilder interface { Init(ctx context.Context) error TaskMetaMgr(pd *pdutil.PdController) taskMetaMgr @@ -179,124 +185,149 @@ func (m *dbTableMetaMgr) AllocTableRowIDs(ctx context.Context, rawRowIDMax int64 return nil, 0, errors.Annotate(err, "enable pessimistic transaction failed") } needAutoID := common.TableHasAutoRowID(m.tr.tableInfo.Core) || m.tr.tableInfo.Core.GetAutoIncrementColInfo() != nil || m.tr.tableInfo.Core.ContainsAutoRandomBits() - err = exec.Transact(ctx, "init table allocator base", func(ctx context.Context, tx *sql.Tx) error { - rows, err := tx.QueryContext( - ctx, - fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName), - m.tr.tableInfo.ID, - ) - if err != nil { - return errors.Trace(err) - } - defer rows.Close() - var ( - metaTaskID, rowIDBase, rowIDMax, maxRowIDMax int64 - totalKvs, totalBytes, checksum uint64 - statusValue string - ) - for rows.Next() { - if err = rows.Scan(&metaTaskID, &rowIDBase, &rowIDMax, &totalKvs, &totalBytes, &checksum, &statusValue); err != nil { - return errors.Trace(err) - } - status, err := parseMetaStatus(statusValue) + tableChecksumingMsg := "Target table is calculating checksum. Please wait until the checksum is finished and try again." + doAllocTableRowIDsFn := func() error { + return exec.Transact(ctx, "init table allocator base", func(ctx context.Context, tx *sql.Tx) error { + rows, err := tx.QueryContext( + ctx, + fmt.Sprintf("SELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from %s WHERE table_id = ? FOR UPDATE", m.tableName), + m.tr.tableInfo.ID, + ) if err != nil { - return err + return errors.Trace(err) } + defer rows.Close() + var ( + metaTaskID, rowIDBase, rowIDMax, maxRowIDMax int64 + totalKvs, totalBytes, checksum uint64 + statusValue string + ) + for rows.Next() { + if err = rows.Scan(&metaTaskID, &rowIDBase, &rowIDMax, &totalKvs, &totalBytes, &checksum, &statusValue); err != nil { + return errors.Trace(err) + } + status, err := parseMetaStatus(statusValue) + if err != nil { + return err + } - // skip finished meta - if status >= metaStatusFinished { - continue - } + // skip finished meta + if status >= metaStatusFinished { + continue + } - if status == metaStatusChecksuming { - return common.ErrAllocTableRowIDs.GenWithStack("Target table is calculating checksum. Please wait until the checksum is finished and try again.") - } + if status == metaStatusChecksuming { + return common.ErrAllocTableRowIDs.GenWithStack(tableChecksumingMsg) + } - if metaTaskID == m.taskID { - curStatus = status - baseChecksum = checksum - baseTotalKvs = totalKvs - baseTotalBytes = totalBytes - if status >= metaStatusRowIDAllocated { - if rowIDMax-rowIDBase != rawRowIDMax { - return common.ErrAllocTableRowIDs.GenWithStack("verify allocator base failed. local: '%d', meta: '%d'", rawRowIDMax, rowIDMax-rowIDBase) + if metaTaskID == m.taskID { + curStatus = status + baseChecksum = checksum + baseTotalKvs = totalKvs + baseTotalBytes = totalBytes + if status >= metaStatusRowIDAllocated { + if rowIDMax-rowIDBase != rawRowIDMax { + return common.ErrAllocTableRowIDs.GenWithStack("verify allocator base failed. local: '%d', meta: '%d'", rawRowIDMax, rowIDMax-rowIDBase) + } + newRowIDBase = rowIDBase + newRowIDMax = rowIDMax + break } - newRowIDBase = rowIDBase - newRowIDMax = rowIDMax - break + continue } - continue - } - // other tasks has finished this logic, we needn't do again. - if status >= metaStatusRowIDAllocated { - newStatus = metaStatusRestoreStarted - } + // other tasks has finished this logic, we needn't do again. + if status >= metaStatusRowIDAllocated { + newStatus = metaStatusRestoreStarted + } - if rowIDMax > maxRowIDMax { - maxRowIDMax = rowIDMax + if rowIDMax > maxRowIDMax { + maxRowIDMax = rowIDMax + } + } + if rows.Err() != nil { + return errors.Trace(rows.Err()) } - } - if rows.Err() != nil { - return errors.Trace(rows.Err()) - } - // no enough info are available, fetch row_id max for table - if curStatus == metaStatusInitial { - if needAutoID && maxRowIDMax == 0 { - // NOTE: currently, if a table contains auto_incremental unique key and _tidb_rowid, - // the `show table next_row_id` will returns the unique key field only. - var autoIDField string - for _, col := range m.tr.tableInfo.Core.Columns { - if mysql.HasAutoIncrementFlag(col.GetFlag()) { - autoIDField = col.Name.L - break - } else if mysql.HasPriKeyFlag(col.GetFlag()) && m.tr.tableInfo.Core.AutoRandomBits > 0 { - autoIDField = col.Name.L - break + // no enough info are available, fetch row_id max for table + if curStatus == metaStatusInitial { + if needAutoID && maxRowIDMax == 0 { + // NOTE: currently, if a table contains auto_incremental unique key and _tidb_rowid, + // the `show table next_row_id` will returns the unique key field only. + var autoIDField string + for _, col := range m.tr.tableInfo.Core.Columns { + if mysql.HasAutoIncrementFlag(col.GetFlag()) { + autoIDField = col.Name.L + break + } else if mysql.HasPriKeyFlag(col.GetFlag()) && m.tr.tableInfo.Core.AutoRandomBits > 0 { + autoIDField = col.Name.L + break + } + } + if len(autoIDField) == 0 && common.TableHasAutoRowID(m.tr.tableInfo.Core) { + autoIDField = model.ExtraHandleName.L + } + if len(autoIDField) == 0 { + return common.ErrAllocTableRowIDs.GenWithStack("table %s contains auto increment id or _tidb_rowid, but target field not found", m.tr.tableName) + } + + autoIDInfos, err := tidb.FetchTableAutoIDInfos(ctx, tx, m.tr.tableName) + if err != nil { + return errors.Trace(err) + } + found := false + for _, info := range autoIDInfos { + if strings.ToLower(info.Column) == autoIDField { + maxRowIDMax = info.NextID - 1 + found = true + break + } + } + if !found { + return common.ErrAllocTableRowIDs.GenWithStack("can't fetch previous auto id base for table %s field '%s'", m.tr.tableName, autoIDField) } } - if len(autoIDField) == 0 && common.TableHasAutoRowID(m.tr.tableInfo.Core) { - autoIDField = model.ExtraHandleName.L - } - if len(autoIDField) == 0 { - return common.ErrAllocTableRowIDs.GenWithStack("table %s contains auto increment id or _tidb_rowid, but target field not found", m.tr.tableName) + newRowIDBase = maxRowIDMax + newRowIDMax = newRowIDBase + rawRowIDMax + // table contains no data, can skip checksum + if needAutoID && newRowIDBase == 0 && newStatus < metaStatusRestoreStarted { + newStatus = metaStatusRestoreStarted } - autoIDInfos, err := tidb.FetchTableAutoIDInfos(ctx, tx, m.tr.tableName) + // nolint:gosec + query := fmt.Sprintf("update %s set row_id_base = ?, row_id_max = ?, status = ? where table_id = ? and task_id = ?", m.tableName) + _, err := tx.ExecContext(ctx, query, newRowIDBase, newRowIDMax, newStatus.String(), m.tr.tableInfo.ID, m.taskID) if err != nil { return errors.Trace(err) } - found := false - for _, info := range autoIDInfos { - if strings.ToLower(info.Column) == autoIDField { - maxRowIDMax = info.NextID - 1 - found = true - break - } - } - if !found { - return common.ErrAllocTableRowIDs.GenWithStack("can't fetch previous auto id base for table %s field '%s'", m.tr.tableName, autoIDField) - } - } - newRowIDBase = maxRowIDMax - newRowIDMax = newRowIDBase + rawRowIDMax - // table contains no data, can skip checksum - if needAutoID && newRowIDBase == 0 && newStatus < metaStatusRestoreStarted { - newStatus = metaStatusRestoreStarted - } - // nolint:gosec - query := fmt.Sprintf("update %s set row_id_base = ?, row_id_max = ?, status = ? where table_id = ? and task_id = ?", m.tableName) - _, err := tx.ExecContext(ctx, query, newRowIDBase, newRowIDMax, newStatus.String(), m.tr.tableInfo.ID, m.taskID) - if err != nil { - return errors.Trace(err) + curStatus = newStatus } - - curStatus = newStatus + return nil + }) + } + // TODO: the retry logic is duplicate with code in local.writeAndIngestByRanges, should encapsulate it later. + // max retry backoff time: 2+4+8+16+30*26=810s + backOffTime := time.Second + for i := 0; i < maxRetryOnStatusConflict; i++ { + err = doAllocTableRowIDsFn() + if err == nil || !strings.Contains(err.Error(), tableChecksumingMsg) { + break } - return nil - }) + // we only retry if it's tableChecksuming error, it happens during parallel import. + // for detail see https://docs.pingcap.com/tidb/stable/tidb-lightning-distributed-import + log.L().Warn("target table is doing checksum, will try again", + zap.Int("retry time", i+1), log.ShortError(err)) + backOffTime *= 2 + if backOffTime > maxBackoffTime { + backOffTime = maxBackoffTime + } + select { + case <-time.After(backOffTime): + case <-ctx.Done(): + return nil, 0, errors.Trace(ctx.Err()) + } + } if err != nil { return nil, 0, errors.Trace(err) } diff --git a/br/pkg/lightning/restore/meta_manager_test.go b/br/pkg/lightning/restore/meta_manager_test.go index 8480bf077d6de..15bbffadd42f5 100644 --- a/br/pkg/lightning/restore/meta_manager_test.go +++ b/br/pkg/lightning/restore/meta_manager_test.go @@ -10,6 +10,7 @@ import ( "time" "github.com/DATA-DOG/go-sqlmock" + "github.com/pingcap/errors" "github.com/pingcap/tidb/br/pkg/lightning/checkpoints" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" @@ -87,7 +88,7 @@ func TestAllocTableRowIDsSingleTable(t *testing.T) { } nextID := int64(1) updateArgs := []driver.Value{int64(0), int64(10), "restore", int64(1), int64(1)} - s.prepareMock(rows, &nextID, updateArgs, nil, nil) + s.prepareMock(rows, &nextID, updateArgs, nil, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -108,7 +109,7 @@ func TestAllocTableRowIDsSingleTableAutoIDNot0(t *testing.T) { nextID := int64(999) updateArgs := []driver.Value{int64(998), int64(1008), "allocated", int64(1), int64(1)} newStatus := "restore" - s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus) + s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -130,7 +131,7 @@ func TestAllocTableRowIDsSingleTableContainsData(t *testing.T) { nextID := int64(999) checksum := verification.MakeKVChecksum(1, 2, 3) updateArgs := []driver.Value{int64(998), int64(1008), "allocated", int64(1), int64(1)} - s.prepareMock(rows, &nextID, updateArgs, &checksum, nil) + s.prepareMock(rows, &nextID, updateArgs, &checksum, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -155,7 +156,7 @@ func TestAllocTableRowIDsSingleTableSkipChecksum(t *testing.T) { nextID := int64(999) newStatus := "restore" updateArgs := []driver.Value{int64(998), int64(1008), "allocated", int64(1), int64(1)} - s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus) + s.prepareMock(rows, &nextID, updateArgs, nil, &newStatus, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -175,7 +176,7 @@ func TestAllocTableRowIDsAllocated(t *testing.T) { {int64(1), int64(998), int64(1008), uint64(0), uint64(0), uint64(0), metaStatusRowIDAllocated.String()}, } checksum := verification.MakeKVChecksum(2, 1, 3) - s.prepareMock(rows, nil, nil, &checksum, nil) + s.prepareMock(rows, nil, nil, &checksum, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -194,7 +195,7 @@ func TestAllocTableRowIDsFinished(t *testing.T) { {int64(1), int64(998), int64(1008), uint64(1), uint64(2), uint64(3), metaStatusRestoreStarted.String()}, } checksum := verification.MakeKVChecksum(2, 1, 3) - s.prepareMock(rows, nil, nil, nil, nil) + s.prepareMock(rows, nil, nil, nil, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -214,7 +215,7 @@ func TestAllocTableRowIDsMultiTasksInit(t *testing.T) { } nextID := int64(1) updateArgs := []driver.Value{int64(0), int64(10), "restore", int64(1), int64(1)} - s.prepareMock(rows, &nextID, updateArgs, nil, nil) + s.prepareMock(rows, &nextID, updateArgs, nil, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -234,7 +235,7 @@ func TestAllocTableRowIDsMultiTasksAllocated(t *testing.T) { {int64(2), int64(0), int64(100), uint64(0), uint64(0), uint64(0), metaStatusRowIDAllocated.String()}, } updateArgs := []driver.Value{int64(100), int64(110), "restore", int64(1), int64(1)} - s.prepareMock(rows, nil, updateArgs, nil, nil) + s.prepareMock(rows, nil, updateArgs, nil, nil, false) ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) require.NoError(t, err) @@ -244,10 +245,49 @@ func TestAllocTableRowIDsMultiTasksAllocated(t *testing.T) { require.Equal(t, 0, s.checksumMgr.callCnt) } -func (s *metaMgrSuite) prepareMock(rowsVal [][]driver.Value, nextRowID *int64, updateArgs []driver.Value, checksum *verification.KVChecksum, updateStatus *string) { +func TestAllocTableRowIDsRetryOnTableInChecksum(t *testing.T) { + s, clean := newMetaMgrSuite(t) + defer clean() + + ctx := context.WithValue(context.Background(), &checksumManagerKey, s.checksumMgr) s.mockDB.ExpectExec("SET SESSION tidb_txn_mode = 'pessimistic';"). WillReturnResult(sqlmock.NewResult(int64(0), int64(0))) + s.mockDB.ExpectBegin() + s.mockDB.ExpectQuery("\\QSELECT task_id, row_id_base, row_id_max, total_kvs_base, total_bytes_base, checksum_base, status from `test`.`table_meta` WHERE table_id = ? FOR UPDATE\\E"). + WithArgs(int64(1)). + WillReturnError(errors.New("mock err")) + s.mockDB.ExpectRollback() + // should not retry + _, _, err := s.mgr.AllocTableRowIDs(ctx, 10) + require.Error(t, err) + require.Contains(t, err.Error(), "mock err") + + rows := [][]driver.Value{ + {int64(1), int64(0), int64(0), uint64(0), uint64(0), uint64(0), metaStatusChecksuming.String()}, + } + s.prepareMock(rows, nil, nil, nil, nil, true) + rows = [][]driver.Value{ + {int64(1), int64(0), int64(0), uint64(0), uint64(0), uint64(0), metaStatusInitial.String()}, + {int64(2), int64(0), int64(100), uint64(0), uint64(0), uint64(0), metaStatusRowIDAllocated.String()}, + } + updateArgs := []driver.Value{int64(100), int64(110), "restore", int64(1), int64(1)} + s.prepareMockInner(rows, nil, updateArgs, nil, nil, false) + // fail, retry and success + ck, rowIDBase, err := s.mgr.AllocTableRowIDs(ctx, 10) + require.NoError(t, err) + require.Equal(t, int64(100), rowIDBase) + require.Nil(t, ck) + + require.Equal(t, 0, s.checksumMgr.callCnt) +} + +func (s *metaMgrSuite) prepareMock(rowsVal [][]driver.Value, nextRowID *int64, updateArgs []driver.Value, checksum *verification.KVChecksum, updateStatus *string, rollback bool) { + s.mockDB.ExpectExec("SET SESSION tidb_txn_mode = 'pessimistic';"). + WillReturnResult(sqlmock.NewResult(int64(0), int64(0))) + s.prepareMockInner(rowsVal, nextRowID, updateArgs, checksum, updateStatus, rollback) +} +func (s *metaMgrSuite) prepareMockInner(rowsVal [][]driver.Value, nextRowID *int64, updateArgs []driver.Value, checksum *verification.KVChecksum, updateStatus *string, rollback bool) { s.mockDB.ExpectBegin() rows := sqlmock.NewRows([]string{"task_id", "row_id_base", "row_id_max", "total_kvs_base", "total_bytes_base", "checksum_base", "status"}) @@ -269,6 +309,11 @@ func (s *metaMgrSuite) prepareMock(rowsVal [][]driver.Value, nextRowID *int64, u WillReturnResult(sqlmock.NewResult(int64(0), int64(1))) } + if rollback { + s.mockDB.ExpectRollback() + return + } + s.mockDB.ExpectCommit() if checksum != nil { From a9e2b767265a04feca4699bbcae6560ecd1a8edf Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 4 Aug 2022 17:58:07 +0800 Subject: [PATCH 09/42] session: fix multiple domain creation racing in domap (#36792) (#36810) close pingcap/tidb#36791 --- session/tidb.go | 24 ++++++------------------ 1 file changed, 6 insertions(+), 18 deletions(-) diff --git a/session/tidb.go b/session/tidb.go index 12ee40da2d4be..c7d93f71d7081 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -44,24 +44,17 @@ import ( ) type domainMap struct { + mu sync.Mutex domains map[string]*domain.Domain - mu sync.RWMutex } func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) { - dm.mu.RLock() + key := store.UUID() - // If this is the only domain instance, and the caller doesn't provide store. - if len(dm.domains) == 1 && store == nil { - for _, r := range dm.domains { - dm.mu.RUnlock() - return r, nil - } - } + dm.mu.Lock() + defer dm.mu.Unlock() - key := store.UUID() d = dm.domains[key] - dm.mu.RUnlock() if d != nil { return } @@ -94,7 +87,8 @@ func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) { if err != nil { return nil, err } - dm.Set(store, d) + + dm.domains[key] = d return } @@ -105,12 +99,6 @@ func (dm *domainMap) Delete(store kv.Storage) { dm.mu.Unlock() } -func (dm *domainMap) Set(store kv.Storage, domain *domain.Domain) { - dm.mu.Lock() - dm.domains[store.UUID()] = domain - dm.mu.Unlock() -} - var ( domap = &domainMap{ domains: map[string]*domain.Domain{}, From a6b54491dd7e8829d3a6f286501099415eb5aacd Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 5 Aug 2022 12:30:06 +0800 Subject: [PATCH 10/42] lightning: retry on error on tikv ingest like 'stale command' (#36878) (#36881) close pingcap/tidb#36877 --- br/pkg/lightning/backend/local/local.go | 18 +++++++----- br/pkg/lightning/backend/local/local_test.go | 31 +++++++++++++++----- br/pkg/lightning/common/errors.go | 18 +++++++----- br/pkg/lightning/common/retry.go | 4 ++- br/pkg/lightning/common/retry_test.go | 6 ++++ errors.toml | 10 +++++++ 6 files changed, 64 insertions(+), 23 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 958be0434ad21..dd8a98600bbde 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -1104,7 +1104,7 @@ WriteAndIngest: err = local.writeAndIngestPairs(ctx, engine, region, pairStart, end, regionSplitSize, regionSplitKeys) local.ingestConcurrency.Recycle(w) if err != nil { - if !common.IsRetryableError(err) { + if !local.isRetryableImportTiKVError(err) { return err } _, regionStart, _ := codec.DecodeBytes(region.Region.StartKey, []byte{}) @@ -1134,7 +1134,7 @@ const ( retryIngest ) -func (local *local) isRetryableTiKVWriteError(err error) bool { +func (local *local) isRetryableImportTiKVError(err error) bool { err = errors.Cause(err) // io.EOF is not retryable in normal case // but on TiKV restart, if we're writing to TiKV(through GRPC) @@ -1164,7 +1164,7 @@ loopWrite: var rangeStats rangeStats metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engine, region, start, end, regionSplitSize, regionSplitKeys) if err != nil { - if !local.isRetryableTiKVWriteError(err) { + if !local.isRetryableImportTiKVError(err) { return err } @@ -1305,7 +1305,7 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, if err == nil || common.IsContextCanceledError(err) { return } - if !common.IsRetryableError(err) { + if !local.isRetryableImportTiKVError(err) { break } log.L().Warn("write and ingest by range failed", @@ -1812,12 +1812,11 @@ func (local *local) isIngestRetryable( } return retryTy, newRegion, common.ErrKVEpochNotMatch.GenWithStack(errPb.GetMessage()) case strings.Contains(errPb.Message, "raft: proposal dropped"): - // TODO: we should change 'Raft raft: proposal dropped' to a error type like 'NotLeader' newRegion, err = getRegion() if err != nil { return retryNone, nil, errors.Trace(err) } - return retryWrite, newRegion, errors.New(errPb.GetMessage()) + return retryWrite, newRegion, common.ErrKVRaftProposalDropped.GenWithStack(errPb.GetMessage()) case errPb.ServerIsBusy != nil: return retryNone, nil, common.ErrKVServerIsBusy.GenWithStack(errPb.GetMessage()) case errPb.RegionNotFound != nil: @@ -1834,8 +1833,13 @@ func (local *local) isIngestRetryable( return retryNone, nil, errors.Trace(err) } return retryWrite, newRegion, common.ErrKVReadIndexNotReady.GenWithStack(errPb.GetMessage()) + case errPb.DiskFull != nil: + return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) } - return retryNone, nil, errors.Errorf("non-retryable error: %s", resp.GetError().GetMessage()) + // all others ingest error, such as stale command, etc. we'll retry it again from writeAndIngestByRange + // here we use a single named-error ErrKVIngestFailed to represent them all + // we can separate them later if it's needed + return retryNone, nil, common.ErrKVIngestFailed.GenWithStack(resp.GetError().GetMessage()) } // return the smallest []byte that is bigger than current bytes. diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index e5931808438a8..b8c3d57df6719 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -39,6 +39,7 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" sst "github.com/pingcap/kvproto/pkg/import_sstpb" "github.com/pingcap/kvproto/pkg/metapb" + berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/lightning/backend" "github.com/pingcap/tidb/br/pkg/lightning/backend/kv" "github.com/pingcap/tidb/br/pkg/lightning/common" @@ -506,11 +507,6 @@ func TestIsIngestRetryable(t *testing.T) { require.Equal(t, retryWrite, retryType) require.Error(t, err) - resp.Error = &errorpb.Error{Message: "unknown error"} - retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) - require.Equal(t, retryNone, retryType) - require.EqualError(t, err, "non-retryable error: unknown error") - resp.Error = &errorpb.Error{ ReadIndexNotReady: &errorpb.ReadIndexNotReady{ Reason: "test", @@ -519,6 +515,27 @@ func TestIsIngestRetryable(t *testing.T) { retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) require.Equal(t, retryWrite, retryType) require.Error(t, err) + + resp.Error = &errorpb.Error{ + Message: "raft: proposal dropped", + } + retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) + require.Equal(t, retryWrite, retryType) + require.True(t, berrors.Is(err, common.ErrKVRaftProposalDropped)) + + resp.Error = &errorpb.Error{ + DiskFull: &errorpb.DiskFull{}, + } + retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) + require.Equal(t, retryNone, retryType) + require.Contains(t, err.Error(), "non-retryable error") + + resp.Error = &errorpb.Error{ + StaleCommand: &errorpb.StaleCommand{}, + } + retryType, _, err = local.isIngestRetryable(ctx, resp, region, metas) + require.Equal(t, retryNone, retryType) + require.True(t, berrors.Is(err, common.ErrKVIngestFailed)) } type testIngester struct{} @@ -1248,6 +1265,6 @@ func TestGetRegionSplitSizeKeys(t *testing.T) { func TestLocalIsRetryableTiKVWriteError(t *testing.T) { l := local{} - require.True(t, l.isRetryableTiKVWriteError(io.EOF)) - require.True(t, l.isRetryableTiKVWriteError(errors.Trace(io.EOF))) + require.True(t, l.isRetryableImportTiKVError(io.EOF)) + require.True(t, l.isRetryableImportTiKVError(errors.Trace(io.EOF))) } diff --git a/br/pkg/lightning/common/errors.go b/br/pkg/lightning/common/errors.go index 4f1c598a84535..b647cc9e64ad4 100644 --- a/br/pkg/lightning/common/errors.go +++ b/br/pkg/lightning/common/errors.go @@ -70,14 +70,16 @@ var ( ErrCreatePDClient = errors.Normalize("create pd client error", errors.RFCCodeText("Lightning:PD:ErrCreatePDClient")) ErrPauseGC = errors.Normalize("pause gc error", errors.RFCCodeText("Lightning:PD:ErrPauseGC")) - ErrCheckKVVersion = errors.Normalize("check tikv version error", errors.RFCCodeText("Lightning:KV:ErrCheckKVVersion")) - ErrCreateKVClient = errors.Normalize("create kv client error", errors.RFCCodeText("Lightning:KV:ErrCreateKVClient")) - ErrCheckMultiIngest = errors.Normalize("check multi-ingest support error", errors.RFCCodeText("Lightning:KV:ErrCheckMultiIngest")) - ErrKVEpochNotMatch = errors.Normalize("epoch not match", errors.RFCCodeText("Lightning:KV:EpochNotMatch")) - ErrKVNotLeader = errors.Normalize("not leader", errors.RFCCodeText("Lightning:KV:NotLeader")) - ErrKVServerIsBusy = errors.Normalize("server is busy", errors.RFCCodeText("Lightning:KV:ServerIsBusy")) - ErrKVRegionNotFound = errors.Normalize("region not found", errors.RFCCodeText("Lightning:KV:RegionNotFound")) - ErrKVReadIndexNotReady = errors.Normalize("read index not ready", errors.RFCCodeText("Lightning:KV:ReadIndexNotReady")) + ErrCheckKVVersion = errors.Normalize("check tikv version error", errors.RFCCodeText("Lightning:KV:ErrCheckKVVersion")) + ErrCreateKVClient = errors.Normalize("create kv client error", errors.RFCCodeText("Lightning:KV:ErrCreateKVClient")) + ErrCheckMultiIngest = errors.Normalize("check multi-ingest support error", errors.RFCCodeText("Lightning:KV:ErrCheckMultiIngest")) + ErrKVEpochNotMatch = errors.Normalize("epoch not match", errors.RFCCodeText("Lightning:KV:EpochNotMatch")) + ErrKVNotLeader = errors.Normalize("not leader", errors.RFCCodeText("Lightning:KV:NotLeader")) + ErrKVServerIsBusy = errors.Normalize("server is busy", errors.RFCCodeText("Lightning:KV:ServerIsBusy")) + ErrKVRegionNotFound = errors.Normalize("region not found", errors.RFCCodeText("Lightning:KV:RegionNotFound")) + ErrKVReadIndexNotReady = errors.Normalize("read index not ready", errors.RFCCodeText("Lightning:KV:ReadIndexNotReady")) + ErrKVIngestFailed = errors.Normalize("ingest tikv failed", errors.RFCCodeText("Lightning:KV:ErrKVIngestFailed")) + ErrKVRaftProposalDropped = errors.Normalize("raft proposal dropped", errors.RFCCodeText("Lightning:KV:ErrKVRaftProposalDropped")) ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend")) ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile")) diff --git a/br/pkg/lightning/common/retry.go b/br/pkg/lightning/common/retry.go index 7cd71dd1f1625..a3dbd2dd539e2 100644 --- a/br/pkg/lightning/common/retry.go +++ b/br/pkg/lightning/common/retry.go @@ -103,7 +103,9 @@ func isSingleRetryableError(err error) bool { case *errors.Error: switch { case berrors.Is(nerr, ErrKVEpochNotMatch), berrors.Is(nerr, ErrKVNotLeader), - berrors.Is(nerr, ErrKVRegionNotFound), berrors.Is(nerr, ErrKVServerIsBusy): + berrors.Is(nerr, ErrKVRegionNotFound), berrors.Is(nerr, ErrKVServerIsBusy), + berrors.Is(nerr, ErrKVReadIndexNotReady), berrors.Is(nerr, ErrKVIngestFailed), + berrors.Is(nerr, ErrKVRaftProposalDropped): // common.ErrKVServerIsBusy is a little duplication with tmysql.ErrTiKVServerBusy // it's because the response of sst.ingest gives us a sst.IngestResponse which doesn't contain error code, // so we have to transform it into a defined code diff --git a/br/pkg/lightning/common/retry_test.go b/br/pkg/lightning/common/retry_test.go index 004cce85622de..78719f28c53a5 100644 --- a/br/pkg/lightning/common/retry_test.go +++ b/br/pkg/lightning/common/retry_test.go @@ -43,10 +43,16 @@ func TestIsRetryableError(t *testing.T) { require.True(t, IsRetryableError(ErrKVEpochNotMatch)) require.True(t, IsRetryableError(ErrKVServerIsBusy)) require.True(t, IsRetryableError(ErrKVRegionNotFound)) + require.True(t, IsRetryableError(ErrKVReadIndexNotReady)) + require.True(t, IsRetryableError(ErrKVIngestFailed)) + require.True(t, IsRetryableError(ErrKVRaftProposalDropped)) require.True(t, IsRetryableError(ErrKVNotLeader.GenWithStack("test"))) require.True(t, IsRetryableError(ErrKVEpochNotMatch.GenWithStack("test"))) require.True(t, IsRetryableError(ErrKVServerIsBusy.GenWithStack("test"))) require.True(t, IsRetryableError(ErrKVRegionNotFound.GenWithStack("test"))) + require.True(t, IsRetryableError(ErrKVReadIndexNotReady.GenWithStack("test"))) + require.True(t, IsRetryableError(ErrKVIngestFailed.GenWithStack("test"))) + require.True(t, IsRetryableError(ErrKVRaftProposalDropped.GenWithStack("test"))) // net: connection refused _, err := net.Dial("tcp", "localhost:65533") diff --git a/errors.toml b/errors.toml index c9e3aa6c9da9b..1bcc9176ba983 100755 --- a/errors.toml +++ b/errors.toml @@ -366,6 +366,16 @@ error = ''' create kv client error ''' +["Lightning:KV:ErrKVIngestFailed"] +error = ''' +ingest tikv failed +''' + +["Lightning:KV:ErrKVRaftProposalDropped"] +error = ''' +raft proposal dropped +''' + ["Lightning:KV:NotLeader"] error = ''' not leader From 475c6a8dfbf352d0f48d15ae24bd363627e7efe7 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 9 Aug 2022 14:04:48 +0800 Subject: [PATCH 11/42] executor,expression: avoid to append nil to warnings (#36304) (#36931) close pingcap/tidb#31569 --- executor/insert.go | 18 ++++++++++++--- executor/insert_common.go | 39 ++++++++++++++++++++++++++----- executor/insert_test.go | 9 ++++---- executor/show_test.go | 4 ++-- executor/write_test.go | 17 +++++++------- expression/integration_test.go | 42 ++++++++++++++++++++++++++-------- expression/scalar_function.go | 5 +--- 7 files changed, 97 insertions(+), 37 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index 7c8c3e1f4b7cc..3b872e5f77928 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -199,7 +199,7 @@ func (e *InsertExec) updateDupRow(ctx context.Context, idxInBatch int, txn kv.Tr oldRow = append(oldRow, extraCols...) } - err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, e.OnDuplicate) + err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, e.OnDuplicate, idxInBatch) if e.ctx.GetSessionVars().StmtCtx.DupKeyAsWarning && kv.ErrKeyExists.Equal(err) { e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) return nil @@ -374,7 +374,7 @@ func (e *InsertExec) initEvalBuffer4Dup() { // doDupRowUpdate updates the duplicate row. func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRow []types.Datum, newRow []types.Datum, - cols []*expression.Assignment) error { + cols []*expression.Assignment, idxInBatch int) error { assignFlag := make([]bool, len(e.Table.WritableCols())) // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values e.curInsertVals.SetDatums(newRow...) @@ -388,6 +388,8 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo // Update old row when the key is duplicated. e.evalBuffer4Dup.SetDatums(e.row4Update...) + sc := e.ctx.GetSessionVars().StmtCtx + warnCnt := int(sc.WarningCount()) for _, col := range cols { if col.LazyErr != nil { return col.LazyErr @@ -396,10 +398,20 @@ func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRo if err1 != nil { return err1 } - e.row4Update[col.Col.Index], err1 = table.CastValue(e.ctx, val, col.Col.ToInfo(), false, false) + c := col.Col.ToInfo() + c.Name = col.ColName + e.row4Update[col.Col.Index], err1 = table.CastValue(e.ctx, val, c, false, false) if err1 != nil { return err1 } + if newWarnings := sc.TruncateWarnings(warnCnt); len(newWarnings) > 0 { + for k := range newWarnings { + // Use `idxInBatch` here for simplicity, since the offset of the batch is unknown under the current context. + newWarnings[k].Err = completeInsertErr(c, &val, idxInBatch, newWarnings[k].Err) + } + sc.AppendWarnings(newWarnings) + warnCnt += len(newWarnings) + } e.evalBuffer4Dup.SetDatum(col.Col.Index, e.row4Update[col.Col.Index]) assignFlag[col.Col.Index] = true } diff --git a/executor/insert_common.go b/executor/insert_common.go index 3c6b812f9c00a..fa77665ab9be8 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -287,12 +287,7 @@ func insertRows(ctx context.Context, base insertCommon) (err error) { return nil } -func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int, err error) error { - if err == nil { - return nil - } - - // Convert the error with full messages. +func completeInsertErr(col *model.ColumnInfo, val *types.Datum, rowIdx int, err error) error { var ( colTp byte colName string @@ -323,6 +318,20 @@ func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int } else if types.ErrWarnDataOutOfRange.Equal(err) { err = types.ErrWarnDataOutOfRange.GenWithStackByArgs(colName, rowIdx+1) } + return err +} + +func (e *InsertValues) handleErr(col *table.Column, val *types.Datum, rowIdx int, err error) error { + if err == nil { + return nil + } + + // Convert the error with full messages. + var c *model.ColumnInfo + if col != nil { + c = col.ColumnInfo + } + err = completeInsertErr(c, val, rowIdx, err) if !e.ctx.GetSessionVars().StmtCtx.DupKeyAsWarning { return err @@ -350,6 +359,8 @@ func (e *InsertValues) evalRow(ctx context.Context, list []expression.Expression } e.evalBuffer.SetDatums(row...) + sc := e.ctx.GetSessionVars().StmtCtx + warnCnt := int(sc.WarningCount()) for i, expr := range list { val, err := expr.Eval(e.evalBuffer.ToRow()) if err != nil { @@ -359,6 +370,13 @@ func (e *InsertValues) evalRow(ctx context.Context, list []expression.Expression if err = e.handleErr(e.insertColumns[i], &val, rowIdx, err); err != nil { return nil, err } + if newWarnings := sc.TruncateWarnings(warnCnt); len(newWarnings) > 0 { + for k := range newWarnings { + newWarnings[k].Err = completeInsertErr(e.insertColumns[i].ColumnInfo, &val, rowIdx, newWarnings[k].Err) + } + sc.AppendWarnings(newWarnings) + warnCnt += len(newWarnings) + } offset := e.insertColumns[i].Offset val1.Copy(&row[offset]) @@ -378,6 +396,8 @@ func (e *InsertValues) fastEvalRow(ctx context.Context, list []expression.Expres } row := make([]types.Datum, rowLen) hasValue := make([]bool, rowLen) + sc := e.ctx.GetSessionVars().StmtCtx + warnCnt := int(sc.WarningCount()) for i, expr := range list { con := expr.(*expression.Constant) val, err := con.Eval(emptyRow) @@ -388,6 +408,13 @@ func (e *InsertValues) fastEvalRow(ctx context.Context, list []expression.Expres if err = e.handleErr(e.insertColumns[i], &val, rowIdx, err); err != nil { return nil, err } + if newWarnings := sc.TruncateWarnings(warnCnt); len(newWarnings) > 0 { + for k := range newWarnings { + newWarnings[k].Err = completeInsertErr(e.insertColumns[i].ColumnInfo, &val, rowIdx, newWarnings[k].Err) + } + sc.AppendWarnings(newWarnings) + warnCnt += len(newWarnings) + } offset := e.insertColumns[i].Offset row[offset], hasValue[offset] = val1, true } diff --git a/executor/insert_test.go b/executor/insert_test.go index ce91449f5fe49..4bcb8dca55958 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -379,7 +379,7 @@ func TestInsertWrongValueForField(t *testing.T) { tk.MustExec(`CREATE TABLE ts (id int DEFAULT NULL, time1 TIMESTAMP NULL DEFAULT NULL)`) tk.MustExec(`SET @@sql_mode=''`) tk.MustExec(`INSERT INTO ts (id, time1) VALUES (1, TIMESTAMP '1018-12-23 00:00:00')`) - tk.MustQuery(`SHOW WARNINGS`).Check(testkit.Rows(`Warning 1292 Incorrect timestamp value: '1018-12-23 00:00:00'`)) + tk.MustQuery(`SHOW WARNINGS`).Check(testkit.Rows(`Warning 1292 Incorrect timestamp value: '1018-12-23 00:00:00' for column 'time1' at row 1`)) tk.MustQuery(`SELECT * FROM ts ORDER BY id`).Check(testkit.Rows(`1 0000-00-00 00:00:00`)) tk.MustExec(`SET @@sql_mode='STRICT_TRANS_TABLES'`) @@ -1677,7 +1677,7 @@ func TestIssue10402(t *testing.T) { tk.MustExec("insert into vctt values ('ab\\n\\n\\n', 'ab\\n\\n\\n'), ('ab\\t\\t\\t', 'ab\\t\\t\\t'), ('ab ', 'ab '), ('ab\\r\\r\\r', 'ab\\r\\r\\r')") require.Equal(t, uint16(4), tk.Session().GetSessionVars().StmtCtx.WarningCount()) warns := tk.Session().GetSessionVars().StmtCtx.GetWarnings() - require.Equal(t, "[{Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 5} {Warning [types:1265]Data truncated, field len 4, data len 6} {Warning [types:1265]Data truncated, field len 4, data len 5}]", + require.Equal(t, "[{Warning [types:1265]Data truncated for column 'v' at row 1} {Warning [types:1265]Data truncated for column 'v' at row 2} {Warning [types:1265]Data truncated for column 'v' at row 3} {Warning [types:1265]Data truncated for column 'v' at row 4}]", fmt.Sprintf("%v", warns)) tk.MustQuery("select * from vctt").Check(testkit.Rows("ab\n\n ab\n\n", "ab\t\t ab\t\t", "ab ab", "ab\r\r ab\r\r")) tk.MustQuery("select length(v), length(c) from vctt").Check(testkit.Rows("4 4", "4 4", "4 2", "4 4")) @@ -1895,7 +1895,8 @@ func TestStringtoDecimal(t *testing.T) { tk.MustGetErrCode("insert into t values('1.2.')", errno.ErrTruncatedWrongValueForField) tk.MustGetErrCode("insert into t values('1,999.00')", errno.ErrTruncatedWrongValueForField) tk.MustExec("insert into t values('12e-3')") - tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DECIMAL value: '0.012'")) + // TODO: MySQL8.0 reports Note 1265 Data truncated for column 'id' at row 1 + tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Warning|1366|Incorrect decimal value: '12e-3' for column 'id' at row 1")) tk.MustQuery("select id from t").Check(testkit.Rows("0")) tk.MustExec("drop table if exists t") } @@ -1910,7 +1911,7 @@ func TestIssue17745(t *testing.T) { tk.MustGetErrCode("insert into tt1 values(89000000000000000000000000000000000000000000000000000000000000000000000000000000000000000)", errno.ErrWarnDataOutOfRange) tk.MustGetErrCode("insert into tt1 values(89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000)", errno.ErrWarnDataOutOfRange) tk.MustExec("insert ignore into tt1 values(89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000)") - tk.MustQuery("show warnings;").Check(testkit.Rows(`Warning 1690 DECIMAL value is out of range in '(64, 0)'`, `Warning 1292 Truncated incorrect DECIMAL value: '789012345678901234567890123456789012345678901234567890123456789012345678900000000'`)) + tk.MustQuery("show warnings;").Check(testkit.Rows(`Warning 1264 Out of range value for column 'c1' at row 1`, `Warning 1292 Truncated incorrect DECIMAL value: '789012345678901234567890123456789012345678901234567890123456789012345678900000000'`)) tk.MustQuery("select c1 from tt1").Check(testkit.Rows("9999999999999999999999999999999999999999999999999999999999999999")) tk.MustGetErrCode("update tt1 set c1 = 89123456789012345678901234567890123456789012345678901234567890123456789012345678900000000", errno.ErrWarnDataOutOfRange) tk.MustExec("drop table if exists tt1") diff --git a/executor/show_test.go b/executor/show_test.go index e4e45bb4b9bb6..107fcf701eaee 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -104,9 +104,9 @@ func TestShowWarnings(t *testing.T) { tk.MustExec("set @@sql_mode=''") tk.MustExec("insert show_warnings values ('a')") require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) - tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: 'a'")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1366|Incorrect int value: 'a' for column 'a' at row 1")) require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) - tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1292|Truncated incorrect DOUBLE value: 'a'")) + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1366|Incorrect int value: 'a' for column 'a' at row 1")) require.Equal(t, uint16(0), tk.Session().GetSessionVars().StmtCtx.WarningCount()) // Test Warning level 'Error' diff --git a/executor/write_test.go b/executor/write_test.go index 2cc17b1e1916b..000afd1800e17 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -213,7 +213,8 @@ func TestInsert(t *testing.T) { tk.MustExec("CREATE TABLE t(a DECIMAL(4,2));") tk.MustExec("INSERT INTO t VALUES (1.000001);") r = tk.MustQuery("SHOW WARNINGS;") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DECIMAL value: '1.000001'")) + // TODO: MySQL8.0 reports Note 1265 Data truncated for column 'a' at row 1 + r.Check(testkit.Rows("Warning 1366 Incorrect decimal value: '1.000001' for column 'a' at row 1")) tk.MustExec("INSERT INTO t VALUES (1.000000);") r = tk.MustQuery("SHOW WARNINGS;") r.Check(testkit.Rows()) @@ -250,15 +251,14 @@ func TestInsert(t *testing.T) { require.True(t, types.ErrWarnDataOutOfRange.Equal(err)) tk.MustExec("set @@sql_mode = '';") tk.MustExec("insert into t value (-1);") - // TODO: the following warning messages are not consistent with MySQL, fix them in the future PRs - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) tk.MustExec("insert into t select -1;") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) tk.MustExec("insert into t select cast(-1 as unsigned);") tk.MustExec("insert into t value (-1.111);") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1.111 overflows bigint")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) tk.MustExec("insert into t value ('-1.111');") - tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 BIGINT UNSIGNED value is out of range in '-1'")) + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1")) tk.MustExec("update t set a = -1 limit 1;") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 constant -1 overflows bigint")) r = tk.MustQuery("select * from t;") @@ -281,8 +281,8 @@ func TestInsert(t *testing.T) { tk.MustExec("create table t(a float unsigned, b double unsigned)") tk.MustExec("insert into t value(-1.1, -1.1), (-2.1, -2.1), (0, 0), (1.1, 1.1)") tk.MustQuery("show warnings"). - Check(testkit.Rows("Warning 1690 constant -1.1 overflows float", "Warning 1690 constant -1.1 overflows double", - "Warning 1690 constant -2.1 overflows float", "Warning 1690 constant -2.1 overflows double")) + Check(testkit.Rows("Warning 1264 Out of range value for column 'a' at row 1", "Warning 1264 Out of range value for column 'b' at row 1", + "Warning 1264 Out of range value for column 'a' at row 2", "Warning 1264 Out of range value for column 'b' at row 2")) tk.MustQuery("select * from t").Check(testkit.Rows("0 0", "0 0", "0 0", "1.1 1.1")) // issue 7061 @@ -531,7 +531,8 @@ func TestInsertIgnore(t *testing.T) { require.NoError(t, err) require.Empty(t, tk.Session().LastMessage()) r = tk.MustQuery("SHOW WARNINGS") - r.Check(testkit.Rows("Warning 1292 Truncated incorrect DOUBLE value: '1a'")) + // TODO: MySQL8.0 reports Warning 1265 Data truncated for column 'a' at row 1 + r.Check(testkit.Rows("Warning 1366 Incorrect bigint value: '1a' for column 'a' at row 1")) // for duplicates with warning testSQL = `drop table if exists t; diff --git a/expression/integration_test.go b/expression/integration_test.go index 31a7e1178a2b6..1a45111155354 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -497,8 +497,8 @@ func TestConvertToBit(t *testing.T) { tk.MustExec("create table t(a tinyint, b bit(63));") tk.MustExec("insert ignore into t values(599999999, -1);") tk.MustQuery("show warnings;").Check(testkit.Rows( - "Warning 1690 constant 599999999 overflows tinyint", - "Warning 1406 Data Too Long, field len 63")) + "Warning 1264 Out of range value for column 'a' at row 1", + "Warning 1406 Data too long for column 'b' at row 1")) tk.MustQuery("select * from t;").Check(testkit.Rows("127 \u007f\xff\xff\xff\xff\xff\xff\xff")) // For issue 24900 @@ -506,8 +506,8 @@ func TestConvertToBit(t *testing.T) { tk.MustExec("create table t(b bit(16));") tk.MustExec("insert ignore into t values(0x3635313836),(0x333830);") tk.MustQuery("show warnings;").Check(testkit.Rows( - "Warning 1406 Data Too Long, field len 16", - "Warning 1406 Data Too Long, field len 16")) + "Warning 1406 Data too long for column 'b' at row 1", + "Warning 1406 Data too long for column 'b' at row 2")) tk.MustQuery("select * from t;").Check(testkit.Rows("\xff\xff", "\xff\xff")) } @@ -5371,7 +5371,7 @@ func TestIssue19892(t *testing.T) { tk.MustExec("TRUNCATE TABLE dd") tk.MustExec("INSERT INTO dd(c) values('0000-00-00 20:00:00')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1")) tk.MustQuery("SELECT c FROM dd").Check(testkit.Rows("0000-00-00 00:00:00")) tk.MustExec("TRUNCATE TABLE dd") @@ -5385,7 +5385,7 @@ func TestIssue19892(t *testing.T) { { tk.MustExec("TRUNCATE TABLE dd") tk.MustExec("INSERT INTO dd(b) values('0000-0-00')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '0000-0-00'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '0000-0-00' for column 'b' at row 1")) tk.MustQuery("SELECT b FROM dd").Check(testkit.Rows("0000-00-00 00:00:00")) tk.MustExec("TRUNCATE TABLE dd") @@ -5406,7 +5406,7 @@ func TestIssue19892(t *testing.T) { tk.MustExec("TRUNCATE TABLE dd") tk.MustGetErrMsg("INSERT INTO dd(c) VALUES ('0000-00-00 20:00:00')", "[table:1292]Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1") tk.MustExec("INSERT IGNORE INTO dd(c) VALUES ('0000-00-00 20:00:00')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect timestamp value: '0000-00-00 20:00:00' for column 'c' at row 1")) tk.MustQuery("SELECT c FROM dd").Check(testkit.Rows("0000-00-00 00:00:00")) tk.MustExec("TRUNCATE TABLE dd") @@ -5457,7 +5457,7 @@ func TestIssue19892(t *testing.T) { { tk.MustExec("TRUNCATE TABLE dd") tk.MustExec("INSERT INTO dd(a) values('2000-01-00')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect date value: '2000-01-00'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect date value: '2000-01-00' for column 'a' at row 1")) tk.MustQuery("SELECT a FROM dd").Check(testkit.Rows("0000-00-00")) tk.MustExec("TRUNCATE TABLE dd") @@ -5485,7 +5485,7 @@ func TestIssue19892(t *testing.T) { tk.MustExec("TRUNCATE TABLE dd") tk.MustGetErrMsg("INSERT INTO dd(b) VALUES ('2000-01-00')", "[table:1292]Incorrect datetime value: '2000-01-00' for column 'b' at row 1") tk.MustExec("INSERT IGNORE INTO dd(b) VALUES ('2000-00-01')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '2000-00-01'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '2000-00-01' for column 'b' at row 1")) tk.MustQuery("SELECT b FROM dd").Check(testkit.Rows("0000-00-00 00:00:00")) tk.MustExec("TRUNCATE TABLE dd") @@ -5538,7 +5538,7 @@ func TestIssue19892(t *testing.T) { tk.MustExec("TRUNCATE TABLE dd") tk.MustGetErrMsg("INSERT INTO dd(b) VALUES ('2000-01-00')", "[table:1292]Incorrect datetime value: '2000-01-00' for column 'b' at row 1") tk.MustExec("INSERT IGNORE INTO dd(b) VALUES ('2000-00-01')") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '2000-00-01'")) + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Incorrect datetime value: '2000-00-01' for column 'b' at row 1")) tk.MustQuery("SELECT b FROM dd").Check(testkit.Rows("0000-00-00 00:00:00")) tk.MustExec("TRUNCATE TABLE dd") @@ -7347,3 +7347,25 @@ func TestIssue34659(t *testing.T) { result = tk.MustQuery("select cast(date_add(cast('00:00:00' as time), interval 1111111 day_microsecond) as char)").Rows() require.Equal(t, [][]interface{}{{"00:00:01.111111"}}, result) } + +func TestIssue31569(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c int primary key, c2 enum('a', 'b'))") + tk.MustExec("set session sql_mode = ''") + tk.MustExec("insert into t values(4, 'a')") + tk.MustExec("insert into t values(4, 0) on duplicate key update c=values(c), c2=values(c2)") + // tidb produces two warnings here (when eval (4, 0) & values(c2)), which is slightly incompatible with mysql + tk.MustQuery("show warnings").Check([][]interface{}{ + {"Warning", "1265", "Data truncated for column 'c2' at row 1"}, + {"Warning", "1265", "Data truncated for column 'c2' at row 1"}, + }) + tk.MustExec("insert into t values(4, 'a') on duplicate key update c=values(c), c2=values(c2)") + tk.MustQuery("show warnings").Check([][]interface{}{}) + tk.MustExec("drop table t") +} diff --git a/expression/scalar_function.go b/expression/scalar_function.go index d58ed6e5792a7..4d489d43c1656 100644 --- a/expression/scalar_function.go +++ b/expression/scalar_function.go @@ -375,10 +375,7 @@ func (sf *ScalarFunction) Eval(row chunk.Row) (d types.Datum, err error) { res, err = types.ParseEnum(tp.GetElems(), str, tp.GetCollate()) if ctx := sf.GetCtx(); ctx != nil { if sc := ctx.GetSessionVars().StmtCtx; sc != nil { - if sc.TruncateAsWarning { - ctx.GetSessionVars().StmtCtx.AppendWarning(err) - err = nil - } + err = sc.HandleTruncate(err) } } } else { From c9d8794c6f87c8785652a05162a6a1298dff223f Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 9 Aug 2022 15:02:47 +0800 Subject: [PATCH 12/42] ddl: prevent returning zero schema version for alter sequence (#36277) (#36418) close pingcap/tidb#36276 --- ddl/sequence.go | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/ddl/sequence.go b/ddl/sequence.go index ceea68d90810b..ad8390bdf2cc2 100644 --- a/ddl/sequence.go +++ b/ddl/sequence.go @@ -259,7 +259,11 @@ func onAlterSequence(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro job.State = model.JobStateCancelled return ver, errors.Trace(err) } - shouldUpdateVer := !reflect.DeepEqual(*tblInfo.Sequence, copySequenceInfo) || restart + same := reflect.DeepEqual(*tblInfo.Sequence, copySequenceInfo) + if same && !restart { + job.State = model.JobStateDone + return ver, errors.Trace(err) + } tblInfo.Sequence = ©SequenceInfo // Restart the sequence value. @@ -276,7 +280,9 @@ func onAlterSequence(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro } // Store the sequence info into kv. - ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, shouldUpdateVer) + // Set shouldUpdateVer always to be true even altering doesn't take effect, since some tools like drainer won't take + // care of SchemaVersion=0. + ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true) if err != nil { return ver, errors.Trace(err) } From dee34722367a90cdcf7a708ef4d2926fa102883b Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Fri, 12 Aug 2022 14:08:49 +0800 Subject: [PATCH 13/42] planner: fix the wrong join reorder produced by some right ouer join (#36936) (#36938) close pingcap/tidb#36912 --- planner/core/rule_join_reorder.go | 6 +-- planner/core/rule_join_reorder_test.go | 52 ++++++++++++++++++++++++++ 2 files changed, 55 insertions(+), 3 deletions(-) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 9a0a507b6dca0..e1d0ce1a7537e 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -98,9 +98,9 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression // If the filters of the outer join is related with multiple leaves of the outer join side. We don't reorder it for now. if join.JoinType == RightOuterJoin { extractedCols := make([]*expression.Column, 0, 8) - expression.ExtractColumnsFromExpressions(extractedCols, join.OtherConditions, nil) - expression.ExtractColumnsFromExpressions(extractedCols, join.RightConditions, nil) - expression.ExtractColumnsFromExpressions(extractedCols, expression.ScalarFuncs2Exprs(join.EqualConditions), nil) + extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.OtherConditions, nil) + extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, join.RightConditions, nil) + extractedCols = expression.ExtractColumnsFromExpressions(extractedCols, expression.ScalarFuncs2Exprs(join.EqualConditions), nil) affectedGroups := 0 for i := range rhsGroup { for _, col := range extractedCols { diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 59a53c41adb04..c7768ccf04ce4 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -329,3 +329,55 @@ func TestJoinOrderHint4Subquery(t *testing.T) { runJoinReorderTestData(t, tk, "TestJoinOrderHint4Subquery") } + +func TestOuterJoinWIthEqCondCrossInnerJoin(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `t1` (`data_status` tinyint(1) DEFAULT '0',`part` tinyint(255) unsigned DEFAULT NULL);") + tk.MustExec("CREATE TABLE `t2` (`id` bigint(20) NOT NULL AUTO_INCREMENT,`routing_rule_switch` tinyint(1) DEFAULT '0',PRIMARY KEY (`id`));") + tk.MustExec("CREATE TABLE `t3` (`fk_id` bigint(20) DEFAULT NULL,`offer_pbu_id` varchar(255) DEFAULT NULL ,`market_id` smallint(6) DEFAULT NULL ,`te_partition` tinyint(255) DEFAULT NULL ,UNIQUE KEY `t_pbu_partition_id` (`offer_pbu_id`,`market_id`,`te_partition`));") + tk.MustExec("insert into t1 values(1,1);") + tk.MustExec("insert into t2 values(1,0);") + tk.MustExec("insert into t3 values(8,'a',3,6);") + + sql := ` +SELECT tt.market_id, + tt.offer_pbu_id +FROM t3 tt + RIGHT JOIN (SELECT pp.offer_pbu_id, + pp.market_id, + t.partition_no + FROM (SELECT p.offer_pbu_id, + p.market_id + FROM t3 p + INNER JOIN t2 e + ON p.fk_id = e.id + AND e.routing_rule_switch = 1) pp, + (SELECT part AS partition_no + FROM t1) t) o + ON tt.market_id = o.market_id + AND tt.offer_pbu_id = o.offer_pbu_id + AND tt.te_partition = o.partition_no;` + tk.MustQuery(sql).Check(testkit.Rows()) + tk.MustQuery("explain format=brief" + sql).Check(testkit.Rows( + "Projection 155781.72 root test.t3.market_id, test.t3.offer_pbu_id", + "└─HashJoin 155781.72 root right outer join, equal:[eq(test.t3.market_id, test.t3.market_id) eq(test.t3.offer_pbu_id, test.t3.offer_pbu_id) eq(test.t3.te_partition, test.t1.part)]", + " ├─IndexReader(Build) 9970.03 root index:Selection", + " │ └─Selection 9970.03 cop[tikv] not(isnull(test.t3.market_id)), not(isnull(test.t3.te_partition))", + " │ └─IndexFullScan 9990.00 cop[tikv] table:tt, index:t_pbu_partition_id(offer_pbu_id, market_id, te_partition) keep order:false, stats:pseudo", + " └─HashJoin(Probe) 125000.00 root CARTESIAN inner join", + " ├─HashJoin(Build) 12.50 root inner join, equal:[eq(test.t2.id, test.t3.fk_id)]", + " │ ├─TableReader(Build) 10.00 root data:Selection", + " │ │ └─Selection 10.00 cop[tikv] eq(test.t2.routing_rule_switch, 1)", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:e keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t3.fk_id))", + " │ └─TableFullScan 10000.00 cop[tikv] table:p keep order:false, stats:pseudo", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + )) +} From 46ebd96a67de626597323b67469dcd54403d1e59 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 17 Aug 2022 12:08:51 +0800 Subject: [PATCH 14/42] lightning: support column starts with slash/number/non-ascii for parquet (#36985) (#37093) close pingcap/tidb#36980, close pingcap/tidb#36981 --- br/pkg/lightning/mydump/parquet_parser.go | 7 +++---- .../special_col_name.parquet | Bin 0 -> 527 bytes br/tests/lightning_parquet/db.sql | 8 ++++++++ br/tests/lightning_parquet/run.sh | 1 + 4 files changed, 12 insertions(+), 4 deletions(-) create mode 100644 br/tests/lightning_parquet/data/test/test.special_col_name/special_col_name.parquet diff --git a/br/pkg/lightning/mydump/parquet_parser.go b/br/pkg/lightning/mydump/parquet_parser.go index 789163c18bb01..792d142b7336e 100644 --- a/br/pkg/lightning/mydump/parquet_parser.go +++ b/br/pkg/lightning/mydump/parquet_parser.go @@ -186,11 +186,10 @@ func NewParquetParser( columns := make([]string, 0, len(reader.Footer.Schema)-1) columnMetas := make([]*parquet.SchemaElement, 0, len(reader.Footer.Schema)-1) - for _, c := range reader.SchemaHandler.SchemaElements { + for i, c := range reader.SchemaHandler.SchemaElements { if c.GetNumChildren() == 0 { - // NOTE: the SchemaElement.Name is capitalized, SchemaHandler.Infos.ExName is the raw column name - // though in this context, there is no difference between these two fields - columns = append(columns, strings.ToLower(c.Name)) + // we need to use the raw name, SchemaElement.Name might be prefixed with PARGO_PERFIX_ + columns = append(columns, strings.ToLower(reader.SchemaHandler.GetExName(i))) // transfer old ConvertedType to LogicalType columnMeta := c if c.ConvertedType != nil && c.LogicalType == nil { diff --git a/br/tests/lightning_parquet/data/test/test.special_col_name/special_col_name.parquet b/br/tests/lightning_parquet/data/test/test.special_col_name/special_col_name.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c89a54d575a3eefb183d97ed741db259ee2509af GIT binary patch literal 527 zcmWG=3^EjD5#VP1|A(Ipg0qw0*GP& zO3R2UfTS5f(lQbp2Qxn`~`_cg{7$_@#*>TMfv$9 zq8tpOJgQ6*Ov#2o_W=FE0^~5qCmR8ILJ%ICp^ Date: Sat, 20 Aug 2022 17:42:52 +0800 Subject: [PATCH 15/42] planner: firstrow agg func should infer return type as nullable for EQAll special case (#34585) (#35039) close pingcap/tidb#34584 --- executor/tiflash_test.go | 21 +++++++++++++++++ expression/aggregation/base_func.go | 2 +- expression/aggregation/base_func_test.go | 29 ++++++++++++++++++++++++ planner/core/expression_rewriter.go | 11 +++++++++ 4 files changed, 62 insertions(+), 1 deletion(-) diff --git a/executor/tiflash_test.go b/executor/tiflash_test.go index dc5ac512c3d59..e35ee76350714 100644 --- a/executor/tiflash_test.go +++ b/executor/tiflash_test.go @@ -117,6 +117,27 @@ func TestReadPartitionTable(t *testing.T) { tk.MustExec("commit") } +func TestAggPushDownApplyAll(t *testing.T) { + store, clean := testkit.CreateMockStore(t, withMockTiFlash(2)) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists foo") + tk.MustExec("drop table if exists bar") + tk.MustExec("create table foo(a int, b int)") + tk.MustExec("create table bar(a double not null, b decimal(65,0) not null)") + tk.MustExec("alter table foo set tiflash replica 1") + tk.MustExec("alter table bar set tiflash replica 1") + tk.MustExec("insert into foo values(0, NULL)") + tk.MustExec("insert into bar values(0, 0)") + + tk.MustExec("set @@session.tidb_allow_mpp=1") + tk.MustExec("set @@session.tidb_enforce_mpp=1") + + tk.MustQuery("select * from foo where a=all(select a from bar where bar.b=foo.b)").Check(testkit.Rows("0 ")) +} + func TestReadUnsigedPK(t *testing.T) { store, clean := testkit.CreateMockStore(t, withMockTiFlash(2)) defer clean() diff --git a/expression/aggregation/base_func.go b/expression/aggregation/base_func.go index 6b6f046105d6e..ffed0ddcee3a3 100644 --- a/expression/aggregation/base_func.go +++ b/expression/aggregation/base_func.go @@ -286,7 +286,7 @@ func (a *baseFuncDesc) typeInfer4MaxMin(ctx sessionctx.Context) { a.Args[0] = expression.BuildCastFunction(ctx, a.Args[0], tp) } a.RetTp = a.Args[0].GetType() - if (a.Name == ast.AggFuncMax || a.Name == ast.AggFuncMin) && a.RetTp.GetType() != mysql.TypeBit { + if a.Name == ast.AggFuncMax || a.Name == ast.AggFuncMin { a.RetTp = a.Args[0].GetType().Clone() a.RetTp.DelFlag(mysql.NotNullFlag) } diff --git a/expression/aggregation/base_func_test.go b/expression/aggregation/base_func_test.go index 2066b7773a1ad..0f4c125153f87 100644 --- a/expression/aggregation/base_func_test.go +++ b/expression/aggregation/base_func_test.go @@ -45,3 +45,32 @@ func TestClone(t *testing.T) { require.Equal(t, col, desc.Args[0]) require.False(t, desc.equal(ctx, cloned)) } + +func TestBaseFunc_InferAggRetType(t *testing.T) { + ctx := mock.NewContext() + doubleType := types.NewFieldType(mysql.TypeDouble) + bitType := types.NewFieldType(mysql.TypeBit) + + funcNames := []string{ + ast.AggFuncMax, ast.AggFuncMin, + } + dataTypes := []*types.FieldType{ + doubleType, bitType, + } + + for _, dataType := range dataTypes { + notNullType := dataType.Clone() + notNullType.AddFlag(mysql.NotNullFlag) + col := &expression.Column{ + UniqueID: 0, + RetType: notNullType, + } + for _, name := range funcNames { + desc, err := newBaseFuncDesc(ctx, name, []expression.Expression{col}) + require.NoError(t, err) + err = desc.TypeInfer(ctx) + require.NoError(t, err) + require.Equal(t, dataType, desc.RetTp) + } + } +} diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 3c20b60438a4d..dd452eb5d238b 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -774,6 +774,17 @@ func (er *expressionRewriter) handleEQAll(lexpr, rexpr expression.Expression, np } plan4Agg.SetChildren(np) plan4Agg.names = append(plan4Agg.names, types.EmptyName) + + // Currently, firstrow agg function is treated like the exact representation of aggregate group key, + // so the data type is the same with group key, even if the group key is not null. + // However, the return type of firstrow should be nullable, we clear the null flag here instead of + // during invoking NewAggFuncDesc, in order to keep compatibility with the existing presumption + // that the return type firstrow does not change nullability, whatsoever. + // Cloning it because the return type is the same object with argument's data type. + newRetTp := firstRowFunc.RetTp.Clone() + newRetTp.DelFlag(mysql.NotNullFlag) + firstRowFunc.RetTp = newRetTp + firstRowResultCol := &expression.Column{ UniqueID: er.sctx.GetSessionVars().AllocPlanColumnID(), RetType: firstRowFunc.RetTp, From 0481a59a9aea87c0ecb01f964f3898e3daa5ba9a Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 17:54:52 +0800 Subject: [PATCH 16/42] executor: fix show database like case sensitive issue#34766 (#34925) (#35060) close pingcap/tidb#34766 --- executor/show.go | 19 +++++++++++++++++++ executor/show_test.go | 17 +++++++++++++++++ planner/core/planbuilder.go | 7 +++++++ planner/core/show_predicate_extractor.go | 23 +++++++++++++++++++++++ planner/core/stringer_test.go | 16 ++++++++++++++++ 5 files changed, 82 insertions(+) diff --git a/executor/show.go b/executor/show.go index 544cac2e3d2df..69aa0b10b7e1b 100644 --- a/executor/show.go +++ b/executor/show.go @@ -400,11 +400,30 @@ func (e *ShowExec) fetchShowDatabases() error { dbs := e.is.AllSchemaNames() checker := privilege.GetPrivilegeManager(e.ctx) sort.Strings(dbs) + var ( + fieldPatternsLike collate.WildcardPattern + FieldFilterEnable bool + fieldFilter string + ) + + if e.Extractor != nil { + extractor := (e.Extractor).(*plannercore.ShowDatabaseExtractor) + if extractor.FieldPatterns != "" { + fieldPatternsLike = collate.GetCollatorByID(collate.CollationName2ID(mysql.UTF8MB4DefaultCollation)).Pattern() + fieldPatternsLike.Compile(extractor.FieldPatterns, byte('\\')) + } + FieldFilterEnable = extractor.Field != "" + fieldFilter = extractor.Field + } // let information_schema be the first database moveInfoSchemaToFront(dbs) for _, d := range dbs { if checker != nil && !checker.DBIsVisible(e.ctx.GetSessionVars().ActiveRoles, d) { continue + } else if FieldFilterEnable && strings.ToLower(d) != fieldFilter { + continue + } else if fieldPatternsLike != nil && !fieldPatternsLike.DoMatch(strings.ToLower(d)) { + continue } e.appendRow([]interface{}{ d, diff --git a/executor/show_test.go b/executor/show_test.go index 107fcf701eaee..34f69a0bf07c3 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1831,3 +1831,20 @@ func TestShowBindingCacheStatus(t *testing.T) { tk.MustQuery("show binding_cache status").Check(testkit.Rows( "1 1 198 Bytes 250 Bytes")) } + +func TestShowDatabasesLike(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: "root", Hostname: "%"}, nil, nil)) + + tk.MustExec("DROP DATABASE IF EXISTS `TEST_$1`") + tk.MustExec("DROP DATABASE IF EXISTS `test_$2`") + tk.MustExec("CREATE DATABASE `TEST_$1`;") + tk.MustExec("CREATE DATABASE `test_$2`;") + + tk.MustQuery("SHOW DATABASES LIKE 'TEST_%'").Check(testkit.Rows("TEST_$1", "test_$2")) + tk.MustQuery("SHOW DATABASES LIKE 'test_%'").Check(testkit.Rows("TEST_$1", "test_$2")) +} diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1c937cc672745..5aa6738cbd0c6 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3031,6 +3031,13 @@ func (b *PlanBuilder) buildShow(ctx context.Context, show *ast.ShowStmt) (Plan, if tableInfo.Meta().TempTableType != model.TempTableNone { return nil, ErrOptOnTemporaryTable.GenWithStackByArgs("show table regions") } + case ast.ShowDatabases: + var extractor ShowDatabaseExtractor + if extractor.Extract(show) { + p.Extractor = &extractor + // Avoid building Selection. + show.Pattern = nil + } } if show.Tp == ast.ShowVariables { var extractor ShowVariablesExtractor diff --git a/planner/core/show_predicate_extractor.go b/planner/core/show_predicate_extractor.go index 352b38b4f8b8c..eb9806c44ee5c 100644 --- a/planner/core/show_predicate_extractor.go +++ b/planner/core/show_predicate_extractor.go @@ -28,6 +28,7 @@ var ( _ ShowPredicateExtractor = &ShowColumnsTableExtractor{} _ ShowPredicateExtractor = &ShowTablesTableExtractor{} _ ShowPredicateExtractor = &ShowVariablesExtractor{} + _ ShowPredicateExtractor = &ShowDatabaseExtractor{} ) // ShowPredicateExtractor is used to extract some predicates from `PatternLikeExpr` clause @@ -167,3 +168,25 @@ func (e *ShowVariablesExtractor) explainInfo() string { } return s } + +// ShowDatabaseExtractor is used to extract some predicates of databases. +type ShowDatabaseExtractor struct { + ShowBaseExtractor +} + +func (e *ShowDatabaseExtractor) explainInfo() string { + r := new(bytes.Buffer) + if len(e.Field) > 0 { + r.WriteString(fmt.Sprintf("database:[%s], ", e.Field)) + } + if len(e.FieldPatterns) > 0 { + r.WriteString(fmt.Sprintf("database_pattern:[%s], ", e.FieldPatterns)) + } + + // remove the last ", " in the message info + s := r.String() + if len(s) > 2 { + return s[:len(s)-2] + } + return s +} diff --git a/planner/core/stringer_test.go b/planner/core/stringer_test.go index 6433ad59b158c..374c0589694ca 100644 --- a/planner/core/stringer_test.go +++ b/planner/core/stringer_test.go @@ -72,6 +72,22 @@ func TestPlanStringer(t *testing.T) { sql: "show tables in test like '%T%'", plan: "Show(table_pattern:[%t%])", }, + { + sql: "show databases like 't'", + plan: "Show(database:[t])", + }, + { + sql: "show databases like 'T'", + plan: "Show(database:[t])", + }, + { + sql: "show databases like 't%'", + plan: "Show(database_pattern:[t%])", + }, + { + sql: "show databases like '%T%'", + plan: "Show(database_pattern:[%t%])", + }, } parser := parser.New() for _, tt := range tests { From 361e4bc01676fbfd1eee85511ec2983a958f8311 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 18:06:52 +0800 Subject: [PATCH 17/42] util/plancodec: add the missing operator IDs to fix the `UnknownPlanID` (#35175) (#35189) close pingcap/tidb#34784, close pingcap/tidb#35153 --- planner/core/plan_test.go | 37 ++++++++++++------- util/plancodec/id.go | 75 +++++++++++++++++++++++---------------- util/plancodec/id_test.go | 23 ++++++------ 3 files changed, 82 insertions(+), 53 deletions(-) diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index f3cdd2840622a..f40b8894293f1 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -175,7 +175,9 @@ func TestEncodeDecodePlan(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t1,t2") tk.MustExec("create table t1 (a int key,b int,c int, index (b));") + tk.MustExec("create table tp (a int ,b int,c int) partition by hash(b) partitions 5;") tk.MustExec("set tidb_enable_collect_execution_info=1;") + tk.MustExec("set tidb_partition_prune_mode='static';") tk.Session().GetSessionVars().PlanID = 0 getPlanTree := func() string { @@ -190,28 +192,37 @@ func TestEncodeDecodePlan(t *testing.T) { } tk.MustExec("select max(a) from t1 where a>0;") planTree := getPlanTree() - require.True(t, strings.Contains(planTree, "time")) - require.True(t, strings.Contains(planTree, "loops")) + require.Contains(t, planTree, "time") + require.Contains(t, planTree, "loops") tk.MustExec("insert into t1 values (1,1,1);") planTree = getPlanTree() - require.True(t, strings.Contains(planTree, "Insert")) - require.True(t, strings.Contains(planTree, "time")) - require.True(t, strings.Contains(planTree, "loops")) + require.Contains(t, planTree, "Insert") + require.Contains(t, planTree, "time") + require.Contains(t, planTree, "loops") tk.MustExec("with cte(a) as (select 1) select * from cte") planTree = getPlanTree() - require.True(t, strings.Contains(planTree, "CTE")) - require.True(t, strings.Contains(planTree, "1->Column#1")) - require.True(t, strings.Contains(planTree, "time")) - require.True(t, strings.Contains(planTree, "loops")) + require.Contains(t, planTree, "CTE") + require.Contains(t, planTree, "1->Column#1") + require.Contains(t, planTree, "time") + require.Contains(t, planTree, "loops") tk.MustExec("with cte(a) as (select 2) select * from cte") planTree = getPlanTree() - require.True(t, strings.Contains(planTree, "CTE")) - require.True(t, strings.Contains(planTree, "2->Column#1")) - require.True(t, strings.Contains(planTree, "time")) - require.True(t, strings.Contains(planTree, "loops")) + require.Contains(t, planTree, "CTE") + require.Contains(t, planTree, "2->Column#1") + require.Contains(t, planTree, "time") + require.Contains(t, planTree, "loops") + + tk.MustExec("select * from tp") + planTree = getPlanTree() + require.Contains(t, planTree, "PartitionUnion") + + tk.MustExec("select row_number() over (partition by c) from t1;") + planTree = getPlanTree() + require.Contains(t, planTree, "Shuffle") + require.Contains(t, planTree, "ShuffleReceiver") } func TestNormalizedDigest(t *testing.T) { diff --git a/util/plancodec/id.go b/util/plancodec/id.go index b6f699c3b9837..2b2e5e7e972a2 100644 --- a/util/plancodec/id.go +++ b/util/plancodec/id.go @@ -174,16 +174,19 @@ const ( typeDataSourceID int = 40 typeLoadDataID int = 41 typeTableSampleID int = 42 - typeTableFullScan int = 43 - typeTableRangeScan int = 44 - typeTableRowIDScan int = 45 - typeIndexFullScan int = 46 - typeIndexRangeScan int = 47 - typeExchangeReceiver int = 48 - typeExchangeSender int = 49 - typeCTE int = 50 - typeCTEDefinition int = 51 - typeCTETable int = 52 + typeTableFullScanID int = 43 + typeTableRangeScanID int = 44 + typeTableRowIDScanID int = 45 + typeIndexFullScanID int = 46 + typeIndexRangeScanID int = 47 + typeExchangeReceiverID int = 48 + typeExchangeSenderID int = 49 + typeCTEID int = 50 + typeCTEDefinitionID int = 51 + typeCTETableID int = 52 + typePartitionUnionID int = 53 + typeShuffleID int = 54 + typeShuffleReceiverID int = 55 ) // TypeStringToPhysicalID converts the plan type string to plan id. @@ -207,6 +210,8 @@ func TypeStringToPhysicalID(tp string) int { return typeJoinID case TypeUnion: return typeUnionID + case TypePartitionUnion: + return typePartitionUnionID case TypeTableScan: return typeTableScanID case TypeMemTableScan: @@ -255,6 +260,10 @@ func TypeStringToPhysicalID(tp string) int { return typeIndexReaderID case TypeWindow: return typeWindowID + case TypeShuffle: + return typeShuffleID + case TypeShuffleReceiver: + return typeShuffleReceiverID case TypeTiKVSingleGather: return typeTiKVSingleGatherID case TypeIndexMerge: @@ -274,25 +283,25 @@ func TypeStringToPhysicalID(tp string) int { case TypeTableSample: return typeTableSampleID case TypeTableFullScan: - return typeTableFullScan + return typeTableFullScanID case TypeTableRangeScan: - return typeTableRangeScan + return typeTableRangeScanID case TypeTableRowIDScan: - return typeTableRowIDScan + return typeTableRowIDScanID case TypeIndexFullScan: - return typeIndexFullScan + return typeIndexFullScanID case TypeIndexRangeScan: - return typeIndexRangeScan + return typeIndexRangeScanID case TypeExchangeReceiver: - return typeExchangeReceiver + return typeExchangeReceiverID case TypeExchangeSender: - return typeExchangeSender + return typeExchangeSenderID case TypeCTE: - return typeCTE + return typeCTEID case TypeCTEDefinition: - return typeCTEDefinition + return typeCTEDefinitionID case TypeCTETable: - return typeCTETable + return typeCTETableID } // Should never reach here. return 0 @@ -319,6 +328,8 @@ func PhysicalIDToTypeString(id int) string { return TypeJoin case typeUnionID: return TypeUnion + case typePartitionUnionID: + return TypePartitionUnion case typeTableScanID: return TypeTableScan case typeMemTableScanID: @@ -367,6 +378,10 @@ func PhysicalIDToTypeString(id int) string { return TypeIndexReader case typeWindowID: return TypeWindow + case typeShuffleID: + return TypeShuffle + case typeShuffleReceiverID: + return TypeShuffleReceiver case typeTiKVSingleGatherID: return TypeTiKVSingleGather case typeIndexMergeID: @@ -383,25 +398,25 @@ func PhysicalIDToTypeString(id int) string { return TypeLoadData case typeTableSampleID: return TypeTableSample - case typeTableFullScan: + case typeTableFullScanID: return TypeTableFullScan - case typeTableRangeScan: + case typeTableRangeScanID: return TypeTableRangeScan - case typeTableRowIDScan: + case typeTableRowIDScanID: return TypeTableRowIDScan - case typeIndexFullScan: + case typeIndexFullScanID: return TypeIndexFullScan - case typeIndexRangeScan: + case typeIndexRangeScanID: return TypeIndexRangeScan - case typeExchangeReceiver: + case typeExchangeReceiverID: return TypeExchangeReceiver - case typeExchangeSender: + case typeExchangeSenderID: return TypeExchangeSender - case typeCTE: + case typeCTEID: return TypeCTE - case typeCTEDefinition: + case typeCTEDefinitionID: return TypeCTEDefinition - case typeCTETable: + case typeCTETableID: return TypeCTETable } diff --git a/util/plancodec/id_test.go b/util/plancodec/id_test.go index f97939e4cc14f..8a7addd23fdba 100644 --- a/util/plancodec/id_test.go +++ b/util/plancodec/id_test.go @@ -68,16 +68,19 @@ func TestPlanIDChanged(t *testing.T) { {typeDataSourceID, 40}, {typeLoadDataID, 41}, {typeTableSampleID, 42}, - {typeTableFullScan, 43}, - {typeTableRangeScan, 44}, - {typeTableRowIDScan, 45}, - {typeIndexFullScan, 46}, - {typeIndexRangeScan, 47}, - {typeExchangeReceiver, 48}, - {typeExchangeSender, 49}, - {typeCTE, 50}, - {typeCTEDefinition, 51}, - {typeCTETable, 52}, + {typeTableFullScanID, 43}, + {typeTableRangeScanID, 44}, + {typeTableRowIDScanID, 45}, + {typeIndexFullScanID, 46}, + {typeIndexRangeScanID, 47}, + {typeExchangeReceiverID, 48}, + {typeExchangeSenderID, 49}, + {typeCTEID, 50}, + {typeCTEDefinitionID, 51}, + {typeCTETableID, 52}, + {typePartitionUnionID, 53}, + {typeShuffleID, 54}, + {typeShuffleReceiverID, 55}, } for _, testcase := range testCases { From 1beb4d4c07135555f3a6fdac7a6940efd6dfc542 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 18:18:52 +0800 Subject: [PATCH 18/42] stmtsummary: fix issue of concurrent map read and write (#35367) (#35387) close pingcap/tidb#35340 --- infoschema/cluster_tables_test.go | 34 +++++++++++++++++++++++++++++++ util/stmtsummary/reader.go | 28 ++++++++++++++----------- 2 files changed, 50 insertions(+), 12 deletions(-) diff --git a/infoschema/cluster_tables_test.go b/infoschema/cluster_tables_test.go index 5cae950bc897e..f8e7a06d07c45 100644 --- a/infoschema/cluster_tables_test.go +++ b/infoschema/cluster_tables_test.go @@ -23,6 +23,7 @@ import ( "runtime" "strconv" "strings" + "sync" "testing" "time" @@ -399,6 +400,39 @@ func TestStmtSummaryEvictedCountTable(t *testing.T) { require.NoError(t, tk.QueryToErr("select * from information_schema.CLUSTER_STATEMENTS_SUMMARY_EVICTED")) } +func TestStmtSummaryIssue35340(t *testing.T) { + var clean func() + s := new(clusterTablesSuite) + s.store, s.dom, clean = testkit.CreateMockStoreAndDomain(t) + defer clean() + + tk := s.newTestKitWithRoot(t) + tk.MustExec("set global tidb_stmt_summary_refresh_interval=1800") + tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 3000") + for i := 0; i < 100; i++ { + user := "user" + strconv.Itoa(i) + tk.MustExec(fmt.Sprintf("create user '%v'@'localhost'", user)) + } + tk.MustExec("flush privileges") + var wg sync.WaitGroup + for i := 0; i < 10; i++ { + wg.Add(1) + go func() { + defer wg.Done() + tk := s.newTestKitWithRoot(t) + for j := 0; j < 100; j++ { + user := "user" + strconv.Itoa(j) + require.True(t, tk.Session().Auth(&auth.UserIdentity{ + Username: user, + Hostname: "localhost", + }, nil, nil)) + tk.MustQuery("select count(*) from information_schema.statements_summary;") + } + }() + } + wg.Wait() +} + func TestStmtSummaryHistoryTableWithUserTimezone(t *testing.T) { // setup suite var clean func() diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index a0003d9390eef..d9c3e338fdd6f 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -129,11 +129,7 @@ func (ssr *stmtSummaryReader) getStmtByDigestRow(ssbd *stmtSummaryByDigest, begi // `ssElement` is lazy expired, so expired elements could also be read. // `beginTime` won't change since `ssElement` is created, so locking is not needed here. - isAuthed := true - if ssr.user != nil && !ssr.hasProcessPriv && ssElement != nil { - _, isAuthed = ssElement.authUsers[ssr.user.Username] - } - if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval || !isAuthed { + if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval { return nil } return ssr.getStmtByDigestElementRow(ssElement, ssbd) @@ -142,6 +138,14 @@ func (ssr *stmtSummaryReader) getStmtByDigestRow(ssbd *stmtSummaryByDigest, begi func (ssr *stmtSummaryReader) getStmtByDigestElementRow(ssElement *stmtSummaryByDigestElement, ssbd *stmtSummaryByDigest) []types.Datum { ssElement.Lock() defer ssElement.Unlock() + isAuthed := true + if ssr.user != nil && !ssr.hasProcessPriv { + _, isAuthed = ssElement.authUsers[ssr.user.Username] + } + if !isAuthed { + return nil + } + datums := make([]types.Datum, len(ssr.columnValueFactories)) for i, factory := range ssr.columnValueFactories { datums[i] = types.NewDatum(factory(ssr, ssElement, ssbd)) @@ -155,12 +159,9 @@ func (ssr *stmtSummaryReader) getStmtByDigestHistoryRow(ssbd *stmtSummaryByDiges rows := make([][]types.Datum, 0, len(ssElements)) for _, ssElement := range ssElements { - isAuthed := true - if ssr.user != nil && !ssr.hasProcessPriv { - _, isAuthed = ssElement.authUsers[ssr.user.Username] - } - if isAuthed { - rows = append(rows, ssr.getStmtByDigestElementRow(ssElement, ssbd)) + record := ssr.getStmtByDigestElementRow(ssElement, ssbd) + if record != nil { + rows = append(rows, record) } } return rows @@ -191,7 +192,10 @@ func (ssr *stmtSummaryReader) getStmtEvictedOtherHistoryRow(ssbde *stmtSummaryBy ssbd := new(stmtSummaryByDigest) for _, seElement := range seElements { - rows = append(rows, ssr.getStmtByDigestElementRow(seElement.otherSummary, ssbd)) + record := ssr.getStmtByDigestElementRow(seElement.otherSummary, ssbd) + if record != nil { + rows = append(rows, record) + } } return rows } From eba9f3e33f47aa3171a5504d998218265fc5d297 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 18:28:52 +0800 Subject: [PATCH 19/42] planner: fix cte-schema-clone will clone the old hashcode of its column if any (#35415) (#35472) close pingcap/tidb#35404 --- cmd/explaintest/r/explain_cte.result | 19 +++++++++++++++++++ cmd/explaintest/t/explain_cte.test | 5 +++++ expression/column.go | 5 +++++ planner/core/logical_plan_builder.go | 2 ++ 4 files changed, 31 insertions(+) diff --git a/cmd/explaintest/r/explain_cte.result b/cmd/explaintest/r/explain_cte.result index cc8804b4fc077..e4837878d3054 100644 --- a/cmd/explaintest/r/explain_cte.result +++ b/cmd/explaintest/r/explain_cte.result @@ -479,3 +479,22 @@ CTE_1 8000.00 root Non-Recursive CTE CTE_0 10000.00 root Non-Recursive CTE └─TableReader(Seed Part) 10000.00 root data:TableFullScan └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo +drop table if exists tbl; +create table tbl (id int); +explain with t1 as (select id from tbl), t2 as (select a.id from t1 a join t1 b on a.id = b.id) select * from t2 where id in (select id from t2); +id estRows task access object operator info +HashJoin_33 8000.00 root inner join, equal:[eq(test.tbl.id, test.tbl.id)] +├─HashAgg_37(Build) 5120.00 root group by:test.tbl.id, funcs:firstrow(test.tbl.id)->test.tbl.id +│ └─Selection_38 8000.00 root not(isnull(test.tbl.id)) +│ └─CTEFullScan_39 10000.00 root CTE:t2 data:CTE_1 +└─Selection_35(Probe) 8000.00 root not(isnull(test.tbl.id)) + └─CTEFullScan_36 10000.00 root CTE:t2 data:CTE_1 +CTE_1 10000.00 root Non-Recursive CTE +└─HashJoin_25(Seed Part) 10000.00 root inner join, equal:[eq(test.tbl.id, test.tbl.id)] + ├─Selection_29(Build) 8000.00 root not(isnull(test.tbl.id)) + │ └─CTEFullScan_30 10000.00 root CTE:b data:CTE_0 + └─Selection_27(Probe) 8000.00 root not(isnull(test.tbl.id)) + └─CTEFullScan_28 10000.00 root CTE:a data:CTE_0 +CTE_0 10000.00 root Non-Recursive CTE +└─TableReader_22(Seed Part) 10000.00 root data:TableFullScan_21 + └─TableFullScan_21 10000.00 cop[tikv] table:tbl keep order:false, stats:pseudo diff --git a/cmd/explaintest/t/explain_cte.test b/cmd/explaintest/t/explain_cte.test index fd6ba042c9fed..87ea10662e1c2 100644 --- a/cmd/explaintest/t/explain_cte.test +++ b/cmd/explaintest/t/explain_cte.test @@ -258,3 +258,8 @@ desc format='brief' with all_data as select v1.tps v1_tps,v2.tps v2_tps from version1 v1, version2 v2 where v1.bench_type =v2.bench_type; + +# issue 35404 +drop table if exists tbl; +create table tbl (id int); +explain with t1 as (select id from tbl), t2 as (select a.id from t1 a join t1 b on a.id = b.id) select * from t2 where id in (select id from t2); diff --git a/expression/column.go b/expression/column.go index c1f3960b2cbf3..b83e0c0203820 100644 --- a/expression/column.go +++ b/expression/column.go @@ -490,6 +490,11 @@ func (col *Column) HashCode(_ *stmtctx.StatementContext) []byte { return col.hashcode } +// CleanHashCode will clean the hashcode you may be cached before. It's used especially in schema-cloned & reallocated-uniqueID's cases. +func (col *Column) CleanHashCode() { + col.hashcode = make([]byte, 0, 9) +} + // ResolveIndices implements Expression interface. func (col *Column) ResolveIndices(schema *Schema) (Expression, error) { newCol := col.Clone() diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index b1c6c6f23d74d..5a76bd9d34385 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -6894,6 +6894,8 @@ func getResultCTESchema(seedSchema *expression.Schema, svar *variable.SessionVar col.RetType = col.RetType.Clone() col.UniqueID = svar.AllocPlanColumnID() col.RetType.DelFlag(mysql.NotNullFlag) + // Since you have reallocated unique id here, the old-cloned-cached hash code is not valid anymore. + col.CleanHashCode() } return res } From 2798dba6ea87a64d3f075a478fc44c8539001146 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 18:40:51 +0800 Subject: [PATCH 20/42] raw_restore: fix the issue that raw restore rewrite the t prefix keys (#35641) (#35697) close pingcap/tidb#35279 --- br/pkg/task/restore_raw.go | 3 +-- br/tests/br_rawkv/run.sh | 28 +++++++++++++++++++++++++++- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/br/pkg/task/restore_raw.go b/br/pkg/task/restore_raw.go index d8fcb46475809..452cccfad8c42 100644 --- a/br/pkg/task/restore_raw.go +++ b/br/pkg/task/restore_raw.go @@ -146,8 +146,7 @@ func RunRestoreRaw(c context.Context, g glue.Glue, cmdName string, cfg *RestoreR !cfg.LogProgress) // RawKV restore does not need to rewrite keys. - rewrite := &restore.RewriteRules{} - err = restore.SplitRanges(ctx, client, ranges, rewrite, updateCh, true) + err = restore.SplitRanges(ctx, client, ranges, nil, updateCh, true) if err != nil { return errors.Trace(err) } diff --git a/br/tests/br_rawkv/run.sh b/br/tests/br_rawkv/run.sh index 97450d3e65fc7..b32cca0f8e41f 100755 --- a/br/tests/br_rawkv/run.sh +++ b/br/tests/br_rawkv/run.sh @@ -98,12 +98,23 @@ run_test() { --key "$TEST_DIR/certs/br.key" \ --mode put --put-data "311121:31, 31112100:32, 311122:33, 31112200:34, 3111220000:35, 311123:36" + + # put some keys starts with t. https://github.com/pingcap/tidb/issues/35279 + # t_128_r_12 ----> 745f3132385f725f3132 + # t_128_r_13 ----> 745f3132385f725f3133 + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode put --put-data "745f3132385f725f3132:31, 745f3132385f725f3133:32" + checksum_ori=$(checksum 31 3130303030303030) checksum_partial=$(checksum 311111 311122) + checksum_t_prefix=$(checksum 745f3132385f725f3131 745f3132385f725f3134) # backup rawkv echo "backup start..." - run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 3130303030303030 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + run_br --pd $PD_ADDR backup raw -s "local://$BACKUP_DIR" --start 31 --end 745f3132385f725f3134 --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" # delete data in range[start-key, end-key) clean 31 3130303030303030 @@ -153,6 +164,21 @@ run_test() { fail_and_exit fi + echo "t prefix restore start..." + run_br --pd $PD_ADDR restore raw -s "local://$BACKUP_DIR" --start "745f3132385f725f3131" --end "745f3132385f725f3134" --format hex --concurrency 4 --crypter.method "aes128-ctr" --crypter.key "0123456789abcdef0123456789abcdef" + bin/rawkv --pd $PD_ADDR \ + --ca "$TEST_DIR/certs/ca.pem" \ + --cert "$TEST_DIR/certs/br.pem" \ + --key "$TEST_DIR/certs/br.key" \ + --mode scan --start-key 745f3132385f725f3131 --end-key 745f3132385f725f3134 + + checksum_new=$(checksum 745f3132385f725f3131 745f3132385f725f3134) + + if [ "$checksum_new" != "$checksum_t_prefix" ];then + echo "checksum failed after restore" + fail_and_exit + fi + export GO_FAILPOINTS="" } From c4a849e44ad208b090556e88e24201b71d5fc8c4 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 18:52:52 +0800 Subject: [PATCH 21/42] stmtctx: add mutex to protect stmtCache(#36159) (#36351) (#36391) close pingcap/tidb#36159 --- sessionctx/stmtctx/stmtctx.go | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index ebcbd72a2ef67..5aed2e633f3b1 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -182,7 +182,12 @@ type StatementContext struct { TaskMapBakTS uint64 // counter for // stmtCache is used to store some statement-related values. - stmtCache map[StmtCacheKey]interface{} + // add mutex to protect stmtCache concurrent access + // https://github.com/pingcap/tidb/issues/36159 + stmtCache struct { + mu sync.Mutex + data map[StmtCacheKey]interface{} + } // Map to store all CTE storages of current SQL. // Will clean up at the end of the execution. @@ -295,23 +300,29 @@ const ( // GetOrStoreStmtCache gets the cached value of the given key if it exists, otherwise stores the value. func (sc *StatementContext) GetOrStoreStmtCache(key StmtCacheKey, value interface{}) interface{} { - if sc.stmtCache == nil { - sc.stmtCache = make(map[StmtCacheKey]interface{}) + sc.stmtCache.mu.Lock() + defer sc.stmtCache.mu.Unlock() + if sc.stmtCache.data == nil { + sc.stmtCache.data = make(map[StmtCacheKey]interface{}) } - if _, ok := sc.stmtCache[key]; !ok { - sc.stmtCache[key] = value + if _, ok := sc.stmtCache.data[key]; !ok { + sc.stmtCache.data[key] = value } - return sc.stmtCache[key] + return sc.stmtCache.data[key] } // ResetInStmtCache resets the cache of given key. func (sc *StatementContext) ResetInStmtCache(key StmtCacheKey) { - delete(sc.stmtCache, key) + sc.stmtCache.mu.Lock() + defer sc.stmtCache.mu.Unlock() + delete(sc.stmtCache.data, key) } // ResetStmtCache resets all cached values. func (sc *StatementContext) ResetStmtCache() { - sc.stmtCache = make(map[StmtCacheKey]interface{}) + sc.stmtCache.mu.Lock() + defer sc.stmtCache.mu.Unlock() + sc.stmtCache.data = make(map[StmtCacheKey]interface{}) } // SQLDigest gets normalized and digest for provided sql. From 21e40f4c9afe114b9148d78396030a0fd5fc22bb Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 20 Aug 2022 19:04:52 +0800 Subject: [PATCH 22/42] statistics: fix "Invalid xxx character string" error when loading new collation stats (#36709) (#36723) close pingcap/tidb#35208 --- statistics/handle/dump.go | 19 +++++++++++-------- statistics/handle/dump_test.go | 2 +- statistics/handle/handle.go | 10 +++++++--- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 71038cd9a74c5..72273b87dd60a 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -300,18 +300,21 @@ func TableStatsFromJSON(tableInfo *model.TableInfo, physicalID int64, jsonTbl *J } hist := statistics.HistogramFromProto(jsonCol.Histogram) sc := &stmtctx.StatementContext{TimeZone: time.UTC} - // Deal with sortKey, the length of sortKey maybe longer than the column's length. - orgLen := colInfo.FieldType.GetFlen() - if types.IsString(colInfo.FieldType.GetType()) { - colInfo.SetFlen(types.UnspecifiedLength) + tmpFT := colInfo.FieldType + // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the + // original value. + // But there's additional conversion logic for new collation data, and the collate key might be longer than + // the FieldType.flen. + // If we use the original FieldType here, there might be errors like "Invalid utf8mb4 character string" + // or "Data too long". + // So we change it to TypeBlob to bypass those logics here. + if colInfo.FieldType.EvalType() == types.ETString && colInfo.FieldType.GetType() != mysql.TypeEnum && colInfo.FieldType.GetType() != mysql.TypeSet { + tmpFT = *types.NewFieldType(mysql.TypeBlob) } - hist, err := hist.ConvertTo(sc, &colInfo.FieldType) + hist, err := hist.ConvertTo(sc, &tmpFT) if err != nil { return nil, errors.Trace(err) } - if types.IsString(colInfo.FieldType.GetType()) { - colInfo.SetFlen(orgLen) - } cm, topN := statistics.CMSketchAndTopNFromProto(jsonCol.CMSketch) fms := statistics.FMSketchFromProto(jsonCol.FMSketch) hist.ID, hist.NullCount, hist.LastUpdateVersion, hist.TotColSize, hist.Correlation = colInfo.ID, jsonCol.NullCount, jsonCol.LastUpdateVersion, jsonCol.TotColSize, jsonCol.Correlation diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index d2c67d49dcdbe..f43fe927f00aa 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -400,7 +400,7 @@ func TestLoadStatsForNewCollation(t *testing.T) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b varchar(3) collate utf8mb4_unicode_ci)") - tk.MustExec("insert into t value(1, 'aaa'), (3, 'aab'), (5, 'bba'), (2, 'bbb'), (4, 'cca'), (6, 'ccc')") + tk.MustExec("insert into t value(1, 'aaa'), (1, 'aaa'), (3, 'aab'), (3, 'aab'), (5, 'bba'), (2, 'bbb'), (4, 'cca'), (6, 'ccc'), (7, 'Ste')") // mark column stats as needed tk.MustExec("select * from t where a = 3") tk.MustExec("select * from t where b = 'bbb'") diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index c213338d44b8e..37bed98516de2 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -1355,9 +1355,13 @@ func (h *Handle) histogramFromStorage(reader *statsReader, tableID int64, colID } else { sc := &stmtctx.StatementContext{TimeZone: time.UTC} d := rows[i].GetDatum(2, &fields[2].Column.FieldType) - // When there's new collation data, the length of bounds of histogram(the collate key) might be - // longer than the FieldType.flen of this column. - // We change it to TypeBlob to bypass the length check here. + // For new collation data, when storing the bounds of the histogram, we store the collate key instead of the + // original value. + // But there's additional conversion logic for new collation data, and the collate key might be longer than + // the FieldType.flen. + // If we use the original FieldType here, there might be errors like "Invalid utf8mb4 character string" + // or "Data too long". + // So we change it to TypeBlob to bypass those logics here. if tp.EvalType() == types.ETString && tp.GetType() != mysql.TypeEnum && tp.GetType() != mysql.TypeSet { tp = types.NewFieldType(mysql.TypeBlob) } From 68c67549c76cac4b8bb8881ec381400b630cd9c4 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sun, 21 Aug 2022 10:14:52 +0800 Subject: [PATCH 23/42] load_data: fix the bug that column list does not work in load data. (#35222) (#35466) close pingcap/tidb#35198 --- errno/errcode.go | 1 + errno/errname.go | 1 + errors.toml | 5 ++ executor/errors.go | 1 + executor/load_data.go | 37 ++++++++++ server/server_test.go | 138 +++++++++++++++++++++++++++++++++++++ server/tidb_serial_test.go | 1 + 7 files changed, 184 insertions(+) diff --git a/errno/errcode.go b/errno/errcode.go index 4296db33d3dd5..83284118d103a 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -979,6 +979,7 @@ const ( ErrWarnOptimizerHintParseError = 8064 ErrWarnOptimizerHintInvalidInteger = 8065 ErrUnsupportedSecondArgumentType = 8066 + ErrColumnNotMatched = 8067 ErrInvalidPluginID = 8101 ErrInvalidPluginManifest = 8102 ErrInvalidPluginName = 8103 diff --git a/errno/errname.go b/errno/errname.go index 3cf46e82fe6b5..aaebb8ed668e1 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1046,6 +1046,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil), ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil), ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil), + ErrColumnNotMatched: mysql.Message("Load data: unmatched columns", nil), ErrLockExpire: mysql.Message("TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction", nil), ErrTableOptionUnionUnsupported: mysql.Message("CREATE/ALTER table with union option is not supported", nil), ErrTableOptionInsertMethodUnsupported: mysql.Message("CREATE/ALTER table with insert method option is not supported", nil), diff --git a/errors.toml b/errors.toml index 1bcc9176ba983..08920b7fe5725 100755 --- a/errors.toml +++ b/errors.toml @@ -1391,6 +1391,11 @@ error = ''' TiDB admin check table failed. ''' +["executor:8067"] +error = ''' +Load data: unmatched columns +''' + ["executor:8114"] error = ''' Unknown plan diff --git a/executor/errors.go b/executor/errors.go index 46d0fdd1ee62d..a5bbf30e97d8b 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -32,6 +32,7 @@ var ( ErrUnsupportedPs = dbterror.ClassExecutor.NewStd(mysql.ErrUnsupportedPs) ErrSubqueryMoreThan1Row = dbterror.ClassExecutor.NewStd(mysql.ErrSubqueryNo1Row) ErrIllegalGrantForTable = dbterror.ClassExecutor.NewStd(mysql.ErrIllegalGrantForTable) + ErrColumnsNotMatched = dbterror.ClassExecutor.NewStd(mysql.ErrColumnNotMatched) ErrCantCreateUserWithGrant = dbterror.ClassExecutor.NewStd(mysql.ErrCantCreateUserWithGrant) ErrPasswordNoMatch = dbterror.ClassExecutor.NewStd(mysql.ErrPasswordNoMatch) diff --git a/executor/load_data.go b/executor/load_data.go index 87ceb964f7e03..fdaf58222f463 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -129,6 +129,36 @@ type FieldMapping struct { UserVar *ast.VariableExpr } +// reorderColumns reorder the e.insertColumns according to the order of columnNames +// Note: We must ensure there must be one-to-one mapping between e.insertColumns and columnNames in terms of column name. +func (e *LoadDataInfo) reorderColumns(columnNames []string) error { + cols := e.insertColumns + + if len(cols) != len(columnNames) { + return ErrColumnsNotMatched + } + + reorderedColumns := make([]*table.Column, len(cols)) + + if columnNames == nil { + return nil + } + + mapping := make(map[string]int) + for idx, colName := range columnNames { + mapping[strings.ToLower(colName)] = idx + } + + for _, col := range cols { + idx := mapping[col.Name.L] + reorderedColumns[idx] = col + } + + e.insertColumns = reorderedColumns + + return nil +} + // initLoadColumns sets columns which the input fields loaded to. func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { var cols []*table.Column @@ -161,6 +191,13 @@ func (e *LoadDataInfo) initLoadColumns(columnNames []string) error { break } } + + // e.insertColumns is appended according to the original tables' column sequence. + // We have to reorder it to follow the use-specified column order which is shown in the columnNames. + if err = e.reorderColumns(columnNames); err != nil { + return err + } + e.rowLen = len(e.insertColumns) // Check column whether is specified only once. err = table.CheckOnce(cols) diff --git a/server/server_test.go b/server/server_test.go index 83f13fdd97b5a..b463303666baa 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -850,6 +850,144 @@ func (cli *testServerClient) checkRows(t *testing.T, rows *sql.Rows, expectedRow require.Equal(t, strings.Join(expectedRows, "\n"), strings.Join(result, "\n")) } +func (cli *testServerClient) runTestLoadDataWithColumnList(t *testing.T, _ *Server) { + fp, err := os.CreateTemp("", "load_data_test.csv") + require.NoError(t, err) + path := fp.Name() + require.NotNil(t, fp) + defer func() { + err = fp.Close() + require.NoError(t, err) + err = os.Remove(path) + require.NoError(t, err) + }() + + _, err = fp.WriteString("dsadasdas\n" + + "\"1\",\"1\",,\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"2\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"3\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"\n" + + "\"1\",\"4\",\"a\",\"2022-04-19\",\"a\",\"2022-04-19 00:00:01\"") + + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (k,id,c,dt,vv,ts)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err := rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + }) + + // Also test cases where column list only specifies partial columns + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (k,id,c)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,,,", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,,,", ",")) + }) + + // Also test for case-insensitivity + cli.runTestsOnNewDB(t, func(config *mysql.Config) { + config.AllowAllFiles = true + config.Params["sql_mode"] = "''" + }, "LoadData", func(db *testkit.DBTestKit) { + db.MustExec("use test") + db.MustExec("drop table if exists t66") + db.MustExec("create table t66 (id int primary key,k int,c varchar(10),dt date,vv char(1),ts datetime)") + // We modify the upper case and lower case in the column list to test the case-insensitivity + db.MustExec(fmt.Sprintf("LOAD DATA LOCAL INFILE '%s' INTO TABLE t66 FIELDS TERMINATED BY ',' ENCLOSED BY '\\\"' IGNORE 1 LINES (K,Id,c,dT,Vv,Ts)", path)) + rows := db.MustQuery("select * from t66") + var ( + id sql.NullString + k sql.NullString + c sql.NullString + dt sql.NullString + vv sql.NullString + ts sql.NullString + ) + columns := []*sql.NullString{&k, &id, &c, &dt, &vv, &ts} + require.Truef(t, rows.Next(), "unexpected data") + err := rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,1,,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,2,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,3,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + require.Truef(t, rows.Next(), "unexpected data") + err = rows.Scan(&id, &k, &c, &dt, &vv, &ts) + require.NoError(t, err) + columnsAsExpected(t, columns, strings.Split("1,4,a,2022-04-19,a,2022-04-19 00:00:01", ",")) + }) +} + +func columnsAsExpected(t *testing.T, columns []*sql.NullString, expected []string) { + require.Equal(t, len(columns), len(expected)) + + for i := 0; i < len(columns); i++ { + require.Equal(t, expected[i], columns[i].String) + } +} + func (cli *testServerClient) runTestLoadData(t *testing.T, server *Server) { fp, err := os.CreateTemp("", "load_data_test.csv") require.NoError(t, err) diff --git a/server/tidb_serial_test.go b/server/tidb_serial_test.go index affce842d3df3..e2b119e4b1331 100644 --- a/server/tidb_serial_test.go +++ b/server/tidb_serial_test.go @@ -40,6 +40,7 @@ func TestLoadData1(t *testing.T) { ts, cleanup := createTidbTestSuite(t) defer cleanup() + ts.runTestLoadDataWithColumnList(t, ts.server) ts.runTestLoadData(t, ts.server) ts.runTestLoadDataWithSelectIntoOutfile(t, ts.server) ts.runTestLoadDataForSlowLog(t, ts.server) From 0f9b69ffd7e41a486231591b90c81ecc58e1cc5d Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sun, 21 Aug 2022 13:40:52 +0800 Subject: [PATCH 24/42] executor: do not append extra cols to the old row when `updateDupRow` (#33656) (#35168) close pingcap/tidb#33608 --- executor/insert.go | 16 +++++++++------- executor/insert_test.go | 14 ++++++++++++++ table/tables/tables.go | 5 +++++ 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/executor/insert.go b/executor/insert.go index 3b872e5f77928..a391d6412aae8 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -193,13 +193,13 @@ func (e *InsertExec) updateDupRow(ctx context.Context, idxInBatch int, txn kv.Tr if err != nil { return err } - // get the extra columns from the SELECT clause and get the final `oldRow`. + // get the extra columns from the SELECT clause. + var extraCols []types.Datum if len(e.ctx.GetSessionVars().CurrInsertBatchExtraCols) > 0 { - extraCols := e.ctx.GetSessionVars().CurrInsertBatchExtraCols[idxInBatch] - oldRow = append(oldRow, extraCols...) + extraCols = e.ctx.GetSessionVars().CurrInsertBatchExtraCols[idxInBatch] } - err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, e.OnDuplicate, idxInBatch) + err = e.doDupRowUpdate(ctx, handle, oldRow, row.row, extraCols, e.OnDuplicate, idxInBatch) if e.ctx.GetSessionVars().StmtCtx.DupKeyAsWarning && kv.ErrKeyExists.Equal(err) { e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) return nil @@ -374,16 +374,18 @@ func (e *InsertExec) initEvalBuffer4Dup() { // doDupRowUpdate updates the duplicate row. func (e *InsertExec) doDupRowUpdate(ctx context.Context, handle kv.Handle, oldRow []types.Datum, newRow []types.Datum, - cols []*expression.Assignment, idxInBatch int) error { + extraCols []types.Datum, cols []*expression.Assignment, idxInBatch int) error { assignFlag := make([]bool, len(e.Table.WritableCols())) // See http://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_values e.curInsertVals.SetDatums(newRow...) e.ctx.GetSessionVars().CurrInsertValues = e.curInsertVals.ToRow() // NOTE: In order to execute the expression inside the column assignment, - // we have to put the value of "oldRow" before "newRow" in "row4Update" to - // be consistent with "Schema4OnDuplicate" in the "Insert" PhysicalPlan. + // we have to put the value of "oldRow" and "extraCols" before "newRow" in + // "row4Update" to be consistent with "Schema4OnDuplicate" in the "Insert" + // PhysicalPlan. e.row4Update = e.row4Update[:0] e.row4Update = append(e.row4Update, oldRow...) + e.row4Update = append(e.row4Update, extraCols...) e.row4Update = append(e.row4Update, newRow...) // Update old row when the key is duplicated. diff --git a/executor/insert_test.go b/executor/insert_test.go index 4bcb8dca55958..5696f201f7a49 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -22,6 +22,7 @@ import ( "testing" "time" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/meta/autoid" @@ -39,6 +40,19 @@ func TestInsertOnDuplicateKey(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() tk := testkit.NewTestKit(t, store) + testInsertOnDuplicateKey(t, tk) +} + +func TestInsertOnDuplicateKeyWithBinlog(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + failpoint.Enable("github.com/pingcap/tidb/table/tables/forceWriteBinlog", "return") + defer failpoint.Disable("github.com/pingcap/tidb/table/tables/forceWriteBinlog") + testInsertOnDuplicateKey(t, tk) +} + +func testInsertOnDuplicateKey(t *testing.T, tk *testkit.TestKit) { tk.MustExec("use test") tk.MustExec(`drop table if exists t1, t2;`) diff --git a/table/tables/tables.go b/table/tables/tables.go index fe8c4f4a513ad..12b1b8639a93a 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1556,6 +1556,11 @@ func (t *TableCommon) Type() table.Type { } func shouldWriteBinlog(ctx sessionctx.Context, tblInfo *model.TableInfo) bool { + failpoint.Inject("forceWriteBinlog", func() { + // Just to cover binlog related code in this package, since the `BinlogClient` is + // still nil, mutations won't be written to pump on commit. + failpoint.Return(true) + }) if ctx.GetSessionVars().BinlogClient == nil { return false } From 55d31c77537dfd7b689450ffc554966c0f84e732 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sun, 21 Aug 2022 17:40:52 +0800 Subject: [PATCH 25/42] infoschema: try on each PD member until one succeeds or all fail (#35285) (#35508) close pingcap/tidb#35268 --- infoschema/tables.go | 39 +++++++++++++++++++++++++++++++++++---- 1 file changed, 35 insertions(+), 4 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 787597f03f63c..765c0bfe6a02e 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -31,7 +31,9 @@ import ( "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/stmtsummary" + "go.uber.org/zap" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" @@ -1645,18 +1647,33 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { if err != nil { return nil, errors.Trace(err) } - var servers = make([]ServerInfo, 0, len(members)) + // TODO: maybe we should unify the PD API request interface. + var ( + memberNum = len(members) + servers = make([]ServerInfo, 0, memberNum) + errs = make([]error, 0, memberNum) + ) + if memberNum == 0 { + return servers, nil + } + // Try on each member until one succeeds or all fail. for _, addr := range members { // Get PD version, git_hash url := fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), addr, pdapi.Status) req, err := http.NewRequest(http.MethodGet, url, nil) if err != nil { - return nil, errors.Trace(err) + ctx.GetSessionVars().StmtCtx.AppendWarning(err) + logutil.BgLogger().Warn("create pd server info request error", zap.String("url", url), zap.Error(err)) + errs = append(errs, err) + continue } req.Header.Add("PD-Allow-follower-handle", "true") resp, err := util.InternalHTTPClient().Do(req) if err != nil { - return nil, errors.Trace(err) + ctx.GetSessionVars().StmtCtx.AppendWarning(err) + logutil.BgLogger().Warn("request pd server info error", zap.String("url", url), zap.Error(err)) + errs = append(errs, err) + continue } var content = struct { Version string `json:"version"` @@ -1666,7 +1683,10 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { err = json.NewDecoder(resp.Body).Decode(&content) terror.Log(resp.Body.Close()) if err != nil { - return nil, errors.Trace(err) + ctx.GetSessionVars().StmtCtx.AppendWarning(err) + logutil.BgLogger().Warn("close pd server info request error", zap.String("url", url), zap.Error(err)) + errs = append(errs, err) + continue } if len(content.Version) > 0 && content.Version[0] == 'v' { content.Version = content.Version[1:] @@ -1681,6 +1701,17 @@ func GetPDServerInfo(ctx sessionctx.Context) ([]ServerInfo, error) { StartTimestamp: content.StartTimestamp, }) } + // Return the errors if all members' requests fail. + if len(errs) == memberNum { + errorMsg := "" + for idx, err := range errs { + errorMsg += err.Error() + if idx < memberNum-1 { + errorMsg += "; " + } + } + return nil, errors.Trace(fmt.Errorf("%s", errorMsg)) + } return servers, nil } From 3c7d7139500d6dbc4711f7f8dcbdd62891a2e2be Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 22 Aug 2022 10:34:53 +0800 Subject: [PATCH 26/42] executor: fix left join on partition table generate invalid lock key (#35732) (#35774) close pingcap/tidb#28073 --- executor/executor.go | 9 +++++++++ executor/union_scan_test.go | 24 ++++++++++++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/executor/executor.go b/executor/executor.go index f903335a91fdf..9aba519361175 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1025,6 +1025,15 @@ func (e *SelectLockExec) Next(ctx context.Context, req *chunk.Chunk) error { physTblID := tblID if physTblColIdx, ok := e.tblID2PhysTblIDColIdx[tblID]; ok { physTblID = row.GetInt64(physTblColIdx) + if physTblID == 0 { + // select * from t1 left join t2 on t1.c = t2.c for update + // The join right side might be added NULL in left join + // In that case, physTblID is 0, so skip adding the lock. + // + // Note, we can't distinguish whether it's the left join case, + // or a bug that TiKV return without correct physical ID column. + continue + } } e.keys = append(e.keys, tablecodec.EncodeRowKeyWithHandle(physTblID, handle)) } diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 4979af65273bd..b585ba5f20c13 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -15,11 +15,13 @@ package executor_test import ( + "encoding/hex" "fmt" "testing" "time" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/store/helper" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/benchdaily" @@ -456,6 +458,28 @@ func TestIssue28073(t *testing.T) { break } require.False(t, exist) + + // Another case, left join on partition table should not generate locks on physical ID = 0 + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (c_int int, c_str varchar(40), primary key (c_int, c_str));") + tk.MustExec("create table t2 (c_int int, c_str varchar(40), primary key (c_int)) partition by hash (c_int) partitions 4;") + tk.MustExec("insert into t1 (`c_int`, `c_str`) values (1, 'upbeat solomon'), (5, 'sharp rubin');") + tk.MustExec("insert into t2 (`c_int`, `c_str`) values (1, 'clever haibt'), (4, 'kind margulis');") + tk.MustExec("begin pessimistic;") + tk.MustQuery("select * from t1 left join t2 on t1.c_int = t2.c_int for update;").Check(testkit.Rows( + "1 upbeat solomon 1 clever haibt", + "5 sharp rubin ", + )) + key, err := hex.DecodeString("7480000000000000005F728000000000000000") + require.NoError(t, err) + h := helper.NewHelper(store.(helper.Storage)) + resp, err := h.GetMvccByEncodedKey(key) + require.NoError(t, err) + require.Nil(t, resp.Info.Lock) + require.Len(t, resp.Info.Writes, 0) + require.Len(t, resp.Info.Values, 0) + + tk.MustExec("rollback;") } func TestIssue32422(t *testing.T) { From 6d0585bed222516b6a01dd70647499113c32c85d Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 22 Aug 2022 13:48:20 +0800 Subject: [PATCH 27/42] *: only add default value for final aggregation to fix the aggregate push down (partition) union case (#35443) (#35772) close pingcap/tidb#35295 --- executor/aggregate_test.go | 40 ++++++++++++++++++++++ executor/builder.go | 10 ++++-- expression/aggregation/descriptor.go | 2 -- planner/core/physical_plans.go | 2 +- planner/core/rule_aggregation_push_down.go | 10 ++++++ planner/core/rule_eliminate_projection.go | 4 +-- planner/core/task.go | 22 ++++++++++-- 7 files changed, 80 insertions(+), 10 deletions(-) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index 764b2e624d7ac..f36e066505ad1 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -1589,3 +1589,43 @@ func TestRandomPanicAggConsume(t *testing.T) { require.EqualError(t, err, "failpoint panic: ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]") } } + +func TestIssue35295(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t100") + // This bug only happens on partition prune mode = 'static' + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec(`CREATE TABLE t100 ( +ID bigint(20) unsigned NOT NULL AUTO_INCREMENT, +col1 int(10) NOT NULL DEFAULT '0' COMMENT 'test', +money bigint(20) NOT NULL COMMENT 'test', +logtime datetime NOT NULL COMMENT '记录时间', +PRIMARY KEY (ID,logtime) +) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin AUTO_INCREMENT=1 COMMENT='test' +PARTITION BY RANGE COLUMNS(logtime) ( +PARTITION p20220608 VALUES LESS THAN ("20220609"), +PARTITION p20220609 VALUES LESS THAN ("20220610"), +PARTITION p20220610 VALUES LESS THAN ("20220611"), +PARTITION p20220611 VALUES LESS THAN ("20220612"), +PARTITION p20220612 VALUES LESS THAN ("20220613"), +PARTITION p20220613 VALUES LESS THAN ("20220614"), +PARTITION p20220614 VALUES LESS THAN ("20220615"), +PARTITION p20220615 VALUES LESS THAN ("20220616"), +PARTITION p20220616 VALUES LESS THAN ("20220617"), +PARTITION p20220617 VALUES LESS THAN ("20220618"), +PARTITION p20220618 VALUES LESS THAN ("20220619"), +PARTITION p20220619 VALUES LESS THAN ("20220620"), +PARTITION p20220620 VALUES LESS THAN ("20220621"), +PARTITION p20220621 VALUES LESS THAN ("20220622"), +PARTITION p20220622 VALUES LESS THAN ("20220623"), +PARTITION p20220623 VALUES LESS THAN ("20220624"), +PARTITION p20220624 VALUES LESS THAN ("20220625") + );`) + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-09 00:00:00');") + tk.MustExec("insert into t100(col1,money,logtime) values (100,10,'2022-06-10 00:00:00');") + tk.MustQuery("SELECT /*+STREAM_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) + tk.MustQuery("SELECT /*+HASH_AGG()*/ col1,sum(money) FROM t100 WHERE logtime>='2022-06-09 00:00:00' AND col1=100 ;").Check(testkit.Rows("100 20")) +} diff --git a/executor/builder.go b/executor/builder.go index a225a7e4a95ec..53e1f7e16df57 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1438,7 +1438,9 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for _, aggDesc := range v.AggFuncs { if aggDesc.HasDistinct || len(aggDesc.OrderByItems) > 0 { @@ -1494,10 +1496,14 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu groupChecker: newVecGroupChecker(b.ctx, v.GroupByItems), aggFuncs: make([]aggfuncs.AggFunc, 0, len(v.AggFuncs)), } + if len(v.GroupByItems) != 0 || aggregation.IsAllFirstRow(v.AggFuncs) { e.defaultVal = nil } else { - e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + // Only do this for final agg, see issue #35295, #30923 + if v.IsFinalAgg() { + e.defaultVal = chunk.NewChunkWithCapacity(retTypes(e), 1) + } } for i, aggDesc := range v.AggFuncs { aggFunc := aggfuncs.Build(b.ctx, aggDesc, i) diff --git a/expression/aggregation/descriptor.go b/expression/aggregation/descriptor.go index 115559022279c..debeaf8d6399e 100644 --- a/expression/aggregation/descriptor.go +++ b/expression/aggregation/descriptor.go @@ -123,8 +123,6 @@ func (a *AggFuncDesc) Split(ordinal []int) (partialAggDesc, finalAggDesc *AggFun partialAggDesc.Mode = Partial1Mode } else if a.Mode == FinalMode { partialAggDesc.Mode = Partial2Mode - } else { - panic("Error happened during AggFuncDesc.Split, the AggFunctionMode is not CompleteMode or FinalMode.") } finalAggDesc = &AggFuncDesc{ Mode: FinalMode, // We only support FinalMode now in final phase. diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index 9f5721049c166..c5c43df0a0a91 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -1077,7 +1077,7 @@ type basePhysicalAgg struct { MppPartitionCols []*property.MPPPartitionColumn } -func (p *basePhysicalAgg) isFinalAgg() bool { +func (p *basePhysicalAgg) IsFinalAgg() bool { if len(p.AggFuncs) > 0 { if p.AggFuncs[0].Mode == aggregation.FinalMode || p.AggFuncs[0].Mode == aggregation.CompleteMode { return true diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 1972e62fed025..496817d87c3d9 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -404,6 +404,16 @@ func (a *aggregationPushDownSolver) tryAggPushDownForUnion(union *LogicalUnionAl if pushedAgg == nil { return nil } + + // Update the agg mode for the pushed down aggregation. + for _, aggFunc := range pushedAgg.AggFuncs { + if aggFunc.Mode == aggregation.CompleteMode { + aggFunc.Mode = aggregation.Partial1Mode + } else if aggFunc.Mode == aggregation.FinalMode { + aggFunc.Mode = aggregation.Partial2Mode + } + } + newChildren := make([]LogicalPlan, 0, len(union.Children())) for _, child := range union.Children() { newChild, err := a.pushAggCrossUnion(pushedAgg, union.Schema(), child) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 5516a242b486d..70a55f7e4e339 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -49,14 +49,14 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { // passing down the aggregation mode to TiFlash. if physicalAgg, ok := p.Children()[0].(*PhysicalHashAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } } if physicalAgg, ok := p.Children()[0].(*PhysicalStreamAgg); ok { if physicalAgg.MppRunMode == Mpp1Phase || physicalAgg.MppRunMode == Mpp2Phase || physicalAgg.MppRunMode == MppScalar { - if physicalAgg.isFinalAgg() { + if physicalAgg.IsFinalAgg() { return false } } diff --git a/planner/core/task.go b/planner/core/task.go index a60cbf2b7ebf0..df07ffd4ab91b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -1338,7 +1338,15 @@ func BuildFinalModeAggregation( finalAggFunc.OrderByItems = byItems finalAggFunc.HasDistinct = aggFunc.HasDistinct - finalAggFunc.Mode = aggregation.CompleteMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.CompleteMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } else { if aggFunc.Name == ast.AggFuncGroupConcat && len(aggFunc.OrderByItems) > 0 { // group_concat can only run in one phase if it has order by items but without distinct property @@ -1418,7 +1426,15 @@ func BuildFinalModeAggregation( } } - finalAggFunc.Mode = aggregation.FinalMode + // In logical optimize phase, the Agg->PartitionUnion->TableReader may become + // Agg1->PartitionUnion->Agg2->TableReader, and the Agg2 is a partial aggregation. + // So in the push down here, we need to add a new if-condition check: + // If the original agg mode is partial already, the finalAggFunc's mode become Partial2. + if aggFunc.Mode == aggregation.CompleteMode { + finalAggFunc.Mode = aggregation.FinalMode + } else if aggFunc.Mode == aggregation.Partial1Mode || aggFunc.Mode == aggregation.Partial2Mode { + finalAggFunc.Mode = aggregation.Partial2Mode + } } finalAggFunc.Args = args @@ -1484,7 +1500,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { } // no avgs // for final agg, always add project due to in-compatibility between TiDB and TiFlash - if len(p.schema.Columns) == len(newSchema.Columns) && !p.isFinalAgg() { + if len(p.schema.Columns) == len(newSchema.Columns) && !p.IsFinalAgg() { return nil } // add remaining columns to exprs From 285172081c5fbeb4e52d3cb3eb3e2cdfbd460f0c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 22 Aug 2022 14:44:20 +0800 Subject: [PATCH 28/42] planner: fix bindings with ignore_plan_cache_hint cannot work (#36427) (#37231) close pingcap/tidb#34596 --- executor/explainfor_test.go | 55 ++++++++++++++++++++++++++++++++++++ planner/core/common_plans.go | 30 ++++++++++++-------- 2 files changed, 73 insertions(+), 12 deletions(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 2b25c8f80a66d..7b547d7caff8d 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -1174,6 +1174,61 @@ func TestHint4PlanCache(t *testing.T) { tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) } +func TestIgnorePlanCacheWithPrepare(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int, index idx_a(a));") + tk.MustExec("drop table if exists r;") + tk.MustExec("create table r(a int);") + + // test use_index + tk.MustExec("prepare stmt from 'select * from t;';") + tk.MustExec("create binding for select * from t using select /*+ use_index(t, idx_a) */ * from t;") + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + + tk.MustExec("create binding for select * from t using select /*+ ignore_plan_cache() */ * from t;") + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + + tk.MustExec("create binding for select * from t using select /*+ use_index(t, idx_a) */ * from t;") + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + + // test straight_join + tk.MustExec("prepare stmt_join from 'select * from t, r where r.a = t.a;';") + tk.MustExec("create binding for select * from t, r where r.a = t.a using select /*+ straight_join() */* from t, r where r.a = t.a;") + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + + tk.MustExec("create binding for select * from t, r where r.a = t.a using select /*+ ignore_plan_cache() */* from t, r where r.a = t.a;") + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("0")) + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + + tk.MustExec("create binding for select * from t, r where r.a = t.a using select /*+ straight_join() */* from t, r where r.a = t.a;") + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_cache;").Check(testkit.Rows("1")) + tk.MustQuery("execute stmt_join;").Check(testkit.Rows()) + tk.MustQuery("select @@last_plan_from_binding;").Check(testkit.Rows("1")) + +} + func TestSelectView4PlanCache(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 8738776fa52b0..33a082880ca09 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -413,34 +413,37 @@ func (e *Execute) setFoundInPlanCache(sctx sessionctx.Context, opt bool) error { } // GetBindSQL4PlanCache used to get the bindSQL for plan cache to build the plan cache key. -func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareStmt) string { +func GetBindSQL4PlanCache(sctx sessionctx.Context, preparedStmt *CachedPrepareStmt) (string, bool) { useBinding := sctx.GetSessionVars().UsePlanBaselines + ignore := false if !useBinding || preparedStmt.PreparedAst.Stmt == nil || preparedStmt.NormalizedSQL4PC == "" || preparedStmt.SQLDigest4PC == "" { - return "" + return "", ignore } if sctx.Value(bindinfo.SessionBindInfoKeyType) == nil { - return "" + return "", ignore } sessionHandle := sctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) bindRecord := sessionHandle.GetBindRecord(preparedStmt.SQLDigest4PC, preparedStmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { - return enabledBinding.BindSQL + ignore = enabledBinding.Hint.ContainTableHint(HintIgnorePlanCache) + return enabledBinding.BindSQL, ignore } } globalHandle := domain.GetDomain(sctx).BindHandle() if globalHandle == nil { - return "" + return "", ignore } bindRecord = globalHandle.GetBindRecord(preparedStmt.SQLDigest4PC, preparedStmt.NormalizedSQL4PC, "") if bindRecord != nil { enabledBinding := bindRecord.FindEnabledBinding() if enabledBinding != nil { - return enabledBinding.BindSQL + ignore = enabledBinding.Hint.ContainTableHint(HintIgnorePlanCache) + return enabledBinding.BindSQL, ignore } } - return "" + return "", ignore } func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema, preparedStmt *CachedPrepareStmt) (err error) { @@ -451,9 +454,12 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, stmtCtx.UseCache = prepared.UseCache var bindSQL string + var ignorePlanCache = false + if prepared.UseCache { - bindSQL = GetBindSQL4PlanCache(sctx, preparedStmt) - if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), preparedStmt.StmtText, preparedStmt.StmtDB, prepared.SchemaVersion); err != nil { + bindSQL, ignorePlanCache = GetBindSQL4PlanCache(sctx, preparedStmt) + if cacheKey, err = NewPlanCacheKey(sctx.GetSessionVars(), preparedStmt.StmtText, + preparedStmt.StmtDB, prepared.SchemaVersion); err != nil { return err } } @@ -479,7 +485,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, } } - if prepared.UseCache && prepared.CachedPlan != nil { // short path for point-get plans + if prepared.UseCache && prepared.CachedPlan != nil && !ignorePlanCache { // short path for point-get plans // Rewriting the expression in the select.where condition will convert its // type from "paramMarker" to "Constant".When Point Select queries are executed, // the expression in the where condition will not be evaluated, @@ -505,7 +511,7 @@ func (e *Execute) getPhysicalPlan(ctx context.Context, sctx sessionctx.Context, stmtCtx.PointExec = true return nil } - if prepared.UseCache { // for general plans + if prepared.UseCache && !ignorePlanCache { // for general plans if cacheValue, exists := sctx.PreparedPlanCache().Get(cacheKey); exists { if err := e.checkPreparedPriv(ctx, sctx, preparedStmt, is); err != nil { return err @@ -582,7 +588,7 @@ REBUILD: if containTableDual(p) && varsNum > 0 { stmtCtx.SkipPlanCache = true } - if prepared.UseCache && !stmtCtx.SkipPlanCache { + if prepared.UseCache && !stmtCtx.SkipPlanCache && !ignorePlanCache { // rebuild key to exclude kv.TiFlash when stmt is not read only if _, isolationReadContainTiFlash := sessVars.IsolationReadEngines[kv.TiFlash]; isolationReadContainTiFlash && !IsReadOnly(stmt, sessVars) { delete(sessVars.IsolationReadEngines, kv.TiFlash) From e50055a789004201a7e1e8fe8a827ce05c3dbd95 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 22 Aug 2022 16:28:20 +0800 Subject: [PATCH 29/42] planner: add missing exchange between hash-partition window and single-partition window (#36025) (#36052) close pingcap/tidb#35990 --- planner/core/exhaust_physical_plans.go | 8 ++++++-- .../testdata/window_push_down_suite_in.json | 3 ++- .../testdata/window_push_down_suite_out.json | 18 ++++++++++++++++++ 3 files changed, 26 insertions(+), 3 deletions(-) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index b4670f1146c2b..cbbc770fdf985 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -2291,7 +2291,7 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P if len(lw.PartitionBy) > 0 { partitionCols := lw.GetPartitionKeys() - // trying to match the required parititions. + // trying to match the required partitions. if prop.MPPPartitionTp == property.HashType { if matches := prop.IsSubsetOf(partitionCols); len(matches) != 0 { partitionCols = choosePartitionKeys(partitionCols, matches) @@ -2306,6 +2306,10 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P childProperty.MPPPartitionTp = property.SinglePartitionType } + if prop.MPPPartitionTp == property.SinglePartitionType && childProperty.MPPPartitionTp != property.SinglePartitionType { + return nil + } + window := PhysicalWindow{ WindowFuncDescs: lw.WindowFuncDescs, PartitionBy: lw.PartitionBy, @@ -2596,7 +2600,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert if len(la.GroupByItems) > 0 { partitionCols := la.GetPotentialPartitionKeys() - // trying to match the required parititions. + // trying to match the required partitions. if prop.MPPPartitionTp == property.HashType { if matches := prop.IsSubsetOf(partitionCols); len(matches) != 0 { partitionCols = choosePartitionKeys(partitionCols, matches) diff --git a/planner/core/testdata/window_push_down_suite_in.json b/planner/core/testdata/window_push_down_suite_in.json index 3f228f5b6a476..27f12930a4cdd 100644 --- a/planner/core/testdata/window_push_down_suite_in.json +++ b/planner/core/testdata/window_push_down_suite_in.json @@ -46,7 +46,8 @@ "explain select count(distinct empid) from (select *, row_number() over () from employee) t", "explain select count(distinct empid) from (select *, row_number() over (partition by deptid) from employee) t", "explain select count(empid) from (select *, row_number() over () a from employee) t group by a", - "explain select count(empid) from (select *, row_number() over (partition by deptid) a from employee) t group by a" + "explain select count(empid) from (select *, row_number() over (partition by deptid) a from employee) t group by a", + "explain select row_number() over w2, row_number() over w1 from employee window w2 as (order by deptid), w1 as (partition by deptid);" ] } ] diff --git a/planner/core/testdata/window_push_down_suite_out.json b/planner/core/testdata/window_push_down_suite_out.json index 2b7b7b893cda4..f769a6b1a4544 100644 --- a/planner/core/testdata/window_push_down_suite_out.json +++ b/planner/core/testdata/window_push_down_suite_out.json @@ -443,6 +443,24 @@ " └─TableFullScan_18 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" ], "Warn": null + }, + { + "SQL": "explain select row_number() over w2, row_number() over w1 from employee window w2 as (order by deptid), w1 as (partition by deptid);", + "Plan": [ + "TableReader_37 10000.00 root data:ExchangeSender_36", + "└─ExchangeSender_36 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_10 10000.00 mpp[tiflash] Column#8, Column#7", + " └─Window_35 10000.00 mpp[tiflash] row_number()->Column#8 over(order by test.employee.deptid rows between current row and current row)", + " └─Sort_20 10000.00 mpp[tiflash] test.employee.deptid", + " └─ExchangeReceiver_19 10000.00 mpp[tiflash] ", + " └─ExchangeSender_18 10000.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Window_13 10000.00 mpp[tiflash] row_number()->Column#7 over(partition by test.employee.deptid rows between current row and current row)", + " └─Sort_17 10000.00 mpp[tiflash] test.employee.deptid", + " └─ExchangeReceiver_16 10000.00 mpp[tiflash] ", + " └─ExchangeSender_15 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.employee.deptid, collate: binary]", + " └─TableFullScan_14 10000.00 mpp[tiflash] table:employee keep order:false, stats:pseudo" + ], + "Warn": null } ] } From 7ddd028b978b5a3e1d5846a762c1fdefb9ed33a7 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 22 Aug 2022 17:04:20 +0800 Subject: [PATCH 30/42] planner: Manual revert of #24282 (#35298) (#37268) ref pingcap/tidb#22079, closes pingcap/tidb#33966, close pingcap/tidb#35181 --- executor/partition_table_test.go | 15 ++++++ planner/core/partition_prune.go | 2 +- planner/core/rule_partition_processor.go | 47 +++++-------------- .../core/testdata/partition_pruner_out.json | 15 +++--- 4 files changed, 37 insertions(+), 42 deletions(-) diff --git a/executor/partition_table_test.go b/executor/partition_table_test.go index e2d098c622f8f..cccb827b0b030 100644 --- a/executor/partition_table_test.go +++ b/executor/partition_table_test.go @@ -3600,3 +3600,18 @@ func TestPartitionTableExplain(t *testing.T) { " └─Selection 1.00 cop[tikv] not(isnull(testpartitiontableexplain.t.b))", " └─TableRangeScan 1.00 cop[tikv] table:t range:[1,1], keep order:false")) } + +func TestIssue35181(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database TestIssue35181") + tk.MustExec("use TestIssue35181") + tk.MustExec("CREATE TABLE `t` (`a` int(11) DEFAULT NULL, `b` int(11) DEFAULT NULL) PARTITION BY RANGE (`a`) (PARTITION `p0` VALUES LESS THAN (2021), PARTITION `p1` VALUES LESS THAN (3000))") + + tk.MustExec("set @@tidb_partition_prune_mode = 'static'") + tk.MustExec(`insert into t select * from t where a=3000`) + tk.MustExec("set @@tidb_partition_prune_mode = 'dynamic'") + tk.MustExec(`insert into t select * from t where a=3000`) +} diff --git a/planner/core/partition_prune.go b/planner/core/partition_prune.go index 441f571b32925..3ab266340829d 100644 --- a/planner/core/partition_prune.go +++ b/planner/core/partition_prune.go @@ -33,7 +33,7 @@ func PartitionPruning(ctx sessionctx.Context, tbl table.PartitionedTable, conds case model.PartitionTypeHash: return s.pruneHashPartition(ctx, tbl, partitionNames, conds, columns, names) case model.PartitionTypeRange: - rangeOr, _, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names, nil) + rangeOr, err := s.pruneRangePartition(ctx, pi, tbl, conds, columns, names) if err != nil { return nil, err } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 57470061787ad..aa0babe749cf1 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -824,26 +824,26 @@ func intersectionRange(start, end, newStart, newEnd int) (int, int) { } func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *model.PartitionInfo, tbl table.PartitionedTable, conds []expression.Expression, - columns []*expression.Column, names types.NameSlice, condsToBePruned *[]expression.Expression) (partitionRangeOR, []expression.Expression, error) { + columns []*expression.Column, names types.NameSlice) (partitionRangeOR, error) { partExpr, err := tbl.(partitionTable).PartitionExpr() if err != nil { - return nil, nil, err + return nil, err } // Partition by range columns. if len(pi.Columns) > 0 { result, err := s.pruneRangeColumnsPartition(ctx, conds, pi, partExpr, columns, names) - return result, nil, err + return result, err } // Partition by range. col, fn, mono, err := makePartitionByFnCol(ctx, columns, names, pi.Expr) if err != nil { - return nil, nil, err + return nil, err } result := fullRange(len(pi.Definitions)) if col == nil { - return result, nil, nil + return result, nil } // Extract the partition column, if the column is not null, it's possible to prune. @@ -858,41 +858,14 @@ func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *mod } result = partitionRangeForCNFExpr(ctx, conds, &pruner, result) - if condsToBePruned == nil { - return result, nil, nil - } - // remove useless predicates after pruning - newConds := make([]expression.Expression, 0, len(*condsToBePruned)) - for _, cond := range *condsToBePruned { - if dataForPrune, ok := pruner.extractDataForPrune(ctx, cond); ok { - switch dataForPrune.op { - case ast.EQ: - unsigned := mysql.HasUnsignedFlag(pruner.col.RetType.GetFlag()) - start, _ := pruneUseBinarySearch(pruner.lessThan, dataForPrune, unsigned) - // if the type of partition key is Int - if pk, ok := partExpr.Expr.(*expression.Column); ok && pk.RetType.EvalType() == types.ETInt { - // see if can be removed - // see issue #22079: https://github.com/pingcap/tidb/issues/22079 for details - if start > 0 && pruner.lessThan.data[start-1] == dataForPrune.c && (pruner.lessThan.data[start]-1) == dataForPrune.c { - continue - } - } - } - } - newConds = append(newConds, cond) - } - - return result, newConds, nil + return result, nil } func (s *partitionProcessor) processRangePartition(ds *DataSource, pi *model.PartitionInfo, opt *logicalOptimizeOp) (LogicalPlan, error) { - used, prunedConds, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names, &ds.pushedDownConds) + used, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names) if err != nil { return nil, err } - if prunedConds != nil { - ds.pushedDownConds = prunedConds - } return s.makeUnionAllChildren(ds, pi, used, opt) } @@ -1279,11 +1252,13 @@ func relaxOP(op string) string { return op } +// pruneUseBinarySearch returns the start and end of which partitions will match. +// If no match (i.e. value > last partition) the start partition will be the number of partition, not the first partition! func pruneUseBinarySearch(lessThan lessThanDataInt, data dataForPrune, unsigned bool) (start int, end int) { length := lessThan.length() switch data.op { case ast.EQ: - // col = 66, lessThan = [4 7 11 14 17] => [5, 6) + // col = 66, lessThan = [4 7 11 14 17] => [5, 5) // col = 14, lessThan = [4 7 11 14 17] => [4, 5) // col = 10, lessThan = [4 7 11 14 17] => [2, 3) // col = 3, lessThan = [4 7 11 14 17] => [0, 1) @@ -1589,6 +1564,8 @@ func (p *rangeColumnsPruner) partitionRangeForExpr(sctx sessionctx.Context, expr return start, end, true } +// pruneUseBinarySearch returns the start and end of which partitions will match. +// If no match (i.e. value > last partition) the start partition will be the number of partition, not the first partition! func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op string, data *expression.Constant) (start int, end int) { var err error var isNull bool diff --git a/planner/core/testdata/partition_pruner_out.json b/planner/core/testdata/partition_pruner_out.json index 39f8a3c00ef76..e63431a14861d 100644 --- a/planner/core/testdata/partition_pruner_out.json +++ b/planner/core/testdata/partition_pruner_out.json @@ -2843,15 +2843,17 @@ { "SQL": "explain format = 'brief' select * from t where a = 1", "Result": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo" + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 1)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p1 keep order:false, stats:pseudo" ] }, { "SQL": "explain format = 'brief' select * from t where a = 2", "Result": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" ] }, { @@ -2899,8 +2901,9 @@ { "SQL": "explain format = 'brief' select * from t where a in (2)", "Result": [ - "TableReader 10000.00 root data:TableFullScan", - "└─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" + "TableReader 10.00 root data:Selection", + "└─Selection 10.00 cop[tikv] eq(test_partition.t.a, 2)", + " └─TableFullScan 10000.00 cop[tikv] table:t, partition:p2 keep order:false, stats:pseudo" ] } ] From 3000d06202589347655a9043937c9ff3bcd08a5e Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 23 Aug 2022 12:44:19 +0800 Subject: [PATCH 31/42] planner: set EnableOuterJoinReorder to false by default (#37264) (#37280) ref pingcap/tidb#37238 --- cmd/explaintest/r/clustered_index.result | 2 ++ cmd/explaintest/r/select.result | 2 ++ cmd/explaintest/t/clustered_index.test | 2 ++ cmd/explaintest/t/select.test | 2 ++ planner/core/rule_join_reorder_test.go | 7 +++++++ planner/core/rule_result_reorder_test.go | 1 + sessionctx/variable/sysvar.go | 2 +- sessionctx/variable/tidb_vars.go | 2 +- sessionctx/variable/varsutil_test.go | 3 +++ 9 files changed, 21 insertions(+), 2 deletions(-) diff --git a/cmd/explaintest/r/clustered_index.result b/cmd/explaintest/r/clustered_index.result index 665c60e40d2b0..c9bfb61c93604 100644 --- a/cmd/explaintest/r/clustered_index.result +++ b/cmd/explaintest/r/clustered_index.result @@ -1,3 +1,4 @@ +set @@tidb_enable_outer_join_reorder=true; drop database if exists with_cluster_index; create database with_cluster_index; drop database if exists wout_cluster_index; @@ -127,3 +128,4 @@ StreamAgg_17 1.00 root funcs:count(Column#9)->Column#7 └─IndexReader_18 1.00 root index:StreamAgg_9 └─StreamAgg_9 1.00 cop[tikv] funcs:count(1)->Column#9 └─IndexRangeScan_16 109.70 cop[tikv] table:tbl_0, index:idx_3(col_0) range:[803163,+inf], keep order:false +set @@tidb_enable_outer_join_reorder=false; diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index 636139109644d..a31c03782bbfb 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -1,3 +1,4 @@ +set @@tidb_enable_outer_join_reorder=true; DROP TABLE IF EXISTS t; CREATE TABLE t ( c1 int, @@ -654,3 +655,4 @@ create table t3(a char(10), primary key (a)); insert into t3 values ('a'); select * from t3 where a > 0x80; Error 1105: Cannot convert string '\x80' from binary to utf8mb4 +set @@tidb_enable_outer_join_reorder=false; diff --git a/cmd/explaintest/t/clustered_index.test b/cmd/explaintest/t/clustered_index.test index 9415781f7caf1..606a768f5b8d4 100644 --- a/cmd/explaintest/t/clustered_index.test +++ b/cmd/explaintest/t/clustered_index.test @@ -1,3 +1,4 @@ +set @@tidb_enable_outer_join_reorder=true; drop database if exists with_cluster_index; create database with_cluster_index; drop database if exists wout_cluster_index; @@ -53,3 +54,4 @@ explain select count(*) from wout_cluster_index.tbl_0 where col_0 <= 0 ; explain select count(*) from with_cluster_index.tbl_0 where col_0 >= 803163 ; explain select count(*) from wout_cluster_index.tbl_0 where col_0 >= 803163 ; +set @@tidb_enable_outer_join_reorder=false; diff --git a/cmd/explaintest/t/select.test b/cmd/explaintest/t/select.test index 64ee824830b51..dbb505bb250f0 100644 --- a/cmd/explaintest/t/select.test +++ b/cmd/explaintest/t/select.test @@ -1,3 +1,4 @@ +set @@tidb_enable_outer_join_reorder=true; DROP TABLE IF EXISTS t; CREATE TABLE t ( @@ -279,3 +280,4 @@ create table t3(a char(10), primary key (a)); insert into t3 values ('a'); --error 1105 select * from t3 where a > 0x80; +set @@tidb_enable_outer_join_reorder=false; diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index c7768ccf04ce4..0659c0a887d43 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -77,6 +77,7 @@ func TestLeadingJoinHint(t *testing.T) { tk.MustExec("create table t6(a int, b int, key(a));") tk.MustExec("create table t7(a int, b int, key(a));") tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") runJoinReorderTestData(t, tk, "TestLeadingJoinHint") // test cases for outer join @@ -117,6 +118,8 @@ func TestJoinOrderHint(t *testing.T) { tk.MustExec("create table t7(a int, b int, key(a));") tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") + // test cases for using the leading hint and straight_join hint at the same time tk.MustExec("select /*+ leading(t1) straight_join() */ * from t1 join t2 on t1.a=t2.a join t3 on t2.b=t3.b") tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1815 We can only use the straight_join hint, when we use the leading hint and straight_join hint at the same time, all leading hints will be invalid")) @@ -239,6 +242,7 @@ func TestJoinOrderHint4StaticPartitionTable(t *testing.T) { tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) tk.MustExec(`set @@tidb_partition_prune_mode="static"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") runJoinReorderTestData(t, tk, "TestJoinOrderHint4StaticPartitionTable") } @@ -255,6 +259,7 @@ func TestJoinOrderHint4DynamicPartitionTable(t *testing.T) { tk.MustExec(`create table t3(a int, b int) partition by hash(b) partitions 3`) tk.MustExec(`set @@tidb_partition_prune_mode="dynamic"`) + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") runJoinReorderTestData(t, tk, "TestJoinOrderHint4DynamicPartitionTable") } @@ -274,6 +279,7 @@ func TestJoinOrderHint4DifferentJoinType(t *testing.T) { tk.MustExec("create table t6(a int, b int, key(a));") tk.MustExec("create table t7(a int, b int, key(a));") tk.MustExec("create table t8(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") runJoinReorderTestData(t, tk, "TestJoinOrderHint4DifferentJoinType") } @@ -288,6 +294,7 @@ func TestJoinOrderHint4TiFlash(t *testing.T) { tk.MustExec("create table t1(a int, b int, key(a));") tk.MustExec("create table t2(a int, b int, key(a));") tk.MustExec("create table t3(a int, b int, key(a));") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") // Create virtual tiflash replica info. dom := domain.GetDomain(tk.Session()) diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 567c1488508d9..82177726ddce1 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -166,6 +166,7 @@ func TestOrderedResultModeOnJoin(t *testing.T) { tk.MustExec("drop table if exists t2") tk.MustExec("create table t1 (a int primary key, b int, c int, d int, key(b))") tk.MustExec("create table t2 (a int primary key, b int, c int, d int, key(b))") + tk.MustExec("set @@tidb_enable_outer_join_reorder=true") runTestData(t, tk, "TestOrderedResultModeOnJoin") } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 169b1be0feec7..ac9e232f85dd8 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -196,7 +196,7 @@ var defaultSysVars = []*SysVar{ s.OptimizerSelectivityLevel = tidbOptPositiveInt32(val, DefTiDBOptimizerSelectivityLevel) return nil }}, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptimizerEnableOuterJoinReorder, Value: BoolToOnOff(DefTiDBEnableOuterJoinReorder), skipInit: true, Type: TypeBool, SetSession: func(s *SessionVars, val string) error { + {Scope: ScopeGlobal | ScopeSession, Name: TiDBOptimizerEnableOuterJoinReorder, Value: BoolToOnOff(DefTiDBEnableOuterJoinReorder), Type: TypeBool, SetSession: func(s *SessionVars, val string) error { s.EnableOuterJoinReorder = TiDBOptOn(val) return nil }}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 12916ed13e874..9141431c95b12 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -775,7 +775,7 @@ const ( DefBroadcastJoinThresholdCount = 10 * 1024 DefTiDBOptimizerSelectivityLevel = 0 DefTiDBOptimizerEnableNewOFGB = false - DefTiDBEnableOuterJoinReorder = true + DefTiDBEnableOuterJoinReorder = false DefTiDBAllowBatchCop = 1 DefTiDBAllowMPPExecution = true DefTiDBHashExchangeWithNewCollation = true diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 28957027b6129..9d30df7604a7f 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -242,6 +242,9 @@ func TestVarsutil(t *testing.T) { err = SetSessionSystemVar(v, TiDBOptimizerEnableOuterJoinReorder, "OFF") require.NoError(t, err) require.Equal(t, false, v.EnableOuterJoinReorder) + err = v.SetSystemVar(TiDBOptimizerEnableOuterJoinReorder, "ON") + require.NoError(t, err) + require.Equal(t, true, v.EnableOuterJoinReorder) require.Equal(t, DefTiDBOptimizerEnableNewOFGB, v.OptimizerEnableNewOnlyFullGroupByCheck) err = SetSessionSystemVar(v, TiDBOptimizerEnableNewOnlyFullGroupByCheck, "off") From bf0e7a1d6f4d2a9bd0a739376774f2e9345b39d6 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 23 Aug 2022 13:30:20 +0800 Subject: [PATCH 32/42] planner: use SEMI_JOIN_REWRITE hint to rewrite the semi join (#35325) (#37283) close pingcap/tidb#35323 --- bindinfo/bind_test.go | 23 + parser/hintparser.go | 1083 +++++++++-------- parser/hintparser.y | 3 + parser/hintparser_test.go | 5 +- parser/misc.go | 1 + planner/core/expression_rewriter.go | 33 +- planner/core/logical_plan_builder.go | 18 +- planner/core/logical_plan_test.go | 2 +- planner/core/logical_plans.go | 1 + planner/core/optimizer.go | 2 + planner/core/physical_plan_test.go | 51 + planner/core/physical_plan_trace_test.go | 4 +- planner/core/planbuilder.go | 9 + planner/core/rule_build_key_info.go | 4 + planner/core/rule_semi_join_rewrite.go | 115 ++ .../core/testdata/integration_suite_in.json | 6 +- .../core/testdata/integration_suite_out.json | 62 + planner/core/testdata/plan_suite_in.json | 11 + planner/core/testdata/plan_suite_out.json | 81 ++ .../testdata/plan_suite_unexported_in.json | 11 + .../testdata/plan_suite_unexported_out.json | 11 + testkit/testkit.go | 22 + 22 files changed, 999 insertions(+), 559 deletions(-) create mode 100644 planner/core/rule_semi_join_rewrite.go diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 896c56471edb3..98947fc0abf8e 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -331,6 +331,29 @@ func TestExplain(t *testing.T) { tk.MustExec("drop global binding for SELECT * from t1 union SELECT * from t1") } +func TestBindSemiJoinRewrite(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + require.True(t, tk.HasKeywordInOperatorInfo("select * from t1 where exists(select 1 from t2 where t1.id=t2.id)", "semi join")) + require.True(t, tk.NotHasKeywordInOperatorInfo("select * from t1 where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t2 where t1.id=t2.id)", "semi join")) + + tk.MustExec(` +create global binding for + select * from t1 where exists(select 1 from t2 where t1.id=t2.id) +using + select * from t1 where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t2 where t1.id=t2.id) +`) + + require.True(t, tk.NotHasKeywordInOperatorInfo("select * from t1 where exists(select 1 from t2 where t1.id=t2.id)", "semi join")) +} + // TestBindingSymbolList tests sql with "?, ?, ?, ?", fixes #13871 func TestBindingSymbolList(t *testing.T) { store, dom, clean := testkit.CreateMockStoreAndDomain(t) diff --git a/parser/hintparser.go b/parser/hintparser.go index b86638e44b682..5b977c307bc39 100644 --- a/parser/hintparser.go +++ b/parser/hintparser.go @@ -41,18 +41,18 @@ type yyhintXError struct { } const ( - yyhintDefault = 57417 + yyhintDefault = 57418 yyhintEOFCode = 57344 yyhintErrCode = 57345 hintAggToCop = 57377 hintBCJoin = 57390 hintBKA = 57355 hintBNL = 57357 - hintDupsWeedOut = 57413 - hintFalse = 57409 - hintFirstMatch = 57414 + hintDupsWeedOut = 57414 + hintFalse = 57410 + hintFirstMatch = 57415 hintForceIndex = 57401 - hintGB = 57412 + hintGB = 57413 hintHashAgg = 57379 hintHashJoin = 57359 hintIdentifier = 57347 @@ -70,10 +70,10 @@ const ( hintJoinSuffix = 57354 hintLeading = 57403 hintLimitToCop = 57400 - hintLooseScan = 57415 - hintMB = 57411 + hintLooseScan = 57416 + hintMB = 57412 hintMRR = 57365 - hintMaterialization = 57416 + hintMaterialization = 57417 hintMaxExecutionTime = 57373 hintMemoryQuota = 57384 hintMerge = 57361 @@ -89,15 +89,16 @@ const ( hintNoSkipScan = 57370 hintNoSwapJoinInputs = 57385 hintNthPlan = 57399 - hintOLAP = 57404 - hintOLTP = 57405 - hintPartition = 57406 + hintOLAP = 57405 + hintOLTP = 57406 + hintPartition = 57407 hintQBName = 57376 hintQueryType = 57386 hintReadConsistentReplica = 57387 hintReadFromStorage = 57388 hintResourceGroup = 57375 hintSMJoin = 57389 + hintSemiJoinRewrite = 57404 hintSemijoin = 57371 hintSetVar = 57374 hintSingleAtIdentifier = 57349 @@ -106,10 +107,10 @@ const ( hintStreamAgg = 57391 hintStringLit = 57350 hintSwapJoinInputs = 57392 - hintTiFlash = 57408 - hintTiKV = 57407 + hintTiFlash = 57409 + hintTiKV = 57408 hintTimeRange = 57397 - hintTrue = 57410 + hintTrue = 57411 hintUseCascades = 57398 hintUseIndex = 57394 hintUseIndexMerge = 57393 @@ -117,125 +118,126 @@ const ( hintUseToja = 57396 yyhintMaxDepth = 200 - yyhintTabOfs = -174 + yyhintTabOfs = -176 ) var ( yyhintXLAT = map[int]int{ - 41: 0, // ')' (131x) - 57377: 1, // hintAggToCop (123x) - 57390: 2, // hintBCJoin (123x) - 57355: 3, // hintBKA (123x) - 57357: 4, // hintBNL (123x) - 57401: 5, // hintForceIndex (123x) - 57379: 6, // hintHashAgg (123x) - 57359: 7, // hintHashJoin (123x) - 57380: 8, // hintIgnoreIndex (123x) - 57378: 9, // hintIgnorePlanCache (123x) - 57363: 10, // hintIndexMerge (123x) - 57381: 11, // hintInlHashJoin (123x) - 57382: 12, // hintInlJoin (123x) - 57383: 13, // hintInlMergeJoin (123x) - 57351: 14, // hintJoinFixedOrder (123x) - 57352: 15, // hintJoinOrder (123x) - 57353: 16, // hintJoinPrefix (123x) - 57354: 17, // hintJoinSuffix (123x) - 57403: 18, // hintLeading (123x) - 57400: 19, // hintLimitToCop (123x) - 57373: 20, // hintMaxExecutionTime (123x) - 57384: 21, // hintMemoryQuota (123x) - 57361: 22, // hintMerge (123x) - 57365: 23, // hintMRR (123x) - 57356: 24, // hintNoBKA (123x) - 57358: 25, // hintNoBNL (123x) - 57360: 26, // hintNoHashJoin (123x) - 57367: 27, // hintNoICP (123x) - 57364: 28, // hintNoIndexMerge (123x) - 57362: 29, // hintNoMerge (123x) - 57366: 30, // hintNoMRR (123x) - 57368: 31, // hintNoRangeOptimization (123x) - 57372: 32, // hintNoSemijoin (123x) - 57370: 33, // hintNoSkipScan (123x) - 57385: 34, // hintNoSwapJoinInputs (123x) - 57399: 35, // hintNthPlan (123x) - 57376: 36, // hintQBName (123x) - 57386: 37, // hintQueryType (123x) - 57387: 38, // hintReadConsistentReplica (123x) - 57388: 39, // hintReadFromStorage (123x) - 57375: 40, // hintResourceGroup (123x) - 57371: 41, // hintSemijoin (123x) - 57374: 42, // hintSetVar (123x) - 57369: 43, // hintSkipScan (123x) - 57389: 44, // hintSMJoin (123x) - 57402: 45, // hintStraightJoin (123x) - 57391: 46, // hintStreamAgg (123x) - 57392: 47, // hintSwapJoinInputs (123x) - 57397: 48, // hintTimeRange (123x) - 57398: 49, // hintUseCascades (123x) - 57394: 50, // hintUseIndex (123x) - 57393: 51, // hintUseIndexMerge (123x) - 57395: 52, // hintUsePlanCache (123x) - 57396: 53, // hintUseToja (123x) - 44: 54, // ',' (121x) - 57413: 55, // hintDupsWeedOut (101x) - 57414: 56, // hintFirstMatch (101x) - 57415: 57, // hintLooseScan (101x) - 57416: 58, // hintMaterialization (101x) - 57408: 59, // hintTiFlash (101x) - 57407: 60, // hintTiKV (101x) - 57409: 61, // hintFalse (100x) - 57404: 62, // hintOLAP (100x) - 57405: 63, // hintOLTP (100x) - 57410: 64, // hintTrue (100x) - 57412: 65, // hintGB (99x) - 57411: 66, // hintMB (99x) - 57347: 67, // hintIdentifier (98x) - 57349: 68, // hintSingleAtIdentifier (83x) - 93: 69, // ']' (77x) - 57406: 70, // hintPartition (71x) - 46: 71, // '.' (67x) - 61: 72, // '=' (67x) - 40: 73, // '(' (62x) - 57344: 74, // $end (24x) - 57437: 75, // QueryBlockOpt (17x) - 57429: 76, // Identifier (13x) - 57346: 77, // hintIntLit (8x) - 57350: 78, // hintStringLit (5x) - 57419: 79, // CommaOpt (4x) - 57425: 80, // HintTable (4x) - 57426: 81, // HintTableList (4x) - 91: 82, // '[' (3x) - 57418: 83, // BooleanHintName (2x) - 57420: 84, // HintIndexList (2x) - 57422: 85, // HintStorageType (2x) - 57423: 86, // HintStorageTypeAndTable (2x) - 57427: 87, // HintTableListOpt (2x) - 57432: 88, // JoinOrderOptimizerHintName (2x) - 57433: 89, // NullaryHintName (2x) - 57436: 90, // PartitionListOpt (2x) - 57439: 91, // StorageOptimizerHintOpt (2x) - 57440: 92, // SubqueryOptimizerHintName (2x) - 57443: 93, // SubqueryStrategy (2x) - 57444: 94, // SupportedIndexLevelOptimizerHintName (2x) - 57445: 95, // SupportedTableLevelOptimizerHintName (2x) - 57446: 96, // TableOptimizerHintOpt (2x) - 57448: 97, // UnsupportedIndexLevelOptimizerHintName (2x) - 57449: 98, // UnsupportedTableLevelOptimizerHintName (2x) - 57421: 99, // HintQueryType (1x) - 57424: 100, // HintStorageTypeAndTableList (1x) - 57428: 101, // HintTrueOrFalse (1x) - 57430: 102, // IndexNameList (1x) - 57431: 103, // IndexNameListOpt (1x) - 57434: 104, // OptimizerHintList (1x) - 57435: 105, // PartitionList (1x) - 57438: 106, // Start (1x) - 57441: 107, // SubqueryStrategies (1x) - 57442: 108, // SubqueryStrategiesOpt (1x) - 57447: 109, // UnitOfBytes (1x) - 57450: 110, // Value (1x) - 57417: 111, // $default (0x) - 57345: 112, // error (0x) - 57348: 113, // hintInvalid (0x) + 41: 0, // ')' (132x) + 57377: 1, // hintAggToCop (124x) + 57390: 2, // hintBCJoin (124x) + 57355: 3, // hintBKA (124x) + 57357: 4, // hintBNL (124x) + 57401: 5, // hintForceIndex (124x) + 57379: 6, // hintHashAgg (124x) + 57359: 7, // hintHashJoin (124x) + 57380: 8, // hintIgnoreIndex (124x) + 57378: 9, // hintIgnorePlanCache (124x) + 57363: 10, // hintIndexMerge (124x) + 57381: 11, // hintInlHashJoin (124x) + 57382: 12, // hintInlJoin (124x) + 57383: 13, // hintInlMergeJoin (124x) + 57351: 14, // hintJoinFixedOrder (124x) + 57352: 15, // hintJoinOrder (124x) + 57353: 16, // hintJoinPrefix (124x) + 57354: 17, // hintJoinSuffix (124x) + 57403: 18, // hintLeading (124x) + 57400: 19, // hintLimitToCop (124x) + 57373: 20, // hintMaxExecutionTime (124x) + 57384: 21, // hintMemoryQuota (124x) + 57361: 22, // hintMerge (124x) + 57365: 23, // hintMRR (124x) + 57356: 24, // hintNoBKA (124x) + 57358: 25, // hintNoBNL (124x) + 57360: 26, // hintNoHashJoin (124x) + 57367: 27, // hintNoICP (124x) + 57364: 28, // hintNoIndexMerge (124x) + 57362: 29, // hintNoMerge (124x) + 57366: 30, // hintNoMRR (124x) + 57368: 31, // hintNoRangeOptimization (124x) + 57372: 32, // hintNoSemijoin (124x) + 57370: 33, // hintNoSkipScan (124x) + 57385: 34, // hintNoSwapJoinInputs (124x) + 57399: 35, // hintNthPlan (124x) + 57376: 36, // hintQBName (124x) + 57386: 37, // hintQueryType (124x) + 57387: 38, // hintReadConsistentReplica (124x) + 57388: 39, // hintReadFromStorage (124x) + 57375: 40, // hintResourceGroup (124x) + 57371: 41, // hintSemijoin (124x) + 57404: 42, // hintSemiJoinRewrite (124x) + 57374: 43, // hintSetVar (124x) + 57369: 44, // hintSkipScan (124x) + 57389: 45, // hintSMJoin (124x) + 57402: 46, // hintStraightJoin (124x) + 57391: 47, // hintStreamAgg (124x) + 57392: 48, // hintSwapJoinInputs (124x) + 57397: 49, // hintTimeRange (124x) + 57398: 50, // hintUseCascades (124x) + 57394: 51, // hintUseIndex (124x) + 57393: 52, // hintUseIndexMerge (124x) + 57395: 53, // hintUsePlanCache (124x) + 57396: 54, // hintUseToja (124x) + 44: 55, // ',' (122x) + 57414: 56, // hintDupsWeedOut (102x) + 57415: 57, // hintFirstMatch (102x) + 57416: 58, // hintLooseScan (102x) + 57417: 59, // hintMaterialization (102x) + 57409: 60, // hintTiFlash (102x) + 57408: 61, // hintTiKV (102x) + 57410: 62, // hintFalse (101x) + 57405: 63, // hintOLAP (101x) + 57406: 64, // hintOLTP (101x) + 57411: 65, // hintTrue (101x) + 57413: 66, // hintGB (100x) + 57412: 67, // hintMB (100x) + 57347: 68, // hintIdentifier (99x) + 57349: 69, // hintSingleAtIdentifier (84x) + 93: 70, // ']' (78x) + 57407: 71, // hintPartition (72x) + 46: 72, // '.' (68x) + 61: 73, // '=' (68x) + 40: 74, // '(' (63x) + 57344: 75, // $end (24x) + 57438: 76, // QueryBlockOpt (17x) + 57430: 77, // Identifier (13x) + 57346: 78, // hintIntLit (8x) + 57350: 79, // hintStringLit (5x) + 57420: 80, // CommaOpt (4x) + 57426: 81, // HintTable (4x) + 57427: 82, // HintTableList (4x) + 91: 83, // '[' (3x) + 57419: 84, // BooleanHintName (2x) + 57421: 85, // HintIndexList (2x) + 57423: 86, // HintStorageType (2x) + 57424: 87, // HintStorageTypeAndTable (2x) + 57428: 88, // HintTableListOpt (2x) + 57433: 89, // JoinOrderOptimizerHintName (2x) + 57434: 90, // NullaryHintName (2x) + 57437: 91, // PartitionListOpt (2x) + 57440: 92, // StorageOptimizerHintOpt (2x) + 57441: 93, // SubqueryOptimizerHintName (2x) + 57444: 94, // SubqueryStrategy (2x) + 57445: 95, // SupportedIndexLevelOptimizerHintName (2x) + 57446: 96, // SupportedTableLevelOptimizerHintName (2x) + 57447: 97, // TableOptimizerHintOpt (2x) + 57449: 98, // UnsupportedIndexLevelOptimizerHintName (2x) + 57450: 99, // UnsupportedTableLevelOptimizerHintName (2x) + 57422: 100, // HintQueryType (1x) + 57425: 101, // HintStorageTypeAndTableList (1x) + 57429: 102, // HintTrueOrFalse (1x) + 57431: 103, // IndexNameList (1x) + 57432: 104, // IndexNameListOpt (1x) + 57435: 105, // OptimizerHintList (1x) + 57436: 106, // PartitionList (1x) + 57439: 107, // Start (1x) + 57442: 108, // SubqueryStrategies (1x) + 57443: 109, // SubqueryStrategiesOpt (1x) + 57448: 110, // UnitOfBytes (1x) + 57451: 111, // Value (1x) + 57418: 112, // $default (0x) + 57345: 113, // error (0x) + 57348: 114, // hintInvalid (0x) } yyhintSymNames = []string{ @@ -281,6 +283,7 @@ var ( "hintReadFromStorage", "hintResourceGroup", "hintSemijoin", + "hintSemiJoinRewrite", "hintSetVar", "hintSkipScan", "hintSMJoin", @@ -357,65 +360,81 @@ var ( yyhintReductions = []struct{ xsym, components int }{ {0, 1}, - {106, 1}, - {104, 1}, - {104, 3}, - {104, 1}, - {104, 3}, - {96, 4}, - {96, 4}, - {96, 4}, - {96, 4}, - {96, 4}, - {96, 4}, - {96, 5}, - {96, 5}, - {96, 5}, - {96, 6}, - {96, 4}, - {96, 4}, - {96, 6}, - {96, 6}, - {96, 5}, - {96, 4}, - {96, 5}, - {91, 5}, - {100, 1}, - {100, 3}, - {86, 4}, - {75, 0}, - {75, 1}, - {79, 0}, - {79, 1}, - {90, 0}, - {90, 4}, + {107, 1}, {105, 1}, {105, 3}, - {87, 1}, - {87, 1}, - {81, 2}, + {105, 1}, + {105, 3}, + {97, 4}, + {97, 4}, + {97, 4}, + {97, 4}, + {97, 4}, + {97, 4}, + {97, 5}, + {97, 5}, + {97, 5}, + {97, 6}, + {97, 4}, + {97, 4}, + {97, 6}, + {97, 6}, + {97, 5}, + {97, 4}, + {97, 5}, + {92, 5}, + {101, 1}, + {101, 3}, + {87, 4}, + {76, 0}, + {76, 1}, + {80, 0}, + {80, 1}, + {91, 0}, + {91, 4}, + {106, 1}, + {106, 3}, + {88, 1}, + {88, 1}, + {82, 2}, + {82, 3}, {81, 3}, - {80, 3}, - {80, 5}, - {84, 4}, - {103, 0}, + {81, 5}, + {85, 4}, + {104, 0}, + {104, 1}, {103, 1}, - {102, 1}, - {102, 3}, - {108, 0}, + {103, 3}, + {109, 0}, + {109, 1}, {108, 1}, - {107, 1}, - {107, 3}, - {110, 1}, + {108, 3}, + {111, 1}, + {111, 1}, + {111, 1}, {110, 1}, {110, 1}, - {109, 1}, - {109, 1}, - {101, 1}, - {101, 1}, - {88, 1}, - {88, 1}, - {88, 1}, + {102, 1}, + {102, 1}, + {89, 1}, + {89, 1}, + {89, 1}, + {99, 1}, + {99, 1}, + {99, 1}, + {99, 1}, + {99, 1}, + {99, 1}, + {99, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, + {96, 1}, {98, 1}, {98, 1}, {98, 1}, @@ -427,423 +446,411 @@ var ( {95, 1}, {95, 1}, {95, 1}, - {95, 1}, - {95, 1}, - {95, 1}, - {95, 1}, - {95, 1}, - {97, 1}, - {97, 1}, - {97, 1}, - {97, 1}, - {97, 1}, - {97, 1}, - {97, 1}, + {93, 1}, + {93, 1}, {94, 1}, {94, 1}, {94, 1}, {94, 1}, - {92, 1}, - {92, 1}, - {93, 1}, - {93, 1}, - {93, 1}, - {93, 1}, - {83, 1}, - {83, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {89, 1}, - {99, 1}, - {99, 1}, - {85, 1}, - {85, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, - {76, 1}, + {84, 1}, + {84, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {90, 1}, + {100, 1}, + {100, 1}, + {86, 1}, + {86, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, + {77, 1}, } yyhintXErrors = map[yyhintXError]string{} - yyhintParseTab = [257][]uint16{ + yyhintParseTab = [259][]uint16{ // 0 - {1: 234, 208, 200, 202, 226, 232, 214, 224, 238, 216, 210, 209, 213, 179, 197, 198, 199, 215, 235, 186, 191, 205, 217, 201, 203, 204, 219, 236, 206, 218, 220, 228, 222, 212, 187, 190, 195, 237, 196, 189, 227, 188, 221, 207, 239, 233, 211, 192, 230, 223, 225, 231, 229, 83: 193, 88: 180, 194, 91: 178, 185, 94: 184, 182, 177, 183, 181, 104: 176, 106: 175}, - {74: 174}, - {1: 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 328, 74: 173, 79: 428}, - {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, 74: 172}, - {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, 74: 170}, + {1: 236, 210, 202, 204, 228, 234, 216, 226, 240, 218, 212, 211, 215, 181, 199, 200, 201, 217, 237, 188, 193, 207, 219, 203, 205, 206, 221, 238, 208, 220, 222, 230, 224, 214, 189, 192, 197, 239, 198, 191, 229, 242, 190, 223, 209, 241, 235, 213, 194, 232, 225, 227, 233, 231, 84: 195, 89: 182, 196, 92: 180, 187, 95: 186, 184, 179, 185, 183, 105: 178, 107: 177}, + {75: 176}, + {1: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 332, 75: 175, 80: 432}, + {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, 75: 174}, + {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, 75: 172}, // 5 - {73: 425}, - {73: 422}, - {73: 419}, - {73: 414}, - {73: 411}, + {74: 429}, + {74: 426}, + {74: 423}, + {74: 418}, + {74: 415}, // 10 - {73: 400}, - {73: 388}, - {73: 384}, - {73: 380}, - {73: 372}, + {74: 404}, + {74: 392}, + {74: 388}, + {74: 384}, + {74: 376}, // 15 - {73: 369}, - {73: 366}, - {73: 359}, - {73: 354}, - {73: 348}, + {74: 373}, + {74: 370}, + {74: 363}, + {74: 358}, + {74: 352}, // 20 - {73: 345}, - {73: 339}, - {73: 240}, - {73: 117}, - {73: 116}, + {74: 349}, + {74: 343}, + {74: 243}, + {74: 119}, + {74: 118}, // 25 - {73: 115}, - {73: 114}, - {73: 113}, - {73: 112}, - {73: 111}, + {74: 117}, + {74: 116}, + {74: 115}, + {74: 114}, + {74: 113}, // 30 - {73: 110}, - {73: 109}, - {73: 108}, - {73: 107}, - {73: 106}, + {74: 112}, + {74: 111}, + {74: 110}, + {74: 109}, + {74: 108}, // 35 - {73: 105}, - {73: 104}, - {73: 103}, - {73: 102}, - {73: 101}, + {74: 107}, + {74: 106}, + {74: 105}, + {74: 104}, + {74: 103}, // 40 - {73: 100}, - {73: 99}, - {73: 98}, - {73: 97}, - {73: 96}, + {74: 102}, + {74: 101}, + {74: 100}, + {74: 99}, + {74: 98}, // 45 - {73: 95}, - {73: 94}, - {73: 93}, - {73: 92}, - {73: 91}, + {74: 97}, + {74: 96}, + {74: 95}, + {74: 94}, + {74: 93}, // 50 - {73: 90}, - {73: 89}, - {73: 88}, - {73: 87}, - {73: 86}, + {74: 92}, + {74: 91}, + {74: 90}, + {74: 89}, + {74: 88}, // 55 - {73: 81}, - {73: 80}, - {73: 79}, - {73: 78}, - {73: 77}, + {74: 83}, + {74: 82}, + {74: 81}, + {74: 80}, + {74: 79}, // 60 - {73: 76}, - {73: 75}, - {73: 74}, - {73: 73}, - {73: 72}, + {74: 78}, + {74: 77}, + {74: 76}, + {74: 75}, + {74: 74}, // 65 - {73: 71}, - {59: 147, 147, 68: 242, 75: 241}, - {59: 247, 246, 85: 245, 244, 100: 243}, - {146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 69: 146, 146, 77: 146}, - {336, 54: 337}, + {74: 73}, + {74: 72}, + {60: 149, 149, 69: 245, 76: 244}, + {60: 250, 249, 86: 248, 247, 101: 246}, + {148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 148, 70: 148, 148, 78: 148}, // 70 - {150, 54: 150}, - {82: 248}, - {82: 68}, - {82: 67}, - {1: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 250, 81: 249}, + {340, 55: 341}, + {152, 55: 152}, + {83: 251}, + {83: 69}, + {83: 68}, // 75 - {54: 334, 69: 333}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 252, 80: 251}, - {137, 54: 137, 69: 137}, - {147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 147, 147, 320, 75: 319}, - {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}, + {1: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 253, 82: 252}, + {55: 338, 70: 337}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 255, 81: 254}, + {139, 55: 139, 70: 139}, + {149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 149, 149, 324, 76: 323}, // 80 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 85 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 90 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 95 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 100 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 105 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 110 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 115 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 120 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 125 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 130 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 135 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 140 - {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}, - {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}, - {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}, + {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}, + {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}, + {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}, // 145 - {143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 69: 143, 323, 90: 332}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 321}, - {147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 147, 147, 75: 322}, - {143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 69: 143, 323, 90: 324}, - {73: 325}, + {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}, + {145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 70: 145, 327, 91: 336}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 325}, + {149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 149, 149, 76: 326}, // 150 - {134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 134, 69: 134}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 327, 105: 326}, - {329, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 328, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 79: 330}, - {141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141, 141}, - {144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 55: 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 78: 144}, + {145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 70: 145, 327, 91: 328}, + {74: 329}, + {136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 136, 70: 136}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 331, 106: 330}, + {333, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 332, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 80: 334}, // 155 - {142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 69: 142}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 331}, - {140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140, 140}, - {135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 135, 69: 135}, - {148, 54: 148}, + {143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143, 143}, + {146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 56: 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 146, 79: 146}, + {144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 144, 70: 144}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 335}, + {142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142, 142}, // 160 - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 252, 80: 335}, - {136, 54: 136, 69: 136}, - {1: 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 151, 74: 151}, - {59: 247, 246, 85: 245, 338}, - {149, 54: 149}, + {137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 137, 70: 137}, + {150, 55: 150}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 255, 81: 339}, + {138, 55: 138, 70: 138}, + {1: 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, 75: 153}, // 165 - {62: 147, 147, 68: 242, 75: 340}, - {62: 342, 343, 99: 341}, - {344}, - {70}, - {69}, + {60: 250, 249, 86: 248, 342}, + {151, 55: 151}, + {63: 149, 149, 69: 245, 76: 344}, + {63: 346, 347, 100: 345}, + {348}, // 170 - {1: 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, 74: 152}, - {147, 68: 242, 75: 346}, - {347}, - {1: 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, 74: 153}, - {61: 147, 64: 147, 68: 242, 75: 349}, + {71}, + {70}, + {1: 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 75: 154}, + {149, 69: 245, 76: 350}, + {351}, // 175 - {61: 352, 64: 351, 101: 350}, - {353}, - {119}, - {118}, - {1: 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 154, 74: 154}, + {1: 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 75: 155}, + {62: 149, 65: 149, 69: 245, 76: 353}, + {62: 356, 65: 355, 102: 354}, + {357}, + {121}, // 180 - {78: 355}, - {54: 328, 78: 145, 356}, - {78: 357}, - {358}, - {1: 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 155, 74: 155}, + {120}, + {1: 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, 75: 156}, + {79: 359}, + {55: 332, 79: 147, 360}, + {79: 361}, // 185 - {68: 242, 75: 360, 77: 147}, - {77: 361}, - {65: 364, 363, 109: 362}, - {365}, - {121}, + {362}, + {1: 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, 75: 157}, + {69: 245, 76: 364, 78: 149}, + {78: 365}, + {66: 368, 367, 110: 366}, // 190 - {120}, - {1: 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, 74: 156}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 367}, - {368}, - {1: 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, 74: 157}, - // 195 - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 370}, - {371}, - {1: 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, 74: 158}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 373}, - {72: 374}, - // 200 - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 377, 378, 376, 110: 375}, - {379}, - {124}, + {369}, {123}, {122}, - // 205 - {1: 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, 74: 159}, - {68: 242, 75: 381, 77: 147}, - {77: 382}, + {1: 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, 75: 158}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 371}, + // 195 + {372}, + {1: 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, 75: 159}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 374}, + {375}, + {1: 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, 75: 160}, + // 200 + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 377}, + {73: 378}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 381, 382, 380, 111: 379}, {383}, - {1: 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, 74: 160}, + {126}, + // 205 + {125}, + {124}, + {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, 75: 161}, + {69: 245, 76: 385, 78: 149}, + {78: 386}, // 210 - {68: 242, 75: 385, 77: 147}, - {77: 386}, {387}, - {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, 74: 161}, - {147, 55: 147, 147, 147, 147, 68: 242, 75: 389}, + {1: 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, 75: 162}, + {69: 245, 76: 389, 78: 149}, + {78: 390}, + {391}, // 215 - {128, 55: 393, 394, 395, 396, 93: 392, 107: 391, 390}, - {399}, - {127, 54: 397}, - {126, 54: 126}, - {85, 54: 85}, + {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, 75: 163}, + {149, 56: 149, 149, 149, 149, 69: 245, 76: 393}, + {130, 56: 397, 398, 399, 400, 94: 396, 108: 395, 394}, + {403}, + {129, 55: 401}, // 220 - {84, 54: 84}, - {83, 54: 83}, - {82, 54: 82}, - {55: 393, 394, 395, 396, 93: 398}, - {125, 54: 125}, + {128, 55: 128}, + {87, 55: 87}, + {86, 55: 86}, + {85, 55: 85}, + {84, 55: 84}, // 225 - {1: 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, 74: 162}, - {1: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 402, 84: 401}, - {410}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 252, 80: 403}, - {145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 328, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 145, 79: 404}, + {56: 397, 398, 399, 400, 94: 402}, + {127, 55: 127}, + {1: 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, 75: 164}, + {1: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 406, 85: 405}, + {414}, // 230 - {132, 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 407, 102: 406, 405}, - {133}, - {131, 54: 408}, - {130, 54: 130}, - {1: 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 409}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 255, 81: 407}, + {147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 332, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 80: 408}, + {134, 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 411, 103: 410, 409}, + {135}, + {133, 55: 412}, // 235 - {129, 54: 129}, - {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, 74: 163}, - {1: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 402, 84: 412}, - {413}, - {1: 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, 74: 164}, + {132, 55: 132}, + {1: 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 413}, + {131, 55: 131}, + {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, 75: 165}, + {1: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 406, 85: 416}, // 240 - {147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 417, 81: 416, 87: 415}, - {418}, - {139, 54: 334}, - {138, 280, 294, 258, 260, 304, 283, 262, 284, 282, 266, 285, 286, 287, 254, 255, 256, 257, 306, 281, 276, 288, 264, 268, 259, 261, 263, 270, 267, 265, 269, 271, 275, 273, 289, 303, 279, 290, 291, 292, 278, 274, 277, 272, 293, 305, 295, 296, 301, 302, 298, 297, 299, 300, 55: 315, 316, 317, 318, 310, 309, 311, 307, 308, 312, 314, 313, 253, 76: 252, 80: 251}, - {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, 74: 165}, + {417}, + {1: 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, 75: 166}, + {149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 421, 82: 420, 88: 419}, + {422}, + {141, 55: 338}, // 245 - {147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 417, 81: 416, 87: 420}, - {421}, - {1: 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, 74: 166}, - {1: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 55: 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 147, 242, 75: 250, 81: 423}, - {424, 54: 334}, + {140, 283, 297, 261, 263, 307, 286, 265, 287, 285, 269, 288, 289, 290, 257, 258, 259, 260, 309, 284, 279, 291, 267, 271, 262, 264, 266, 273, 270, 268, 272, 274, 278, 276, 292, 306, 282, 293, 294, 295, 281, 277, 310, 280, 275, 296, 308, 298, 299, 304, 305, 301, 300, 302, 303, 56: 319, 320, 321, 322, 314, 313, 315, 311, 312, 316, 318, 317, 256, 77: 255, 81: 254}, + {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, 75: 167}, + {149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 421, 82: 420, 88: 424}, + {425}, + {1: 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, 75: 168}, // 250 - {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, 74: 167}, - {147, 68: 242, 75: 426}, - {427}, - {1: 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, 74: 168}, - {1: 234, 208, 200, 202, 226, 232, 214, 224, 238, 216, 210, 209, 213, 179, 197, 198, 199, 215, 235, 186, 191, 205, 217, 201, 203, 204, 219, 236, 206, 218, 220, 228, 222, 212, 187, 190, 195, 237, 196, 189, 227, 188, 221, 207, 239, 233, 211, 192, 230, 223, 225, 231, 229, 83: 193, 88: 180, 194, 91: 430, 185, 94: 184, 182, 429, 183, 181}, + {1: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 56: 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 149, 245, 76: 253, 82: 427}, + {428, 55: 338}, + {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, 75: 169}, + {149, 69: 245, 76: 430}, + {431}, // 255 - {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, 74: 171}, - {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, 74: 169}, + {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, 75: 170}, + {1: 236, 210, 202, 204, 228, 234, 216, 226, 240, 218, 212, 211, 215, 181, 199, 200, 201, 217, 237, 188, 193, 207, 219, 203, 205, 206, 221, 238, 208, 220, 222, 230, 224, 214, 189, 192, 197, 239, 198, 191, 229, 242, 190, 223, 209, 241, 235, 213, 194, 232, 225, 227, 233, 231, 84: 195, 89: 182, 196, 92: 434, 187, 95: 186, 184, 433, 185, 183}, + {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, 75: 173}, + {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, 75: 171}, } ) @@ -883,7 +890,7 @@ func yyhintlex1(yylex yyhintLexer, lval *yyhintSymType) (n int) { } func yyhintParse(yylex yyhintLexer, parser *hintParser) int { - const yyError = 112 + const yyError = 113 yyEx, _ := yylex.(yyhintLexerEx) var yyn int diff --git a/parser/hintparser.y b/parser/hintparser.y index e3c3dacf4faff..7f0eadfa17685 100644 --- a/parser/hintparser.y +++ b/parser/hintparser.y @@ -107,6 +107,7 @@ import ( hintForceIndex "FORCE_INDEX" hintStraightJoin "STRAIGHT_JOIN" hintLeading "LEADING" + hintSemiJoinRewrite "SEMI_JOIN_REWRITE" /* Other keywords */ hintOLAP "OLAP" @@ -583,6 +584,7 @@ NullaryHintName: | "READ_CONSISTENT_REPLICA" | "IGNORE_PLAN_CACHE" | "STRAIGHT_JOIN" +| "SEMI_JOIN_REWRITE" HintQueryType: "OLAP" @@ -649,6 +651,7 @@ Identifier: | "FORCE_INDEX" | "STRAIGHT_JOIN" | "LEADING" +| "SEMI_JOIN_REWRITE" /* other keywords */ | "OLAP" | "OLTP" diff --git a/parser/hintparser_test.go b/parser/hintparser_test.go index 5c252b2d4af78..e8df92de37e00 100644 --- a/parser/hintparser_test.go +++ b/parser/hintparser_test.go @@ -262,7 +262,7 @@ func TestParseHint(t *testing.T) { }, }, { - input: "READ_FROM_STORAGE(@foo TIKV[a, b], TIFLASH[c, d]) HASH_AGG() READ_FROM_STORAGE(TIKV[e])", + input: "READ_FROM_STORAGE(@foo TIKV[a, b], TIFLASH[c, d]) HASH_AGG() SEMI_JOIN_REWRITE() READ_FROM_STORAGE(TIKV[e])", output: []*ast.TableOptimizerHint{ { HintName: model.NewCIStr("READ_FROM_STORAGE"), @@ -285,6 +285,9 @@ func TestParseHint(t *testing.T) { { HintName: model.NewCIStr("HASH_AGG"), }, + { + HintName: model.NewCIStr("SEMI_JOIN_REWRITE"), + }, { HintName: model.NewCIStr("READ_FROM_STORAGE"), HintData: model.NewCIStr("TIKV"), diff --git a/parser/misc.go b/parser/misc.go index 1a4539ec4506a..e2c8e3cbf6427 100644 --- a/parser/misc.go +++ b/parser/misc.go @@ -937,6 +937,7 @@ var hintTokenMap = map[string]int{ "FORCE_INDEX": hintForceIndex, "STRAIGHT_JOIN": hintStraightJoin, "LEADING": hintLeading, + "SEMI_JOIN_REWRITE": hintSemiJoinRewrite, // TiDB hint aliases "TIDB_HJ": hintHashJoin, diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index dd452eb5d238b..6cdd36ae15efe 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -315,7 +315,9 @@ func (er *expressionRewriter) constructBinaryOpFunction(l expression.Expression, } } -func (er *expressionRewriter) buildSubquery(ctx context.Context, subq *ast.SubqueryExpr) (LogicalPlan, error) { +// buildSubquery translates the subquery ast to plan. +// Currently, only the EXIST can apply the rewrite hint(rewrite the semi join to inner join with aggregation). +func (er *expressionRewriter) buildSubquery(ctx context.Context, subq *ast.SubqueryExpr, rewriteHintCanTakeEffect bool) (np LogicalPlan, hasSemiJoinRewriteHint bool, err error) { if er.schema != nil { outerSchema := er.schema.Clone() er.b.outerSchemas = append(er.b.outerSchemas, outerSchema) @@ -325,18 +327,25 @@ func (er *expressionRewriter) buildSubquery(ctx context.Context, subq *ast.Subqu er.b.outerNames = er.b.outerNames[0 : len(er.b.outerNames)-1] }() } + // Store the old value before we enter the subquery and reset they to default value. + oldRewriteHintCanTakeEffect := er.b.checkSemiJoinHint + er.b.checkSemiJoinHint = rewriteHintCanTakeEffect + oldHasHint := er.b.hasValidSemiJoinHint + er.b.hasValidSemiJoinHint = false outerWindowSpecs := er.b.windowSpecs defer func() { er.b.windowSpecs = outerWindowSpecs + er.b.checkSemiJoinHint = oldRewriteHintCanTakeEffect + er.b.hasValidSemiJoinHint = oldHasHint }() - np, err := er.b.buildResultSetNode(ctx, subq.Query) + np, err = er.b.buildResultSetNode(ctx, subq.Query) if err != nil { - return nil, err + return nil, false, err } // Pop the handle map generated by the subquery. er.b.handleHelper.popMap() - return np, nil + return np, er.b.hasValidSemiJoinHint, nil } // Enter implements Visitor interface. @@ -500,7 +509,7 @@ func (er *expressionRewriter) buildSemiApplyFromEqualSubq(np LogicalPlan, l, r e if er.err != nil { return } - er.p, er.err = er.b.buildSemiApply(er.p, np, []expression.Expression{condition}, er.asScalar, not) + er.p, er.err = er.b.buildSemiApply(er.p, np, []expression.Expression{condition}, er.asScalar, not, false) } func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast.CompareSubqueryExpr) (ast.Node, bool) { @@ -516,7 +525,7 @@ func (er *expressionRewriter) handleCompareSubquery(ctx context.Context, v *ast. er.err = errors.Errorf("Unknown compare type %T", v.R) return v, true } - np, err := er.buildSubquery(ctx, subq) + np, _, err := er.buildSubquery(ctx, subq, false) if err != nil { er.err = err return v, true @@ -690,7 +699,7 @@ func (er *expressionRewriter) buildQuantifierPlan(plan4Agg *LogicalAggregation, // plan4Agg.buildProjectionIfNecessary() if !er.asScalar { // For Semi LogicalApply without aux column, the result is no matter false or null. So we can add it to join predicate. - er.p, er.err = er.b.buildSemiApply(er.p, plan4Agg, []expression.Expression{cond}, false, false) + er.p, er.err = er.b.buildSemiApply(er.p, plan4Agg, []expression.Expression{cond}, false, false, false) return } // If we treat the result as a scalar value, we will add a projection with a extra column to output true, false or null. @@ -810,14 +819,14 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex er.err = errors.Errorf("Unknown exists type %T", v.Sel) return v, true } - np, err := er.buildSubquery(ctx, subq) + np, hasRewriteHint, err := er.buildSubquery(ctx, subq, true) if err != nil { er.err = err return v, true } np = er.popExistsSubPlan(np) if er.b.disableSubQueryPreprocessing || len(ExtractCorrelatedCols4LogicalPlan(np)) > 0 { - er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, v.Not) + er.p, er.err = er.b.buildSemiApply(er.p, np, nil, er.asScalar, v.Not, hasRewriteHint) if er.err != nil || !er.asScalar { return v, true } @@ -884,7 +893,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte er.err = errors.Errorf("Unknown compare type %T", v.Sel) return v, true } - np, err := er.buildSubquery(ctx, subq) + np, _, err := er.buildSubquery(ctx, subq, false) if err != nil { er.err = err return v, true @@ -968,7 +977,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte } er.p = join } else { - er.p, er.err = er.b.buildSemiApply(er.p, np, expression.SplitCNFItems(checkCondition), asScalar, v.Not) + er.p, er.err = er.b.buildSemiApply(er.p, np, expression.SplitCNFItems(checkCondition), asScalar, v.Not, false) if er.err != nil { return v, true } @@ -985,7 +994,7 @@ func (er *expressionRewriter) handleInSubquery(ctx context.Context, v *ast.Patte func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.SubqueryExpr) (ast.Node, bool) { ci := er.b.prepareCTECheckForSubQuery() defer resetCTECheckForSubQuery(ci) - np, err := er.buildSubquery(ctx, v) + np, _, err := er.buildSubquery(ctx, v, false) if err != nil { er.err = err return v, true diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5a76bd9d34385..d6e2fa9d92b0c 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -117,6 +117,8 @@ const ( HintIgnorePlanCache = "ignore_plan_cache" // HintLimitToCop is a hint enforce pushing limit or topn to coprocessor. HintLimitToCop = "limit_to_cop" + // HintSemiJoinRewrite is a hint to force we rewrite the semi join operator as much as possible. + HintSemiJoinRewrite = "semi_join_rewrite" ) const ( @@ -3626,6 +3628,12 @@ func (b *PlanBuilder) pushTableHints(hints []*ast.TableOptimizerHint, currentLev leadingJoinOrder = append(leadingJoinOrder, tableNames2HintTableInfo(b.ctx, hint.HintName.L, hint.Tables, b.hintProcessor, currentLevel)...) } leadingHintCnt++ + case HintSemiJoinRewrite: + if !b.checkSemiJoinHint { + b.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause.")) + continue + } + b.hasValidSemiJoinHint = true default: // ignore hints that not implemented } @@ -4866,10 +4874,10 @@ func (b *PlanBuilder) buildApplyWithJoinType(outerPlan, innerPlan LogicalPlan, t } // buildSemiApply builds apply plan with outerPlan and innerPlan, which apply semi-join for every row from outerPlan and the whole innerPlan. -func (b *PlanBuilder) buildSemiApply(outerPlan, innerPlan LogicalPlan, condition []expression.Expression, asScalar, not bool) (LogicalPlan, error) { +func (b *PlanBuilder) buildSemiApply(outerPlan, innerPlan LogicalPlan, condition []expression.Expression, asScalar, not, considerRewrite bool) (LogicalPlan, error) { b.optFlag = b.optFlag | flagPredicatePushDown | flagBuildKeyInfo | flagDecorrelate - join, err := b.buildSemiJoin(outerPlan, innerPlan, condition, asScalar, not) + join, err := b.buildSemiJoin(outerPlan, innerPlan, condition, asScalar, not, considerRewrite) if err != nil { return nil, err } @@ -4905,7 +4913,7 @@ func (b *PlanBuilder) buildMaxOneRow(p LogicalPlan) LogicalPlan { return maxOneRow } -func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onCondition []expression.Expression, asScalar bool, not bool) (*LogicalJoin, error) { +func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onCondition []expression.Expression, asScalar, not, forceRewrite bool) (*LogicalJoin, error) { joinPlan := LogicalJoin{}.Init(b.ctx, b.getSelectOffset()) for i, expr := range onCondition { onCondition[i] = expr.Decorrelate(outerPlan.Schema()) @@ -4959,6 +4967,10 @@ func (b *PlanBuilder) buildSemiJoin(outerPlan, innerPlan LogicalPlan, onConditio return nil, errors.New("Join hints are conflict, you can only specify one type of join") } } + if forceRewrite { + joinPlan.preferJoinType |= preferRewriteSemiJoin + b.optFlag |= flagSemiJoinRewrite + } return joinPlan, nil } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index b6f279f81c4e5..4d67356260908 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -505,7 +505,7 @@ func TestSubquery(t *testing.T) { p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) require.NoError(t, err) if lp, ok := p.(LogicalPlan); ok { - p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, lp) + p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagSemiJoinRewrite, lp) require.NoError(t, err) } testdata.OnRecord(func() { diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 00f6ccc372137..8a5d75244b23d 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -117,6 +117,7 @@ const ( preferHashJoin preferMergeJoin preferBCJoin + preferRewriteSemiJoin preferHashAgg preferStreamAgg ) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index b1d8dcd04ced4..f816ec0e8dc24 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -57,6 +57,7 @@ const ( flagStabilizeResults flagBuildKeyInfo flagDecorrelate + flagSemiJoinRewrite flagEliminateAgg flagEliminateProjection flagMaxMinEliminate @@ -77,6 +78,7 @@ var optRuleList = []logicalOptRule{ &resultReorder{}, &buildKeySolver{}, &decorrelateSolver{}, + &semiJoinRewriter{}, &aggregationEliminator{}, &projectionEliminator{}, &maxMinEliminator{}, diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index fe5c5cba7da00..ec9c71d6fa245 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -911,6 +911,57 @@ func TestAggregationHints(t *testing.T) { } } +func TestSemiJoinRewriteHints(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(a int, b int, c int)") + + sessionVars := tk.Session().GetSessionVars() + sessionVars.SetHashAggFinalConcurrency(1) + sessionVars.SetHashAggPartialConcurrency(1) + + var input []string + var output []struct { + SQL string + Plan []string + Warning string + } + planSuiteData := core.GetPlanSuiteData() + planSuiteData.GetTestCases(t, &input, &output) + ctx := context.Background() + p := parser.New() + is := infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable()}) + for i, test := range input { + comment := fmt.Sprintf("case: %v sql: %v", i, test) + tk.Session().GetSessionVars().StmtCtx.SetWarnings(nil) + + stmt, err := p.ParseOneStmt(test, "", "") + require.NoError(t, err, comment) + + _, _, err = planner.Optimize(ctx, tk.Session(), stmt, is) + require.NoError(t, err) + warnings := tk.Session().GetSessionVars().StmtCtx.GetWarnings() + + testdata.OnRecord(func() { + output[i].SQL = test + output[i].Plan = testdata.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief'" + test).Rows()) + if len(warnings) > 0 { + output[i].Warning = warnings[0].Err.Error() + } + }) + tk.MustQuery("explain format = 'brief'" + test).Check(testkit.Rows(output[i].Plan...)) + if output[i].Warning == "" { + require.Len(t, warnings, 0) + } else { + require.Len(t, warnings, 1, fmt.Sprintf("%v", warnings)) + require.Equal(t, stmtctx.WarnLevelWarning, warnings[0].Level) + require.Equal(t, output[i].Warning, warnings[0].Err.Error()) + } + } +} + func TestExplainJoinHints(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/planner/core/physical_plan_trace_test.go b/planner/core/physical_plan_trace_test.go index 9988fb6cacfa9..30cb5d6d1a976 100644 --- a/planner/core/physical_plan_trace_test.go +++ b/planner/core/physical_plan_trace_test.go @@ -83,9 +83,7 @@ func TestPhysicalOptimizeWithTraceEnabled(t *testing.T) { domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) plan, err := builder.Build(context.TODO(), stmt) require.NoError(t, err) - flag := uint64(0) - flag = flag | 1<<3 | 1<<8 - _, _, err = core.DoOptimize(context.TODO(), sctx, flag, plan.(core.LogicalPlan)) + _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) require.NoError(t, err) otrace := sctx.GetSessionVars().StmtCtx.OptimizeTracer.Physical require.NotNil(t, otrace) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 5aa6738cbd0c6..e758bcbea3293 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -504,6 +504,15 @@ type PlanBuilder struct { // disableSubQueryPreprocessing indicates whether to pre-process uncorrelated sub-queries in rewriting stage. disableSubQueryPreprocessing bool + + // checkSemiJoinHint checks whether the SEMI_JOIN_REWRITE hint is possible to be applied on the current SELECT stmt. + // We need this variable for the hint since the hint is set in subquery, but we check its availability in its outer scope. + // e.g. select * from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t1 where t.a=t1.a) + // Whether the hint can be applied or not is checked after the subquery is fully built. + checkSemiJoinHint bool + // hasValidSemijoinHint would tell the outer APPLY/JOIN operator that there's valid hint to be checked later + // if there's SEMI_JOIN_REWRITE hint and we find checkSemiJoinHint is true. + hasValidSemiJoinHint bool } type handleColHelper struct { diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 22ec84d150bc4..361a821c8f2cd 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -48,6 +48,10 @@ func (la *LogicalAggregation) BuildKeyInfo(selfSchema *expression.Schema, childS return } la.logicalSchemaProducer.BuildKeyInfo(selfSchema, childSchema) + la.buildSelfKeyInfo(selfSchema) +} + +func (la *LogicalAggregation) buildSelfKeyInfo(selfSchema *expression.Schema) { groupByCols := la.GetGroupByCols() if len(groupByCols) == len(la.GroupByItems) && len(la.GroupByItems) > 0 { indices := selfSchema.ColumnsIndices(groupByCols) diff --git a/planner/core/rule_semi_join_rewrite.go b/planner/core/rule_semi_join_rewrite.go new file mode 100644 index 0000000000000..0bce68183b59e --- /dev/null +++ b/planner/core/rule_semi_join_rewrite.go @@ -0,0 +1,115 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "context" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/expression/aggregation" + "github.com/pingcap/tidb/parser/ast" +) + +type semiJoinRewriter struct { +} + +func (smj *semiJoinRewriter) optimize(_ context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { + return smj.recursivePlan(p) +} + +func (smj *semiJoinRewriter) name() string { + return "semi_join_rewrite" +} + +func (smj *semiJoinRewriter) recursivePlan(p LogicalPlan) (LogicalPlan, error) { + newChildren := make([]LogicalPlan, 0, len(p.Children())) + for _, child := range p.Children() { + newChild, err := smj.recursivePlan(child) + if err != nil { + return nil, err + } + newChildren = append(newChildren, newChild) + } + p.SetChildren(newChildren...) + join, ok := p.(*LogicalJoin) + // If it's not a join, or not a (outer) semi join. We just return it since no optimization is needed. + // Actually the check of the preferRewriteSemiJoin is a superset of checking the join type. We remain them for a better understanding. + if !ok || !(join.JoinType == SemiJoin || join.JoinType == LeftOuterSemiJoin) || (join.preferJoinType&preferRewriteSemiJoin == 0) { + return p, nil + } + + if join.JoinType == LeftOuterSemiJoin { + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("SEMI_JOIN_REWRITE() is inapplicable for LeftOuterSemiJoin.")) + return p, nil + } + + // If we have jumped the above if condition. We can make sure that the current join is a non-correlated one. + + // If there's left condition or other condition, we cannot rewrite + if len(join.LeftConditions) > 0 || len(join.OtherConditions) > 0 { + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("SEMI_JOIN_REWRITE() is inapplicable for SemiJoin with left conditions or other conditions.")) + return p, nil + } + + innerChild := join.Children()[1] + + // If there's right conditions: + // - If it's semi join, then right condition should be pushed. + // - If it's outer semi join, then it still should be pushed since the outer join should not remain any cond of the inner side. + // But the aggregation we added may block the predicate push down since we've not maintained the functional dependency to pass the equiv class to guide the push down. + // So we create a selection before we build the aggregation. + if len(join.RightConditions) > 0 { + sel := LogicalSelection{Conditions: make([]expression.Expression, len(join.RightConditions))}.Init(p.SCtx(), innerChild.SelectBlockOffset()) + copy(sel.Conditions, join.RightConditions) + sel.SetChildren(innerChild) + innerChild = sel + } + + subAgg := LogicalAggregation{ + AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(join.EqualConditions)), + GroupByItems: make([]expression.Expression, 0, len(join.EqualConditions)), + }.Init(p.SCtx(), p.Children()[1].SelectBlockOffset()) + + aggOutputCols := make([]*expression.Column, 0, len(join.EqualConditions)) + for i := range join.EqualConditions { + innerCol := join.EqualConditions[i].GetArgs()[1].(*expression.Column) + firstRow, err := aggregation.NewAggFuncDesc(join.SCtx(), ast.AggFuncFirstRow, []expression.Expression{innerCol}, false) + if err != nil { + return nil, err + } + subAgg.AggFuncs = append(subAgg.AggFuncs, firstRow) + subAgg.GroupByItems = append(subAgg.GroupByItems, innerCol) + aggOutputCols = append(aggOutputCols, innerCol) + } + subAgg.SetChildren(innerChild) + subAgg.SetSchema(expression.NewSchema(aggOutputCols...)) + subAgg.buildSelfKeyInfo(subAgg.Schema()) + + innerJoin := LogicalJoin{ + JoinType: InnerJoin, + EqualConditions: make([]*expression.ScalarFunction, 0, len(join.EqualConditions)), + }.Init(p.SCtx(), p.SelectBlockOffset()) + innerJoin.SetChildren(join.Children()[0], subAgg) + innerJoin.SetSchema(expression.MergeSchema(join.Children()[0].Schema(), subAgg.schema)) + innerJoin.AttachOnConds(expression.ScalarFuncs2Exprs(join.EqualConditions)) + + proj := LogicalProjection{ + Exprs: expression.Column2Exprs(join.Children()[0].Schema().Columns), + }.Init(p.SCtx(), p.SelectBlockOffset()) + proj.SetChildren(innerJoin) + proj.SetSchema(join.Children()[0].Schema()) + + return proj, nil +} diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index a2a6c7e655efc..e3aff32273794 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -285,7 +285,9 @@ "cases": [ // Query with WHERE or ON should have the same plan, i.e, the Apply has been decorrelated. "explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a)", - "explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a)" + "explain format = 'brief' select * from t where exists (select 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a)", + "explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a)", + "explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a)" ] }, { @@ -499,6 +501,8 @@ "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k)", + "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index f3f200872b93e..10a8475503219 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1535,6 +1535,34 @@ "└─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] + }, + { + "SQL": "explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 where t1.a = t2.a and t1.a = t.a)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "│ ├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] + }, + { + "SQL": "explain format = 'brief' select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 join t t2 on t1.a = t2.a and t1.a = t.a)", + "Plan": [ + "HashJoin 10000.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─HashJoin 12500.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "│ ├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ] } ] }, @@ -2960,6 +2988,40 @@ " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 8.00 root data:ExchangeSender", + " └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 8.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Projection 2.00 mpp[tiflash] test.d1_t.d1_k", + " │ └─HashAgg 2.00 mpp[tiflash] group by:test.d1_t.d1_k, funcs:firstrow(test.d1_t.d1_k)->test.d1_t.d1_k", + " │ └─ExchangeReceiver 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + ] + }, + { + "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", + "Plan": [ + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 6.40 root data:ExchangeSender", + " └─ExchangeSender 6.40 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 6.40 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + ] + }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ diff --git a/planner/core/testdata/plan_suite_in.json b/planner/core/testdata/plan_suite_in.json index ce4c38cbc695d..627621aa1bfe6 100644 --- a/planner/core/testdata/plan_suite_in.json +++ b/planner/core/testdata/plan_suite_in.json @@ -783,5 +783,16 @@ "select * from employee e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.deptid = e2.d", "select * from (select deptid+1 d, count(empid) a from employee group by d) e1 join (select deptid+1 d, count(empid) a from employee group by d) e2 on e1.d = e2.d" ] + }, + { + "name": "TestSemiJoinRewriteHints", + "cases": [ + "select /*+ SEMI_JOIN_REWRITE() */ * from t", + "select * from t where a > (select /*+ SEMI_JOIN_REWRITE() */ min(b) from t t1 where t1.c = t.c)", + "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 where t1.a=t.a)", + "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ t.b from t t1 where t1.a=t.a)", + "select exists(select /*+ SEMI_JOIN_REWRITE() */ * from t t1 where t1.a=t.a) from t", + "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 where t1.a > t.a)" + ] } ] diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 43c97667fd49d..49b92cb6e9afa 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -3078,5 +3078,86 @@ ] } ] + }, + { + "Name": "TestSemiJoinRewriteHints", + "Cases": [ + { + "SQL": "select /*+ SEMI_JOIN_REWRITE() */ * from t", + "Plan": [ + "TableReader 10000.00 root data:TableFullScan", + "└─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause." + }, + { + "SQL": "select * from t where a > (select /*+ SEMI_JOIN_REWRITE() */ min(b) from t t1 where t1.c = t.c)", + "Plan": [ + "HashJoin 7992.00 root inner join, equal:[eq(test.t.c, test.t.c)], other cond:gt(test.t.a, Column#9)", + "├─Selection(Build) 6393.60 root not(isnull(Column#9))", + "│ └─HashAgg 7992.00 root group by:test.t.c, funcs:min(Column#10)->Column#9, funcs:firstrow(test.t.c)->test.t.c", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t.c, funcs:min(test.t.b)->Column#10", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.c))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t.a)), not(isnull(test.t.c))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "[planner:1815]The SEMI_JOIN_REWRITE hint is not used correctly, maybe it's not in a subquery or the subquery is not EXISTS clause." + }, + { + "SQL": "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 where t1.a=t.a)", + "Plan": [ + "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "" + }, + { + "SQL": "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ t.b from t t1 where t1.a=t.a)", + "Plan": [ + "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", + "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", + "│ └─TableReader 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "" + }, + { + "SQL": "select exists(select /*+ SEMI_JOIN_REWRITE() */ * from t t1 where t1.a=t.a) from t", + "Plan": [ + "HashJoin 10000.00 root left outer semi join, equal:[eq(test.t.a, test.t.a)]", + "├─TableReader(Build) 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "[planner:1815]SEMI_JOIN_REWRITE() is inapplicable for LeftOuterSemiJoin." + }, + { + "SQL": "select * from t where exists (select /*+ SEMI_JOIN_REWRITE() */ 1 from t t1 where t1.a > t.a)", + "Plan": [ + "HashJoin 7992.00 root CARTESIAN semi join, other cond:gt(test.t.a, test.t.a)", + "├─TableReader(Build) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + ], + "Warning": "[planner:1815]SEMI_JOIN_REWRITE() is inapplicable for SemiJoin with left conditions or other conditions." + } + ] } ] diff --git a/planner/core/testdata/plan_suite_unexported_in.json b/planner/core/testdata/plan_suite_unexported_in.json index 98c7b9b9b5985..9855bbc7a7689 100644 --- a/planner/core/testdata/plan_suite_unexported_in.json +++ b/planner/core/testdata/plan_suite_unexported_in.json @@ -130,6 +130,17 @@ "select t1.b from t t1 where t1.b = (select max(t2.a) from t t2 where t1.b=t2.b order by t1.a)", "select t1.b from t t1 where t1.b in (select t2.b from t t2 where t2.a = t1.a order by t2.a)", "select t1.b from t t1 where exists(select t2.b from t t2 where t2.a = t1.a order by t2.a)", + "select t1.b from t t1 where exists(select /*+ SEMI_JOIN_REWRITE() */ t2.b from t t2 where t2.a = t1.a order by t2.a)", + "select a from t where exists(select 1 from t as x where x.a < t.a)", + "select a from t where exists(select 1 from t as x where x.a = t.a and t.a < 1 and x.a < 1)", + "select a from t where exists(select 1 from t as x where x.a = t.a and x.a < 1) and a < 1", + "select a from t where exists(select 1 from t as x where x.a = t.a) and exists(select 1 from t as x where x.a = t.a)", + "select a from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a < t.a)", + "select a from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a and t.a < 1 and x.a < 1)", + "select a from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a and x.a < 1) and a < 1", + "select a from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a) and exists(select 1 from t as x where x.a = t.a)", + "select a from t where exists(select 1 from t as x where x.a = t.a) and exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a)", + "select a from t where exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a) and exists(select /*+ SEMI_JOIN_REWRITE() */ 1 from t as x where x.a = t.a)", // `Sort` will not be eliminated, if it is not the top level operator. "select t1.b from t t1 where t1.b = (select t2.b from t t2 where t2.a = t1.a order by t2.a limit 1)", "select (select 1 from t t1 where t1.a = t2.a) from t t2" diff --git a/planner/core/testdata/plan_suite_unexported_out.json b/planner/core/testdata/plan_suite_unexported_out.json index 218880d0aa713..630ed1a11db24 100644 --- a/planner/core/testdata/plan_suite_unexported_out.json +++ b/planner/core/testdata/plan_suite_unexported_out.json @@ -117,6 +117,17 @@ "Join{DataScan(t1)->DataScan(t2)->Aggr(max(test.t.a),firstrow(test.t.b))}(test.t.b,test.t.b)->Projection->Sel([eq(test.t.b, Column#25)])->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)(test.t.b,test.t.b)->Projection", "Join{DataScan(t1)->DataScan(t2)}(test.t.a,test.t.a)->Projection", + "Join{DataScan(t1)->DataScan(t2)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->Projection", + "Join{DataScan(t)->DataScan(x)}->Projection", + "Join{DataScan(t)->DataScan(x)}(test.t.a,test.t.a)->Projection", + "Join{DataScan(t)->DataScan(x)}(test.t.a,test.t.a)->Sel([lt(test.t.a, 1)])->Projection", + "Join{Join{DataScan(t)->DataScan(x)}(test.t.a,test.t.a)->DataScan(x)}(test.t.a,test.t.a)->Projection", + "Join{DataScan(t)->DataScan(x)}->Projection", + "Join{DataScan(t)->DataScan(x)}(test.t.a,test.t.a)->Projection", + "Join{DataScan(t)->DataScan(x)->Sel([lt(test.t.a, 1)])->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->Sel([lt(test.t.a, 1)])->Projection", + "Join{Join{DataScan(t)->DataScan(x)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->DataScan(x)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->Projection", + "Join{Join{DataScan(t)->DataScan(x)}(test.t.a,test.t.a)->DataScan(x)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->Projection", + "Join{Join{DataScan(t)->DataScan(x)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->DataScan(x)->Aggr(firstrow(test.t.a))}(test.t.a,test.t.a)->Projection->Projection", "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t.a, test.t.a)])->Projection->Sort->Limit}->Projection->Sel([eq(test.t.b, test.t.b)])->Projection", "Apply{DataScan(t2)->DataScan(t1)->Sel([eq(test.t.a, test.t.a)])->Projection}->Projection" ] diff --git a/testkit/testkit.go b/testkit/testkit.go index 4631861445b39..69679e95866a3 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -182,6 +182,28 @@ func (tk *TestKit) HasPlan(sql string, plan string, args ...interface{}) bool { return false } +// HasKeywordInOperatorInfo checks if the result execution plan contains specific keyword in the operator info. +func (tk *TestKit) HasKeywordInOperatorInfo(sql string, keyword string, args ...interface{}) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Contains(rs.rows[i][4], keyword) { + return true + } + } + return false +} + +// NotHasKeywordInOperatorInfo checks if the result execution plan doesn't contain specific keyword in the operator info. +func (tk *TestKit) NotHasKeywordInOperatorInfo(sql string, keyword string, args ...interface{}) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Contains(rs.rows[i][4], keyword) { + return false + } + } + return true +} + // HasPlan4ExplainFor checks if the result execution plan contains specific plan. func (tk *TestKit) HasPlan4ExplainFor(result *Result, plan string) bool { for i := range result.rows { From dcffd65b367b547cdd6f9e5687b9dd944e8b187c Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 23 Aug 2022 13:42:19 +0800 Subject: [PATCH 33/42] lightning: hide sensitive log for server mode lightning (#36375) (#36408) close pingcap/tidb#36374 --- Makefile | 1 + br/pkg/lightning/lightning.go | 3 ++- br/pkg/utils/sensitive.go | 23 ++++++++++++++++ br/pkg/utils/sensitive_test.go | 41 +++++++++++++++++++++++++++++ dumpling/install.sh | 3 +++ dumpling/tests/s3/run.sh | 48 ++++++++++++++++++++++------------ 6 files changed, 102 insertions(+), 17 deletions(-) create mode 100644 br/pkg/utils/sensitive.go create mode 100644 br/pkg/utils/sensitive_test.go diff --git a/Makefile b/Makefile index b6b3cab07a462..9c92292dfe714 100644 --- a/Makefile +++ b/Makefile @@ -421,6 +421,7 @@ dumpling_tidy: dumpling_bins: @which bin/tidb-server @which bin/minio + @which bin/mc @which bin/tidb-lightning @which bin/sync_diff_inspector diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index 3c6f0256a740c..dc389bebbc6ab 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -615,7 +615,8 @@ func (l *Lightning) handlePostTask(w http.ResponseWriter, req *http.Request) { writeJSONError(w, http.StatusBadRequest, "cannot read request", err) return } - log.L().Info("received task config", zap.ByteString("content", data)) + filteredData := utils.HideSensitive(string(data)) + log.L().Info("received task config", zap.String("content", filteredData)) cfg := config.NewConfig() if err = cfg.LoadFromGlobal(l.globalCfg); err != nil { diff --git a/br/pkg/utils/sensitive.go b/br/pkg/utils/sensitive.go new file mode 100644 index 0000000000000..fcc31ee30b78d --- /dev/null +++ b/br/pkg/utils/sensitive.go @@ -0,0 +1,23 @@ +// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. + +package utils + +import ( + "regexp" +) + +var ( + passwordPatterns = `(password[\s]*=[\s]*(\\")?)(.*?)((\\")?\\n)` + + passwordRegexp *regexp.Regexp +) + +func init() { + passwordRegexp = regexp.MustCompile(passwordPatterns) +} + +// HideSensitive replace password with ******. +func HideSensitive(input string) string { + output := passwordRegexp.ReplaceAllString(input, "$1******$4") + return output +} diff --git a/br/pkg/utils/sensitive_test.go b/br/pkg/utils/sensitive_test.go new file mode 100644 index 0000000000000..a14ce0619eb85 --- /dev/null +++ b/br/pkg/utils/sensitive_test.go @@ -0,0 +1,41 @@ +// Copyright 2022 PingCAP, Inc. Licensed under Apache-2.0. + +package utils + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestHideSensitive(t *testing.T) { + strs := []struct { + old string + new string + }{ + { + `host = "127.0.0.1"\n user = "root"\n password = "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs="\n port = 3306\n`, + `host = "127.0.0.1"\n user = "root"\n password = ******\n port = 3306\n`, + }, + { + `host = "127.0.0.1"\n user = "root"\n password = ""\n port = 3306\n`, + `host = "127.0.0.1"\n user = "root"\n password = ******\n port = 3306\n`, + }, + { + `host = "127.0.0.1"\n user = "root"\n password= "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs="\n port = 3306\n`, + `host = "127.0.0.1"\n user = "root"\n password= ******\n port = 3306\n`, + }, + { + `host = "127.0.0.1"\n user = "root"\n password =""\n port = 3306\n`, + `host = "127.0.0.1"\n user = "root"\n password =******\n port = 3306\n`, + }, + { + `host = "127.0.0.1"\n user = "root"\n password=""\n port = 3306\n`, + `host = "127.0.0.1"\n user = "root"\n password=******\n port = 3306\n`, + }, + } + for i, str := range strs { + t.Logf("case #%d\n", i) + require.Equal(t, str.new, HideSensitive(str.old)) + } +} diff --git a/dumpling/install.sh b/dumpling/install.sh index e51ee861b099c..65fd56e454c97 100644 --- a/dumpling/install.sh +++ b/dumpling/install.sh @@ -27,3 +27,6 @@ mv tidb/bin/tidb-server bin/ # download minio wget https://dl.min.io/server/minio/release/linux-amd64/minio -O bin/minio chmod a+x bin/minio + +wget https://dl.minio.io/client/mc/release/linux-amd64/mc -O bin/mc +chmod a+x bin/mc diff --git a/dumpling/tests/s3/run.sh b/dumpling/tests/s3/run.sh index a5011e4707946..ede8c316106ed 100755 --- a/dumpling/tests/s3/run.sh +++ b/dumpling/tests/s3/run.sh @@ -47,38 +47,54 @@ run_sql "create database $DB_NAME DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;" (cd "$(dirname "$0")" && GO111MODULE=on go build -o out) $DUMPLING_BASE_NAME/out -B $DB_NAME -T $TABLE_NAME -P 3306 -w 16 -# run dumpling! HOST_DIR=${DUMPLING_OUTPUT_DIR} + +# run dumpling local +export DUMPLING_OUTPUT_DIR=${HOST_DIR}/local +run_dumpling + +# run dumpling s3 export DUMPLING_OUTPUT_DIR=s3://mybucket/dump export DUMPLING_TEST_DATABASE=$DB_NAME export AWS_REGION=us-east-1 export AWS_ACCESS_KEY_ID="$MINIO_ACCESS_KEY" export AWS_SECRET_ACCESS_KEY="$MINIO_SECRET_KEY" run_dumpling --s3.endpoint="http://$S3_ENDPOINT/" -ls "${HOST_DIR}" +ls "${HOST_DIR}" -R + +mkdir -p "${HOST_DIR}/remote" + +bin/mc config host add minio http://127.0.0.1:5000 testid testkey8 +bin/mc cp minio/mybucket/dump/s3-schema-create.sql "${HOST_DIR}/remote/s3-schema-create.sql" +bin/mc cp minio/mybucket/dump/s3.t-schema.sql "${HOST_DIR}/remote/s3.t-schema.sql" +bin/mc cp minio/mybucket/dump/s3.t.000000000.sql "${HOST_DIR}/remote/s3.t.000000000.sql" -file_should_exist "$DBPATH/mybucket/dump/s3-schema-create.sql" -file_should_exist "$DBPATH/mybucket/dump/s3.t-schema.sql" -file_should_exist "$DBPATH/mybucket/dump/s3.t.000000000.sql" +diff "${HOST_DIR}/local/s3-schema-create.sql" "${HOST_DIR}/remote/s3-schema-create.sql" +diff "${HOST_DIR}/local/s3.t-schema.sql" "${HOST_DIR}/remote/s3.t-schema.sql" +diff "${HOST_DIR}/local/s3.t.000000000.sql" "${HOST_DIR}/remote/s3.t.000000000.sql" -cnt=`grep -o "('aaaaaaaaaa')" $DBPATH/mybucket/dump/s3.t.000000000.sql|wc -l` +cnt=`grep -o "('aaaaaaaaaa')" ${HOST_DIR}/remote/s3.t.000000000.sql|wc -l` echo "1st records count is ${cnt}" [ $cnt = 5000000 ] # run dumpling with compress option -mv "$DBPATH/mybucket/dump" "$DBPATH/mybucket/expect" +export DUMPLING_OUTPUT_DIR=s3://mybucket/dump-compress + run_dumpling --s3.endpoint="http://$S3_ENDPOINT/" --compress "gzip" -file_should_exist "$DBPATH/mybucket/dump/s3-schema-create.sql.gz" -file_should_exist "$DBPATH/mybucket/dump/s3.t-schema.sql.gz" -file_should_exist "$DBPATH/mybucket/dump/s3.t.000000000.sql.gz" -gzip "$DBPATH/mybucket/dump/s3-schema-create.sql.gz" -d -diff "$DBPATH/mybucket/expect/s3-schema-create.sql" "$DBPATH/mybucket/dump/s3-schema-create.sql" +mkdir -p "${HOST_DIR}/compress" + +bin/mc cp minio/mybucket/dump-compress/s3-schema-create.sql.gz "${HOST_DIR}/compress/s3-schema-create.sql.gz" +bin/mc cp minio/mybucket/dump-compress/s3.t-schema.sql.gz "${HOST_DIR}/compress/s3.t-schema.sql.gz" +bin/mc cp minio/mybucket/dump-compress/s3.t.000000000.sql.gz "${HOST_DIR}/compress/s3.t.000000000.sql.gz" + +gzip "${HOST_DIR}/compress/s3-schema-create.sql.gz" -d +diff "${HOST_DIR}/local/s3-schema-create.sql" "${HOST_DIR}/compress/s3-schema-create.sql" -gzip "$DBPATH/mybucket/dump/s3.t-schema.sql.gz" -d -diff "$DBPATH/mybucket/expect/s3.t-schema.sql" "$DBPATH/mybucket/dump/s3.t-schema.sql" +gzip "${HOST_DIR}/compress/s3.t-schema.sql.gz" -d +diff "${HOST_DIR}/local/s3.t-schema.sql" "${HOST_DIR}/compress/s3.t-schema.sql" -gzip "$DBPATH/mybucket/dump/s3.t.000000000.sql.gz" -d -diff "$DBPATH/mybucket/expect/s3.t.000000000.sql" "$DBPATH/mybucket/dump/s3.t.000000000.sql" +gzip "${HOST_DIR}/compress/s3.t.000000000.sql.gz" -d +diff "${HOST_DIR}/local/s3.t.000000000.sql" "${HOST_DIR}/compress/s3.t.000000000.sql" run_sql "drop database if exists \`$DB_NAME\`;" From a13a18af20470f12634e71a68224e3439a83cb11 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Tue, 23 Aug 2022 14:00:20 +0800 Subject: [PATCH 34/42] planner: fix outer join reorder will push down its outer join condition (#37245) (#37295) close pingcap/tidb#37238 --- cmd/explaintest/r/explain_complex.result | 25 ++++++++ cmd/explaintest/t/explain_complex.test | 16 +++++ planner/core/rule_join_reorder.go | 78 ++++++++++++++++++------ 3 files changed, 102 insertions(+), 17 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index 43f7c0f915f14..b68393c99fcfe 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -261,3 +261,28 @@ Sort 1.00 root test.org_department.left_value └─TableReader(Probe) 9.99 root data:Selection └─Selection 9.99 cop[tikv] eq(test.org_employee_position.status, 1000), not(isnull(test.org_employee_position.position_id)) └─TableFullScan 10000.00 cop[tikv] table:ep keep order:false, stats:pseudo +create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2)); +create table test.Tab_B (id int primary key,name varchar(20)); +create table test.Tab_C (id int primary key,name varchar(20),amt decimal(11,2)); +insert into test.Tab_A values(2,2,2,'A01','01',112,111); +insert into test.Tab_A values(4,4,4,'A02','02',112,111); +insert into test.Tab_B values(2,'B01'); +insert into test.Tab_B values(4,'B02'); +insert into test.Tab_C values(2,'C01',22); +insert into test.Tab_C values(4,'C01',5); +explain select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112; +id estRows task access object operator info +Projection_8 15.62 root test.tab_a.name, test.tab_b.name, test.tab_a.amt, test.tab_c.amt, ifnull(test.tab_c.amt, 0)->Column#13 +└─IndexJoin_13 15.62 root left outer join, inner:TableReader_10, outer key:test.tab_a.cid, inner key:test.tab_c.id, equal cond:eq(test.tab_a.cid, test.tab_c.id), left cond:eq(test.tab_a.type, "01") + ├─IndexJoin_24(Build) 12.50 root left outer join, inner:TableReader_21, outer key:test.tab_a.bid, inner key:test.tab_b.id, equal cond:eq(test.tab_a.bid, test.tab_b.id) + │ ├─TableReader_33(Build) 10.00 root data:Selection_32 + │ │ └─Selection_32 10.00 cop[tikv] eq(test.tab_a.num, 112) + │ │ └─TableFullScan_31 10000.00 cop[tikv] table:Tab_A keep order:false, stats:pseudo + │ └─TableReader_21(Probe) 1.00 root data:TableRangeScan_20 + │ └─TableRangeScan_20 1.00 cop[tikv] table:Tab_B range: decided by [test.tab_a.bid], keep order:false, stats:pseudo + └─TableReader_10(Probe) 1.00 root data:TableRangeScan_9 + └─TableRangeScan_9 1.00 cop[tikv] table:Tab_C range: decided by [test.tab_a.cid], keep order:false, stats:pseudo +select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112; +AAA BBB Aamt Bamt IFNULL(Tab_C.amt, 0) +A01 B01 111.00 22.00 22.00 +A02 B02 111.00 NULL 0 diff --git a/cmd/explaintest/t/explain_complex.test b/cmd/explaintest/t/explain_complex.test index 2447db8f2d53c..39a2baa357f1a 100644 --- a/cmd/explaintest/t/explain_complex.test +++ b/cmd/explaintest/t/explain_complex.test @@ -174,3 +174,19 @@ CREATE TABLE org_position ( ) ENGINE=InnoDB DEFAULT CHARSET=utf8; explain format = 'brief' SELECT d.id, d.ctx, d.name, d.left_value, d.right_value, d.depth, d.leader_id, d.status, d.created_on, d.updated_on FROM org_department AS d LEFT JOIN org_position AS p ON p.department_id = d.id AND p.status = 1000 LEFT JOIN org_employee_position AS ep ON ep.position_id = p.id AND ep.status = 1000 WHERE (d.ctx = 1 AND (ep.user_id = 62 OR d.id = 20 OR d.id = 20) AND d.status = 1000) GROUP BY d.id ORDER BY d.left_value; + +create table test.Tab_A (id int primary key,bid int,cid int,name varchar(20),type varchar(20),num int,amt decimal(11,2)); +create table test.Tab_B (id int primary key,name varchar(20)); +create table test.Tab_C (id int primary key,name varchar(20),amt decimal(11,2)); + +insert into test.Tab_A values(2,2,2,'A01','01',112,111); +insert into test.Tab_A values(4,4,4,'A02','02',112,111); +insert into test.Tab_B values(2,'B01'); +insert into test.Tab_B values(4,'B02'); +insert into test.Tab_C values(2,'C01',22); +insert into test.Tab_C values(4,'C01',5); + +explain select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112; + +select Tab_A.name AAA,Tab_B.name BBB,Tab_A.amt Aamt, Tab_C.amt Bamt,IFNULL(Tab_C.amt, 0) FROM Tab_A left join Tab_B on Tab_A.bid=Tab_B.id left join Tab_C on Tab_A.cid=Tab_C.id and Tab_A.type='01' where Tab_A.num=112; + diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index e1d0ce1a7537e..460ce11909a4d 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -34,7 +34,7 @@ import ( // For example: "InnerJoin(InnerJoin(a, b), LeftJoin(c, d))" // results in a join group {a, b, c, d}. func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression.ScalarFunction, - otherConds []expression.Expression, joinTypes []JoinType, hintInfo []*tableHintInfo, hasOuterJoin bool) { + otherConds []expression.Expression, joinTypes []*joinTypeWithExtMsg, hintInfo []*tableHintInfo, hasOuterJoin bool) { join, isJoin := p.(*LogicalJoin) if isJoin && join.preferJoinOrder { // When there is a leading hint, the hint may not take effect for other reasons. @@ -129,12 +129,26 @@ func extractJoinGroup(p LogicalPlan) (group []LogicalPlan, eqEdges []*expression } eqEdges = append(eqEdges, join.EqualConditions...) - otherConds = append(otherConds, join.OtherConditions...) - otherConds = append(otherConds, join.LeftConditions...) - otherConds = append(otherConds, join.RightConditions...) - for range join.EqualConditions { - joinTypes = append(joinTypes, join.JoinType) + tmpOtherConds := make(expression.CNFExprs, 0, len(join.OtherConditions)+len(join.LeftConditions)+len(join.RightConditions)) + tmpOtherConds = append(tmpOtherConds, join.OtherConditions...) + tmpOtherConds = append(tmpOtherConds, join.LeftConditions...) + tmpOtherConds = append(tmpOtherConds, join.RightConditions...) + if join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin { + for range join.EqualConditions { + abType := &joinTypeWithExtMsg{JoinType: join.JoinType} + // outer join's other condition should be bound with the connecting edge. + // although we bind the outer condition to **anyone** of the join type, it will be extracted **only once** when make a new join. + abType.outerBindCondition = tmpOtherConds + joinTypes = append(joinTypes, abType) + } + } else { + for range join.EqualConditions { + abType := &joinTypeWithExtMsg{JoinType: join.JoinType} + joinTypes = append(joinTypes, abType) + } + otherConds = append(otherConds, tmpOtherConds...) } + return group, eqEdges, otherConds, joinTypes, hintInfo, hasOuterJoin } @@ -146,6 +160,11 @@ type jrNode struct { cumCost float64 } +type joinTypeWithExtMsg struct { + JoinType + outerBindCondition []expression.Expression +} + func (s *joinReOrderSolver) optimize(ctx context.Context, p LogicalPlan, opt *logicalOptimizeOp) (LogicalPlan, error) { tracer := &joinReorderTrace{cost: map[string]float64{}, opt: opt} tracer.traceJoinReorder(p) @@ -172,7 +191,7 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP // Not support outer join reorder when using the DP algorithm isSupportDP := true for _, joinType := range joinTypes { - if joinType != InnerJoin { + if joinType.JoinType != InnerJoin { isSupportDP = false break } @@ -294,7 +313,7 @@ type baseSingleGroupJoinOrderSolver struct { curJoinGroup []*jrNode otherConds []expression.Expression eqEdges []*expression.ScalarFunction - joinTypes []JoinType + joinTypes []*joinTypeWithExtMsg leadingJoinGroup LogicalPlan } @@ -322,7 +341,7 @@ func (s *baseSingleGroupJoinOrderSolver) generateLeadingJoinGroup(curJoinGroup [ leadingJoinGroup = leadingJoinGroup[1:] for len(leadingJoinGroup) > 0 { var usedEdges []*expression.ScalarFunction - var joinType JoinType + var joinType *joinTypeWithExtMsg leadingJoin, leadingJoinGroup[0], usedEdges, joinType = s.checkConnection(leadingJoin, leadingJoinGroup[0]) leadingJoin, s.otherConds = s.makeJoin(leadingJoin, leadingJoinGroup[0], usedEdges, joinType) leadingJoinGroup = leadingJoinGroup[1:] @@ -359,8 +378,8 @@ func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode LogicalPlan) } // checkConnection used to check whether two nodes have equal conditions or not. -func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan LogicalPlan) (leftNode, rightNode LogicalPlan, usedEdges []*expression.ScalarFunction, joinType JoinType) { - joinType = InnerJoin +func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan LogicalPlan) (leftNode, rightNode LogicalPlan, usedEdges []*expression.ScalarFunction, joinType *joinTypeWithExtMsg) { + joinType = &joinTypeWithExtMsg{JoinType: InnerJoin} leftNode, rightNode = leftPlan, rightPlan for idx, edge := range s.eqEdges { lCol := edge.GetArgs()[0].(*expression.Column) @@ -370,7 +389,7 @@ func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan Log usedEdges = append(usedEdges, edge) } else if rightPlan.Schema().Contains(lCol) && leftPlan.Schema().Contains(rCol) { joinType = s.joinTypes[idx] - if joinType != InnerJoin { + if joinType.JoinType != InnerJoin { rightNode, leftNode = leftPlan, rightPlan usedEdges = append(usedEdges, edge) } else { @@ -383,12 +402,19 @@ func (s *baseSingleGroupJoinOrderSolver) checkConnection(leftPlan, rightPlan Log } // makeJoin build join tree for the nodes which have equal conditions to connect them. -func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPlan, eqEdges []*expression.ScalarFunction, joinType JoinType) (LogicalPlan, []expression.Expression) { +func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPlan, eqEdges []*expression.ScalarFunction, joinType *joinTypeWithExtMsg) (LogicalPlan, []expression.Expression) { remainOtherConds := make([]expression.Expression, len(s.otherConds)) copy(remainOtherConds, s.otherConds) - var otherConds []expression.Expression - var leftConds []expression.Expression - var rightConds []expression.Expression + var ( + otherConds []expression.Expression + leftConds []expression.Expression + rightConds []expression.Expression + + // for outer bind conditions + obOtherConds []expression.Expression + obLeftConds []expression.Expression + obRightConds []expression.Expression + ) mergedSchema := expression.MergeSchema(leftPlan.Schema(), rightPlan.Schema()) remainOtherConds, leftConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { @@ -400,7 +426,25 @@ func (s *baseSingleGroupJoinOrderSolver) makeJoin(leftPlan, rightPlan LogicalPla remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { return expression.ExprFromSchema(expr, mergedSchema) }) - return s.newJoinWithEdges(leftPlan, rightPlan, eqEdges, otherConds, leftConds, rightConds, joinType), remainOtherConds + if len(joinType.outerBindCondition) > 0 { + remainOBOtherConds := make([]expression.Expression, len(joinType.outerBindCondition)) + copy(remainOBOtherConds, joinType.outerBindCondition) + remainOBOtherConds, obLeftConds = expression.FilterOutInPlace(remainOBOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, leftPlan.Schema()) && !expression.ExprFromSchema(expr, rightPlan.Schema()) + }) + remainOBOtherConds, obRightConds = expression.FilterOutInPlace(remainOBOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, rightPlan.Schema()) && !expression.ExprFromSchema(expr, leftPlan.Schema()) + }) + // _ here make the linter happy. + _, obOtherConds = expression.FilterOutInPlace(remainOBOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + // case like: (A * B) left outer join C on (A.a = C.a && B.b > 0) will remain B.b > 0 in remainOBOtherConds (while this case + // has been forbidden by: filters of the outer join is related with multiple leaves of the outer join side in #34603) + // so noway here we got remainOBOtherConds remained. + } + return s.newJoinWithEdges(leftPlan, rightPlan, eqEdges, + append(otherConds, obOtherConds...), append(leftConds, obLeftConds...), append(rightConds, obRightConds...), joinType.JoinType), remainOtherConds } // makeBushyJoin build bushy tree for the nodes which have no equal condition to connect them. From 1f7e2ce63ddddd5e772e7f4efa17492d2017ed26 Mon Sep 17 00:00:00 2001 From: Yilin Chen Date: Tue, 23 Aug 2022 15:08:20 +0800 Subject: [PATCH 35/42] *: update client-go to improve availability on tikv failure (#37256) close pingcap/tidb#34906 --- executor/main_test.go | 3 +++ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/executor/main_test.go b/executor/main_test.go index 50f5ac5bc98d3..60511ea1d6455 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -18,6 +18,7 @@ import ( "fmt" "testing" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/testkit" @@ -53,6 +54,8 @@ func TestMain(m *testing.M) { conf.Experimental.AllowsExpressionIndex = true }) tikv.EnableFailpoints() + failpoint.Enable("tikvclient/injectLiveness", `return("reachable")`) + defer failpoint.Disable("tikvclient/injectLiveness") opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), diff --git a/go.mod b/go.mod index 1ab50a370ee6b..c47a8ba917b65 100644 --- a/go.mod +++ b/go.mod @@ -63,7 +63,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.1-0.20220531092439-efebaeb9fe53 + github.com/tikv/client-go/v2 v2.0.1-0.20220614073425-1693f8c71524 github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 github.com/twmb/murmur3 v1.1.3 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 86e3125807621..48646ce77dde5 100644 --- a/go.sum +++ b/go.sum @@ -755,8 +755,8 @@ github.com/stretchr/testify v1.7.2-0.20220504104629-106ec21d14df/go.mod h1:6Fq8o github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tikv/client-go/v2 v2.0.1-0.20220531092439-efebaeb9fe53 h1:zalDvjC3IhixTcqU1HQYJQtTI3npketLDprwLw1eGqI= -github.com/tikv/client-go/v2 v2.0.1-0.20220531092439-efebaeb9fe53/go.mod h1:VTlli8fRRpcpISj9I2IqroQmcAFfaTyBquiRhofOcDs= +github.com/tikv/client-go/v2 v2.0.1-0.20220614073425-1693f8c71524 h1:nFTlY55m4gaRML/H44qw2Vg0KpkTISrsHJl5shzfm/g= +github.com/tikv/client-go/v2 v2.0.1-0.20220614073425-1693f8c71524/go.mod h1:VTlli8fRRpcpISj9I2IqroQmcAFfaTyBquiRhofOcDs= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710 h1:jxgmKOscXSjaFEKQGRyY5qOpK8hLqxs2irb/uDJMtwk= github.com/tikv/pd/client v0.0.0-20220307081149-841fa61e9710/go.mod h1:AtvppPwkiyUgQlR1W9qSqfTB+OsOIu19jDCOxOsPkmU= github.com/tklauser/go-sysconf v0.3.9 h1:JeUVdAOWhhxVcU6Eqr/ATFHgXk/mmiItdKeJPev3vTo= From 08ed961a0cec4426b69eee253ceb68eee7bf81b1 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 24 Aug 2022 00:22:19 +0800 Subject: [PATCH 36/42] session: handle nil store for plugins (#37321) (#37322) close pingcap/tidb#37319 --- session/tidb.go | 12 ++++++++++-- session/tidb_test.go | 8 ++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/session/tidb.go b/session/tidb.go index c7d93f71d7081..2e531b24e3adb 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -49,11 +49,19 @@ type domainMap struct { } func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) { - key := store.UUID() - dm.mu.Lock() defer dm.mu.Unlock() + if store == nil { + for _, d := range dm.domains { + // return available domain if any + return d, nil + } + return nil, errors.New("can not find available domain for a nil store") + } + + key := store.UUID() + d = dm.domains[key] if d != nil { return diff --git a/session/tidb_test.go b/session/tidb_test.go index 70831a8f64d89..e2808080b7c1c 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -26,6 +26,14 @@ import ( "github.com/stretchr/testify/require" ) +func TestDomapHandleNil(t *testing.T) { + // this is required for enterprise plugins + // ref: https://github.com/pingcap/tidb/issues/37319 + require.NotPanics(t, func() { + _, _ = domap.Get(nil) + }) +} + func TestSysSessionPoolGoroutineLeak(t *testing.T) { store, dom := createStoreAndBootstrap(t) defer func() { require.NoError(t, store.Close()) }() From 0b00c97b56bb80c9b9a71ed0f0f22d392037ae52 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 24 Aug 2022 13:22:21 +0800 Subject: [PATCH 37/42] br: fix broken integration test (#36411) (#37325) --- br/tests/br_other/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/br/tests/br_other/run.sh b/br/tests/br_other/run.sh index 5e3a54053b6e3..91fa7c3d795b1 100644 --- a/br/tests/br_other/run.sh +++ b/br/tests/br_other/run.sh @@ -52,7 +52,7 @@ run_br debug backupmeta validate -s "local://$TEST_DIR/$DB" --offset 100 run_br validate checksum -s "local://$TEST_DIR/$DB" # Test validate checksum -for sst in $TEST_DIR/$DB/*.sst; do +for sst in $TEST_DIR/$DB/*/*.sst; do echo "corrupted!" >> $sst echo "$sst corrupted!" break From 36dd73f6764e0f27d284745bfa69e384e1b44210 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 24 Aug 2022 15:40:21 +0800 Subject: [PATCH 38/42] executor: handle NULL values properly when `LOAD DATA INFILE` (#36124) (#36700) close pingcap/tidb#34358 --- executor/load_data.go | 26 +++++++++++++++++++------- executor/set.go | 3 +-- executor/write_test.go | 20 ++++++++++++++++++++ sessionctx/variable/session.go | 7 +++++++ 4 files changed, 47 insertions(+), 9 deletions(-) diff --git a/executor/load_data.go b/executor/load_data.go index fdaf58222f463..e02ef3ce5d42d 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -623,12 +623,21 @@ func (e *LoadDataInfo) SetMessage() { func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []field) []types.Datum { row := make([]types.Datum, 0, len(e.insertColumns)) + sessionVars := e.Ctx.GetSessionVars() + setVar := func(name string, col *field) { + sessionVars.UsersLock.Lock() + if col == nil || col.isNull() { + sessionVars.UnsetUserVar(name) + } else { + sessionVars.SetUserVar(name, string(col.str), mysql.DefaultCollationName) + } + sessionVars.UsersLock.Unlock() + } for i := 0; i < len(e.FieldMappings); i++ { if i >= len(cols) { if e.FieldMappings[i].Column == nil { - sessionVars := e.Ctx.GetSessionVars() - sessionVars.SetUserVar(e.FieldMappings[i].UserVar.Name, "", mysql.DefaultCollationName) + setVar(e.FieldMappings[i].UserVar.Name, nil) continue } @@ -643,14 +652,11 @@ func (e *LoadDataInfo) colsToRow(ctx context.Context, cols []field) []types.Datu } if e.FieldMappings[i].Column == nil { - sessionVars := e.Ctx.GetSessionVars() - sessionVars.SetUserVar(e.FieldMappings[i].UserVar.Name, string(cols[i].str), mysql.DefaultCollationName) + setVar(e.FieldMappings[i].UserVar.Name, &cols[i]) continue } - // The field with only "\N" in it is handled as NULL in the csv file. - // See http://dev.mysql.com/doc/refman/5.7/en/load-data.html - if cols[i].maybeNull && string(cols[i].str) == "N" { + if cols[i].isNull() { row = append(row, types.NewDatum(nil)) continue } @@ -695,6 +701,12 @@ type field struct { enclosed bool } +func (f *field) isNull() bool { + // The field with only "\N" in it is handled as NULL in the csv file. + // See http://dev.mysql.com/doc/refman/5.7/en/load-data.html + return f.maybeNull && len(f.str) == 1 && f.str[0] == 'N' +} + type fieldWriter struct { pos int ReadBuf []byte diff --git a/executor/set.go b/executor/set.go index a79055abb5dbe..1e82ab84cd8c7 100644 --- a/executor/set.go +++ b/executor/set.go @@ -89,8 +89,7 @@ func (e *SetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { } sessionVars.UsersLock.Lock() if value.IsNull() { - delete(sessionVars.Users, name) - delete(sessionVars.UserVarTypes, name) + sessionVars.UnsetUserVar(name) } else { sessionVars.Users[name] = value sessionVars.UserVarTypes[name] = v.Expr.GetType() diff --git a/executor/write_test.go b/executor/write_test.go index 000afd1800e17..3315a73097fbf 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1975,6 +1975,26 @@ func TestIssue18681(t *testing.T) { require.Equal(t, uint16(0), sc.WarningCount()) } +func TestIssue34358(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + ctx := tk.Session().(sessionctx.Context) + defer ctx.SetValue(executor.LoadDataVarKey, nil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists load_data_test") + tk.MustExec("create table load_data_test (a varchar(10), b varchar(10))") + + tk.MustExec("load data local infile '/tmp/nonexistence.csv' into table load_data_test ( @v1, @v2 ) set a = @v1, b = @v2") + ld, ok := ctx.Value(executor.LoadDataVarKey).(*executor.LoadDataInfo) + require.True(t, ok) + require.NotNil(t, ld) + checkCases([]testCase{ + {nil, []byte("\\N\n"), []string{"|"}, nil, "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0"}, + }, ld, t, tk, ctx, "select * from load_data_test", "delete from load_data_test") +} + func TestLoadData(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" store, clean := testkit.CreateMockStore(t) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index aa7fa72ad0ab3..bcc8820d9aa00 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1474,6 +1474,13 @@ func (s *SessionVars) SetUserVar(varName string, svalue string, collation string } } +// UnsetUserVar unset an user defined variable by name. +func (s *SessionVars) UnsetUserVar(varName string) { + varName = strings.ToLower(varName) + delete(s.Users, varName) + delete(s.UserVarTypes, varName) +} + // SetLastInsertID saves the last insert id to the session context. // TODO: we may store the result for last_insert_id sys var later. func (s *SessionVars) SetLastInsertID(insertID uint64) { From a5fd7d13dd234ca7dec2b8fd65831b8d7a2f5c39 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Wed, 24 Aug 2022 17:40:21 +0800 Subject: [PATCH 39/42] lightning: Fix panic when downstream table schema has changed (#36805) (#37233) close pingcap/tidb#34163 --- br/pkg/lightning/backend/local/local.go | 5 +++ br/pkg/lightning/common/errors.go | 27 ++++++++-------- errors.toml | 5 +++ types/mydecimal.go | 21 ++++++++++--- types/mydecimal_test.go | 41 ++++++++++++++++++++----- util/codec/codec.go | 7 ++++- util/codec/decimal.go | 8 +++-- 7 files changed, 87 insertions(+), 27 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index dd8a98600bbde..d16f1860a7fce 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -58,6 +58,7 @@ import ( "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/mathutil" tikverror "github.com/tikv/client-go/v2/error" @@ -1482,6 +1483,10 @@ func (local *local) ResolveDuplicateRows(ctx context.Context, tbl table.Table, t if err == nil { return nil } + if types.ErrBadNumber.Equal(err) { + logger.Warn("delete duplicate rows encounter error", log.ShortError(err)) + return common.ErrResolveDuplicateRows.Wrap(err).GenWithStackByArgs(tableName) + } if log.IsContextCanceledError(err) { return err } diff --git a/br/pkg/lightning/common/errors.go b/br/pkg/lightning/common/errors.go index b647cc9e64ad4..6a4f559b77430 100644 --- a/br/pkg/lightning/common/errors.go +++ b/br/pkg/lightning/common/errors.go @@ -81,19 +81,20 @@ var ( ErrKVIngestFailed = errors.Normalize("ingest tikv failed", errors.RFCCodeText("Lightning:KV:ErrKVIngestFailed")) ErrKVRaftProposalDropped = errors.Normalize("raft proposal dropped", errors.RFCCodeText("Lightning:KV:ErrKVRaftProposalDropped")) - ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend")) - ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile")) - ErrOpenDuplicateDB = errors.Normalize("open duplicate db error", errors.RFCCodeText("Lightning:Restore:ErrOpenDuplicateDB")) - ErrSchemaNotExists = errors.Normalize("table `%s`.`%s` schema not found", errors.RFCCodeText("Lightning:Restore:ErrSchemaNotExists")) - ErrInvalidSchemaStmt = errors.Normalize("invalid schema statement: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidSchemaStmt")) - ErrCreateSchema = errors.Normalize("create schema failed, table: %s, stmt: %s", errors.RFCCodeText("Lightning:Restore:ErrCreateSchema")) - ErrUnknownColumns = errors.Normalize("unknown columns in header (%s) for table %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownColumns")) - ErrChecksumMismatch = errors.Normalize("checksum mismatched remote vs local => (checksum: %d vs %d) (total_kvs: %d vs %d) (total_bytes:%d vs %d)", errors.RFCCodeText("Lighting:Restore:ErrChecksumMismatch")) - ErrRestoreTable = errors.Normalize("restore table %s failed", errors.RFCCodeText("Lightning:Restore:ErrRestoreTable")) - ErrEncodeKV = errors.Normalize("encode kv error in file %s at offset %d", errors.RFCCodeText("Lightning:Restore:ErrEncodeKV")) - ErrAllocTableRowIDs = errors.Normalize("allocate table row id error", errors.RFCCodeText("Lightning:Restore:ErrAllocTableRowIDs")) - ErrInvalidMetaStatus = errors.Normalize("invalid meta status: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidMetaStatus")) - ErrTableIsChecksuming = errors.Normalize("table '%s' is checksuming", errors.RFCCodeText("Lightning:Restore:ErrTableIsChecksuming")) + ErrUnknownBackend = errors.Normalize("unknown backend %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownBackend")) + ErrCheckLocalFile = errors.Normalize("cannot find local file for table: %s engineDir: %s", errors.RFCCodeText("Lightning:Restore:ErrCheckLocalFile")) + ErrOpenDuplicateDB = errors.Normalize("open duplicate db error", errors.RFCCodeText("Lightning:Restore:ErrOpenDuplicateDB")) + ErrSchemaNotExists = errors.Normalize("table `%s`.`%s` schema not found", errors.RFCCodeText("Lightning:Restore:ErrSchemaNotExists")) + ErrInvalidSchemaStmt = errors.Normalize("invalid schema statement: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidSchemaStmt")) + ErrCreateSchema = errors.Normalize("create schema failed, table: %s, stmt: %s", errors.RFCCodeText("Lightning:Restore:ErrCreateSchema")) + ErrUnknownColumns = errors.Normalize("unknown columns in header (%s) for table %s", errors.RFCCodeText("Lightning:Restore:ErrUnknownColumns")) + ErrChecksumMismatch = errors.Normalize("checksum mismatched remote vs local => (checksum: %d vs %d) (total_kvs: %d vs %d) (total_bytes:%d vs %d)", errors.RFCCodeText("Lighting:Restore:ErrChecksumMismatch")) + ErrRestoreTable = errors.Normalize("restore table %s failed", errors.RFCCodeText("Lightning:Restore:ErrRestoreTable")) + ErrEncodeKV = errors.Normalize("encode kv error in file %s at offset %d", errors.RFCCodeText("Lightning:Restore:ErrEncodeKV")) + ErrAllocTableRowIDs = errors.Normalize("allocate table row id error", errors.RFCCodeText("Lightning:Restore:ErrAllocTableRowIDs")) + ErrInvalidMetaStatus = errors.Normalize("invalid meta status: '%s'", errors.RFCCodeText("Lightning:Restore:ErrInvalidMetaStatus")) + ErrTableIsChecksuming = errors.Normalize("table '%s' is checksuming", errors.RFCCodeText("Lightning:Restore:ErrTableIsChecksuming")) + ErrResolveDuplicateRows = errors.Normalize("resolve duplicate rows error on table '%s'", errors.RFCCodeText("Lightning:Restore:ErrResolveDuplicateRows")) ) type withStack struct { diff --git a/errors.toml b/errors.toml index 08920b7fe5725..83c228ef02704 100755 --- a/errors.toml +++ b/errors.toml @@ -501,6 +501,11 @@ error = ''' open duplicate db error ''' +["Lightning:Restore:ErrResolveDuplicateRows"] +error = ''' +resolve duplicate rows error on table '%s' +''' + ["Lightning:Restore:ErrRestoreTable"] error = ''' restore table %s failed diff --git a/types/mydecimal.go b/types/mydecimal.go index c1e09808c6d20..d1f158fa2a1bb 100644 --- a/types/mydecimal.go +++ b/types/mydecimal.go @@ -1401,7 +1401,13 @@ func (d *MyDecimal) FromBin(bin []byte, precision, frac int) (binSize int, err e if bin[binIdx]&0x80 > 0 { mask = 0 } - binSize = DecimalBinSize(precision, frac) + binSize, err = DecimalBinSize(precision, frac) + if err != nil { + return 0, err + } + if binSize < 0 || binSize > 40 { + return 0, ErrBadNumber + } dCopy := make([]byte, 40) dCopy = dCopy[:binSize] copy(dCopy, bin) @@ -1477,13 +1483,16 @@ func (d *MyDecimal) FromBin(bin []byte, precision, frac int) (binSize int, err e } // DecimalBinSize returns the size of array to hold a binary representation of a decimal. -func DecimalBinSize(precision, frac int) int { +func DecimalBinSize(precision, frac int) (int, error) { digitsInt := precision - frac wordsInt := digitsInt / digitsPerWord wordsFrac := frac / digitsPerWord xInt := digitsInt - wordsInt*digitsPerWord xFrac := frac - wordsFrac*digitsPerWord - return wordsInt*wordSize + dig2bytes[xInt] + wordsFrac*wordSize + dig2bytes[xFrac] + if xInt < 0 || xInt >= len(dig2bytes) || xFrac < 0 || xFrac >= len(dig2bytes) { + return 0, ErrBadNumber + } + return wordsInt*wordSize + dig2bytes[xInt] + wordsFrac*wordSize + dig2bytes[xFrac], nil } func readWord(b []byte, size int) int32 { @@ -2340,7 +2349,11 @@ func DecimalPeak(b []byte) (int, error) { } precision := int(b[0]) frac := int(b[1]) - return DecimalBinSize(precision, frac) + 2, nil + binSize, err := DecimalBinSize(precision, frac) + if err != nil { + return 0, err + } + return binSize + 2, nil } // NewDecFromInt creates a MyDecimal from int. diff --git a/types/mydecimal_test.go b/types/mydecimal_test.go index 07e8df28c28b9..a375bd400cef0 100644 --- a/types/mydecimal_test.go +++ b/types/mydecimal_test.go @@ -475,17 +475,44 @@ func TestToBinFromBin(t *testing.T) { var dec MyDecimal dec.FromInt(1) errTests := []struct { - prec int - frac int + prec int + frac int + ToBinErr error + FromBinErr error }{ - {82, 1}, - {-1, 1}, - {10, 31}, - {10, -1}, + {82, 1, ErrBadNumber, ErrTruncated}, + {-1, 1, ErrBadNumber, ErrBadNumber}, + {10, 31, ErrBadNumber, ErrBadNumber}, + {10, -1, ErrBadNumber, ErrBadNumber}, } for _, tt := range errTests { _, err := dec.ToBin(tt.prec, tt.frac) - require.True(t, ErrBadNumber.Equal(err)) + require.Equal(t, tt.ToBinErr, err) + err = dec.FromString([]byte{'0'}) + require.NoError(t, err) + buf, err := dec.ToBin(1, 0) + require.NoError(t, err) + _, err = dec.FromBin(buf, tt.prec, tt.frac) + require.Equal(t, tt.FromBinErr, err) + } +} + +func TestDecimalBinSize(t *testing.T) { + type tcase struct { + precision int + frac int + output int + err error + } + tests := []tcase{ + {3, 1, 2, nil}, + {-1, 0, 0, ErrBadNumber}, + {3, 5, 0, ErrBadNumber}, + } + for _, tt := range tests { + binSize, err := DecimalBinSize(tt.precision, tt.frac) + require.Equal(t, tt.output, binSize) + require.Equal(t, tt.err, err) } } diff --git a/util/codec/codec.go b/util/codec/codec.go index 9c5099a4c4ae9..74e8a510ae1fd 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -157,7 +157,12 @@ func EstimateValueSize(sc *stmtctx.StatementContext, val types.Datum) (int, erro case types.KindString, types.KindBytes: l = valueSizeOfBytes(val.GetBytes()) case types.KindMysqlDecimal: - l = valueSizeOfDecimal(val.GetMysqlDecimal(), val.Length(), val.Frac()) + 1 + var err error + l, err = valueSizeOfDecimal(val.GetMysqlDecimal(), val.Length(), val.Frac()) + if err != nil { + return 0, err + } + l = l + 1 case types.KindMysqlEnum: l = valueSizeOfUnsignedInt(val.GetMysqlEnum().Value) case types.KindMysqlSet: diff --git a/util/codec/decimal.go b/util/codec/decimal.go index ac204674f8546..0454febac0a61 100644 --- a/util/codec/decimal.go +++ b/util/codec/decimal.go @@ -34,11 +34,15 @@ func EncodeDecimal(b []byte, dec *types.MyDecimal, precision, frac int) ([]byte, return b, errors.Trace(err) } -func valueSizeOfDecimal(dec *types.MyDecimal, precision, frac int) int { +func valueSizeOfDecimal(dec *types.MyDecimal, precision, frac int) (int, error) { if precision == 0 { precision, frac = dec.PrecisionAndFrac() } - return types.DecimalBinSize(precision, frac) + 2 + binSize, err := types.DecimalBinSize(precision, frac) + if err != nil { + return 0, err + } + return binSize + 2, nil } // DecodeDecimal decodes bytes to decimal. From 5263a0abda61f102122735049fd0dfadc7b7f8b2 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 25 Aug 2022 14:30:22 +0800 Subject: [PATCH 40/42] planner: forbid load data with empty field terminator (#36500) (#36686) close pingcap/tidb#33298 --- executor/write_test.go | 18 ++++++++++++++++++ planner/core/planbuilder.go | 4 ++++ 2 files changed, 22 insertions(+) diff --git a/executor/write_test.go b/executor/write_test.go index 3315a73097fbf..ae61dba931d2d 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -1995,6 +1995,24 @@ func TestIssue34358(t *testing.T) { }, ld, t, tk, ctx, "select * from load_data_test", "delete from load_data_test") } +func TestIssue33298(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + ctx := tk.Session().(sessionctx.Context) + defer ctx.SetValue(executor.LoadDataVarKey, nil) + + tk.MustExec("use test") + tk.MustExec("drop table if exists load_data_test") + tk.MustExec("create table load_data_test (a varchar(10), b varchar(10))") + + // According to https://dev.mysql.com/doc/refman/8.0/en/load-data.html , fixed-row format should be used when fields + // terminated by '' and enclosed by ''. However, tidb doesn't support it yet and empty terminator leads to infinite + // loop in `indexOfTerminator` (see https://github.com/pingcap/tidb/issues/33298). + require.Error(t, tk.ExecToErr("load data local infile '/tmp/nonexistence.csv' into table load_data_test fields terminated by ''")) + require.Error(t, tk.ExecToErr("load data local infile '/tmp/nonexistence.csv' into table load_data_test fields terminated by '' enclosed by ''")) +} + func TestLoadData(t *testing.T) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" store, clean := testkit.CreateMockStore(t) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e758bcbea3293..aab670ad79612 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -3891,6 +3891,10 @@ func (b *PlanBuilder) buildSelectPlanOfInsert(ctx context.Context, insert *ast.I } func (b *PlanBuilder) buildLoadData(ctx context.Context, ld *ast.LoadDataStmt) (Plan, error) { + // quick fix for https://github.com/pingcap/tidb/issues/33298 + if ld.FieldsInfo != nil && len(ld.FieldsInfo.Terminated) == 0 { + return nil, ErrNotSupportedYet.GenWithStackByArgs("load data with empty field terminator") + } p := LoadData{ IsLocal: ld.IsLocal, OnDuplicate: ld.OnDuplicate, From 368597ea94adb08675f565e8ab7f2209d39e14f2 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sun, 4 Sep 2022 12:08:25 +0800 Subject: [PATCH 41/42] planner: fix update plan's projection elimination will cause column resolution error. (#37582) (#37597) close pingcap/tidb#37568 --- planner/core/cbo_test.go | 5 +++++ planner/core/logical_plan_builder.go | 5 ++++- planner/core/optimizer.go | 2 ++ planner/core/rule_eliminate_projection.go | 7 +++++++ sessionctx/stmtctx/stmtctx.go | 2 ++ 5 files changed, 20 insertions(+), 1 deletion(-) diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index a0aa419b95b86..4071cf51f71a3 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -741,6 +741,11 @@ func TestUpdateProjEliminate(t *testing.T) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("explain update t t1, (select distinct b from t) t2 set t1.b = t2.b") + + tk.MustExec("drop table if exists tb1, tb2") + tk.MustExec("create table tb1(a int, b int, primary key(a))") + tk.MustExec("create table tb2 (a int, b int, c int, d datetime, primary key(c),key idx_u(a));") + tk.MustExec("update tb1 set tb1.b=(select tb2.b from tb2 where tb2.a=tb1.a order by c desc limit 1);") } func TestTiFlashCostModel(t *testing.T) { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d6e2fa9d92b0c..37a1aa7c4b1ec 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5149,7 +5149,7 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( }.Init(b.ctx) updt.names = p.OutputNames() // We cannot apply projection elimination when building the subplan, because - // columns in orderedList cannot be resolved. + // columns in orderedList cannot be resolved. (^flagEliminateProjection should also be applied in postOptimize) updt.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag&^flagEliminateProjection, p) if err != nil { return nil, err @@ -5392,6 +5392,9 @@ func (b *PlanBuilder) buildUpdateLists(ctx context.Context, tableList []*ast.Tab allAssignmentsAreConstant = false } p = np + if col, ok := newExpr.(*expression.Column); ok { + b.ctx.GetSessionVars().StmtCtx.ColRefFromUpdatePlan = append(b.ctx.GetSessionVars().StmtCtx.ColRefFromUpdatePlan, col.UniqueID) + } newList = append(newList, &expression.Assignment{Col: col, ColName: name.ColName, Expr: newExpr}) dbName := name.DBName.L // To solve issue#10028, we need to get database name by the table alias name. diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index f816ec0e8dc24..14ad6dd9732e5 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -364,6 +364,8 @@ func mergeContinuousSelections(p PhysicalPlan) { } func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { + // some cases from update optimize will require avoiding projection elimination. + // see comments ahead of call of DoOptimize in function of buildUpdate(). plan = eliminatePhysicalProjection(plan) plan = InjectExtraProjection(plan) mergeContinuousSelections(plan) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 70a55f7e4e339..68a20d10f59b9 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -72,6 +72,13 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { if p.Schema().Len() != child.Schema().Len() { return false } + for _, ref := range p.ctx.GetSessionVars().StmtCtx.ColRefFromUpdatePlan { + for _, one := range p.Schema().Columns { + if ref == one.UniqueID { + return false + } + } + } for i, expr := range p.Exprs { col, ok := expr.(*expression.Column) if !ok || !col.Equal(nil, child.Schema().Columns[i]) { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 5aed2e633f3b1..b069e8ab05416 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -253,6 +253,8 @@ type StatementContext struct { IsSQLRegistered atomic2.Bool // IsSQLAndPlanRegistered uses to indicate whether the SQL and plan has been registered for TopSQL. IsSQLAndPlanRegistered atomic2.Bool + // ColRefFromPlan mark the column ref used by assignment in update statement. + ColRefFromUpdatePlan []int64 } // StmtHints are SessionVars related sql hints. From 696f08efadec351b8d9c9aaef48a5cbc5f5d53b2 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Mon, 5 Sep 2022 11:00:55 +0800 Subject: [PATCH 42/42] executor: enable TiDBEnableWindowFunction in SystemVar (#37077) (#37138) close pingcap/tidb#35916, close pingcap/tidb#37045 --- executor/show_test.go | 11 +++++++++++ session/session.go | 4 ++++ 2 files changed, 15 insertions(+) diff --git a/executor/show_test.go b/executor/show_test.go index 34f69a0bf07c3..6b6e92361377e 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -1832,6 +1832,17 @@ func TestShowBindingCacheStatus(t *testing.T) { "1 1 198 Bytes 250 Bytes")) } +func TestShowViewWithWindowFunction(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE `test1` (`id` int(0) NOT NULL,`num` int(0) DEFAULT NULL) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;") + tk.MustExec("create or replace view test1_v as(select id,row_number() over (partition by num) from test1);") + tk.MustQuery("desc test1_v;").Check(testkit.Rows("id int(0) NO ", "row_number() over (partition by num) bigint(21) YES ")) +} + func TestShowDatabasesLike(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/session/session.go b/session/session.go index a43873e26a2fd..f9d923d7042bd 100644 --- a/session/session.go +++ b/session/session.go @@ -1180,6 +1180,10 @@ func createSessionFunc(store kv.Storage) pools.Factory { if err != nil { return nil, errors.Trace(err) } + err = se.sessionVars.SetSystemVar(variable.TiDBEnableWindowFunction, variable.BoolToOnOff(variable.DefEnableWindowFunction)) + if err != nil { + return nil, errors.Trace(err) + } se.sessionVars.CommonGlobalLoaded = true se.sessionVars.InRestrictedSQL = true // Internal session uses default format to prevent memory leak problem.