diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 8d3000b5a2cc7..e3eda9fd389e1 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -21,6 +21,7 @@ import ( "fmt" "os" "strconv" + "strings" "testing" "time" @@ -2453,6 +2454,37 @@ func (s *testSerialSuite) TestOptimizeOnlyOnce(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/checkOptimizeCountOne"), IsNil) } +func (s *testSerialSuite) TestIssue26377(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_global_temporary_table = true") + tk.MustExec("set @@tidb_enable_noop_functions=1;") + tk.MustExec("drop table if exists t1,tmp1") + tk.MustExec("create table t1(a int(11))") + tk.MustExec("create global temporary table tmp1(a int(11), key idx_a(a)) on commit delete rows;") + tk.MustExec("create temporary table tmp2(a int(11), key idx_a(a));") + + queries := []string{ + "create global binding for select * from t1 inner join tmp1 on t1.a=tmp1.a using select * from t1 inner join tmp1 on t1.a=tmp1.a;", + "create global binding for select * from t1 where t1.a in (select a from tmp1) using select * from t1 where t1.a in (select a from tmp1 use index (idx_a));", + "create global binding for select a from t1 union select a from tmp1 using select a from t1 union select a from tmp1 use index (idx_a);", + "create global binding for select t1.a, (select a from tmp1 where tmp1.a=1) as t2 from t1 using select t1.a, (select a from tmp1 where tmp1.a=1) as t2 from t1;", + "create global binding for select * from (select * from tmp1) using select * from (select * from tmp1);", + } + genLocalTemporarySQL := func(sql string) string { + return strings.Replace(sql, "tmp1", "tmp2", -1) + } + for _, query := range queries { + localSQL := genLocalTemporarySQL(query) + queries = append(queries, localSQL) + } + + for _, q := range queries { + tk.MustGetErrCode(q, errno.ErrOptOnTemporaryTable) + } +} + func (s *testSuite) TestCaptureFilter(c *C) { tk := testkit.NewTestKit(c, s.store) s.cleanBindingEnv(tk) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 1703d57086012..4a78cd6b4c125 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -2755,8 +2755,8 @@ func (w *Writer) appendRowsUnsorted(ctx context.Context, kvs []common.KvPair) er l := len(w.writeBatch) cnt := w.batchCount var lastKey []byte - if len(w.writeBatch) > 0 { - lastKey = w.writeBatch[len(w.writeBatch)-1].Key + if cnt > 0 { + lastKey = w.writeBatch[cnt-1].Key } for _, pair := range kvs { if w.isWriteBatchSorted && bytes.Compare(lastKey, pair.Key) > 0 { diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 5a63a70f838ea..79fbc83cf92dd 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -349,7 +349,7 @@ func testLocalWriter(c *C, needSort bool, partitialSort bool) { f.wg.Add(1) go f.ingestSSTLoop() sorted := needSort && !partitialSort - w, err := openLocalWriter(context.Background(), &backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1<<20) + w, err := openLocalWriter(context.Background(), &backend.LocalWriterConfig{IsKVSorted: sorted}, f, 1024) c.Assert(err, IsNil) ctx := context.Background() diff --git a/br/pkg/lightning/mydump/parquet_parser_test.go b/br/pkg/lightning/mydump/parquet_parser_test.go index 9c68f80c0d2ae..41fea53a79068 100644 --- a/br/pkg/lightning/mydump/parquet_parser_test.go +++ b/br/pkg/lightning/mydump/parquet_parser_test.go @@ -5,7 +5,6 @@ import ( "io" "path/filepath" "strconv" - "time" . "github.com/pingcap/check" "github.com/pingcap/tidb/br/pkg/storage" @@ -82,13 +81,6 @@ func (s testParquetParserSuite) TestParquetParser(c *C) { } func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { - // those deprecated TIME/TIMESTAMP types depend on the local timezone! - prevTZ := time.Local - time.Local = time.FixedZone("UTC+8", 8*60*60) - defer func() { - time.Local = prevTZ - }() - type Test struct { Date int32 `parquet:"name=date, type=DATE"` TimeMillis int32 `parquet:"name=timemillis, type=TIME_MILLIS"` @@ -114,7 +106,7 @@ func (s testParquetParserSuite) TestParquetVariousTypes(c *C) { v := &Test{ Date: 18564, // 2020-10-29 - TimeMillis: 62775123, // 17:26:15.123 (note all time are in UTC+8!) + TimeMillis: 62775123, // 17:26:15.123 TimeMicros: 62775123456, // 17:26:15.123 TimestampMillis: 1603963672356, // 2020-10-29T09:27:52.356Z TimestampMicros: 1603963672356956, // 2020-10-29T09:27:52.356956Z diff --git a/br/pkg/restore/backoff.go b/br/pkg/restore/backoff.go index 1f39ff78d17e7..80e8c91edbf34 100644 --- a/br/pkg/restore/backoff.go +++ b/br/pkg/restore/backoff.go @@ -56,7 +56,8 @@ func (bo *importerBackoffer) NextBackoff(err error) time.Duration { bo.delayTime = 2 * bo.delayTime bo.attempt-- } else { - switch errors.Cause(err) { // nolint:errorlint + e := errors.Cause(err) + switch e { // nolint:errorlint case berrors.ErrKVEpochNotMatch, berrors.ErrKVDownloadFailed, berrors.ErrKVIngestFailed: bo.delayTime = 2 * bo.delayTime bo.attempt-- @@ -65,7 +66,7 @@ func (bo *importerBackoffer) NextBackoff(err error) time.Duration { bo.delayTime = 0 bo.attempt = 0 default: - switch status.Code(err) { + switch status.Code(e) { case codes.Unavailable, codes.Aborted: bo.delayTime = 2 * bo.delayTime bo.attempt-- diff --git a/br/pkg/restore/import.go b/br/pkg/restore/import.go index 9cd7f0a2fa8e0..83ec6c59571d2 100644 --- a/br/pkg/restore/import.go +++ b/br/pkg/restore/import.go @@ -317,6 +317,10 @@ func (importer *FileImporter) Import( log.Debug("failpoint restore-storage-error injected.", zap.String("msg", msg)) e = errors.Annotate(e, msg) }) + failpoint.Inject("restore-gRPC-error", func(_ failpoint.Value) { + log.Warn("the connection to TiKV has been cut by a neko, meow :3") + e = status.Error(codes.Unavailable, "the connection to TiKV has been cut by a neko, meow :3") + }) if e != nil { remainFiles = remainFiles[i:] return errors.Trace(e) diff --git a/br/pkg/restore/split.go b/br/pkg/restore/split.go index 35244b9b1bd60..77db3f8f94e49 100644 --- a/br/pkg/restore/split.go +++ b/br/pkg/restore/split.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" "github.com/pingcap/tidb/br/pkg/logutil" + "github.com/pingcap/tidb/br/pkg/redact" "github.com/pingcap/tidb/br/pkg/rtree" "github.com/pingcap/tidb/br/pkg/utils" "github.com/tikv/pd/pkg/codec" @@ -294,6 +295,33 @@ func (rs *RegionSplitter) ScatterRegions(ctx context.Context, newRegions []*Regi } } +func checkRegionConsistency(startKey, endKey []byte, regions []*RegionInfo) error { + // current pd can't guarantee the consistency of returned regions + if len(regions) == 0 { + return errors.Annotatef(berrors.ErrPDBatchScanRegion, "scan region return empty result, startKey: %s, endkey: %s", + redact.Key(startKey), redact.Key(endKey)) + } + + if bytes.Compare(regions[0].Region.StartKey, startKey) > 0 { + return errors.Annotatef(berrors.ErrPDBatchScanRegion, "first region's startKey > startKey, startKey: %s, regionStartKey: %s", + redact.Key(startKey), redact.Key(regions[0].Region.StartKey)) + } else if len(regions[len(regions)-1].Region.EndKey) != 0 && bytes.Compare(regions[len(regions)-1].Region.EndKey, endKey) < 0 { + return errors.Annotatef(berrors.ErrPDBatchScanRegion, "last region's endKey < startKey, startKey: %s, regionStartKey: %s", + redact.Key(endKey), redact.Key(regions[len(regions)-1].Region.EndKey)) + } + + cur := regions[0] + for _, r := range regions[1:] { + if !bytes.Equal(cur.Region.EndKey, r.Region.StartKey) { + return errors.Annotatef(berrors.ErrPDBatchScanRegion, "region endKey not equal to next region startKey, endKey: %s, startKey: %s", + redact.Key(cur.Region.EndKey), redact.Key(r.Region.StartKey)) + } + cur = r + } + + return nil +} + // PaginateScanRegion scan regions with a limit pagination and // return all regions at once. // It reduces max gRPC message size. @@ -305,50 +333,61 @@ func PaginateScanRegion( hex.EncodeToString(startKey), hex.EncodeToString(endKey)) } - regions := []*RegionInfo{} - scanStartKey := startKey - for { - batch, err := client.ScanRegions(ctx, scanStartKey, endKey, limit) - if err != nil { - return nil, errors.Trace(err) - } - regions = append(regions, batch...) - if len(batch) < limit { - // No more region - break + var regions []*RegionInfo + err := utils.WithRetry(ctx, func() error { + regions = []*RegionInfo{} + scanStartKey := startKey + for { + batch, err := client.ScanRegions(ctx, scanStartKey, endKey, limit) + if err != nil { + return errors.Trace(err) + } + regions = append(regions, batch...) + if len(batch) < limit { + // No more region + break + } + scanStartKey = batch[len(batch)-1].Region.GetEndKey() + if len(scanStartKey) == 0 || + (len(endKey) > 0 && bytes.Compare(scanStartKey, endKey) >= 0) { + // All key space have scanned + break + } } - scanStartKey = batch[len(batch)-1].Region.GetEndKey() - if len(scanStartKey) == 0 || - (len(endKey) > 0 && bytes.Compare(scanStartKey, endKey) >= 0) { - // All key space have scanned - break + if err := checkRegionConsistency(startKey, endKey, regions); err != nil { + log.Warn("failed to scan region, retrying", logutil.ShortError(err)) + return err } - } + return nil + }, newScanRegionBackoffer()) - // current pd can't guarantee the consistency of returned regions - if len(regions) == 0 { - return nil, errors.Annotatef(berrors.ErrPDBatchScanRegion, "scan region return empty result, startKey: %s, endkey: %s", - hex.EncodeToString(startKey), hex.EncodeToString(endKey)) - } + return regions, err +} - if bytes.Compare(regions[0].Region.StartKey, startKey) > 0 { - return nil, errors.Annotatef(berrors.ErrPDBatchScanRegion, "first region's startKey > startKey, startKey: %s, regionStartKey: %s", - hex.EncodeToString(startKey), hex.EncodeToString(regions[0].Region.StartKey)) - } else if len(regions[len(regions)-1].Region.EndKey) != 0 && bytes.Compare(regions[len(regions)-1].Region.EndKey, endKey) < 0 { - return nil, errors.Annotatef(berrors.ErrPDBatchScanRegion, "last region's endKey < startKey, startKey: %s, regionStartKey: %s", - hex.EncodeToString(endKey), hex.EncodeToString(regions[len(regions)-1].Region.EndKey)) +type scanRegionBackoffer struct { + attempt int +} + +func newScanRegionBackoffer() utils.Backoffer { + return &scanRegionBackoffer{ + attempt: 3, } +} - cur := regions[0] - for _, r := range regions[1:] { - if !bytes.Equal(cur.Region.EndKey, r.Region.StartKey) { - return nil, errors.Annotatef(berrors.ErrPDBatchScanRegion, "region endKey not equal to next region startKey, endKey: %s, startKey: %s", - hex.EncodeToString(cur.Region.EndKey), hex.EncodeToString(r.Region.StartKey)) - } - cur = r +// NextBackoff returns a duration to wait before retrying again +func (b *scanRegionBackoffer) NextBackoff(err error) time.Duration { + if berrors.ErrPDBatchScanRegion.Equal(err) { + // 500ms * 3 could be enough for splitting remain regions in the hole. + b.attempt-- + return 500 * time.Millisecond } + b.attempt = 0 + return 0 +} - return regions, nil +// Attempt returns the remain attempt times +func (b *scanRegionBackoffer) Attempt() int { + return b.attempt } // getSplitKeys checks if the regions should be split by the new prefix of the rewrites rule and the end key of diff --git a/br/pkg/storage/s3.go b/br/pkg/storage/s3.go index d07fe6b30ca23..8fe1b6b3ae24c 100644 --- a/br/pkg/storage/s3.go +++ b/br/pkg/storage/s3.go @@ -26,6 +26,7 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/spf13/pflag" "go.uber.org/zap" ) @@ -43,6 +44,7 @@ const ( maxRetries = 7 // max number of retries when meets error maxErrorRetries = 3 + ec2MetaAddress = "169.254.169.254" // the maximum number of byte to read for seek. maxSkipOffsetByRead = 1 << 16 // 64KB @@ -736,6 +738,29 @@ type retryerWithLog struct { client.DefaultRetryer } +func isDeadlineExceedError(err error) bool { + // TODO find a better way. + // Known challenges: + // + // If we want to unwrap the r.Error: + // 1. the err should be an awserr.Error (let it be awsErr) + // 2. awsErr.OrigErr() should be an *url.Error (let it be urlErr). + // 3. urlErr.Err should be a http.httpError (which is private). + // + // If we want to reterive the error from the request context: + // The error of context in the HTTPRequest (i.e. r.HTTPRequest.Context().Err() ) is nil. + return strings.Contains(err.Error(), "context deadline exceeded") +} + +func (rl retryerWithLog) ShouldRetry(r *request.Request) bool { + if isDeadlineExceedError(r.Error) && r.HTTPRequest.URL.Host == ec2MetaAddress { + // fast fail for unreachable linklocal address in EC2 containers. + log.Warn("failed to get EC2 metadata. skipping.", logutil.ShortError(r.Error)) + return false + } + return rl.DefaultRetryer.ShouldRetry(r) +} + func (rl retryerWithLog) RetryRules(r *request.Request) time.Duration { backoffTime := rl.DefaultRetryer.RetryRules(r) if backoffTime > 0 { diff --git a/br/tests/br_full/run.sh b/br/tests/br_full/run.sh index 4fab79686352e..0d15794788b26 100755 --- a/br/tests/br_full/run.sh +++ b/br/tests/br_full/run.sh @@ -70,7 +70,7 @@ for ct in limit lz4 zstd; do # restore full echo "restore with $ct backup start..." - export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-storage-error=1*return(\"connection refused\")" + export GO_FAILPOINTS="github.com/pingcap/tidb/br/pkg/restore/restore-storage-error=1*return(\"connection refused\");github.com/pingcap/tidb/br/pkg/restore/restore-gRPC-error=1*return(true)" run_br restore full -s "local://$TEST_DIR/$DB-$ct" --pd $PD_ADDR --ratelimit 1024 export GO_FAILPOINTS="" diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index 78d631987f1d0..0cb7623929c11 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -496,6 +496,19 @@ desc format = 'brief' select b from t; id estRows task access object operator info IndexReader 10000.00 root index:IndexFullScan └─IndexFullScan 10000.00 cop[tikv] table:t, index:b(b) keep order:false, stats:pseudo +create table t01(a varchar(20)); +insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙"); +alter table t01 add index eidx ((concat_ws('expression_index', a, 'test'))); +select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) not like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test')); +a +insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙"); +select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test')); +a +齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙 +齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙 +drop table if exists t1; +create table t1(a char, b varchar(20), c char, d varchar(20)); +alter table t1 add index eidx ((export_set(3, a, c, ',', 5))); create table t02 (a varchar(20)); insert into t02 values ('a'), ('b'), ('c'); select * from t02 where lower(a) < 'c'; diff --git a/cmd/explaintest/t/explain_generate_column_substitute.test b/cmd/explaintest/t/explain_generate_column_substitute.test index be461f9a37184..c6060780d62f5 100644 --- a/cmd/explaintest/t/explain_generate_column_substitute.test +++ b/cmd/explaintest/t/explain_generate_column_substitute.test @@ -217,6 +217,16 @@ create table t(a int, b int as (a+1), key((a+1)), key(b)); desc format = 'brief' select a+1 from t; desc format = 'brief' select b from t; +create table t01(a varchar(20)); +insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙"); +alter table t01 add index eidx ((concat_ws('expression_index', a, 'test'))); +select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) not like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test')); +insert into t01 values ("齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙"); +select * from t01 use index (eidx) where (concat_ws('expression_index', a, 'test')) like (concat_ws('expression_index', "齆斮聒蚆髙锐潊贩哨啅捸爖斥圱犳飁綴纜牖蚙", 'test')); + +drop table if exists t1; +create table t1(a char, b varchar(20), c char, d varchar(20)); +alter table t1 add index eidx ((export_set(3, a, c, ',', 5))); create table t02 (a varchar(20)); insert into t02 values ('a'), ('b'), ('c'); select * from t02 where lower(a) < 'c'; diff --git a/config/config.go b/config/config.go index 90abd091501f0..b4d69a7541c99 100644 --- a/config/config.go +++ b/config/config.go @@ -548,6 +548,8 @@ type IsolationRead struct { type Experimental struct { // Whether enable global kill. EnableGlobalKill bool `toml:"enable-global-kill" json:"-"` + // Whether enable charset feature. + EnableNewCharset bool `toml:"enable-new-charset" json:"-"` } var defTiKVCfg = tikvcfg.DefaultConfig() @@ -670,6 +672,7 @@ var defaultConf = Config{ }, Experimental: Experimental{ EnableGlobalKill: false, + EnableNewCharset: false, }, EnableCollectExecutionInfo: true, EnableTelemetry: true, diff --git a/config/config_test.go b/config/config_test.go index e56d43cdda09e..3c6715c44d065 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -286,6 +286,7 @@ deadlock-history-collect-retryable = true require.Equal(t, uint64(30), conf.StoresRefreshInterval) require.Equal(t, uint(123), conf.PessimisticTxn.DeadlockHistoryCapacity) require.True(t, conf.PessimisticTxn.DeadlockHistoryCollectRetryable) + require.False(t, conf.Experimental.EnableNewCharset) _, err = f.WriteString(` [log.file] diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index e2e0e3dfeba8a..bb045d6ff80b0 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2951,10 +2951,6 @@ func (s *testIntegrationSuite3) TestCreateTemporaryTable(c *C) { ON DUPLICATE KEY UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) tk.MustExec(updateSafePoint) - - // Considering snapshot, local temporary table is always visible. - tk.MustExec("set @@tidb_snapshot = '2016-01-01 15:04:05.999999'") - tk.MustExec("select * from overlap") } func (s *testIntegrationSuite3) TestAvoidCreateViewOnLocalTemporaryTable(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index f671d8f219fea..4e8260e51c17f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -6797,13 +6797,17 @@ func (s *testSerialDBSuite) TestAddIndexFailOnCaseWhenCanExit(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/MockCaseWhenParseFailure"), IsNil) }() tk := testkit.NewTestKit(c, s.store) + originalVal := variable.GetDDLErrorCountLimit() + tk.MustExec("set @@global.tidb_ddl_error_count_limit = 1") + defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %d", originalVal)) + tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int)") tk.MustExec("insert into t values(1, 1)") _, err := tk.Exec("alter table t add index idx(b)") c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:-1]DDL job rollback, error msg: job.ErrCount:512, mock unknown type: ast.whenClause.") + c.Assert(err.Error(), Equals, "[ddl:-1]DDL job rollback, error msg: job.ErrCount:1, mock unknown type: ast.whenClause.") tk.MustExec("drop table if exists t") } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7fd8e63461f58..127d2f9c78c18 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2587,6 +2587,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast needsOverwriteCols := needToOverwriteColCharset(spec.Options) err = d.AlterTableCharsetAndCollate(sctx, ident, toCharset, toCollate, needsOverwriteCols) handledCharsetOrCollate = true + case ast.TableOptionEngine: default: err = errUnsupportedAlterTableOption } diff --git a/ddl/error.go b/ddl/error.go index 1c2f37ce5751f..42ba073829e72 100644 --- a/ddl/error.go +++ b/ddl/error.go @@ -295,4 +295,6 @@ var ( errFunctionalIndexOnJSONOrGeometryFunction = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnJSONOrGeometryFunction) // errDependentByFunctionalIndex returns when the dropped column depends by expression index. errDependentByFunctionalIndex = dbterror.ClassDDL.NewStd(mysql.ErrDependentByFunctionalIndex) + // errFunctionalIndexOnBlob when the expression of expression index returns blob or text. + errFunctionalIndexOnBlob = dbterror.ClassDDL.NewStd(mysql.ErrFunctionalIndexOnBlob) ) diff --git a/ddl/index.go b/ddl/index.go index 244e177f2fb31..27cfb7ba1e8e7 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -140,6 +140,9 @@ func checkIndexColumn(col *model.ColumnInfo, indexColumnLen int) error { // Length must be specified and non-zero for BLOB and TEXT column indexes. if types.IsTypeBlob(col.FieldType.Tp) { if indexColumnLen == types.UnspecifiedLength { + if col.Hidden { + return errFunctionalIndexOnBlob + } return errors.Trace(errBlobKeyWithoutLength.GenWithStackByArgs(col.Name.O)) } if indexColumnLen == types.ErrorLength { diff --git a/ddl/label/rule.go b/ddl/label/rule.go index fe2d5035686fd..b1529f598f5e6 100644 --- a/ddl/label/rule.go +++ b/ddl/label/rule.go @@ -93,18 +93,20 @@ func (r *Rule) Reset(id int64, dbName, tableName string, partName ...string) *Ru } var hasDBKey, hasTableKey, hasPartitionKey bool - for _, label := range r.Labels { - if label.Key == dbKey { - label.Value = dbName + for i := range r.Labels { + switch r.Labels[i].Key { + case dbKey: + r.Labels[i].Value = dbName hasDBKey = true - } - if label.Key == tableKey { - label.Value = tableName + case tableKey: + r.Labels[i].Value = tableName hasTableKey = true - } - if isPartition && label.Key == partitionKey { - label.Value = partName[0] - hasPartitionKey = true + case partitionKey: + if isPartition { + r.Labels[i].Value = partName[0] + hasPartitionKey = true + } + default: } } diff --git a/ddl/label/rule_test.go b/ddl/label/rule_test.go index 21b291425fe95..ea498d4a9e210 100644 --- a/ddl/label/rule_test.go +++ b/ddl/label/rule_test.go @@ -32,7 +32,7 @@ func (t *testRuleSuite) TestApplyAttributesSpec(c *C) { c.Assert(rule.Labels[1].Key, Equals, "attr2") } -func (t *testRuleSuite) TestResetID(c *C) { +func (t *testRuleSuite) TestReset(c *C) { rule := NewRule() rule.Reset(1, "db1", "t1") c.Assert(rule.ID, Equals, "schema/db1/t1") @@ -46,4 +46,14 @@ func (t *testRuleSuite) TestResetID(c *C) { r1 := rule.Clone() c.Assert(rule, DeepEquals, r1) + + r2 := rule.Reset(2, "db2", "t2", "p2") + c.Assert(r2.ID, Equals, "schema/db2/t2/p2") + c.Assert(r2.Labels, HasLen, 3) + c.Assert(rule.Labels[0].Value, Equals, "db2") + c.Assert(rule.Labels[1].Value, Equals, "t2") + c.Assert(rule.Labels[2].Value, Equals, "p2") + r = r2.Rule.(map[string]string) + c.Assert(r["start_key"], Equals, "7480000000000000ff025f720000000000fa") + c.Assert(r["end_key"], Equals, "7480000000000000ff035f720000000000fa") } diff --git a/ddl/table.go b/ddl/table.go index 4766b583affab..36ea2a6174b00 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -518,6 +518,45 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return 0, errors.Wrapf(err, "failed to notify PD the placement rules") } + tableRuleID := fmt.Sprintf(label.TableIDFormat, label.IDPrefix, job.SchemaName, tblInfo.Name.L) + ids := []string{tableRuleID} + if tblInfo.GetPartitionInfo() != nil { + for _, def := range tblInfo.GetPartitionInfo().Definitions { + ids = append(ids, fmt.Sprintf(label.PartitionIDFormat, label.IDPrefix, job.SchemaName, tblInfo.Name.L, def.Name.L)) + } + } + + oldRules, err := infosync.GetLabelRules(context.TODO(), ids) + if err != nil { + job.State = model.JobStateCancelled + return 0, errors.Wrapf(err, "failed to get PD the label rule") + } + + var newRules []*label.Rule + for _, r := range oldRules { + if r.ID == tableRuleID { + newRules = append(newRules, r.Clone().Reset(newTableID, job.SchemaName, tblInfo.Name.L)) + } + } + + if tblInfo.GetPartitionInfo() != nil { + for _, r := range oldRules { + for _, def := range tblInfo.GetPartitionInfo().Definitions { + if r.ID == fmt.Sprintf(label.PartitionIDFormat, label.IDPrefix, job.SchemaName, tblInfo.Name.L, def.Name.L) { + newRules = append(newRules, r.Clone().Reset(def.ID, job.SchemaName, tblInfo.Name.L, def.Name.L)) + } + } + } + } + + // update the key range with same rule id. + patch := label.NewRulePatch(newRules, nil) + err = infosync.UpdateLabelRules(context.TODO(), patch) + if err != nil { + job.State = model.JobStateCancelled + return 0, errors.Wrapf(err, "failed to notify PD the label rules") + } + // Clear the tiflash replica available status. if tblInfo.TiFlashReplica != nil { tblInfo.TiFlashReplica.AvailablePartitionIDs = nil diff --git a/docs/MAINTAINERS.md b/docs/MAINTAINERS.md deleted file mode 100644 index 29021c74a9add..0000000000000 --- a/docs/MAINTAINERS.md +++ /dev/null @@ -1,8 +0,0 @@ -## Maintainers - -- [dongxu](https://github.com/c4pt0r) -- [Ewan Chou](https://github.com/coocood) -- [goroutine](https://github.com/ngaut) -- [qiuyesuifeng](https://github.com/qiuyesuifeng) -- [Shen Li](https://github.com/shenli) -- [siddontang](https://github.com/siddontang) diff --git a/domain/infosync/info.go b/domain/infosync/info.go index e6bd04cffd0b7..e885bbeb175e4 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -892,3 +892,38 @@ func GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) { } return rules, err } + +// GetLabelRules gets the label rules according to the given IDs from PD. +func GetLabelRules(ctx context.Context, ruleIDs []string) ([]*label.Rule, error) { + if len(ruleIDs) == 0 { + return nil, nil + } + + is, err := getGlobalInfoSyncer() + if err != nil { + return nil, err + } + + if is.etcdCli == nil { + return nil, nil + } + + addrs := is.etcdCli.Endpoints() + + if len(addrs) == 0 { + return nil, errors.Errorf("pd unavailable") + } + + ids, err := json.Marshal(ruleIDs) + if err != nil { + return nil, err + } + + rules := []*label.Rule{} + res, err := doRequest(ctx, addrs, path.Join(pdapi.Config, "region-label", "rules", "ids"), "GET", bytes.NewReader(ids)) + + if err == nil && res != nil { + err = json.Unmarshal(res, &rules) + } + return rules, err +} diff --git a/errno/errcode.go b/errno/errcode.go index 1b8f1f0bc4d55..638e1b889a902 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -891,7 +891,7 @@ const ( ErrFunctionalIndexRefAutoIncrement = 3754 ErrCannotDropColumnFunctionalIndex = 3755 ErrFunctionalIndexPrimaryKey = 3756 - ErrFunctionalIndexOnLob = 3757 + ErrFunctionalIndexOnBlob = 3757 ErrFunctionalIndexFunctionIsNotAllowed = 3758 ErrFulltextFunctionalIndex = 3759 ErrSpatialFunctionalIndex = 3760 @@ -1048,6 +1048,8 @@ const ( ErrDDLReorgElementNotExist = 8235 ErrPlacementPolicyCheck = 8236 ErrInvalidAttributesSpec = 8237 + ErrPlacementPolicyExists = 8238 + ErrPlacementPolicyNotExists = 8239 // TiKV/PD/TiFlash errors. ErrPDServerTimeout = 9001 diff --git a/errno/errname.go b/errno/errname.go index df323a213220e..025caa8f0e4dd 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -887,7 +887,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrFunctionalIndexRefAutoIncrement: mysql.Message("Expression index '%s' cannot refer to an auto-increment column", nil), ErrCannotDropColumnFunctionalIndex: mysql.Message("Cannot drop column '%s' because it is used by an expression index. In order to drop the column, you must remove the expression index", nil), ErrFunctionalIndexPrimaryKey: mysql.Message("The primary key cannot be an expression index", nil), - ErrFunctionalIndexOnLob: mysql.Message("Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST", nil), + ErrFunctionalIndexOnBlob: mysql.Message("Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST", nil), ErrFunctionalIndexFunctionIsNotAllowed: mysql.Message("Expression of expression index '%s' contains a disallowed function", nil), ErrFulltextFunctionalIndex: mysql.Message("Fulltext expression index is not supported", nil), ErrSpatialFunctionalIndex: mysql.Message("Spatial expression index is not supported", nil), @@ -1050,11 +1050,13 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrPartitionStatsMissing: mysql.Message("Build table: %s global-level stats failed due to missing partition-level stats", nil), ErrNotSupportedWithSem: mysql.Message("Feature '%s' is not supported when security enhanced mode is enabled", nil), - ErrInvalidPlacementSpec: mysql.Message("Invalid placement policy '%s': %s", nil), - ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil), - ErrMultiStatementDisabled: mysql.Message("client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk", nil), - ErrAsOf: mysql.Message("invalid as of timestamp: %s", nil), - ErrInvalidAttributesSpec: mysql.Message("Invalid attributes '%s': %s", nil), + ErrInvalidPlacementSpec: mysql.Message("Invalid placement policy '%s': %s", nil), + ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil), + ErrMultiStatementDisabled: mysql.Message("client has multi-statement capability disabled. Run SET GLOBAL tidb_multi_statement_mode='ON' after you understand the security risk", nil), + ErrAsOf: mysql.Message("invalid as of timestamp: %s", nil), + ErrInvalidAttributesSpec: mysql.Message("Invalid attributes '%s': %s", nil), + ErrPlacementPolicyExists: mysql.Message("Can't create placement policy '%-.192s'; policy exists", nil), + ErrPlacementPolicyNotExists: mysql.Message("Unknown placement policy '%-.192s'", nil), // TiKV/PD errors. ErrPDServerTimeout: mysql.Message("PD server timeout", nil), diff --git a/errors.toml b/errors.toml index cd2bdd0b89abc..a6143d74bcd2a 100644 --- a/errors.toml +++ b/errors.toml @@ -1061,6 +1061,16 @@ error = ''' DDL reorg element does not exist ''' +["meta:8238"] +error = ''' +Can't create placement policy '%-.192s'; policy exists +''' + +["meta:8239"] +error = ''' +Unknown placement policy '%-.192s' +''' + ["planner:1044"] error = ''' Access denied for user '%-.48s'@'%-.64s' to database '%-.192s' diff --git a/executor/adapter.go b/executor/adapter.go index 2b86471b998f9..98dde31851570 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -826,7 +826,7 @@ func (a *ExecStmt) logAudit() { if audit.OnGeneralEvent != nil { cmd := mysql.Command2Str[byte(atomic.LoadUint32(&a.Ctx.GetSessionVars().CommandValue))] ctx := context.WithValue(context.Background(), plugin.ExecStartTimeCtxKey, a.Ctx.GetSessionVars().StartTime) - audit.OnGeneralEvent(ctx, sessVars, plugin.Log, cmd) + audit.OnGeneralEvent(ctx, sessVars, plugin.Completed, cmd) } return nil }) diff --git a/executor/builder.go b/executor/builder.go index c888facf4e369..e95c638919bc5 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -4525,6 +4525,10 @@ func (b *executorBuilder) validCanReadTemporaryTable(tbl *model.TableInfo) error sessionVars := b.ctx.GetSessionVars() + if tbl.TempTableType == model.TempTableLocal && sessionVars.SnapshotTS != 0 { + return errors.New("can not read local temporary table when 'tidb_snapshot' is set") + } + if sessionVars.TxnCtx.IsStaleness || b.isStaleness { return errors.New("can not stale read temporary table") } diff --git a/executor/cte_serial_test.go b/executor/cte_serial_test.go new file mode 100644 index 0000000000000..b8b04551b0c6e --- /dev/null +++ b/executor/cte_serial_test.go @@ -0,0 +1,85 @@ +// Copyright 2021 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 executor_test + +import ( + "fmt" + "math/rand" + "sort" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestSpillToDisk(t *testing.T) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.OOMUseTmpStorage = true + }) + + store, close := testkit.CreateMockStore(t) + defer close() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testCTEStorageSpill", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testCTEStorageSpill")) + tk.MustExec("set tidb_mem_quota_query = 1073741824;") + }() + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill")) + }() + + // Use duplicated rows to test UNION DISTINCT. + tk.MustExec("set tidb_mem_quota_query = 1073741824;") + insertStr := "insert into t1 values(0)" + rowNum := 1000 + vals := make([]int, rowNum) + vals[0] = 0 + for i := 1; i < rowNum; i++ { + v := rand.Intn(100) + vals[i] = v + insertStr += fmt.Sprintf(", (%d)", v) + } + tk.MustExec("drop table if exists t1;") + tk.MustExec("create table t1(c1 int);") + tk.MustExec(insertStr) + tk.MustExec("set tidb_mem_quota_query = 40000;") + tk.MustExec("set cte_max_recursion_depth = 500000;") + sql := fmt.Sprintf("with recursive cte1 as ( "+ + "select c1 from t1 "+ + "union "+ + "select c1 + 1 c1 from cte1 where c1 < %d) "+ + "select c1 from cte1 order by c1;", rowNum) + rows := tk.MustQuery(sql) + + memTracker := tk.Session().GetSessionVars().StmtCtx.MemTracker + diskTracker := tk.Session().GetSessionVars().StmtCtx.DiskTracker + require.Greater(t, memTracker.MaxConsumed(), int64(0)) + require.Greater(t, diskTracker.MaxConsumed(), int64(0)) + + sort.Ints(vals) + resRows := make([]string, 0, rowNum) + for i := vals[0]; i <= rowNum; i++ { + resRows = append(resRows, fmt.Sprintf("%d", i)) + } + rows.Check(testkit.Rows(resRows...)) +} diff --git a/executor/cte_test.go b/executor/cte_test.go index 27bfb9f86c30c..3a9c3a5a7987f 100644 --- a/executor/cte_test.go +++ b/executor/cte_test.go @@ -15,68 +15,20 @@ package executor_test import ( - "context" "fmt" - "math/rand" - "sort" + "testing" - "github.com/pingcap/check" - - "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/session" - "github.com/pingcap/tidb/sessionctx" - - "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/config" - "github.com/pingcap/tidb/store/mockstore" - "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" ) -var _ = check.Suite(&CTETestSuite{&baseCTETestSuite{}}) -var _ = check.SerialSuites(&CTESerialTestSuite{&baseCTETestSuite{}}) - -type baseCTETestSuite struct { - store kv.Storage - dom *domain.Domain - sessionCtx sessionctx.Context - session session.Session - ctx context.Context -} - -type CTETestSuite struct { - *baseCTETestSuite -} - -type CTESerialTestSuite struct { - *baseCTETestSuite -} - -func (test *baseCTETestSuite) SetUpSuite(c *check.C) { - var err error - test.store, err = mockstore.NewMockStore() - c.Assert(err, check.IsNil) - - test.dom, err = session.BootstrapSession(test.store) - c.Assert(err, check.IsNil) - - test.sessionCtx = mock.NewContext() - - test.session, err = session.CreateSession4Test(test.store) - c.Assert(err, check.IsNil) - test.session.SetConnectionID(0) - - test.ctx = context.Background() -} +func TestBasicCTE(t *testing.T) { + t.Parallel() -func (test *baseCTETestSuite) TearDownSuite(c *check.C) { - test.dom.Close() - test.store.Close() -} + store, close := testkit.CreateMockStore(t) + defer close() -func (test *CTETestSuite) TestBasicCTE(c *check.C) { - tk := testkit.NewTestKit(c, test.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test") rows := tk.MustQuery("with recursive cte1 as (" + @@ -121,63 +73,13 @@ func (test *CTETestSuite) TestBasicCTE(c *check.C) { rows.Check(testkit.Rows("1 1", "2 1", "3 1", "4 1", "5 1")) } -func (test *CTESerialTestSuite) TestSpillToDisk(c *check.C) { - defer config.RestoreFunc()() - config.UpdateGlobal(func(conf *config.Config) { - conf.OOMUseTmpStorage = true - }) - - tk := testkit.NewTestKit(c, test.store) - tk.MustExec("use test;") +func TestUnionDistinct(t *testing.T) { + t.Parallel() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testCTEStorageSpill", "return(true)"), check.IsNil) - defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testCTEStorageSpill"), check.IsNil) - tk.MustExec("set tidb_mem_quota_query = 1073741824;") - }() - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill", "return(true)"), check.IsNil) - defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/testSortedRowContainerSpill"), check.IsNil) - }() - - // Use duplicated rows to test UNION DISTINCT. - tk.MustExec("set tidb_mem_quota_query = 1073741824;") - insertStr := "insert into t1 values(0)" - rowNum := 1000 - vals := make([]int, rowNum) - vals[0] = 0 - for i := 1; i < rowNum; i++ { - v := rand.Intn(100) - vals[i] = v - insertStr += fmt.Sprintf(", (%d)", v) - } - tk.MustExec("drop table if exists t1;") - tk.MustExec("create table t1(c1 int);") - tk.MustExec(insertStr) - tk.MustExec("set tidb_mem_quota_query = 40000;") - tk.MustExec("set cte_max_recursion_depth = 500000;") - sql := fmt.Sprintf("with recursive cte1 as ( "+ - "select c1 from t1 "+ - "union "+ - "select c1 + 1 c1 from cte1 where c1 < %d) "+ - "select c1 from cte1 order by c1;", rowNum) - rows := tk.MustQuery(sql) - - memTracker := tk.Se.GetSessionVars().StmtCtx.MemTracker - diskTracker := tk.Se.GetSessionVars().StmtCtx.DiskTracker - c.Assert(memTracker.MaxConsumed(), check.Greater, int64(0)) - c.Assert(diskTracker.MaxConsumed(), check.Greater, int64(0)) - - sort.Ints(vals) - resRows := make([]string, 0, rowNum) - for i := vals[0]; i <= rowNum; i++ { - resRows = append(resRows, fmt.Sprintf("%d", i)) - } - rows.Check(testkit.Rows(resRows...)) -} + store, close := testkit.CreateMockStore(t) + defer close() -func (test *CTETestSuite) TestUnionDistinct(c *check.C) { - tk := testkit.NewTestKit(c, test.store) + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") // Basic test. UNION/UNION ALL intersects. @@ -200,14 +102,18 @@ func (test *CTETestSuite) TestUnionDistinct(c *check.C) { rows.Check(testkit.Rows("1", "2", "3", "4")) } -func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { - tk := testkit.NewTestKit(c, test.store) +func TestCTEMaxRecursionDepth(t *testing.T) { + t.Parallel() + + store, close := testkit.CreateMockStore(t) + defer close() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") tk.MustExec("set @@cte_max_recursion_depth = -1;") err := tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 100) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") // If there is no recursive part, query runs ok. rows := tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") rows.Check(testkit.Rows("1", "2")) @@ -216,11 +122,9 @@ func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { tk.MustExec("set @@cte_max_recursion_depth = 0;") err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 0) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 1) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") // If there is no recursive part, query runs ok. rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") rows.Check(testkit.Rows("1", "2")) @@ -233,8 +137,7 @@ func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 1) select * from cte1;") rows.Check(testkit.Rows("1")) err = tk.QueryToErr("with recursive cte1(c1) as (select 1 union select c1 + 1 c1 from cte1 where c1 < 2) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 2 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 2 iterations. Try increasing @@cte_max_recursion_depth to a larger value") // If there is no recursive part, query runs ok. rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2) select * from cte1 order by c1;") rows.Check(testkit.Rows("1", "2")) @@ -242,8 +145,13 @@ func (test *CTETestSuite) TestCTEMaxRecursionDepth(c *check.C) { rows.Check(testkit.Rows("1", "2")) } -func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { - tk := testkit.NewTestKit(c, test.store) +func TestCTEWithLimit(t *testing.T) { + t.Parallel() + + store, close := testkit.CreateMockStore(t) + defer close() + + tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") // Basic recursive tests. @@ -268,16 +176,14 @@ func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { rows.Check(testkit.Rows("2")) err := tk.QueryToErr("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 1 offset 3) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 3 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 3 iterations. Try increasing @@cte_max_recursion_depth to a larger value") tk.MustExec("set cte_max_recursion_depth=1000;") rows = tk.MustQuery("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 5 offset 996) select * from cte1;") rows.Check(testkit.Rows("996", "997", "998", "999", "1000")) err = tk.QueryToErr("with recursive cte1(c1) as (select 0 union select c1 + 1 from cte1 limit 5 offset 997) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1001 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 1001 iterations. Try increasing @@cte_max_recursion_depth to a larger value") rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select c1 + 1 from cte1 limit 0 offset 1) select * from cte1") rows.Check(testkit.Rows()) @@ -312,7 +218,7 @@ func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { // Error: ERROR 1221 (HY000): Incorrect usage of UNION and LIMIT. // Limit can only be at the end of SQL stmt. err = tk.ExecToErr("with recursive cte1(c1) as (select c1 from t1 limit 1 offset 1 union select c1 + 1 from cte1 limit 0 offset 1) select * from cte1") - c.Assert(err.Error(), check.Equals, "[planner:1221]Incorrect usage of UNION and LIMIT") + require.EqualError(t, err, "[planner:1221]Incorrect usage of UNION and LIMIT") // Basic non-recusive tests. rows = tk.MustQuery("with recursive cte1(c1) as (select 1 union select 2 order by 1 limit 1 offset 1) select * from cte1") @@ -375,8 +281,7 @@ func (test *CTETestSuite) TestCTEWithLimit(c *check.C) { rows.Check(testkit.Rows()) // MySQL err: ERROR 1365 (22012): Division by 0. Because it gives error when computing 1/c1. err = tk.QueryToErr("with recursive cte1 as (select 1/c1 c1 from t1 union select c1 + 1 c1 from cte1 where c1 < 2 limit 1) select * from cte1;") - c.Assert(err, check.NotNil) - c.Assert(err.Error(), check.Equals, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") + require.EqualError(t, err, "[executor:3636]Recursive query aborted after 1 iterations. Try increasing @@cte_max_recursion_depth to a larger value") tk.MustExec("set cte_max_recursion_depth = 1000;") tk.MustExec("drop table if exists t1;") diff --git a/executor/executor_test.go b/executor/executor_test.go index 6d1edbc3bf7fe..f638a45b9fd69 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -66,6 +66,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/deadlockhistory" "github.com/pingcap/tidb/util/gcutil" "github.com/pingcap/tidb/util/israce" @@ -114,6 +115,7 @@ var _ = Suite(&testSuiteP1{&baseTestSuite{}}) var _ = Suite(&testSuiteP2{&baseTestSuite{}}) var _ = Suite(&testSuite1{}) var _ = SerialSuites(&testSerialSuite2{}) +var _ = SerialSuites(&testSuiteWithCliBaseCharset{}) var _ = Suite(&testSuite2{&baseTestSuite{}}) var _ = Suite(&testSuite3{&baseTestSuite{}}) var _ = Suite(&testSuite4{&baseTestSuite{}}) @@ -232,7 +234,7 @@ func (s *testPrepareSuite) TearDownSuite(c *C) { func (s *baseTestSuite) TearDownSuite(c *C) { s.domain.Close() - s.store.Close() + c.Assert(s.store.Close(), IsNil) } func (s *globalIndexSuite) SetUpSuite(c *C) { @@ -3349,6 +3351,20 @@ func (c *checkRequestClient) SendRequest(ctx context.Context, addr string, req * return resp, err } +type testSuiteWithCliBaseCharset struct { + testSuiteWithCliBase +} + +func (s *testSuiteWithCliBaseCharset) SetUpSuite(c *C) { + collate.SetCharsetFeatEnabledForTest(true) + s.testSuiteWithCliBase.SetUpSuite(c) +} + +func (s *testSuiteWithCliBaseCharset) TearDownSuite(c *C) { + s.testSuiteWithCliBase.TearDownSuite(c) + collate.SetCharsetFeatEnabledForTest(false) +} + type testSuiteWithCliBase struct { store kv.Storage dom *domain.Domain @@ -4979,6 +4995,16 @@ func (s *testSuiteP2) TestAddDateBuiltinWithWarnings(c *C) { tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1292|Incorrect datetime value: '2001-01-00'")) } +func (s *testSuiteP2) TestIssue27232(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a timestamp)") + tk.MustExec("insert into t values (\"1970-07-23 10:04:59\"), (\"2038-01-19 03:14:07\")") + tk.MustQuery("select * from t where date_sub(a, interval 10 month) = date_sub(\"1970-07-23 10:04:59\", interval 10 month)").Check(testkit.Rows("1970-07-23 10:04:59")) + tk.MustQuery("select * from t where timestampadd(hour, 1, a ) = timestampadd(hour, 1, \"2038-01-19 03:14:07\")").Check(testkit.Rows("2038-01-19 03:14:07")) +} + func (s *testSuiteP2) TestStrToDateBuiltinWithWarnings(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@sql_mode='NO_ZERO_DATE'") @@ -5627,6 +5653,69 @@ func (s *testSerialSuite2) TestUnsignedFeedback(c *C) { c.Assert(result.Rows()[2][6], Equals, "range:[0,+inf], keep order:false") } +func (s *testSuiteWithCliBaseCharset) TestCharsetFeature(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustQuery("show charset").Check(testkit.Rows( + "ascii US ASCII ascii_bin 1", + "binary binary binary 1", + "gbk Chinese Internal Code Specification gbk_bin 2", + "latin1 Latin1 latin1_bin 1", + "utf8 UTF-8 Unicode utf8_bin 3", + "utf8mb4 UTF-8 Unicode utf8mb4_bin 4", + )) + tk.MustQuery("show collation").Check(testkit.Rows( + "ascii_bin ascii 65 Yes Yes 1", + "binary binary 63 Yes Yes 1", + "gbk_bin gbk 87 Yes 1", + "latin1_bin latin1 47 Yes Yes 1", + "utf8_bin utf8 83 Yes Yes 1", + "utf8_general_ci utf8 33 Yes 1", + "utf8_unicode_ci utf8 192 Yes 1", + "utf8mb4_bin utf8mb4 46 Yes Yes 1", + "utf8mb4_general_ci utf8mb4 45 Yes 1", + "utf8mb4_unicode_ci utf8mb4 224 Yes 1", + )) + + tk.MustExec("set names gbk;") + tk.MustQuery("select @@character_set_connection;").Check(testkit.Rows("gbk")) + tk.MustQuery("select @@collation_connection;").Check(testkit.Rows("gbk_bin")) + tk.MustExec("set @@character_set_client=gbk;") + tk.MustQuery("select @@character_set_client;").Check(testkit.Rows("gbk")) + tk.MustExec("set names utf8mb4;") + tk.MustExec("set @@character_set_connection=gbk;") + tk.MustQuery("select @@character_set_connection;").Check(testkit.Rows("gbk")) + tk.MustQuery("select @@collation_connection;").Check(testkit.Rows("gbk_bin")) + + tk.MustQuery("select _gbk 'a'").Check(testkit.Rows("a")) + + tk.MustExec("use test") + tk.MustExec("create table t1(a char(10) charset gbk);") + tk.MustExec("create table t2(a char(10) charset gbk collate gbk_bin);") + tk.MustExec("create table t3(a char(10)) charset gbk;") + tk.MustExec("alter table t3 add column b char(10) charset gbk;") + tk.MustQuery("show create table t3").Check(testkit.Rows("t3 CREATE TABLE `t3` (\n" + + " `a` char(10) DEFAULT NULL,\n" + + " `b` char(10) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin", + )) + tk.MustExec("create table t4(a char(10));") + tk.MustExec("alter table t4 add column b char(10) charset gbk;") + tk.MustQuery("show create table t4").Check(testkit.Rows("t4 CREATE TABLE `t4` (\n" + + " `a` char(10) DEFAULT NULL,\n" + + " `b` char(10) CHARACTER SET gbk COLLATE gbk_bin DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + + tk.MustExec("create database test_gbk charset gbk;") + tk.MustExec("use test_gbk") + tk.MustExec("create table t1(a char(10));") + tk.MustQuery("show create table t1").Check(testkit.Rows("t1 CREATE TABLE `t1` (\n" + + " `a` char(10) DEFAULT NULL\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_bin", + )) +} + func (s *testSerialSuite2) TestIssue23567(c *C) { tk := testkit.NewTestKit(c, s.store) oriProbability := statistics.FeedbackProbability.Load() @@ -8540,101 +8629,6 @@ func (s testSerialSuite) TestExprBlackListForEnum(c *C) { c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue) } -func (s testSerialSuite) TestTemporaryTableNoNetwork(c *C) { - s.assertTemporaryTableNoNetwork(c, model.TempTableGlobal) - s.assertTemporaryTableNoNetwork(c, model.TempTableLocal) -} - -func (s testSerialSuite) assertTemporaryTableNoNetwork(c *C, temporaryTableType model.TempTableType) { - // Test that table reader/index reader/index lookup on the temporary table do not need to visit TiKV. - tk := testkit.NewTestKit(c, s.store) - tk1 := testkit.NewTestKit(c, s.store) - - tk.MustExec("use test") - tk1.MustExec("use test") - tk.MustExec("drop table if exists normal, tmp_t") - tk.MustExec("create table normal (id int, a int, index(a))") - if temporaryTableType == model.TempTableGlobal { - tk.MustExec("set tidb_enable_global_temporary_table=true") - tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows") - } else if temporaryTableType == model.TempTableLocal { - tk.MustExec("set tidb_enable_noop_functions=true") - tk.MustExec("create temporary table tmp_t (id int primary key, a int, b int, index(a))") - } else { - c.Fail() - } - - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", "return(true)"), IsNil) - defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy"), IsNil) - }() - - tk.MustExec("begin") - tk.MustExec("insert into tmp_t values (1, 1, 1)") - tk.MustExec("insert into tmp_t values (2, 2, 2)") - - // Make sure the fail point works. - // With that failpoint, all requests to the TiKV is discard. - rs, err := tk1.Exec("select * from normal") - c.Assert(err, IsNil) - blocked := make(chan struct{}) - ctx, cancelFunc := context.WithCancel(context.Background()) - go func() { - _, err := session.ResultSetToStringSlice(ctx, tk1.Se, rs) - blocked <- struct{}{} - c.Assert(err, NotNil) - }() - select { - case <-blocked: - c.Error("The query should block when the failpoint is enabled") - case <-time.After(200 * time.Millisecond): - } - cancelFunc() - - // Check the temporary table do not send request to TiKV. - // PointGet - c.Assert(tk.HasPlan("select * from tmp_t where id=1", "Point_Get"), IsTrue) - tk.MustQuery("select * from tmp_t where id=1").Check(testkit.Rows("1 1 1")) - // BatchPointGet - c.Assert(tk.HasPlan("select * from tmp_t where id in (1, 2)", "Batch_Point_Get"), IsTrue) - tk.MustQuery("select * from tmp_t where id in (1, 2)").Check(testkit.Rows("1 1 1", "2 2 2")) - // Table reader - c.Assert(tk.HasPlan("select * from tmp_t", "TableReader"), IsTrue) - tk.MustQuery("select * from tmp_t").Check(testkit.Rows("1 1 1", "2 2 2")) - // Index reader - c.Assert(tk.HasPlan("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t", "IndexReader"), IsTrue) - tk.MustQuery("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t").Check(testkit.Rows("1", "2")) - // Index lookup - c.Assert(tk.HasPlan("select /*+ USE_INDEX(tmp_t, a) */ b from tmp_t where a = 1", "IndexLookUp"), IsTrue) - tk.MustQuery("select /*+ USE_INDEX(tmp_t, a) */ b from tmp_t where a = 1").Check(testkit.Rows("1")) - tk.MustExec("rollback") - - // prepare some data for local temporary table, when for global temporary table, the below operations have no effect. - tk.MustExec("insert into tmp_t value(10, 10, 10)") - tk.MustExec("insert into tmp_t value(11, 11, 11)") - - // Pessimistic lock - tk.MustExec("begin pessimistic") - tk.MustExec("insert into tmp_t values (3, 3, 3)") - tk.MustExec("insert ignore into tmp_t values (4, 4, 4)") - tk.MustExec("insert into tmp_t values (5, 5, 5) on duplicate key update a=100") - tk.MustExec("insert into tmp_t values (10, 10, 10) on duplicate key update a=100") - tk.MustExec("insert ignore into tmp_t values (10, 10, 10) on duplicate key update id=11") - tk.MustExec("replace into tmp_t values(6, 6, 6)") - tk.MustExec("replace into tmp_t values(11, 100, 100)") - tk.MustExec("update tmp_t set id = id + 1 where a = 1") - tk.MustExec("delete from tmp_t where a > 1") - tk.MustQuery("select count(*) from tmp_t where a >= 1 for update") - tk.MustExec("rollback") - - // Check 'for update' will not write any lock too when table is unmodified - tk.MustExec("begin pessimistic") - tk.MustExec("select * from tmp_t where id=1 for update") - tk.MustExec("select * from tmp_t where id in (1, 2, 3) for update") - tk.MustExec("select * from tmp_t where id > 1 for update") - tk.MustExec("rollback") -} - func (s *testResourceTagSuite) TestResourceGroupTag(c *C) { if israce.RaceEnabled { c.Skip("unstable, skip it and fix it before 20210622") @@ -8955,9 +8949,13 @@ func (s *testStaleTxnSuite) TestInvalidReadTemporaryTable(c *C) { tk.MustExec("set @@tidb_snapshot=NOW(6)") for _, query := range queries { + // forbidden historical read local temporary table + if strings.Contains(query.sql, "tmp2") { + tk.MustGetErrMsg(query.sql, "can not read local temporary table when 'tidb_snapshot' is set") + continue + } // Will success here for compatibility with some tools like dumping - rs := tk.MustQuery(query.sql) - rs.Check(testkit.Rows()) + tk.MustQuery(query.sql).Check(testkit.Rows()) } } diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 8ae6047fc8003..5396585b04683 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -2102,11 +2102,6 @@ func (e *stmtSummaryTableRetriever) retrieve(ctx context.Context, sctx sessionct return nil, nil } e.retrieved = true - user := sctx.GetSessionVars().User - isSuper := false - if pm := privilege.GetPrivilegeManager(sctx); pm != nil { - isSuper = pm.RequestVerificationWithUser("", "", "", mysql.SuperPriv, user) - } var err error var instanceAddr string @@ -2118,7 +2113,8 @@ func (e *stmtSummaryTableRetriever) retrieve(ctx context.Context, sctx sessionct return nil, err } } - reader := stmtsummary.NewStmtSummaryReader(user, isSuper, e.columns, instanceAddr) + user := sctx.GetSessionVars().User + reader := stmtsummary.NewStmtSummaryReader(user, hasPriv(sctx, mysql.ProcessPriv), e.columns, instanceAddr) var rows [][]types.Datum switch e.table.Name.O { case infoschema.TableStatementsSummary, diff --git a/executor/main_test.go b/executor/main_test.go index dbd7e4c057811..b5ac20bf97e21 100644 --- a/executor/main_test.go +++ b/executor/main_test.go @@ -15,13 +15,18 @@ package executor import ( - "os" "testing" "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" ) func TestMain(m *testing.M) { testbridge.WorkaroundGoCheckFlags() - os.Exit(m.Run()) + opts := []goleak.Option{ + goleak.IgnoreTopFunction("go.etcd.io/etcd/pkg/logutil.(*MergeLogger).outputLoop"), + goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), + goleak.IgnoreTopFunction("gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun"), + } + goleak.VerifyTestMain(m, opts...) } diff --git a/executor/sample_test.go b/executor/sample_test.go index 315ac10eaf1ac..d200919081e84 100644 --- a/executor/sample_test.go +++ b/executor/sample_test.go @@ -60,6 +60,11 @@ func (s *testTableSampleSuite) SetUpSuite(c *C) { s.domain = d } +func (s *testTableSampleSuite) TearDownSuite(c *C) { + s.domain.Close() + c.Assert(s.store.Close(), IsNil) +} + func (s *testTableSampleSuite) initSampleTest(c *C) *testkit.TestKit { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) tk := testkit.NewTestKit(c, s.store) @@ -214,6 +219,17 @@ func (s *testTableSampleSuite) TestTableSampleWithPartition(c *C) { c.Assert(len(rows), Equals, 0) rows = tk.MustQuery("select * from t partition (p1) tablesample regions();").Rows() c.Assert(len(rows), Equals, 1) + + // Test https://github.com/pingcap/tidb/issues/27349. + tk.MustExec("drop table if exists t;") + tk.MustExec(`create table t (a int, b int, unique key idx(a)) partition by range (a) ( + partition p0 values less than (0), + partition p1 values less than (10), + partition p2 values less than (30), + partition p3 values less than (maxvalue));`) + tk.MustExec("insert into t values (2, 2), (31, 31), (12, 12);") + tk.MustQuery("select _tidb_rowid from t tablesample regions() order by _tidb_rowid;"). + Check(testkit.Rows("1", "2", "3")) // The order of _tidb_rowid should be correct. } func (s *testTableSampleSuite) TestTableSampleGeneratedColumns(c *C) { diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 72f4ebb95b9f2..bb81ab9c9d270 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -114,7 +114,7 @@ func (s *seqTestSuite) SetUpSuite(c *C) { func (s *seqTestSuite) TearDownSuite(c *C) { s.domain.Close() - s.store.Close() + c.Assert(s.store.Close(), IsNil) } func (s *seqTestSuite) TestEarlyClose(c *C) { diff --git a/executor/show_test.go b/executor/show_test.go index 613d4e3c8c43c..574ec467b6e66 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -28,6 +28,7 @@ import ( parsertypes "github.com/pingcap/parser/types" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/infoschema" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege/privileges" "github.com/pingcap/tidb/session" @@ -1044,6 +1045,37 @@ func (s *testAutoRandomSuite) TestAutoIdCache(c *C) { )) } +func (s *testSuite5) TestShowCreateStmtIgnoreLocalTemporaryTables(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_noop_functions=true") + + // SHOW CREATE VIEW ignores local temporary table with the same name + tk.MustExec("drop view if exists v1") + tk.MustExec("create view v1 as select 1") + tk.MustExec("create temporary table v1 (a int)") + tk.MustQuery("show create table v1").Check(testutil.RowsWithSep("|", + ""+ + "v1 CREATE TEMPORARY TABLE `v1` (\n"+ + " `a` int(11) DEFAULT NULL\n"+ + ") ENGINE=memory DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", + )) + tk.MustExec("drop view v1") + err := tk.ExecToErr("show create view v1") + c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) + + // SHOW CREATE SEQUENCE ignores local temporary table with the same name + tk.MustExec("drop view if exists seq1") + tk.MustExec("create sequence seq1") + tk.MustExec("create temporary table seq1 (a int)") + tk.MustQuery("show create sequence seq1").Check(testutil.RowsWithSep("|", + "seq1 CREATE SEQUENCE `seq1` start with 1 minvalue 1 maxvalue 9223372036854775806 increment by 1 cache 1000 nocycle ENGINE=InnoDB", + )) + tk.MustExec("drop sequence seq1") + err = tk.ExecToErr("show create sequence seq1") + c.Assert(infoschema.ErrTableNotExists.Equal(err), IsTrue) +} + func (s *testAutoRandomSuite) TestAutoRandomBase(c *C) { c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil) defer func() { diff --git a/executor/slow_query.go b/executor/slow_query.go index 8633f7a5e00f0..46abdb9c201d0 100755 --- a/executor/slow_query.go +++ b/executor/slow_query.go @@ -433,7 +433,11 @@ func (e *slowQueryRetriever) parseSlowLog(ctx context.Context, sctx sessionctx.C if err != nil { t := slowLogTask{} t.resultCh = make(chan parsedSlowLog, 1) - e.taskList <- t + select { + case <-ctx.Done(): + return + case e.taskList <- t: + } e.sendParsedSlowLogCh(ctx, t, parsedSlowLog{nil, err}) } if len(logs) == 0 || len(logs[0]) == 0 { diff --git a/executor/temporary_table_serial_test.go b/executor/temporary_table_serial_test.go new file mode 100644 index 0000000000000..31606e3a583a8 --- /dev/null +++ b/executor/temporary_table_serial_test.go @@ -0,0 +1,139 @@ +// Copyright 2021 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 executor_test + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + "github.com/stretchr/testify/require" +) + +func TestTemporaryTableNoNetwork(t *testing.T) { + t.Run("global", func(t *testing.T) { + assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) { + tk.MustExec("set tidb_enable_global_temporary_table=true") + tk.MustExec("create global temporary table tmp_t (id int primary key, a int, b int, index(a)) on commit delete rows") + }) + }) + + t.Run("local", func(t *testing.T) { + assertTemporaryTableNoNetwork(t, func(tk *testkit.TestKit) { + tk.MustExec("set tidb_enable_noop_functions=true") + tk.MustExec("create temporary table tmp_t (id int primary key, a int, b int, index(a))") + }) + }) +} + +func assertTemporaryTableNoNetwork(t *testing.T, createTable func(*testkit.TestKit)) { + var done sync.WaitGroup + defer done.Wait() + + store, clean := testkit.CreateMockStore(t) + defer clean() + + // Test that table reader/index reader/index lookup on the temporary table do not need to visit TiKV. + tk := testkit.NewTestKit(t, store) + tk1 := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk1.MustExec("use test") + tk.MustExec("drop table if exists normal, tmp_t") + tk.MustExec("create table normal (id int, a int, index(a))") + createTable(tk) + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy", "return(true)")) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/mockstore/unistore/rpcServerBusy")) + }() + + tk.MustExec("begin") + tk.MustExec("insert into tmp_t values (1, 1, 1)") + tk.MustExec("insert into tmp_t values (2, 2, 2)") + + // Make sure the fail point works. + // With that failpoint, all requests to the TiKV is discard. + rs, err := tk1.Exec("select * from normal") + require.NoError(t, err) + + blocked := make(chan struct{}, 1) + ctx, cancelFunc := context.WithCancel(context.Background()) + done.Add(1) + go func() { + defer done.Done() + _, _ = session.ResultSetToStringSlice(ctx, tk1.Session(), rs) + blocked <- struct{}{} + }() + + select { + case <-blocked: + cancelFunc() + require.FailNow(t, "The query should block when the failpoint is enabled.") + case <-time.After(200 * time.Millisecond): + cancelFunc() + } + + // Check the temporary table do not send request to TiKV. + // PointGet + require.True(t, tk.HasPlan("select * from tmp_t where id=1", "Point_Get")) + tk.MustQuery("select * from tmp_t where id=1").Check(testkit.Rows("1 1 1")) + + // BatchPointGet + require.True(t, tk.HasPlan("select * from tmp_t where id in (1, 2)", "Batch_Point_Get")) + tk.MustQuery("select * from tmp_t where id in (1, 2)").Check(testkit.Rows("1 1 1", "2 2 2")) + + // Table reader + require.True(t, tk.HasPlan("select * from tmp_t", "TableReader")) + tk.MustQuery("select * from tmp_t").Check(testkit.Rows("1 1 1", "2 2 2")) + + // Index reader + require.True(t, tk.HasPlan("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t", "IndexReader")) + tk.MustQuery("select /*+ USE_INDEX(tmp_t, a) */ a from tmp_t").Check(testkit.Rows("1", "2")) + + // Index lookup + require.True(t, tk.HasPlan("select /*+ USE_INDEX(tmp_t, a) */ b from tmp_t where a = 1", "IndexLookUp")) + tk.MustQuery("select /*+ USE_INDEX(tmp_t, a) */ b from tmp_t where a = 1").Check(testkit.Rows("1")) + tk.MustExec("rollback") + + // prepare some data for local temporary table, when for global temporary table, the below operations have no effect. + tk.MustExec("insert into tmp_t value(10, 10, 10)") + tk.MustExec("insert into tmp_t value(11, 11, 11)") + + // Pessimistic lock + tk.MustExec("begin pessimistic") + tk.MustExec("insert into tmp_t values (3, 3, 3)") + tk.MustExec("insert ignore into tmp_t values (4, 4, 4)") + tk.MustExec("insert into tmp_t values (5, 5, 5) on duplicate key update a=100") + tk.MustExec("insert into tmp_t values (10, 10, 10) on duplicate key update a=100") + tk.MustExec("insert ignore into tmp_t values (10, 10, 10) on duplicate key update id=11") + tk.MustExec("replace into tmp_t values(6, 6, 6)") + tk.MustExec("replace into tmp_t values(11, 100, 100)") + tk.MustExec("update tmp_t set id = id + 1 where a = 1") + tk.MustExec("delete from tmp_t where a > 1") + tk.MustQuery("select count(*) from tmp_t where a >= 1 for update") + tk.MustExec("rollback") + + // Check 'for update' will not write any lock too when table is unmodified + tk.MustExec("begin pessimistic") + tk.MustExec("select * from tmp_t where id=1 for update") + tk.MustExec("select * from tmp_t where id in (1, 2, 3) for update") + tk.MustExec("select * from tmp_t where id > 1 for update") + tk.MustExec("rollback") +} diff --git a/executor/update_test.go b/executor/update_test.go index d22b8af74353a..683fe7455574a 100644 --- a/executor/update_test.go +++ b/executor/update_test.go @@ -61,7 +61,7 @@ func (s *testUpdateSuite) SetUpSuite(c *C) { func (s *testUpdateSuite) TearDownSuite(c *C) { s.domain.Close() - s.store.Close() + c.Assert(s.store.Close(), IsNil) } func (s *testUpdateSuite) TearDownTest(c *C) { diff --git a/expression/builtin_arithmetic_vec.go b/expression/builtin_arithmetic_vec.go index 12eb0bb1a63dd..4ec42ce198084 100644 --- a/expression/builtin_arithmetic_vec.go +++ b/expression/builtin_arithmetic_vec.go @@ -921,7 +921,7 @@ func (b *builtinArithmeticPlusIntSig) plusUS(result *chunk.Column, lhi64s, rhi64 } return types.ErrOverflow.GenWithStackByArgs("BIGINT UNSIGNED", fmt.Sprintf("(%s + %s)", b.args[0].String(), b.args[1].String())) } - if rh > 0 && uint64(lh) > math.MaxUint64-uint64(lh) { + if rh > 0 && uint64(lh) > math.MaxUint64-uint64(rh) { if result.IsNull(i) { continue } diff --git a/expression/builtin_string.go b/expression/builtin_string.go index f81de802c49ef..94210daf924ba 100644 --- a/expression/builtin_string.go +++ b/expression/builtin_string.go @@ -366,8 +366,8 @@ func (c *concatWSFunctionClass) getFunction(ctx sessionctx.Context, args []Expre bf.tp.Flen = mysql.MaxBlobWidth logutil.BgLogger().Warn("unexpected `Flen` value(-1) in CONCAT_WS's args", zap.Int("arg's index", i)) } - bf.tp.Flen += argType.Flen } + bf.tp.Flen += argType.Flen } // add separator @@ -1588,8 +1588,8 @@ func (c *hexFunctionClass) getFunction(ctx sessionctx.Context, args []Expression return nil, err } bf.tp.Charset, bf.tp.Collate = ctx.GetSessionVars().GetCharsetInfo() - // Use UTF-8 as default - bf.tp.Flen = args[0].GetType().Flen * 3 * 2 + // Use UTF8MB4 as default. + bf.tp.Flen = args[0].GetType().Flen * 4 * 2 sig := &builtinHexStrArgSig{bf} sig.setPbCode(tipb.ScalarFuncSig_HexStrArg) return sig, nil @@ -1665,10 +1665,10 @@ func (c *unhexFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi argEvalTp := argType.EvalType() switch argEvalTp { case types.ETString, types.ETDatetime, types.ETTimestamp, types.ETDuration, types.ETJson: - // Use UTF-8 as default charset, so there're (Flen * 3 + 1) / 2 byte-pairs - retFlen = (argType.Flen*3 + 1) / 2 + // Use UTF8MB4 as default charset, so there're (Flen * 4 + 1) / 2 byte-pairs. + retFlen = (argType.Flen*4 + 1) / 2 case types.ETInt, types.ETReal, types.ETDecimal: - // For number value, there're (Flen + 1) / 2 byte-pairs + // For number value, there're (Flen + 1) / 2 byte-pairs. retFlen = (argType.Flen + 1) / 2 default: return nil, errors.Errorf("Unhex invalid args, need int or string but get %s", argType) @@ -3073,7 +3073,16 @@ func (c *exportSetFunctionClass) getFunction(ctx sessionctx.Context, args []Expr if err != nil { return nil, err } - bf.tp.Flen = mysql.MaxBlobWidth + // Calculate the flen as MySQL does. + l := args[1].GetType().Flen + if args[2].GetType().Flen > l { + l = args[2].GetType().Flen + } + sepL := 1 + if len(args) > 3 { + sepL = args[3].GetType().Flen + } + bf.tp.Flen = (l*64 + sepL*63) * 4 switch len(args) { case 3: sig = &builtinExportSet3ArgSig{bf} @@ -3407,7 +3416,15 @@ func (c *fromBase64FunctionClass) getFunction(ctx sessionctx.Context, args []Exp if err != nil { return nil, err } - bf.tp.Flen = mysql.MaxBlobWidth + // The calculation of Flen is the same as MySQL. + if args[0].GetType().Flen == types.UnspecifiedLength { + bf.tp.Flen = types.UnspecifiedLength + } else { + bf.tp.Flen = args[0].GetType().Flen * 3 + if bf.tp.Flen > mysql.MaxBlobWidth { + bf.tp.Flen = mysql.MaxBlobWidth + } + } valStr, _ := ctx.GetSessionVars().GetSystemVar(variable.MaxAllowedPacket) maxAllowedPacket, err := strconv.ParseUint(valStr, 10, 64) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index d1da8382f164f..1d388d34a8590 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -3356,6 +3356,16 @@ func (c *addDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err != nil { return nil, err } + if dateEvalTp == types.ETDatetime && args[0].GetType().Tp == mysql.TypeTimestamp { + tp := types.NewFieldType(mysql.TypeDatetime) + tp.Decimal = args[0].GetType().Decimal + tp.Flen = mysql.MaxDatetimeWidthNoFsp + if tp.Decimal > 0 { + tp.Flen = tp.Flen + 1 + tp.Decimal + } + types.SetBinChsClnFlag(tp) + args[0] = BuildCastFunction(ctx, args[0], tp) + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeFullWidth, types.UnspecifiedLength } @@ -4030,6 +4040,16 @@ func (c *subDateFunctionClass) getFunction(ctx sessionctx.Context, args []Expres if err != nil { return nil, err } + if dateEvalTp == types.ETDatetime && args[0].GetType().Tp == mysql.TypeTimestamp { + tp := types.NewFieldType(mysql.TypeDatetime) + tp.Decimal = args[0].GetType().Decimal + tp.Flen = mysql.MaxDatetimeWidthNoFsp + if tp.Decimal > 0 { + tp.Flen = tp.Flen + 1 + tp.Decimal + } + types.SetBinChsClnFlag(tp) + args[0] = BuildCastFunction(ctx, args[0], tp) + } bf.tp.Flen, bf.tp.Decimal = mysql.MaxDatetimeFullWidth, types.UnspecifiedLength } diff --git a/expression/expr_to_pb_test.go b/expression/expr_to_pb_test.go index ecf607b8f21ed..365de5f11b618 100644 --- a/expression/expr_to_pb_test.go +++ b/expression/expr_to_pb_test.go @@ -866,6 +866,26 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) + // ScalarFuncSig_RoundDecimal + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_RoundWithFracReal + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeDouble), realColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_RoundWithFracInt + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeLonglong), intColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + + // ScalarFuncSig_RoundWithFracDecimal + function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn, intColumn) + c.Assert(err, IsNil) + exprs = append(exprs, function) + // concat function, err = NewFunction(mock.NewContext(), ast.Concat, types.NewFieldType(mysql.TypeString), stringColumn, intColumn, realColumn) c.Assert(err, IsNil) @@ -965,11 +985,6 @@ func (s *testEvaluatorSuite) TestExprPushDownToFlash(c *C) { c.Assert(err, IsNil) exprs = append(exprs, function) - // RoundDecimal: can not be pushed - function, err = NewFunction(mock.NewContext(), ast.Round, types.NewFieldType(mysql.TypeNewDecimal), decimalColumn) - c.Assert(err, IsNil) - exprs = append(exprs, function) - pushed, remained := PushDownExprs(sc, exprs, client, kv.TiFlash) c.Assert(len(pushed), Equals, 0) c.Assert(len(remained), Equals, len(exprs)) diff --git a/expression/expression.go b/expression/expression.go index 0b9e5817b6b24..d31a519281b96 100644 --- a/expression/expression.go +++ b/expression/expression.go @@ -1066,7 +1066,8 @@ func scalarExprSupportedByFlash(function *ScalarFunction) bool { } case ast.Round: switch function.Function.PbCode() { - case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal: + case tipb.ScalarFuncSig_RoundInt, tipb.ScalarFuncSig_RoundReal, tipb.ScalarFuncSig_RoundDec, + tipb.ScalarFuncSig_RoundWithFracInt, tipb.ScalarFuncSig_RoundWithFracReal, tipb.ScalarFuncSig_RoundWithFracDec: return true } case ast.Extract: diff --git a/expression/integration_test.go b/expression/integration_test.go index c33e1c102c562..2efcae665713c 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -5051,6 +5051,11 @@ func (s *testIntegrationSuite) TestTiDBInternalFunc(c *C) { result = tk.MustQuery(sql) rs = fmt.Sprintf(`{"index_id":1,"index_vals":{"a":null,"b":null,"c":null},"table_id":%d}`, tbl.Meta().ID) result.Check(testkit.Rows(rs)) + + // https://github.com/pingcap/tidb/issues/27434. + hexKey = "7480000000000000375F69800000000000000103800000000001D4C1023B6458" + sql = fmt.Sprintf("select tidb_decode_key( '%s' )", hexKey) + tk.MustQuery(sql).Check(testkit.Rows(hexKey)) } func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { @@ -10260,3 +10265,9 @@ func (s *testIntegrationSuite) TestIssue27236(c *C) { row = tk.MustQuery(`select extract(hour_second from c1) from t order by c1;`) row.Check(testkit.Rows("-8385959", "7005959")) } + +func (s *testIntegrationSuite) TestIssue26977(c *C) { + tk := testkit.NewTestKit(c, s.store) + result := tk.MustQuery("select a + 1 as f from (select cast(0xfffffffffffffff0 as unsigned) as a union select cast(1 as unsigned)) t having f != 2;") + result.Check(testkit.Rows("18446744073709551601")) +} diff --git a/expression/typeinfer_test.go b/expression/typeinfer_test.go index 335fa4b1aaa25..293ea722a2824 100644 --- a/expression/typeinfer_test.go +++ b/expression/typeinfer_test.go @@ -244,8 +244,8 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"CONCAT(c_bchar, 0x80)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 23, types.UnspecifiedLength}, {"CONCAT('T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0 | mysql.NotNullFlag, 4, types.UnspecifiedLength}, {"CONCAT('T', 'i', 'DB', c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 24, types.UnspecifiedLength}, - {"CONCAT_WS('-', 'T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0 | mysql.NotNullFlag, 6, types.UnspecifiedLength}, - {"CONCAT_WS(',', 'TiDB', c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 25, types.UnspecifiedLength}, + {"CONCAT_WS('-', 'T', 'i', 'DB')", mysql.TypeVarString, charset.CharsetUTF8MB4, 0 | mysql.NotNullFlag, 7, types.UnspecifiedLength}, + {"CONCAT_WS(',', 'TiDB', c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 26, types.UnspecifiedLength}, {"left(c_int_d, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"right(c_int_d, c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"lower(c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, @@ -261,10 +261,10 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"bit_length(c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, {"substring_index(c_int_d, '.', 1)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"substring_index(c_binary, '.', 1)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, - {"hex(c_char)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 120, types.UnspecifiedLength}, + {"hex(c_char)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 160, types.UnspecifiedLength}, {"hex(c_int_d)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 22, types.UnspecifiedLength}, {"unhex(c_int_d)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 6, types.UnspecifiedLength}, - {"unhex(c_char)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 30, types.UnspecifiedLength}, + {"unhex(c_char)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 40, types.UnspecifiedLength}, {"ltrim(c_char)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, {"ltrim(c_binary)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength}, {"rtrim(c_char)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 20, types.UnspecifiedLength}, @@ -301,22 +301,22 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"rpad(c_char, c_int_d, c_binary)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, {"rpad(c_char, c_int_d, c_char )", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_int_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_bigint_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_float_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_double_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_decimal )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_datetime )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_time_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_timestamp_d)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_char )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_varchar )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_text_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_binary )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_varbinary )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_blob_d )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_set )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"from_base64(c_enum )", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, + {"from_base64(c_int_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_bigint_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_float_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, types.UnspecifiedLength, types.UnspecifiedLength}, + {"from_base64(c_double_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, types.UnspecifiedLength, types.UnspecifiedLength}, + {"from_base64(c_decimal )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 24, types.UnspecifiedLength}, + {"from_base64(c_datetime )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 66, types.UnspecifiedLength}, + {"from_base64(c_time_d )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 30, types.UnspecifiedLength}, + {"from_base64(c_timestamp_d)", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 57, types.UnspecifiedLength}, + {"from_base64(c_char )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_varchar )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_text_d )", mysql.TypeMediumBlob, charset.CharsetBin, mysql.BinaryFlag, 196605, types.UnspecifiedLength}, + {"from_base64(c_binary )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_varbinary )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 60, types.UnspecifiedLength}, + {"from_base64(c_blob_d )", mysql.TypeMediumBlob, charset.CharsetBin, mysql.BinaryFlag, 196605, types.UnspecifiedLength}, + {"from_base64(c_set )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 15, types.UnspecifiedLength}, + {"from_base64(c_enum )", mysql.TypeVarString, charset.CharsetBin, mysql.BinaryFlag, 3, types.UnspecifiedLength}, {"bin(c_int_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 64, types.UnspecifiedLength}, {"bin(c_bigint_d )", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 64, types.UnspecifiedLength}, @@ -472,9 +472,9 @@ func (s *testInferTypeSuite) createTestCase4StrFuncs() []typeInferTestCase { {"insert(c_binary, c_int_d, c_int_d, c_varchar)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, {"insert(c_binary, c_int_d, c_int_d, c_binary)", mysql.TypeLongBlob, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"export_set(c_double_d, c_text_d, c_text_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"export_set(c_double_d, c_text_d, c_text_d, c_text_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, - {"export_set(c_double_d, c_text_d, c_text_d, c_text_d, c_int_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, + {"export_set(c_double_d, c_text_d, c_text_d)", mysql.TypeMediumBlob, charset.CharsetUTF8MB4, 0, 16777212, types.UnspecifiedLength}, + {"export_set(c_double_d, c_text_d, c_text_d, c_text_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, 33291780, types.UnspecifiedLength}, + {"export_set(c_double_d, c_text_d, c_text_d, c_text_d, c_int_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, 33291780, types.UnspecifiedLength}, {"format(c_double_d, c_double_d)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, {"format(c_double_d, c_double_d, c_binary)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, 0, mysql.MaxBlobWidth, types.UnspecifiedLength}, diff --git a/go.mod b/go.mod index 70a650dfc1d23..585ba1eab819e 100644 --- a/go.mod +++ b/go.mod @@ -66,7 +66,7 @@ require ( github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.7.0 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210816071108-df2119f51be1 + github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8 github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d github.com/twmb/murmur3 v1.1.3 github.com/uber-go/atomic v1.4.0 diff --git a/go.sum b/go.sum index ce3140581ec3f..3b10da810030c 100644 --- a/go.sum +++ b/go.sum @@ -694,8 +694,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls= github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E= github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210816071108-df2119f51be1 h1:/CRgfKWlH9Go5BFoV+dO04oLRWWOgAZ8q1effzEPRMw= -github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210816071108-df2119f51be1/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8 h1:/S06bCy/r/zuAe+uSjkywM7RHq4rXySApIom3uBVsIU= +github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o= github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw= github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d h1:AFm1Dzw+QRUevWRfrFp45CPPkuK/zdSWcfxI10z+WVE= github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA= diff --git a/infoschema/builder.go b/infoschema/builder.go index 88847ba544c5a..d86ef74e3ddce 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/util/domainutil" + "github.com/pingcap/tidb/util/placementpolicy" ) // Builder builds a new InfoSchema. @@ -485,6 +486,7 @@ func (b *Builder) InitWithOldInfoSchema(oldSchema InfoSchema) *Builder { b.is.schemaMetaVersion = oldIS.schemaMetaVersion b.copySchemasMap(oldIS) b.copyBundlesMap(oldIS) + b.copyPoliciesMap(oldIS) copy(b.is.sortedTablesBuckets, oldIS.sortedTablesBuckets) return b } @@ -502,6 +504,15 @@ func (b *Builder) copyBundlesMap(oldIS *infoSchema) { } } +func (b *Builder) copyPoliciesMap(oldIS *infoSchema) { + is := b.is + is.policyMutex.Lock() + defer is.policyMutex.Unlock() + for _, v := range oldIS.PlacementPolicies() { + is.policyMap[v.Name.L] = v + } +} + // copySchemaTables creates a new schemaTables instance when a table in the database has changed. // It also does modifications on the new one because old schemaTables must be read-only. // Note: please make sure the dbName is in lowercase. @@ -589,6 +600,7 @@ func NewBuilder(store kv.Storage) *Builder { store: store, is: &infoSchema{ schemaMap: map[string]*schemaTables{}, + policyMap: map[string]*placementpolicy.PolicyInfo{}, ruleBundleMap: map[string]*placement.Bundle{}, sortedTablesBuckets: make([]sortedTables, bucketCount), }, diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index e9131bff5d822..b13d0ede1fdbe 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/placementpolicy" ) // InfoSchema is the interface used to retrieve the schema information. @@ -94,6 +95,10 @@ type infoSchema struct { ruleBundleMutex sync.RWMutex ruleBundleMap map[string]*placement.Bundle + // policyMap stores all placement policies. + policyMutex sync.RWMutex + policyMap map[string]*placementpolicy.PolicyInfo + schemaMap map[string]*schemaTables // sortedTablesBuckets is a slice of sortedTables, a table's bucket index is (tableID % bucketCount). @@ -107,6 +112,7 @@ type infoSchema struct { func MockInfoSchema(tbList []*model.TableInfo) InfoSchema { result := &infoSchema{} result.schemaMap = make(map[string]*schemaTables) + result.policyMap = make(map[string]*placementpolicy.PolicyInfo) result.ruleBundleMap = make(map[string]*placement.Bundle) result.sortedTablesBuckets = make([]sortedTables, bucketCount) dbInfo := &model.DBInfo{ID: 0, Name: model.NewCIStr("test"), Tables: tbList} @@ -131,6 +137,7 @@ func MockInfoSchema(tbList []*model.TableInfo) InfoSchema { func MockInfoSchemaWithSchemaVer(tbList []*model.TableInfo, schemaVer int64) InfoSchema { result := &infoSchema{} result.schemaMap = make(map[string]*schemaTables) + result.policyMap = make(map[string]*placementpolicy.PolicyInfo) result.ruleBundleMap = make(map[string]*placement.Bundle) result.sortedTablesBuckets = make([]sortedTables, bucketCount) dbInfo := &model.DBInfo{ID: 0, Name: model.NewCIStr("test"), Tables: tbList} @@ -347,6 +354,24 @@ func HasAutoIncrementColumn(tbInfo *model.TableInfo) (bool, string) { return false, "" } +// PolicyByName is used to find the policy. +func (is *infoSchema) PolicyByName(name string) (*placementpolicy.PolicyInfo, bool) { + is.policyMutex.RLock() + defer is.policyMutex.RUnlock() + t, r := is.policyMap[name] + return t, r +} + +func (is *infoSchema) PlacementPolicies() []*placementpolicy.PolicyInfo { + is.policyMutex.RLock() + defer is.policyMutex.RUnlock() + policies := make([]*placementpolicy.PolicyInfo, 0, len(is.policyMap)) + for _, policy := range is.policyMap { + policies = append(policies, policy) + } + return policies +} + func (is *infoSchema) BundleByName(name string) (*placement.Bundle, bool) { is.ruleBundleMutex.RLock() defer is.ruleBundleMutex.RUnlock() diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index ca10d6214ac5e..881ebd6499a25 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1149,51 +1149,61 @@ func (s *testTableSuite) TestStmtSummaryTable(c *C) { from information_schema.statements_summary`, ).Check(testkit.Rows()) - // Create a new session to test - tk = s.newTestKitWithRoot(c) - tk.MustExec("set global tidb_enable_stmt_summary = on") tk.MustExec("set global tidb_stmt_summary_history_size = 24") +} + +func (s *testTableSuite) TestStmtSummaryTablePriv(c *C) { + tk := s.newTestKitWithRoot(c) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b varchar(10), key k(a))") + defer tk.MustExec("drop table if exists t") + + // Disable refreshing summary. + tk.MustExec("set global tidb_stmt_summary_refresh_interval = 999999999") + tk.MustQuery("select @@global.tidb_stmt_summary_refresh_interval").Check(testkit.Rows("999999999")) + // Clear all statements. + tk.MustExec("set global tidb_enable_stmt_summary = 0") + tk.MustExec("set global tidb_enable_stmt_summary = 1") // Create a new user to test statements summary table privilege + tk.MustExec("drop user if exists 'test_user'@'localhost'") tk.MustExec("create user 'test_user'@'localhost'") - tk.MustExec("grant select on *.* to 'test_user'@'localhost'") - tk.Se.Auth(&auth.UserIdentity{ - Username: "root", - Hostname: "%", - AuthUsername: "root", - AuthHostname: "%", - }, nil, nil) + defer tk.MustExec("drop user if exists 'test_user'@'localhost'") + tk.MustExec("grant select on test.t to 'test_user'@'localhost'") tk.MustExec("select * from t where a=1") result := tk.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") - // Super user can query all records. c.Assert(len(result.Rows()), Equals, 1) result = tk.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") c.Assert(len(result.Rows()), Equals, 1) - tk.Se.Auth(&auth.UserIdentity{ + + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk1.Se.Auth(&auth.UserIdentity{ Username: "test_user", Hostname: "localhost", AuthUsername: "test_user", AuthHostname: "localhost", }, nil, nil) - result = tk.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") // Ordinary users can not see others' records c.Assert(len(result.Rows()), Equals, 0) - result = tk.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") c.Assert(len(result.Rows()), Equals, 0) - tk.MustExec("select * from t where a=1") - result = tk.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + tk1.MustExec("select * from t where b=1") + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + // Ordinary users can see his own records c.Assert(len(result.Rows()), Equals, 1) - tk.MustExec("select * from t where a=1") - result = tk.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") c.Assert(len(result.Rows()), Equals, 1) - // use root user to set variables back - tk.Se.Auth(&auth.UserIdentity{ - Username: "root", - Hostname: "%", - AuthUsername: "root", - AuthHostname: "%", - }, nil, nil) + + tk.MustExec("grant process on *.* to 'test_user'@'localhost'") + result = tk1.MustQuery("select * from information_schema.statements_summary where digest_text like 'select * from `t`%'") + // Users with 'PROCESS' privileges can query all records. + c.Assert(len(result.Rows()), Equals, 2) + result = tk1.MustQuery("select * from information_schema.statements_summary_history where digest_text like 'select * from `t`%'") + c.Assert(len(result.Rows()), Equals, 2) } func (s *testTableSuite) TestIssue18845(c *C) { diff --git a/meta/meta.go b/meta/meta.go index e222cc126ddd3..e4872e7d08740 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -35,11 +35,13 @@ import ( "github.com/pingcap/tidb/structure" "github.com/pingcap/tidb/util/dbterror" "github.com/pingcap/tidb/util/logutil" + placement "github.com/pingcap/tidb/util/placementpolicy" "go.uber.org/zap" ) var ( globalIDMutex sync.Mutex + policyIDMutex sync.Mutex ) // Meta structure: @@ -70,6 +72,25 @@ var ( mRandomIDPrefix = "TARID" mBootstrapKey = []byte("BootstrapKey") mSchemaDiffPrefix = "Diff" + mPolicies = []byte("Policies") + mPolicyPrefix = "Policy" + mPolicyGlobalID = []byte("PolicyGlobalID") + mPolicyMagicByte = CurrentMagicByteVer +) + +const ( + // CurrentMagicByteVer is the current magic byte version, used for future meta compatibility. + CurrentMagicByteVer byte = 0x00 + // PolicyMagicByte handler + // 0x00 - 0x3F: Json Handler + // 0x40 - 0x7F: Reserved + // 0x80 - 0xBF: Reserved + // 0xC0 - 0xFF: Reserved + + // type means how to handle the serialized data. + typeUnknown int = 0 + typeJSON int = 1 + // todo: customized handler. ) var ( @@ -77,6 +98,10 @@ var ( ErrDBExists = dbterror.ClassMeta.NewStd(mysql.ErrDBCreateExists) // ErrDBNotExists is the error for db not exists. ErrDBNotExists = dbterror.ClassMeta.NewStd(mysql.ErrBadDB) + // ErrPolicyExists is the error for policy exists. + ErrPolicyExists = dbterror.ClassMeta.NewStd(errno.ErrPlacementPolicyExists) + // ErrPolicyNotExists is the error for policy not exists. + ErrPolicyNotExists = dbterror.ClassMeta.NewStd(errno.ErrPlacementPolicyNotExists) // ErrTableExists is the error for table exists. ErrTableExists = dbterror.ClassMeta.NewStd(mysql.ErrTableExists) // ErrTableNotExists is the error for table not exists. @@ -145,6 +170,15 @@ func (m *Meta) GetGlobalID() (int64, error) { return m.txn.GetInt64(mNextGlobalIDKey) } +// GetPolicyID gets current policy global id. +func (m *Meta) GetPolicyID() (int64, error) { + return m.txn.GetInt64(mPolicyGlobalID) +} + +func (m *Meta) policyKey(policyID int64) []byte { + return []byte(fmt.Sprintf("%s:%d", mPolicyPrefix, policyID)) +} + func (m *Meta) dbKey(dbID int64) []byte { return []byte(fmt.Sprintf("%s:%d", mDBPrefix, dbID)) } @@ -268,6 +302,22 @@ func (m *Meta) GenSchemaVersion() (int64, error) { return m.txn.Inc(mSchemaVersionKey, 1) } +func (m *Meta) checkPolicyExists(policyKey []byte) error { + v, err := m.txn.HGet(mPolicies, policyKey) + if err == nil && v == nil { + err = ErrPolicyNotExists.GenWithStack("policy doesn't exist") + } + return errors.Trace(err) +} + +func (m *Meta) checkPolicyNotExists(policyKey []byte) error { + v, err := m.txn.HGet(mPolicies, policyKey) + if err == nil && v != nil { + err = ErrPolicyExists.GenWithStack("policy already exists") + } + return errors.Trace(err) +} + func (m *Meta) checkDBExists(dbKey []byte) error { v, err := m.txn.HGet(mDBs, dbKey) if err == nil && v == nil { @@ -300,6 +350,46 @@ func (m *Meta) checkTableNotExists(dbKey []byte, tableKey []byte) error { return errors.Trace(err) } +// CreatePolicy creates a policy. +func (m *Meta) CreatePolicy(policy *placement.PolicyInfo) error { + if policy.ID != 0 { + policyKey := m.policyKey(policy.ID) + if err := m.checkPolicyNotExists(policyKey); err != nil { + return errors.Trace(err) + } + } else { + // Autofill the policy ID. + policyIDMutex.Lock() + genID, err := m.txn.Inc(mPolicyGlobalID, 1) + if err != nil { + return errors.Trace(err) + } + policyIDMutex.Unlock() + policy.ID = genID + } + policyKey := m.policyKey(policy.ID) + data, err := json.Marshal(policy) + if err != nil { + return errors.Trace(err) + } + return m.txn.HSet(mPolicies, policyKey, attachMagicByte(data)) +} + +// UpdatePolicy updates a policy. +func (m *Meta) UpdatePolicy(policy *placement.PolicyInfo) error { + policyKey := m.policyKey(policy.ID) + + if err := m.checkPolicyExists(policyKey); err != nil { + return errors.Trace(err) + } + + data, err := json.Marshal(policy) + if err != nil { + return errors.Trace(err) + } + return m.txn.HSet(mPolicies, policyKey, attachMagicByte(data)) +} + // CreateDatabase creates a database with db info. func (m *Meta) CreateDatabase(dbInfo *model.DBInfo) error { dbKey := m.dbKey(dbInfo.ID) @@ -552,6 +642,73 @@ func (m *Meta) GetDatabase(dbID int64) (*model.DBInfo, error) { return dbInfo, errors.Trace(err) } +// ListPolicies shows all policies. +func (m *Meta) ListPolicies() ([]*placement.PolicyInfo, error) { + res, err := m.txn.HGetAll(mPolicies) + if err != nil { + return nil, errors.Trace(err) + } + + policies := make([]*placement.PolicyInfo, 0, len(res)) + for _, r := range res { + value, err := detachMagicByte(r.Value) + if err != nil { + return nil, errors.Trace(err) + } + policy := &placement.PolicyInfo{} + err = json.Unmarshal(value, policy) + if err != nil { + return nil, errors.Trace(err) + } + policies = append(policies, policy) + } + return policies, nil +} + +// GetPolicy gets the database value with ID. +func (m *Meta) GetPolicy(policyID int64) (*placement.PolicyInfo, error) { + policyKey := m.policyKey(policyID) + value, err := m.txn.HGet(mPolicies, policyKey) + if err != nil || value == nil { + return nil, errors.Trace(err) + } + value, err = detachMagicByte(value) + if err != nil { + return nil, errors.Trace(err) + } + + policy := &placement.PolicyInfo{} + err = json.Unmarshal(value, policy) + return policy, errors.Trace(err) +} + +func attachMagicByte(data []byte) []byte { + data = append(data, 0) + copy(data[1:], data) + data[0] = mPolicyMagicByte + return data +} + +func detachMagicByte(value []byte) ([]byte, error) { + magic, data := value[:1], value[1:] + switch whichMagicType(magic[0]) { + case typeJSON: + if magic[0] != CurrentMagicByteVer { + return nil, errors.New("incompatible magic type handling module") + } + return data, nil + default: + return nil, errors.New("unknown magic type handling module") + } +} + +func whichMagicType(b byte) int { + if b <= 0x3F { + return typeJSON + } + return typeUnknown +} + // GetTable gets the table value in database with tableID. func (m *Meta) GetTable(dbID int64, tableID int64) (*model.TableInfo, error) { // Check if db exists. diff --git a/meta/meta_test.go b/meta/meta_test.go index 15cf516eeaa81..d5512a1145ced 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -29,9 +29,80 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/placementpolicy" "github.com/stretchr/testify/require" ) +func TestPlacementPolicy(t *testing.T) { + t.Parallel() + + store, err := mockstore.NewMockStore() + require.NoError(t, err) + + defer func() { + err := store.Close() + require.NoError(t, err) + }() + + txn, err := store.Begin() + require.NoError(t, err) + + // test the independent policy ID allocation. + m := meta.NewMeta(txn) + + // test the meta storage of placemnt policy. + policy := &placementpolicy.PolicyInfo{ + Name: model.NewCIStr("aa"), + PrimaryRegion: "my primary", + Regions: "my regions", + Learners: 1, + Followers: 2, + Voters: 3, + Schedule: "even", + Constraints: "+disk=ssd", + LearnerConstraints: "+zone=shanghai", + } + err = m.CreatePolicy(policy) + require.NoError(t, err) + require.Equal(t, policy.ID, int64(1)) + + err = m.CreatePolicy(policy) + require.NotNil(t, err) + require.True(t, meta.ErrPolicyExists.Equal(err)) + + val, err := m.GetPolicy(1) + require.NoError(t, err) + require.Equal(t, policy, val) + + // mock updating the placement policy. + policy.Name = model.NewCIStr("bb") + policy.LearnerConstraints = "+zone=nanjing" + err = m.UpdatePolicy(policy) + require.NoError(t, err) + + val, err = m.GetPolicy(1) + require.NoError(t, err) + require.Equal(t, policy, val) + + ps, err := m.ListPolicies() + require.NoError(t, err) + require.Equal(t, []*placementpolicy.PolicyInfo{policy}, ps) + + err = txn.Commit(context.Background()) + require.NoError(t, err) + + // fetch the stored value after committing. + txn, err = store.Begin() + require.NoError(t, err) + + m = meta.NewMeta(txn) + val, err = m.GetPolicy(1) + require.NoError(t, err) + require.Equal(t, policy, val) + err = txn.Commit(context.Background()) + require.NoError(t, err) +} + func TestMeta(t *testing.T) { t.Parallel() diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 04292c9a60d0c..c4c97f01074c2 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1742,6 +1742,12 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, cand if !prop.IsEmpty() && !candidate.isMatchProp { return invalidTask, nil } + if candidate.isMatchProp { + // TableSample on partition table can't keep order. + if ds.tableInfo.GetPartitionInfo() != nil { + return invalidTask, nil + } + } p := PhysicalTableSample{ TableSampleInfo: ds.SampleInfo, TableInfo: ds.table, diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 218b1c7e95c08..7b75afee6a6d2 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4266,3 +4266,18 @@ func (s *testIntegrationSuite) TestIssue27242(c *C) { c.Assert(err, NotNil) c.Assert(err.Error(), Matches, ".*Incorrect timestamp value.*") } + +func (s *testIntegrationSerialSuite) TestTemporaryTableForCte(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + tk.MustExec("set @@tidb_enable_noop_functions=1") + tk.MustExec("create temporary table tmp1(a int, b int, c int);") + tk.MustExec("insert into tmp1 values (1,1,1),(2,2,2),(3,3,3),(4,4,4);") + rows := tk.MustQuery("with cte1 as (with cte2 as (select * from tmp1) select * from cte2) select * from cte1 left join tmp1 on cte1.c=tmp1.c;") + rows.Check(testkit.Rows("1 1 1 1 1 1", "2 2 2 2 2 2", "3 3 3 3 3 3", "4 4 4 4 4 4")) + rows = tk.MustQuery("with cte1 as (with cte2 as (select * from tmp1) select * from cte2) select * from cte1 t1 left join cte1 t2 on t1.c=t2.c;") + rows.Check(testkit.Rows("1 1 1 1 1 1", "2 2 2 2 2 2", "3 3 3 3 3 3", "4 4 4 4 4 4")) + rows = tk.MustQuery("WITH RECURSIVE cte(a) AS (SELECT 1 UNION SELECT a+1 FROM tmp1 WHERE a < 5) SELECT * FROM cte order by a;") + rows.Check(testkit.Rows("1", "2", "3", "4", "5")) +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 5bb3c34e327be..055cbb0390056 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -5875,11 +5875,48 @@ func buildWindowSpecs(specs []ast.WindowSpec) (map[string]*ast.WindowSpec, error return specsMap, nil } +func unfoldSelectList(list *ast.SetOprSelectList, unfoldList *ast.SetOprSelectList) { + for _, sel := range list.Selects { + switch s := sel.(type) { + case *ast.SelectStmt: + unfoldList.Selects = append(unfoldList.Selects, s) + case *ast.SetOprSelectList: + unfoldSelectList(s, unfoldList) + } + } +} + // extractTableList extracts all the TableNames from node. // If asName is true, extract AsName prior to OrigName. // Privilege check should use OrigName, while expression may use AsName. +// TODO: extracting all tables by vistor model maybe a better way func extractTableList(node ast.ResultSetNode, input []*ast.TableName, asName bool) []*ast.TableName { switch x := node.(type) { + case *ast.SubqueryExpr: + input = extractTableList(x.Query, input, asName) + case *ast.SelectStmt: + input = extractTableList(x.From.TableRefs, input, asName) + switch w := x.Where.(type) { + case *ast.PatternInExpr: + if s, ok := w.Sel.(*ast.SubqueryExpr); ok { + input = extractTableList(s, input, asName) + } + case *ast.ExistsSubqueryExpr: + if s, ok := w.Sel.(*ast.SubqueryExpr); ok { + input = extractTableList(s, input, asName) + } + } + for _, f := range x.Fields.Fields { + if s, ok := f.Expr.(*ast.SubqueryExpr); ok { + input = extractTableList(s, input, asName) + } + } + case *ast.SetOprStmt: + l := &ast.SetOprSelectList{} + unfoldSelectList(x.SelectList, l) + for _, s := range l.Selects { + input = extractTableList(s.(ast.ResultSetNode), input, asName) + } case *ast.Join: input = extractTableList(x.Left, input, asName) input = extractTableList(x.Right, input, asName) diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 90b3bd1c9c233..a2fe1bd413b14 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -163,6 +163,7 @@ type preprocessor struct { ctx sessionctx.Context flag preprocessorFlag stmtTp byte + showTp ast.ShowStmtType // tableAliasInJoin is a stack that keeps the table alias names for joins. // len(tableAliasInJoin) may bigger than 1 because the left/right child of join may be subquery that contains `JOIN` @@ -226,6 +227,7 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { p.checkDropDatabaseGrammar(node) case *ast.ShowStmt: p.stmtTp = TypeShow + p.showTp = node.Tp p.resolveShowStmt(node) case *ast.SetOprSelectList: p.checkSetOprSelectList(node) @@ -369,7 +371,16 @@ func (p *preprocessor) tableByName(tn *ast.TableName) (table.Table, error) { return nil, errors.Trace(ErrNoDB) } sName := model.NewCIStr(currentDB) - tbl, err := p.ensureInfoSchema().TableByName(sName, tn.Name) + is := p.ensureInfoSchema() + + // for 'SHOW CREATE VIEW/SEQUENCE ...' statement, ignore local temporary tables. + if p.stmtTp == TypeShow && (p.showTp == ast.ShowCreateView || p.showTp == ast.ShowCreateSequence) { + if tempAttachedIs, ok := is.(*infoschema.TemporaryTableAttachedInfoSchema); ok { + is = tempAttachedIs.InfoSchema + } + } + + tbl, err := is.TableByName(sName, tn.Name) if err != nil { // We should never leak that the table doesn't exist (i.e. attach ErrTableNotExists) // unless we know that the user has permissions to it, should it exist. @@ -414,11 +425,14 @@ func (p *preprocessor) checkBindGrammar(originNode, hintedNode ast.StmtNode, def var resNode ast.ResultSetNode switch n := originNode.(type) { case *ast.SelectStmt: - resNode = n.From.TableRefs + resNode = n + case *ast.SetOprStmt: + resNode = n case *ast.DeleteStmt: resNode = n.TableRefs.TableRefs case *ast.UpdateStmt: resNode = n.TableRefs.TableRefs + //TODO: What about insert into (select * from t) case *ast.InsertStmt: resNode = n.Table.TableRefs } diff --git a/plugin/audit.go b/plugin/audit.go index c64255e1b293d..95efcf8599be2 100644 --- a/plugin/audit.go +++ b/plugin/audit.go @@ -24,29 +24,27 @@ import ( type GeneralEvent byte const ( - // Log presents log event. - Log GeneralEvent = iota - // Error presents error event. + // Starting represents a GeneralEvent that is about to start + Starting GeneralEvent = iota + // Completed represents a GeneralEvent that has completed + Completed + // Error represents a GeneralEvent that has error (and typically couldn't start) Error - // Result presents result event. - Result - // Status presents status event. - Status ) // ConnectionEvent presents TiDB connection event. type ConnectionEvent byte const ( - // Connected presents new connection establish event(finish auth). + // Connected represents new connection establish event(finish auth). Connected ConnectionEvent = iota - // Disconnect presents disconnect event. + // Disconnect represents disconnect event. Disconnect - // ChangeUser presents change user. + // ChangeUser represents change user. ChangeUser - // PreAuth presents event before start auth. + // PreAuth represents event before start auth. PreAuth - // Reject presents event reject connection event. + // Reject represents event reject connection event. Reject ) diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index ed9fb40c04c3e..a5b806aa67ae2 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -79,7 +79,7 @@ func OnInit(ctx context.Context, manifest *plugin.Manifest) error { // OnShutdown implements TiDB plugin's OnShutdown SPI. func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { - fmt.Println("## conn_ip_examples OnShutdown called ##") + fmt.Println("## conn_ip_example OnShutdown called ##") fmt.Printf("---- context: %s\n", ctx) fmt.Printf("---- read cfg in shutdown [key: conn_ip_example_key, value: %s]\n", variable.GetSysVar("conn_ip_example_key").Value) atomic.SwapInt32(&connection, 0) @@ -89,19 +89,22 @@ func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { // OnGeneralEvent implements TiDB Audit plugin's OnGeneralEvent SPI. func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { fmt.Println("## conn_ip_example OnGeneralEvent called ##") - fmt.Printf("---- new connection by %s\n", ctx.Value("ip")) if sctx != nil { fmt.Printf("---- session status: %d\n", sctx.Status) + digest, _ := sctx.StmtCtx.SQLDigest() + fmt.Printf("---- statement sql: %s, digest: %s\n", sctx.StmtCtx.OriginalSQL, digest) + if len(sctx.StmtCtx.Tables) > 0 { + fmt.Printf("---- statement tables: %#v\n", sctx.StmtCtx.Tables) + } + fmt.Printf("---- executed by user: %#v\n", sctx.User) } switch event { - case plugin.Log: - fmt.Println("---- event: Log") + case plugin.Starting: + fmt.Println("---- event: Statement Starting") + case plugin.Completed: + fmt.Println("---- event: Statement Completed") case plugin.Error: - fmt.Println("---- event: Error") - case plugin.Result: - fmt.Println("---- event: Result") - case plugin.Status: - fmt.Println("---- event: Status") + fmt.Println("---- event: ERROR!") default: fmt.Println("---- event: unrecognized") } @@ -117,6 +120,7 @@ func OnConnectionEvent(ctx context.Context, event plugin.ConnectionEvent, info * fmt.Println("## conn_ip_example onConnectionEvent called ##") fmt.Printf("---- conenct event: %s, reason: [%s]\n", event, reason) fmt.Printf("---- connection host: %s\n", info.Host) + fmt.Printf("---- connection details: %s@%s/%s type: %s\n", info.User, info.Host, info.DB, info.ConnectionType) atomic.AddInt32(&connection, 1) return nil } diff --git a/plugin/conn_ip_example/conn_ip_example_test.go b/plugin/conn_ip_example/conn_ip_example_test.go index 811722123cbe1..8b7cd09b04018 100644 --- a/plugin/conn_ip_example/conn_ip_example_test.go +++ b/plugin/conn_ip_example/conn_ip_example_test.go @@ -65,7 +65,7 @@ func TestLoadPlugin(t *testing.T) { require.NoErrorf(t, err, "init plugin [%s] fail, error [%s]\n", pluginSign, err) err = plugin.ForeachPlugin(plugin.Audit, func(auditPlugin *plugin.Plugin) error { - plugin.DeclareAuditManifest(auditPlugin.Manifest).OnGeneralEvent(context.Background(), nil, plugin.Log, "QUERY") + plugin.DeclareAuditManifest(auditPlugin.Manifest).OnGeneralEvent(context.Background(), nil, plugin.Completed, "QUERY") return nil }) require.NoErrorf(t, err, "query event fail, error [%s]\n", err) diff --git a/plugin/plugin.go b/plugin/plugin.go index 368cb79e785c8..572b676aa3450 100644 --- a/plugin/plugin.go +++ b/plugin/plugin.go @@ -438,7 +438,7 @@ func (p *Plugin) supportsFlush(pluginName string) error { return errors.Errorf("plugin '%s' is not ready", pluginName) } if p.Manifest.flushWatcher == nil { - return errors.Errorf("plugin %s does not support flush", pluginName) + return errors.Errorf("plugin %s does not support flush, or PD is not available", pluginName) } return nil } diff --git a/plugin/spi_test.go b/plugin/spi_test.go index 02cd623f01c5b..672656f6a200d 100644 --- a/plugin/spi_test.go +++ b/plugin/spi_test.go @@ -47,7 +47,7 @@ func TestExportManifest(t *testing.T) { err := exported.OnInit(context.Background(), exported) require.NoError(t, err) audit := plugin.DeclareAuditManifest(exported) - audit.OnGeneralEvent(context.Background(), nil, plugin.Log, "QUERY") + audit.OnGeneralEvent(context.Background(), nil, plugin.Completed, "QUERY") require.True(t, callRecorder.NotifyEventCalled) require.True(t, callRecorder.OnInitCalled) } diff --git a/server/column_test.go b/server/column_test.go index 97445b470b51a..ad68e95d69b06 100644 --- a/server/column_test.go +++ b/server/column_test.go @@ -15,16 +15,15 @@ package server import ( - . "github.com/pingcap/check" + "testing" + "github.com/pingcap/parser/mysql" + "github.com/stretchr/testify/require" ) -type ColumnTestSuite struct { -} - -var _ = Suite(new(ColumnTestSuite)) +func TestDumpColumn(t *testing.T) { + t.Parallel() -func (s ColumnTestSuite) TestDumpColumn(c *C) { info := ColumnInfo{ Schema: "testSchema", Table: "testTable", @@ -41,18 +40,20 @@ func (s ColumnTestSuite) TestDumpColumn(c *C) { } r := info.Dump(nil) exp := []byte{0x3, 0x64, 0x65, 0x66, 0xa, 0x74, 0x65, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x9, 0x74, 0x65, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0xc, 0x74, 0x65, 0x73, 0x74, 0x4f, 0x72, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0x8, 0x74, 0x65, 0x73, 0x74, 0x4e, 0x61, 0x6d, 0x65, 0xb, 0x74, 0x65, 0x73, 0x74, 0x4f, 0x72, 0x67, 0x4e, 0x61, 0x6d, 0x65, 0xc, 0x6a, 0x0, 0x1, 0x0, 0x0, 0x0, 0xe, 0x0, 0x0, 0x1, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5, 0x2} - c.Assert(r, DeepEquals, exp) + require.Equal(t, exp, r) - c.Assert(dumpFlag(mysql.TypeSet, 0), Equals, uint16(mysql.SetFlag)) - c.Assert(dumpFlag(mysql.TypeEnum, 0), Equals, uint16(mysql.EnumFlag)) - c.Assert(dumpFlag(mysql.TypeString, 0), Equals, uint16(0)) + require.Equal(t, uint16(mysql.SetFlag), dumpFlag(mysql.TypeSet, 0)) + require.Equal(t, uint16(mysql.EnumFlag), dumpFlag(mysql.TypeEnum, 0)) + require.Equal(t, uint16(0), dumpFlag(mysql.TypeString, 0)) - c.Assert(dumpType(mysql.TypeSet), Equals, mysql.TypeString) - c.Assert(dumpType(mysql.TypeEnum), Equals, mysql.TypeString) - c.Assert(dumpType(mysql.TypeBit), Equals, mysql.TypeBit) + require.Equal(t, mysql.TypeString, dumpType(mysql.TypeSet)) + require.Equal(t, mysql.TypeString, dumpType(mysql.TypeEnum)) + require.Equal(t, mysql.TypeBit, dumpType(mysql.TypeBit)) } -func (s ColumnTestSuite) TestColumnNameLimit(c *C) { +func TestColumnNameLimit(t *testing.T) { + t.Parallel() + aLongName := make([]byte, 0, 300) for i := 0; i < 300; i++ { aLongName = append(aLongName, 'a') @@ -73,5 +74,5 @@ func (s ColumnTestSuite) TestColumnNameLimit(c *C) { } r := info.Dump(nil) exp := []byte{0x3, 0x64, 0x65, 0x66, 0xa, 0x74, 0x65, 0x73, 0x74, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x9, 0x74, 0x65, 0x73, 0x74, 0x54, 0x61, 0x62, 0x6c, 0x65, 0xc, 0x74, 0x65, 0x73, 0x74, 0x4f, 0x72, 0x67, 0x54, 0x61, 0x62, 0x6c, 0x65, 0xfc, 0x0, 0x1, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0x61, 0xb, 0x74, 0x65, 0x73, 0x74, 0x4f, 0x72, 0x67, 0x4e, 0x61, 0x6d, 0x65, 0xc, 0x6a, 0x0, 0x1, 0x0, 0x0, 0x0, 0xe, 0x0, 0x0, 0x1, 0x0, 0x0, 0x2, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x5, 0x2} - c.Assert(r, DeepEquals, exp) + require.Equal(t, exp, r) } diff --git a/server/conn.go b/server/conn.go index 400a6bf57b55f..a2135bfb3d8fa 100644 --- a/server/conn.go +++ b/server/conn.go @@ -977,6 +977,7 @@ func (cc *clientConn) Run(ctx context.Context) { startTime := time.Now() if err = cc.dispatch(ctx, data); err != nil { + cc.audit(plugin.Error) // tell the plugin API there was a dispatch error if terror.ErrorEqual(err, io.EOF) { cc.addMetrics(data[0], startTime, nil) disconnectNormal.Inc() @@ -1627,6 +1628,21 @@ func (cc *clientConn) handlePlanRecreator(ctx context.Context, info executor.Pla return "", errors.New("plan recreator: not supporting info type") } +func (cc *clientConn) audit(eventType plugin.GeneralEvent) { + err := plugin.ForeachPlugin(plugin.Audit, func(p *plugin.Plugin) error { + audit := plugin.DeclareAuditManifest(p.Manifest) + if audit.OnGeneralEvent != nil { + cmd := mysql.Command2Str[byte(atomic.LoadUint32(&cc.ctx.GetSessionVars().CommandValue))] + ctx := context.WithValue(context.Background(), plugin.ExecStartTimeCtxKey, cc.ctx.GetSessionVars().StartTime) + audit.OnGeneralEvent(ctx, cc.ctx.GetSessionVars(), eventType, cmd) + } + return nil + }) + if err != nil { + terror.Log(err) + } +} + // handleQuery executes the sql query string and writes result set or result ok to the client. // As the execution time of this function represents the performance of TiDB, we do time log and metrics here. // There is a special query `load data` that does not return result, which is handled differently. @@ -1823,6 +1839,7 @@ func (cc *clientConn) handleStmt(ctx context.Context, stmt ast.StmtNode, warns [ ctx = context.WithValue(ctx, execdetails.StmtExecDetailKey, &execdetails.StmtExecDetails{}) ctx = context.WithValue(ctx, util.ExecDetailsKey, &util.ExecDetails{}) reg := trace.StartRegion(ctx, "ExecuteStmt") + cc.audit(plugin.Starting) rs, err := cc.ctx.ExecuteStmt(ctx, stmt) reg.End() // The session tracker detachment from global tracker is solved in the `rs.Close` in most cases. diff --git a/session/session.go b/session/session.go index 0d6ec5330b146..75b8cb84ef38c 100644 --- a/session/session.go +++ b/session/session.go @@ -2436,6 +2436,9 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { if newCollationEnabled { collate.EnableNewCollations() + if cfg.Experimental.EnableNewCharset { + collate.EnableNewCharset() + } } newMemoryQuotaQuery, err := loadDefMemQuotaQuery(se) diff --git a/session/session_test.go b/session/session_test.go index 887c3b6abfb99..7137167a34484 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -4486,7 +4486,7 @@ func (s *testTxnStateSerialSuite) TestRunning(c *C) { tk.MustExec("create table t(a int);") tk.MustExec("insert into t(a) values (1);") tk.MustExec("begin pessimistic;") - c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockStmtSlow", "pause"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/beforeLockKeys", "pause"), IsNil) ch := make(chan struct{}) go func() { tk.MustExec("select * from t for update;") @@ -4496,7 +4496,7 @@ func (s *testTxnStateSerialSuite) TestRunning(c *C) { time.Sleep(100 * time.Millisecond) info := tk.Se.TxnInfo() c.Assert(info.State, Equals, txninfo.TxnRunning) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockStmtSlow"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/beforeLockKeys"), IsNil) <-ch } diff --git a/statistics/handle/handle_test.go b/statistics/handle/handle_test.go index 533a15ad54e2f..53b9549d5e7ba 100644 --- a/statistics/handle/handle_test.go +++ b/statistics/handle/handle_test.go @@ -878,7 +878,7 @@ func (s *testStatsSuite) TestBuildGlobalLevelStats(c *C) { } // nolint:unused -func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, tblName, dbName string) { +func (s *testSerialStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, tblName, dbName string) { tk.MustExec("create database if not exists " + dbName) tk.MustExec("use " + dbName) tk.MustExec("drop table if exists " + tblName) @@ -902,7 +902,7 @@ func (s *testStatsSuite) prepareForGlobalStatsWithOpts(c *C, tk *testkit.TestKit } // nolint:unused -func (s *testStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, t string, p string, topn, buckets int) { +func (s *testSerialStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, t string, p string, topn, buckets int) { delta := buckets/2 + 1 for _, isIdx := range []int{0, 1} { c.Assert(len(tk.MustQuery(fmt.Sprintf("show stats_topn where table_name='%v' and partition_name='%v' and is_index=%v", t, p, isIdx)).Rows()), Equals, topn) @@ -914,7 +914,7 @@ func (s *testStatsSuite) checkForGlobalStatsWithOpts(c *C, tk *testkit.TestKit, } } -func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { +func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") } @@ -953,7 +953,7 @@ func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts(c *C) { } } -func (s *testStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { +func (s *testSerialStatsSuite) TestAnalyzeGlobalStatsWithOpts2(c *C) { if israce.RaceEnabled { c.Skip("exhaustive types test, skip race test") } diff --git a/testkit/testkit.go b/testkit/testkit.go index 19d515fb2eba7..1f0564d974911 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -19,6 +19,7 @@ package testkit import ( "context" "fmt" + "strings" "testing" "github.com/pingcap/errors" @@ -60,10 +61,10 @@ func (tk *TestKit) Session() session.Session { func (tk *TestKit) MustExec(sql string, args ...interface{}) { res, err := tk.Exec(sql, args...) comment := fmt.Sprintf("sql:%s, %v, error stack %v", sql, args, errors.ErrorStack(err)) - tk.require.Nil(err, comment) + tk.require.NoError(err, comment) if res != nil { - tk.require.Nil(res.Close()) + tk.require.NoError(res.Close()) } } @@ -84,7 +85,7 @@ func (tk *TestKit) QueryToErr(sql string, args ...interface{}) error { tk.require.NoError(err, comment) tk.require.NotNil(res, comment) _, resErr := session.GetRows4Test(context.Background(), tk.session, res) - tk.require.Nil(res.Close()) + tk.require.NoError(res.Close()) return resErr } @@ -101,6 +102,17 @@ func (tk *TestKit) ResultSetToResultWithCtx(ctx context.Context, rs sqlexec.Reco return &Result{rows: rows, comment: comment, assert: tk.assert, require: tk.require} } +// HasPlan checks if the result execution plan contains specific plan. +func (tk *TestKit) HasPlan(sql string, plan string, args ...interface{}) bool { + rs := tk.MustQuery("explain "+sql, args...) + for i := range rs.rows { + if strings.Contains(rs.rows[i][0], plan) { + return true + } + } + return false +} + // Exec executes a sql statement using the prepared stmt API func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, error) { ctx := context.Background() @@ -149,9 +161,18 @@ func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, err return rs, nil } +// ExecToErr executes a sql statement and discard results. +func (tk *TestKit) ExecToErr(sql string, args ...interface{}) error { + res, err := tk.Exec(sql, args...) + if res != nil { + tk.require.NoError(res.Close()) + } + return err +} + func newSession(t *testing.T, store kv.Storage) session.Session { se, err := session.CreateSession4Test(store) - require.Nil(t, err) + require.NoError(t, err) se.SetConnectionID(testKitIDGenerator.Inc()) return se } diff --git a/tools/check/check-timeout.go b/tools/check/check-timeout.go index 22f62dfa1de93..942bf76b7efd3 100644 --- a/tools/check/check-timeout.go +++ b/tools/check/check-timeout.go @@ -99,6 +99,7 @@ func init() { "testIntegrationSuite2.TestPartitionCancelAddPrimaryKey", "testDBSuite1.TestAddIndexWithSplitTable", "testSerialDBSuite.TestAddIndexWithShardRowID", + "testColumnTypeChangeSuite.TestColumnTypeChangeFromDateTimeTypeToOthers", } for _, v := range tmp { allowList[v] = struct{}{} diff --git a/util/admin/admin_integration_test.go b/util/admin/admin_integration_test.go index fc222265713ac..056c6c826b5d8 100644 --- a/util/admin/admin_integration_test.go +++ b/util/admin/admin_integration_test.go @@ -45,6 +45,7 @@ func TestAdminCheckTable(t *testing.T) { tk.MustExec("ALTER TABLE t1 ADD COLUMN cc2 VARCHAR(36) NULL DEFAULT ''") tk.MustExec("ALTER TABLE t1 ADD INDEX idx1 (cc1);") tk.MustExec("ALTER TABLE t1 ADD INDEX idx2 (cc2);") + tk.MustExec("ALTER TABLE t1 engine=innodb;") tk.MustExec("admin check table t1;") // For add index on virtual column diff --git a/util/codec/bytes.go b/util/codec/bytes.go index c061146ba191a..c4f61442822fe 100644 --- a/util/codec/bytes.go +++ b/util/codec/bytes.go @@ -161,7 +161,7 @@ func DecodeCompactBytes(b []byte) ([]byte, []byte, error) { if err != nil { return nil, nil, errors.Trace(err) } - if int64(len(b)) < n { + if n < 0 || int64(len(b)) < n { return nil, nil, errors.Errorf("insufficient bytes to decode value, expected length: %v", n) } return b[n:], b[:n], nil diff --git a/util/collate/charset.go b/util/collate/charset.go new file mode 100644 index 0000000000000..76f00b3774518 --- /dev/null +++ b/util/collate/charset.go @@ -0,0 +1,84 @@ +// Copyright 2021 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 collate + +import ( + "strings" + + "github.com/pingcap/parser/charset" +) + +var ( + enableCharsetFeat bool +) + +// EnableNewCharset enables the charset feature. +func EnableNewCharset() { + enableCharsetFeat = true + addCharset() +} + +// SetCharsetFeatEnabledForTest set charset feature enabled. Only used in test. +// It will also enable or disable new collation. +func SetCharsetFeatEnabledForTest(flag bool) { + enableCharsetFeat = flag + SetNewCollationEnabledForTest(flag) + if flag { + addCharset() + } else { + removeCharset() + } +} + +// CharsetFeatEnabled return true if charset feature is enabled. +func CharsetFeatEnabled() bool { + return enableCharsetFeat +} + +func addCharset() { + for _, c := range experimentalCharsetInfo { + charset.AddCharset(c) + for _, coll := range charset.GetCollations() { + if strings.EqualFold(coll.CharsetName, c.Name) { + charset.AddCollation(coll) + } + } + } + + for name, collator := range experimentalCollation { + newCollatorMap[name] = collator + newCollatorIDMap[CollationName2ID(name)] = collator + } +} + +func removeCharset() { + for _, c := range experimentalCharsetInfo { + charset.RemoveCharset(c.Name) + } + + for name := range experimentalCollation { + delete(newCollatorMap, name) + delete(newCollatorIDMap, CollationName2ID(name)) + } +} + +// All the experimental supported charset should be in the following table, only used when charset feature is enable. +var experimentalCharsetInfo = []*charset.Charset{ + {Name: charset.CharsetGBK, DefaultCollation: charset.CollationGBKBin, Collations: make(map[string]*charset.Collation), Desc: "Chinese Internal Code Specification", Maxlen: 2}, +} + +var experimentalCollation = map[string]Collator{ + charset.CollationGBKBin: &gbkBinCollator{}, +} diff --git a/util/collate/collate_test.go b/util/collate/collate_test.go index 0b779cfc2fcaa..bbe530af34037 100644 --- a/util/collate/collate_test.go +++ b/util/collate/collate_test.go @@ -177,6 +177,7 @@ func TestGetCollator(t *testing.T) { SetNewCollationEnabledForTest(true) defer SetNewCollationEnabledForTest(false) require.IsType(t, &binCollator{}, GetCollator("binary")) + require.IsType(t, &binPaddingCollator{}, GetCollator("gbk_bin")) require.IsType(t, &binPaddingCollator{}, GetCollator("utf8mb4_bin")) require.IsType(t, &binPaddingCollator{}, GetCollator("utf8_bin")) require.IsType(t, &generalCICollator{}, GetCollator("utf8mb4_general_ci")) @@ -186,6 +187,7 @@ func TestGetCollator(t *testing.T) { require.IsType(t, &zhPinyinTiDBASCSCollator{}, GetCollator("utf8mb4_zh_pinyin_tidb_as_cs")) require.IsType(t, &binPaddingCollator{}, GetCollator("default_test")) require.IsType(t, &binCollator{}, GetCollatorByID(63)) + require.IsType(t, &binPaddingCollator{}, GetCollatorByID(87)) require.IsType(t, &binPaddingCollator{}, GetCollatorByID(46)) require.IsType(t, &binPaddingCollator{}, GetCollatorByID(83)) require.IsType(t, &generalCICollator{}, GetCollatorByID(45)) @@ -214,4 +216,9 @@ func TestGetCollator(t *testing.T) { require.IsType(t, &binCollator{}, GetCollatorByID(192)) require.IsType(t, &binCollator{}, GetCollatorByID(2048)) require.IsType(t, &binCollator{}, GetCollatorByID(9999)) + + SetCharsetFeatEnabledForTest(true) + defer SetCharsetFeatEnabledForTest(false) + require.IsType(t, &gbkBinCollator{}, GetCollator("gbk_bin")) + require.IsType(t, &gbkBinCollator{}, GetCollatorByID(87)) } diff --git a/util/collate/gbk_bin.go b/util/collate/gbk_bin.go new file mode 100644 index 0000000000000..0ac832ef62cfe --- /dev/null +++ b/util/collate/gbk_bin.go @@ -0,0 +1,20 @@ +// Copyright 2021 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 collate + +// gbkBinCollator is collator for gbk_bin, use binCollator instead temporary. +type gbkBinCollator struct { + binCollator +} diff --git a/util/placementpolicy/policy.go b/util/placementpolicy/policy.go new file mode 100644 index 0000000000000..4768e76ed43aa --- /dev/null +++ b/util/placementpolicy/policy.go @@ -0,0 +1,36 @@ +// Copyright 2021 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 placementpolicy + +import ( + "github.com/pingcap/parser/model" +) + +// PolicyInfo is the struct to store the placement policy. +type PolicyInfo struct { + ID int64 `json:"id"` + Name model.CIStr `json:"name"` + PrimaryRegion string `json:"primary_region"` + Regions string `json:"regions"` + Learners uint64 `json:"learners"` + Followers uint64 `json:"followers"` + Voters uint64 `json:"voters"` + Schedule string `json:"schedule"` + Constraints string `json:"constraints"` + LearnerConstraints string `json:"learner_constraints"` + FollowerConstraints string `json:"follower_constraints"` + VoterConstraints string `json:"voter_constraints"` + State model.SchemaState `json:"state"` +} diff --git a/util/stmtsummary/reader.go b/util/stmtsummary/reader.go index 16e88550656be..f337060415de0 100644 --- a/util/stmtsummary/reader.go +++ b/util/stmtsummary/reader.go @@ -31,8 +31,9 @@ import ( // stmtSummaryReader uses to read the statement summaries data and convert to []datum row. type stmtSummaryReader struct { - user *auth.UserIdentity - isSuper bool + user *auth.UserIdentity + // If the user has the 'PROCESS' privilege, he can read all the statements. + hasProcessPriv bool columns []*model.ColumnInfo instanceAddr string ssMap *stmtSummaryByDigestMap @@ -40,13 +41,13 @@ type stmtSummaryReader struct { } // NewStmtSummaryReader return a new statement summaries reader. -func NewStmtSummaryReader(user *auth.UserIdentity, isSuper bool, cols []*model.ColumnInfo, instanceAddr string) *stmtSummaryReader { +func NewStmtSummaryReader(user *auth.UserIdentity, hasProcessPriv bool, cols []*model.ColumnInfo, instanceAddr string) *stmtSummaryReader { reader := &stmtSummaryReader{ - user: user, - isSuper: isSuper, - columns: cols, - instanceAddr: instanceAddr, - ssMap: StmtSummaryByDigestMap, + user: user, + hasProcessPriv: hasProcessPriv, + columns: cols, + instanceAddr: instanceAddr, + ssMap: StmtSummaryByDigestMap, } // initialize column value factories. reader.columnValueFactories = make([]columnValueFactory, len(reader.columns)) @@ -120,7 +121,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.isSuper { + if ssr.user != nil && !ssr.hasProcessPriv && ssElement != nil { _, isAuthed = ssElement.authUsers[ssr.user.Username] } if ssElement == nil || ssElement.beginTime < beginTimeForCurInterval || !isAuthed { @@ -146,7 +147,7 @@ func (ssr *stmtSummaryReader) getStmtByDigestHistoryRow(ssbd *stmtSummaryByDiges rows := make([][]types.Datum, 0, len(ssElements)) for _, ssElement := range ssElements { isAuthed := true - if ssr.user != nil && !ssr.isSuper { + if ssr.user != nil && !ssr.hasProcessPriv { _, isAuthed = ssElement.authUsers[ssr.user.Username] } if isAuthed { diff --git a/util/stmtsummary/statement_summary_test.go b/util/stmtsummary/statement_summary_test.go index 16a9b43875ca0..851ccd66ba457 100644 --- a/util/stmtsummary/statement_summary_test.go +++ b/util/stmtsummary/statement_summary_test.go @@ -1418,26 +1418,26 @@ func TestAccessPrivilege(t *testing.T) { reader := newStmtSummaryReaderForTest(ssMap) reader.user = user - reader.isSuper = false + reader.hasProcessPriv = false datums := reader.GetStmtSummaryCurrentRows() require.Len(t, datums, loops) reader.user = badUser - reader.isSuper = false + reader.hasProcessPriv = false datums = reader.GetStmtSummaryCurrentRows() require.Len(t, datums, 0) - reader.isSuper = true + reader.hasProcessPriv = true datums = reader.GetStmtSummaryCurrentRows() require.Len(t, datums, loops) reader.user = user - reader.isSuper = false + reader.hasProcessPriv = false datums = reader.GetStmtSummaryHistoryRows() require.Len(t, datums, loops) reader.user = badUser - reader.isSuper = false + reader.hasProcessPriv = false datums = reader.GetStmtSummaryHistoryRows() require.Len(t, datums, 0) - reader.isSuper = true + reader.hasProcessPriv = true datums = reader.GetStmtSummaryHistoryRows() require.Len(t, datums, loops) }