diff --git a/br/pkg/pdutil/pd.go b/br/pkg/pdutil/pd.go index 599eefe30afb5..16b8ce3dbac58 100644 --- a/br/pkg/pdutil/pd.go +++ b/br/pkg/pdutil/pd.go @@ -238,7 +238,9 @@ func NewPdController( pdClient, err := pd.NewClientWithContext( ctx, addrs, securityOption, pd.WithGRPCDialOptions(maxCallMsgSize...), - pd.WithCustomTimeoutOption(10*time.Second), + // If the time too short, we may scatter a region many times, because + // the interface `ScatterRegions` may time out. + pd.WithCustomTimeoutOption(60*time.Second), pd.WithMaxErrorRetry(3), ) if err != nil { diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index be6a41112aa98..8b26a5e9378a4 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -390,6 +390,9 @@ func (rs *RegionSplitter) ScatterRegions(ctx context.Context, newRegions []*Regi }) return nil } + if err != nil { + log.Warn("scatter region meet error", logutil.ShortError(err)) + } return err // the retry is for the temporary network errors during sending request. }, &exponentialBackoffer{attempt: 3, baseBackoff: 500 * time.Millisecond}) diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index 7c2d5549d6608..045ce62d42389 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -125,6 +125,9 @@ func (c *pdClient) ScatterRegions(ctx context.Context, regionInfo []*RegionInfo) regionsID := make([]uint64, 0, len(regionInfo)) for _, v := range regionInfo { regionsID = append(regionsID, v.Region.Id) + log.Debug("scattering regions", logutil.Key("start", v.Region.StartKey), + logutil.Key("end", v.Region.EndKey), + zap.Uint64("id", v.Region.Id)) } resp, err := c.client.ScatterRegions(ctx, regionsID) if err != nil { diff --git a/br/pkg/storage/parse_test.go b/br/pkg/storage/parse_test.go index 90f72d0778407..20e441f86aecf 100644 --- a/br/pkg/storage/parse_test.go +++ b/br/pkg/storage/parse_test.go @@ -194,3 +194,49 @@ func TestFormatBackendURL(t *testing.T) { }) require.Equal(t, "azure://bucket/some%20prefix/", backendURL.String()) } + +func TestParseRawURL(t *testing.T) { + cases := []struct { + url string + schema string + host string + path string + accessKey string + secretAccessKey string + }{ + { + url: `s3://bucket/prefix/path?access-key=NXN7IPIOSAAKDEEOLMAF&secret-access-key=nREY/7DtPaIbYKrKlEEMMF/ExCiJEX=XMLPUANw`, + schema: "s3", + host: "bucket", + path: "/prefix/path", + accessKey: "NXN7IPIOSAAKDEEOLMAF", // fake ak/sk + secretAccessKey: "nREY/7DtPaIbYKrKlEEMMF/ExCiJEX=XMLPUANw", // w/o "+" + }, + { + url: `s3://bucket/prefix/path?access-key=NXN7IPIOSAAKDEEOLMAF&secret-access-key=nREY/7Dt+PaIbYKrKlEEMMF/ExCiJEX=XMLPUANw`, + schema: "s3", + host: "bucket", + path: "/prefix/path", + accessKey: "NXN7IPIOSAAKDEEOLMAF", // fake ak/sk + secretAccessKey: "nREY/7Dt+PaIbYKrKlEEMMF/ExCiJEX=XMLPUANw", // with "+" + }, + } + + for _, c := range cases { + storageRawURL := c.url + storageURL, err := ParseRawURL(storageRawURL) + require.NoError(t, err) + + require.Equal(t, c.schema, storageURL.Scheme) + require.Equal(t, c.host, storageURL.Host) + require.Equal(t, c.path, storageURL.Path) + + require.Equal(t, 1, len(storageURL.Query()["access-key"])) + accessKey := storageURL.Query()["access-key"][0] + require.Equal(t, c.accessKey, accessKey) + + require.Equal(t, 1, len(storageURL.Query()["secret-access-key"])) + secretAccessKey := storageURL.Query()["secret-access-key"][0] + require.Equal(t, c.secretAccessKey, secretAccessKey) + } +} diff --git a/executor/brie.go b/executor/brie.go index 1e14d970acbb0..9eeb4e48cbdd5 100644 --- a/executor/brie.go +++ b/executor/brie.go @@ -17,7 +17,6 @@ package executor import ( "bytes" "context" - "net/url" "strings" "sync" "sync/atomic" @@ -225,7 +224,7 @@ func (b *executorBuilder) buildBRIE(s *ast.BRIEStmt, schema *expression.Schema) }, } - storageURL, err := url.Parse(s.Storage) + storageURL, err := storage.ParseRawURL(s.Storage) if err != nil { b.err = errors.Annotate(err, "invalid destination URL") return nil diff --git a/executor/grant.go b/executor/grant.go index 387ced7e69f40..2aabd3c4aca77 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -105,8 +105,9 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.Chunk) error { return err } } - // Note the table name compare is case sensitive here. - if tbl != nil && tbl.Meta().Name.String() != e.Level.TableName { + // Note the table name compare is not case sensitive here. + // In TiDB, system variable lower_case_table_names = 2 which means name comparisons are not case-sensitive. + if tbl != nil && tbl.Meta().Name.L != strings.ToLower(e.Level.TableName) { return infoschema.ErrTableNotExists.GenWithStackByArgs(dbName, e.Level.TableName) } if len(e.Level.DBName) > 0 { diff --git a/executor/grant_test.go b/executor/grant_test.go index 05d64b0d7b698..0519c778fe1bd 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -449,9 +449,9 @@ func TestGrantOnNonExistTable(t *testing.T) { require.True(t, terror.ErrorEqual(err, infoschema.ErrTableNotExists)) tk.MustExec("create table if not exists xx (id int)") - // Case sensitive - _, err = tk.Exec("grant Select,Insert on XX to 'genius'") - require.True(t, terror.ErrorEqual(err, infoschema.ErrTableNotExists)) + // Case insensitive, differ from MySQL default behaviour. + // In TiDB, system variable lower_case_table_names = 2, which means compare table name using lower case. + tk.MustExec("grant Select,Insert on XX to 'genius'") _, err = tk.Exec("grant Select,Insert on xx to 'genius'") require.NoError(t, err) @@ -606,3 +606,23 @@ func TestNonExistTableIllegalGrant(t *testing.T) { // Column level, not existing table, illegal privilege tk.MustGetErrCode("grant create temporary tables (NotExistsCol) on NotExistsD29302.NotExistsT29302 to u29302;", mysql.ErrWrongUsage) } + +func TestIssue34610(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("DROP DATABASE IF EXISTS d1;") + tk.MustExec("CREATE DATABASE d1;") + tk.MustExec("USE d1;") + tk.MustExec("CREATE USER user_1@localhost;") + defer func() { + tk.MustExec("DROP DATABASE d1;") + tk.MustExec("DROP USER user_1@localhost;") + }() + + tk.MustExec("CREATE TABLE T1(f1 INT);") + tk.MustGetErrCode("CREATE TABLE t1(f1 INT);", mysql.ErrTableExists) + tk.MustExec("GRANT SELECT ON T1 to user_1@localhost;") + tk.MustExec("GRANT SELECT ON t1 to user_1@localhost;") +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index de0104ab8613e..d216de6accc17 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -826,10 +826,10 @@ func (b *PlanBuilder) buildJoin(ctx context.Context, joinNode *ast.Join) (Logica // on the "USING" clause. // // According to the standard SQL, columns are ordered in the following way: -// 1. coalesced common columns of "leftPlan" and "rightPlan", in the order they -// appears in "leftPlan". -// 2. the rest columns in "leftPlan", in the order they appears in "leftPlan". -// 3. the rest columns in "rightPlan", in the order they appears in "rightPlan". +// 1. coalesced common columns of "leftPlan" and "rightPlan", in the order they +// appears in "leftPlan". +// 2. the rest columns in "leftPlan", in the order they appears in "leftPlan". +// 3. the rest columns in "rightPlan", in the order they appears in "rightPlan". func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error { filter := make(map[string]bool, len(join.Using)) for _, col := range join.Using { @@ -850,9 +850,10 @@ func (b *PlanBuilder) buildUsingClause(p *LogicalJoin, leftPlan, rightPlan Logic // buildNaturalJoin builds natural join output schema. It finds out all the common columns // then using the same mechanism as buildUsingClause to eliminate redundant columns and build join conditions. // According to standard SQL, producing this display order: -// All the common columns -// Every column in the first (left) table that is not a common column -// Every column in the second (right) table that is not a common column +// +// All the common columns +// Every column in the first (left) table that is not a common column +// Every column in the second (right) table that is not a common column func (b *PlanBuilder) buildNaturalJoin(p *LogicalJoin, leftPlan, rightPlan LogicalPlan, join *ast.Join) error { err := b.coalesceCommonColumns(p, leftPlan, rightPlan, join.Tp, nil) if err != nil { @@ -1491,7 +1492,11 @@ func unionJoinFieldType(a, b *types.FieldType) *types.FieldType { } resultTp.SetDecimal(mathutil.Max(a.GetDecimal(), b.GetDecimal())) // `flen - decimal` is the fraction before '.' - resultTp.SetFlen(mathutil.Max(a.GetFlen()-a.GetDecimal(), b.GetFlen()-b.GetDecimal()) + resultTp.GetDecimal()) + if a.GetFlen() == -1 || b.GetFlen() == -1 { + resultTp.SetFlen(-1) + } else { + resultTp.SetFlen(mathutil.Max(a.GetFlen()-a.GetDecimal(), b.GetFlen()-b.GetDecimal()) + resultTp.GetDecimal()) + } types.TryToFixFlenOfDatetime(resultTp) if resultTp.EvalType() != types.ETInt && (a.EvalType() == types.ETInt || b.EvalType() == types.ETInt) && resultTp.GetFlen() < mysql.MaxIntWidth { resultTp.SetFlen(mysql.MaxIntWidth) @@ -1502,6 +1507,9 @@ func unionJoinFieldType(a, b *types.FieldType) *types.FieldType { // Set the flen of the union column using the max flen in children. func (b *PlanBuilder) setUnionFlen(resultTp *types.FieldType, cols []expression.Expression) { + if resultTp.GetFlen() == -1 { + return + } isBinary := resultTp.GetCharset() == charset.CharsetBin for i := 0; i < len(cols); i++ { childTp := cols[i].GetType() @@ -1805,7 +1813,9 @@ func (b *PlanBuilder) buildUnion(ctx context.Context, selects []LogicalPlan, aft // divideUnionSelectPlans resolves union's select stmts to logical plans. // and divide result plans into "union-distinct" and "union-all" parts. // divide rule ref: -// https://dev.mysql.com/doc/refman/5.7/en/union.html +// +// https://dev.mysql.com/doc/refman/5.7/en/union.html +// // "Mixed UNION types are treated such that a DISTINCT union overrides any ALL union to its left." func (b *PlanBuilder) divideUnionSelectPlans(ctx context.Context, selects []LogicalPlan, setOprTypes []*ast.SetOprType) (distinctSelects []LogicalPlan, allSelects []LogicalPlan, err error) { firstUnionAllIdx := 0 diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 05cf1e802b9f3..0dbcbaf2ad91a 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -2931,3 +2931,17 @@ func TestPlanCacheWithRCWhenInfoSchemaChange(t *testing.T) { tk2.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 0")) tk2.MustQuery("select @@last_plan_from_cache").Check(testkit.Rows("0")) } + +func TestIssue37901(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 t4`) + tk.MustExec(`create table t4 (a date)`) + tk.MustExec(`prepare st1 from "insert into t4(a) select dt from (select ? as dt from dual union all select sysdate() ) a";`) + tk.MustExec(`set @t='2022-01-01 00:00:00.000000'`) + tk.MustExec(`execute st1 using @t`) + tk.MustQuery(`select count(*) from t4`).Check(testkit.Rows("2")) +}