diff --git a/.github/workflows/issue_assigned.yml b/.github/workflows/issue_assigned.yml deleted file mode 100644 index 90735cf64b36c..0000000000000 --- a/.github/workflows/issue_assigned.yml +++ /dev/null @@ -1,20 +0,0 @@ -name: Auto Assign Project Local - -on: - issues: - types: [assigned] -env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - -jobs: - assign_to_project_column: - runs-on: ubuntu-latest - name: Assign to Project Column - steps: - - name: Run issues assignment to Robust test - uses: srggrs/assign-one-project-github-action@1.2.0 - if: | - contains(github.event.issue.labels.*.name, 'component/test') - with: - project: 'https://github.com/pingcap/tidb/projects/32' - column_name: 'Task Assigned' diff --git a/br/cmd/tidb-lightning-ctl/main.go b/br/cmd/tidb-lightning-ctl/main.go index 7758621ac5bf3..66b616af57e3e 100644 --- a/br/cmd/tidb-lightning-ctl/main.go +++ b/br/cmd/tidb-lightning-ctl/main.go @@ -438,7 +438,11 @@ func importEngine(ctx context.Context, cfg *config.Config, tls *common.TLS, engi return errors.Trace(err) } - return errors.Trace(ce.Import(ctx)) + regionSplitSize := int64(cfg.TikvImporter.RegionSplitSize) + if regionSplitSize == 0 { + regionSplitSize = int64(config.SplitRegionSize) + } + return errors.Trace(ce.Import(ctx, regionSplitSize)) } func cleanupEngine(ctx context.Context, cfg *config.Config, tls *common.TLS, engine string) error { diff --git a/br/pkg/backup/client.go b/br/pkg/backup/client.go index a31acfb85979e..184f2d4294fff 100644 --- a/br/pkg/backup/client.go +++ b/br/pkg/backup/client.go @@ -268,10 +268,6 @@ func BuildBackupRangeAndSchema( continue } - idAlloc := autoid.NewAllocator(storage, dbInfo.ID, false, autoid.RowIDAllocType) - seqAlloc := autoid.NewAllocator(storage, dbInfo.ID, false, autoid.SequenceType) - randAlloc := autoid.NewAllocator(storage, dbInfo.ID, false, autoid.AutoRandomType) - tables, err := m.ListTables(dbInfo.ID) if err != nil { return nil, nil, errors.Trace(err) @@ -294,14 +290,19 @@ func BuildBackupRangeAndSchema( zap.String("table", tableInfo.Name.O), ) + tblVer := autoid.AllocOptionTableInfoVersion(tableInfo.Version) + idAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.RowIDAllocType, tblVer) + seqAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.SequenceType, tblVer) + randAlloc := autoid.NewAllocator(storage, dbInfo.ID, tableInfo.ID, false, autoid.AutoRandomType, tblVer) + var globalAutoID int64 switch { case tableInfo.IsSequence(): - globalAutoID, err = seqAlloc.NextGlobalAutoID(tableInfo.ID) + globalAutoID, err = seqAlloc.NextGlobalAutoID() case tableInfo.IsView() || !utils.NeedAutoID(tableInfo): // no auto ID for views or table without either rowID nor auto_increment ID. default: - globalAutoID, err = idAlloc.NextGlobalAutoID(tableInfo.ID) + globalAutoID, err = idAlloc.NextGlobalAutoID() } if err != nil { return nil, nil, errors.Trace(err) @@ -311,7 +312,7 @@ func BuildBackupRangeAndSchema( if tableInfo.PKIsHandle && tableInfo.ContainsAutoRandomBits() { // this table has auto_random id, we need backup and rebase in restoration var globalAutoRandID int64 - globalAutoRandID, err = randAlloc.NextGlobalAutoID(tableInfo.ID) + globalAutoRandID, err = randAlloc.NextGlobalAutoID() if err != nil { return nil, nil, errors.Trace(err) } diff --git a/br/pkg/lightning/backend/backend.go b/br/pkg/lightning/backend/backend.go index 0a775c4a2b015..29b8981f000cc 100644 --- a/br/pkg/lightning/backend/backend.go +++ b/br/pkg/lightning/backend/backend.go @@ -151,7 +151,7 @@ type AbstractBackend interface { // ImportEngine imports engine data to the backend. If it returns ErrDuplicateDetected, // it means there is duplicate detected. For this situation, all data in the engine must be imported. // It's safe to reset or cleanup this engine. - ImportEngine(ctx context.Context, engineUUID uuid.UUID) error + ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error @@ -310,7 +310,7 @@ func (be Backend) CheckDiskQuota(quota int64) ( // into the target and then reset the engine to empty. This method will not // close the engine. Make sure the engine is flushed manually before calling // this method. -func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID) error { +func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { // DO NOT call be.abstract.CloseEngine()! The engine should still be writable after // calling UnsafeImportAndReset(). closedEngine := ClosedEngine{ @@ -320,7 +320,7 @@ func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID uuid: engineUUID, }, } - if err := closedEngine.Import(ctx); err != nil { + if err := closedEngine.Import(ctx, regionSplitSize); err != nil { return err } return be.abstract.ResetEngine(ctx, engineUUID) @@ -436,12 +436,12 @@ func (en engine) unsafeClose(ctx context.Context, cfg *EngineConfig) (*ClosedEng } // Import the data written to the engine into the target. -func (engine *ClosedEngine) Import(ctx context.Context) error { +func (engine *ClosedEngine) Import(ctx context.Context, regionSplitSize int64) error { var err error for i := 0; i < importMaxRetryTimes; i++ { task := engine.logger.With(zap.Int("retryCnt", i)).Begin(zap.InfoLevel, "import") - err = engine.backend.ImportEngine(ctx, engine.uuid) + err = engine.backend.ImportEngine(ctx, engine.uuid, regionSplitSize) if !common.IsRetryableError(err) { task.End(zap.ErrorLevel, err) return err diff --git a/br/pkg/lightning/backend/backend_test.go b/br/pkg/lightning/backend/backend_test.go index 42404f5f3a29b..db79f4b28c806 100644 --- a/br/pkg/lightning/backend/backend_test.go +++ b/br/pkg/lightning/backend/backend_test.go @@ -58,7 +58,7 @@ func (s *backendSuite) TestOpenCloseImportCleanUpEngine(c *C) { Return(nil). After(openCall) importCall := s.mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID). + ImportEngine(ctx, engineUUID, gomock.Any()). Return(nil). After(closeCall) s.mockBackend.EXPECT(). @@ -70,7 +70,7 @@ func (s *backendSuite) TestOpenCloseImportCleanUpEngine(c *C) { c.Assert(err, IsNil) closedEngine, err := engine.Close(ctx, nil) c.Assert(err, IsNil) - err = closedEngine.Import(ctx) + err = closedEngine.Import(ctx, 1) c.Assert(err, IsNil) err = closedEngine.Cleanup(ctx) c.Assert(err, IsNil) @@ -252,12 +252,12 @@ func (s *backendSuite) TestImportFailedNoRetry(c *C) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any()). Return(errors.Annotate(context.Canceled, "fake unrecoverable import error")) closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) c.Assert(err, IsNil) - err = closedEngine.Import(ctx) + err = closedEngine.Import(ctx, 1) c.Assert(err, ErrorMatches, "fake unrecoverable import error.*") } @@ -269,14 +269,14 @@ func (s *backendSuite) TestImportFailedWithRetry(c *C) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any()). Return(errors.New("fake recoverable import error")). MinTimes(2) s.mockBackend.EXPECT().RetryImportDelay().Return(time.Duration(0)).AnyTimes() closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) c.Assert(err, IsNil) - err = closedEngine.Import(ctx) + err = closedEngine.Import(ctx, 1) c.Assert(err, ErrorMatches, ".*fake recoverable import error") } @@ -288,16 +288,16 @@ func (s *backendSuite) TestImportFailedRecovered(c *C) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any()). Return(errors.New("fake recoverable import error")) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any()). Return(nil) s.mockBackend.EXPECT().RetryImportDelay().Return(time.Duration(0)).AnyTimes() closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) c.Assert(err, IsNil) - err = closedEngine.Import(ctx) + err = closedEngine.Import(ctx, 1) c.Assert(err, IsNil) } diff --git a/br/pkg/lightning/backend/importer/importer.go b/br/pkg/lightning/backend/importer/importer.go index dc292bdb7e870..f4cb73930d80c 100644 --- a/br/pkg/lightning/backend/importer/importer.go +++ b/br/pkg/lightning/backend/importer/importer.go @@ -201,7 +201,7 @@ func (importer *importer) Flush(_ context.Context, _ uuid.UUID) error { return nil } -func (importer *importer) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error { +func (importer *importer) ImportEngine(ctx context.Context, engineUUID uuid.UUID, _ int64) error { importer.lock.Lock() defer importer.lock.Unlock() req := &import_kvpb.ImportEngineRequest{ diff --git a/br/pkg/lightning/backend/importer/importer_test.go b/br/pkg/lightning/backend/importer/importer_test.go index 524a523d2e31d..5d75d1badc245 100644 --- a/br/pkg/lightning/backend/importer/importer_test.go +++ b/br/pkg/lightning/backend/importer/importer_test.go @@ -219,7 +219,7 @@ func (s *importerSuite) TestCloseImportCleanupEngine(c *C) { engine, err := s.engine.Close(s.ctx, nil) c.Assert(err, IsNil) - err = engine.Import(s.ctx) + err = engine.Import(s.ctx, 1) c.Assert(err, IsNil) err = engine.Cleanup(s.ctx) c.Assert(err, IsNil) diff --git a/br/pkg/lightning/backend/kv/allocator.go b/br/pkg/lightning/backend/kv/allocator.go index ff86f72524e60..1c33e346b6bf6 100644 --- a/br/pkg/lightning/backend/kv/allocator.go +++ b/br/pkg/lightning/backend/kv/allocator.go @@ -40,7 +40,7 @@ func NewPanickingAllocators(base int64) autoid.Allocators { } // Rebase implements the autoid.Allocator interface -func (alloc *panickingAllocator) Rebase(tableID, newBase int64, allocIDs bool) error { +func (alloc *panickingAllocator) Rebase(newBase int64, allocIDs bool) error { // CAS for { oldBase := atomic.LoadInt64(alloc.base) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 4a78cd6b4c125..ddaad8c357717 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -807,9 +807,7 @@ type local struct { pdAddr string g glue.Glue - localStoreDir string - regionSplitSize int64 - regionSplitKeys int64 + localStoreDir string rangeConcurrency *worker.Pool ingestConcurrency *worker.Pool @@ -939,12 +937,6 @@ func NewLocalBackend( } } - regionSplitSize := int64(cfg.RegionSplitSize) - regionSplitKeys := int64(regionMaxKeyCount) - if regionSplitSize > defaultRegionSplitSize { - regionSplitKeys = int64(float64(regionSplitSize) / float64(defaultRegionSplitSize) * float64(regionMaxKeyCount)) - } - local := &local{ engines: sync.Map{}, pdCtl: pdCtl, @@ -953,10 +945,7 @@ func NewLocalBackend( pdAddr: pdAddr, g: g, - localStoreDir: localFile, - regionSplitSize: regionSplitSize, - regionSplitKeys: regionSplitKeys, - + localStoreDir: localFile, rangeConcurrency: worker.NewPool(ctx, rangeConcurrency, "range"), ingestConcurrency: worker.NewPool(ctx, rangeConcurrency*2, "ingest"), tcpConcurrency: rangeConcurrency, @@ -1185,11 +1174,6 @@ func (local *local) RetryImportDelay() time.Duration { return defaultRetryBackoffTime } -func (local *local) MaxChunkSize() int { - // a batch size write to leveldb - return int(local.regionSplitSize) -} - func (local *local) ShouldPostProcess() bool { return true } @@ -1365,6 +1349,8 @@ func (local *local) WriteToTiKV( engineFile *File, region *split.RegionInfo, start, end []byte, + regionSplitSize int64, + regionSplitKeys int64, ) ([]*sst.SSTMeta, Range, rangeStats, error) { for _, peer := range region.Region.GetPeers() { var e error @@ -1463,7 +1449,7 @@ func (local *local) WriteToTiKV( size := int64(0) totalCount := int64(0) firstLoop := true - regionMaxSize := local.regionSplitSize * 4 / 3 + regionMaxSize := regionSplitSize * 4 / 3 for iter.First(); iter.Valid(); iter.Next() { size += int64(len(iter.Key()) + len(iter.Value())) @@ -1492,7 +1478,7 @@ func (local *local) WriteToTiKV( bytesBuf.Reset() firstLoop = false } - if size >= regionMaxSize || totalCount >= local.regionSplitKeys { + if size >= regionMaxSize || totalCount >= regionSplitKeys { break } } @@ -1624,7 +1610,7 @@ func splitRangeBySizeProps(fullRange Range, sizeProps *sizeProperties, sizeLimit return ranges } -func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File) ([]Range, error) { +func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File, regionSplitSize int64, regionSplitKeys int64) ([]Range, error) { iter := newKeyIter(ctx, engineFile, &pebble.IterOptions{}) defer iter.Close() @@ -1653,7 +1639,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File) engineFileLength := engineFile.Length.Load() // <= 96MB no need to split into range - if engineFileTotalSize <= local.regionSplitSize && engineFileLength <= local.regionSplitKeys { + if engineFileTotalSize <= regionSplitSize && engineFileLength <= regionSplitKeys { ranges := []Range{{start: firstKey, end: endKey}} return ranges, nil } @@ -1664,7 +1650,7 @@ func (local *local) readAndSplitIntoRange(ctx context.Context, engineFile *File) } ranges := splitRangeBySizeProps(Range{start: firstKey, end: endKey}, sizeProps, - local.regionSplitSize, local.regionSplitKeys) + regionSplitSize, regionSplitKeys) log.L().Info("split engine key ranges", zap.Stringer("engine", engineFile.UUID), zap.Int64("totalSize", engineFileTotalSize), zap.Int64("totalCount", engineFileLength), @@ -1678,6 +1664,8 @@ func (local *local) writeAndIngestByRange( ctxt context.Context, engineFile *File, start, end []byte, + regionSplitSize int64, + regionSplitKeys int64, ) error { ito := &pebble.IterOptions{ LowerBound: start, @@ -1736,7 +1724,7 @@ WriteAndIngest: zap.Binary("end", region.Region.GetEndKey()), zap.Reflect("peers", region.Region.GetPeers())) w := local.ingestConcurrency.Apply() - err = local.writeAndIngestPairs(ctx, engineFile, region, pairStart, end) + err = local.writeAndIngestPairs(ctx, engineFile, region, pairStart, end, regionSplitSize, regionSplitKeys) local.ingestConcurrency.Recycle(w) if err != nil { if common.IsContextCanceledError(err) { @@ -1774,6 +1762,8 @@ func (local *local) writeAndIngestPairs( engineFile *File, region *split.RegionInfo, start, end []byte, + regionSplitSize int64, + regionSplitKeys int64, ) error { var err error @@ -1782,7 +1772,7 @@ loopWrite: var metas []*sst.SSTMeta var finishedRange Range var rangeStats rangeStats - metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engineFile, region, start, end) + metas, finishedRange, rangeStats, err = local.WriteToTiKV(ctx, engineFile, region, start, end, regionSplitSize, regionSplitKeys) if err != nil { if common.IsContextCanceledError(err) { return err @@ -1889,7 +1879,7 @@ loopWrite: return errors.Trace(err) } -func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File, ranges []Range) error { +func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File, ranges []Range, regionSplitSize int64, regionSplitKeys int64) error { if engineFile.Length.Load() == 0 { // engine is empty, this is likes because it's a index engine but the table contains no index log.L().Info("engine contains no data", zap.Stringer("uuid", engineFile.UUID)) @@ -1921,7 +1911,7 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engineFile *File // max retry backoff time: 2+4+8+16=30s backOffTime := time.Second for i := 0; i < maxRetryTimes; i++ { - err = local.writeAndIngestByRange(ctx, engineFile, startKey, endKey) + err = local.writeAndIngestByRange(ctx, engineFile, startKey, endKey, regionSplitSize, regionSplitKeys) if err == nil || common.IsContextCanceledError(err) { return } @@ -1967,7 +1957,7 @@ func (r *syncedRanges) reset() { r.Unlock() } -func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error { +func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { lf := local.lockEngine(engineUUID, importMutexStateImport) if lf == nil { // skip if engine not exist. See the comment of `CloseEngine` for more detail. @@ -1981,9 +1971,13 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) erro log.L().Info("engine contains no kv, skip import", zap.Stringer("engine", engineUUID)) return nil } + regionSplitKeys := int64(regionMaxKeyCount) + if regionSplitSize > defaultRegionSplitSize { + regionSplitKeys = int64(float64(regionSplitSize) / float64(defaultRegionSplitSize) * float64(regionMaxKeyCount)) + } // split sorted file into range by 96MB size per file - ranges, err := local.readAndSplitIntoRange(ctx, lf) + ranges, err := local.readAndSplitIntoRange(ctx, lf, regionSplitSize, regionSplitKeys) if err != nil { return err } @@ -1999,10 +1993,10 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) erro // if all the kv can fit in one region, skip split regions. TiDB will split one region for // the table when table is created. - needSplit := len(unfinishedRanges) > 1 || lfTotalSize > local.regionSplitSize || lfLength > local.regionSplitKeys + needSplit := len(unfinishedRanges) > 1 || lfTotalSize > regionSplitSize || lfLength > regionSplitKeys // split region by given ranges for i := 0; i < maxRetryTimes; i++ { - err = local.SplitAndScatterRegionByRanges(ctx, unfinishedRanges, lf.tableInfo, needSplit) + err = local.SplitAndScatterRegionByRanges(ctx, unfinishedRanges, lf.tableInfo, needSplit, regionSplitSize) if err == nil || common.IsContextCanceledError(err) { break } @@ -2016,7 +2010,7 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID) erro } // start to write to kv and ingest - err = local.writeAndIngestByRanges(ctx, lf, unfinishedRanges) + err = local.writeAndIngestByRanges(ctx, lf, unfinishedRanges, regionSplitSize, regionSplitKeys) if err != nil { log.L().Error("write and ingest engine failed", log.ShortError(err)) return err diff --git a/br/pkg/lightning/backend/local/local_unix.go b/br/pkg/lightning/backend/local/local_unix.go index d2383aca745c1..57b2fcec4149b 100644 --- a/br/pkg/lightning/backend/local/local_unix.go +++ b/br/pkg/lightning/backend/local/local_unix.go @@ -25,8 +25,8 @@ import ( ) const ( - // mininum max open files value - minRLimit = 1024 + // maximum max open files value + maxRLimit = 1000000 ) func GetSystemRLimit() (Rlim_t, error) { @@ -39,8 +39,8 @@ func GetSystemRLimit() (Rlim_t, error) { // In Local-backend, we need to read and write a lot of L0 SST files, so we need // to check system max open files limit. func VerifyRLimit(estimateMaxFiles Rlim_t) error { - if estimateMaxFiles < minRLimit { - estimateMaxFiles = minRLimit + if estimateMaxFiles > maxRLimit { + estimateMaxFiles = maxRLimit } var rLimit syscall.Rlimit err := syscall.Getrlimit(syscall.RLIMIT_NOFILE, &rLimit) diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index 2d9dc5c48cbdd..bc7a7a65d2a4c 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -67,6 +67,7 @@ func (local *local) SplitAndScatterRegionByRanges( ranges []Range, tableInfo *checkpoints.TidbTableInfo, needSplit bool, + regionSplitSize int64, ) error { if len(ranges) == 0 { return nil @@ -270,7 +271,7 @@ func (local *local) SplitAndScatterRegionByRanges( if !ok { log.L().Warn("region stats not found", zap.Uint64("region", regionID)) } - if len(keys) == 1 && regionSize < local.regionSplitSize { + if len(keys) == 1 && regionSize < regionSplitSize { skippedKeys++ } select { diff --git a/br/pkg/lightning/backend/local/localhelper_test.go b/br/pkg/lightning/backend/local/localhelper_test.go index 11149a000fbe2..502f9a1be7d8a 100644 --- a/br/pkg/lightning/backend/local/localhelper_test.go +++ b/br/pkg/lightning/backend/local/localhelper_test.go @@ -424,7 +424,7 @@ func (s *localSuite) doTestBatchSplitRegionByRanges(ctx context.Context, c *C, h start = end } - err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true) + err = local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000) if len(errPat) == 0 { c.Assert(err, IsNil) } else { @@ -643,7 +643,7 @@ func (s *localSuite) doTestBatchSplitByRangesWithClusteredIndex(c *C, hook clien start = e } - err := local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true) + err := local.SplitAndScatterRegionByRanges(ctx, ranges, nil, true, 1000) c.Assert(err, IsNil) startKey := codec.EncodeBytes([]byte{}, rangeKeys[0]) diff --git a/br/pkg/lightning/backend/noop/noop.go b/br/pkg/lightning/backend/noop/noop.go index 37ca4fd8e77a2..ca095844024d8 100644 --- a/br/pkg/lightning/backend/noop/noop.go +++ b/br/pkg/lightning/backend/noop/noop.go @@ -78,7 +78,7 @@ func (b noopBackend) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, return nil } -func (b noopBackend) ImportEngine(ctx context.Context, engineUUID uuid.UUID) error { +func (b noopBackend) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { return nil } diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index b2259ffb5c8a3..092893ab9d2d9 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -368,7 +368,7 @@ func (be *tidbBackend) CollectRemoteDuplicateRows(ctx context.Context, tbl table panic("Unsupported Operation") } -func (be *tidbBackend) ImportEngine(context.Context, uuid.UUID) error { +func (be *tidbBackend) ImportEngine(context.Context, uuid.UUID, int64) error { return nil } diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index 4c2fbba4c1c55..a112ed4d67418 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -436,7 +436,7 @@ func NewConfig() *Config { OnDuplicate: ReplaceOnDup, MaxKVPairs: 4096, SendKVPairs: 32768, - RegionSplitSize: SplitRegionSize, + RegionSplitSize: 0, DiskQuota: ByteSize(math.MaxInt64), }, PostRestore: PostRestore{ @@ -740,9 +740,6 @@ func (cfg *Config) DefaultVarsForImporterAndLocalBackend(ctx context.Context) { if cfg.TikvImporter.RangeConcurrency == 0 { cfg.TikvImporter.RangeConcurrency = 16 } - if cfg.TikvImporter.RegionSplitSize == 0 { - cfg.TikvImporter.RegionSplitSize = SplitRegionSize - } if cfg.TiDB.BuildStatsConcurrency == 0 { cfg.TiDB.BuildStatsConcurrency = defaultBuildStatsConcurrency } diff --git a/br/pkg/lightning/config/const.go b/br/pkg/lightning/config/const.go index 78ad85c2944d7..4f262eaddbcca 100644 --- a/br/pkg/lightning/config/const.go +++ b/br/pkg/lightning/config/const.go @@ -20,10 +20,11 @@ import ( const ( // mydumper - ReadBlockSize ByteSize = 64 * units.KiB - MinRegionSize ByteSize = 256 * units.MiB - MaxRegionSize ByteSize = 256 * units.MiB - SplitRegionSize ByteSize = 96 * units.MiB + ReadBlockSize ByteSize = 64 * units.KiB + MinRegionSize ByteSize = 256 * units.MiB + MaxRegionSize ByteSize = 256 * units.MiB + SplitRegionSize ByteSize = 96 * units.MiB + MaxSplitRegionSizeRatio int = 10 BufferSizeScale = 5 diff --git a/br/pkg/lightning/lightning.go b/br/pkg/lightning/lightning.go index a25751e923f36..2596b7a7ac8d7 100644 --- a/br/pkg/lightning/lightning.go +++ b/br/pkg/lightning/lightning.go @@ -667,7 +667,10 @@ func checkSystemRequirement(cfg *config.Config, dbsMeta []*mydump.MDDatabaseMeta // region-concurrency: number of LocalWriters writing SST files. // 2*totalSize/memCacheSize: number of Pebble MemCache files. - estimateMaxFiles := local.Rlim_t(cfg.App.RegionConcurrency) + local.Rlim_t(topNTotalSize)/local.Rlim_t(cfg.TikvImporter.EngineMemCacheSize)*2 + maxDBFiles := topNTotalSize / int64(cfg.TikvImporter.LocalWriterMemCacheSize) * 2 + // the pebble db and all import routine need upto maxDBFiles fds for read and write. + maxOpenDBFiles := maxDBFiles * (1 + int64(cfg.TikvImporter.RangeConcurrency)) + estimateMaxFiles := local.Rlim_t(cfg.App.RegionConcurrency) + local.Rlim_t(maxOpenDBFiles) if err := local.VerifyRLimit(estimateMaxFiles); err != nil { return err } diff --git a/br/pkg/lightning/lightning_test.go b/br/pkg/lightning/lightning_test.go index 337adc96f9882..8bae6d89ad9e9 100644 --- a/br/pkg/lightning/lightning_test.go +++ b/br/pkg/lightning/lightning_test.go @@ -447,7 +447,8 @@ func (s *lightningServerSuite) TestCheckSystemRequirement(c *C) { cfg.App.CheckRequirements = true cfg.App.TableConcurrency = 4 cfg.TikvImporter.Backend = config.BackendLocal - cfg.TikvImporter.EngineMemCacheSize = 512 * units.MiB + cfg.TikvImporter.LocalWriterMemCacheSize = 128 * units.MiB + cfg.TikvImporter.RangeConcurrency = 16 dbMetas := []*mydump.MDDatabaseMeta{ { @@ -485,15 +486,14 @@ func (s *lightningServerSuite) TestCheckSystemRequirement(c *C) { }, } - // with max open files 1024, the max table size will be: 65536MB - err := failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/GetRlimitValue", "return(2049)") + err := failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/GetRlimitValue", "return(139439)") c.Assert(err, IsNil) err = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/SetRlimitError", "return(true)") c.Assert(err, IsNil) defer func() { _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/SetRlimitError") }() - // with this dbMetas, the estimated fds will be 2050, so should return error + // with this dbMetas, the estimated fds will be 139440, so should return error err = checkSystemRequirement(cfg, dbMetas) c.Assert(err, NotNil) @@ -501,7 +501,7 @@ func (s *lightningServerSuite) TestCheckSystemRequirement(c *C) { c.Assert(err, IsNil) // the min rlimit should be bigger than the default min value (16384) - err = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/GetRlimitValue", "return(8200)") + err = failpoint.Enable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/GetRlimitValue", "return(139440)") defer func() { _ = failpoint.Disable("github.com/pingcap/tidb/br/pkg/lightning/backend/local/GetRlimitValue") }() diff --git a/br/pkg/lightning/restore/check_info.go b/br/pkg/lightning/restore/check_info.go index 023231691ec36..7ad0f11642a65 100644 --- a/br/pkg/lightning/restore/check_info.go +++ b/br/pkg/lightning/restore/check_info.go @@ -594,7 +594,11 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab // tidb_rowid have a default value. defaultCols[model.ExtraHandleName.String()] = struct{}{} - for _, dataFile := range tableInfo.DataFiles { + // only check the first file of this table. + if len(tableInfo.DataFiles) > 0 { + dataFile := tableInfo.DataFiles[0] + log.L().Info("datafile to check", zap.String("db", tableInfo.DB), + zap.String("table", tableInfo.Name), zap.String("path", dataFile.FileMeta.Path)) // get columns name from data file. dataFileMeta := dataFile.FileMeta @@ -608,7 +612,7 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab } if colsFromDataFile == nil && colCountFromDataFile == 0 { log.L().Info("file contains no data, skip checking against schema validity", zap.String("path", dataFileMeta.Path)) - continue + return msgs, nil } if colsFromDataFile == nil { @@ -669,9 +673,6 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab tableInfo.DB, tableInfo.Name, col, col)) } } - if len(msgs) > 0 { - return msgs, nil - } } return msgs, nil } diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 85757927134aa..f9e91f5810725 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1452,12 +1452,12 @@ func (tr *TableRestore) restoreTable( // rebase the allocator so it exceeds the number of rows. if tr.tableInfo.Core.PKIsHandle && tr.tableInfo.Core.ContainsAutoRandomBits() { cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoRandID) - if err := tr.alloc.Get(autoid.AutoRandomType).Rebase(tr.tableInfo.ID, cp.AllocBase, false); err != nil { + if err := tr.alloc.Get(autoid.AutoRandomType).Rebase(cp.AllocBase, false); err != nil { return false, err } } else { cp.AllocBase = mathutil.MaxInt64(cp.AllocBase, tr.tableInfo.Core.AutoIncID) - if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(tr.tableInfo.ID, cp.AllocBase, false); err != nil { + if err := tr.alloc.Get(autoid.RowIDAllocType).Rebase(cp.AllocBase, false); err != nil { return false, err } } @@ -1621,7 +1621,8 @@ func (rc *Controller) enforceDiskQuota(ctx context.Context) { task := logger.Begin(zap.WarnLevel, "importing large engines for disk quota") var importErr error for _, engine := range largeEngines { - if err := rc.backend.UnsafeImportAndReset(ctx, engine); err != nil { + // Use a larger split region size to avoid split the same region by many times. + if err := rc.backend.UnsafeImportAndReset(ctx, engine, int64(config.SplitRegionSize)*int64(config.MaxSplitRegionSizeRatio)); err != nil { importErr = multierr.Append(importErr, err) } } diff --git a/br/pkg/lightning/restore/restore_test.go b/br/pkg/lightning/restore/restore_test.go index 7758d7d81aa11..2764f51f52d78 100644 --- a/br/pkg/lightning/restore/restore_test.go +++ b/br/pkg/lightning/restore/restore_test.go @@ -852,7 +852,7 @@ func (s *tableRestoreSuite) TestImportKVSuccess(c *C) { importer := backend.MakeBackend(mockBackend) chptCh := make(chan saveCp) defer close(chptCh) - rc := &Controller{saveCpCh: chptCh} + rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} go func() { for range chptCh { } @@ -865,7 +865,7 @@ func (s *tableRestoreSuite) TestImportKVSuccess(c *C) { CloseEngine(ctx, nil, engineUUID). Return(nil) mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID). + ImportEngine(ctx, engineUUID, gomock.Any()). Return(nil) mockBackend.EXPECT(). CleanupEngine(ctx, engineUUID). @@ -884,7 +884,7 @@ func (s *tableRestoreSuite) TestImportKVFailure(c *C) { importer := backend.MakeBackend(mockBackend) chptCh := make(chan saveCp) defer close(chptCh) - rc := &Controller{saveCpCh: chptCh} + rc := &Controller{saveCpCh: chptCh, cfg: config.NewConfig()} go func() { for range chptCh { } @@ -897,7 +897,7 @@ func (s *tableRestoreSuite) TestImportKVFailure(c *C) { CloseEngine(ctx, nil, engineUUID). Return(nil) mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID). + ImportEngine(ctx, engineUUID, gomock.Any()). Return(errors.Annotate(context.Canceled, "fake import error")) closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) @@ -2205,6 +2205,65 @@ func (s *tableRestoreSuite) TestSchemaIsValid(c *C) { }, }, }, + // Case 4: + // table4 has two datafiles for table. we only check the first file. + // we expect the check success. + { + []*config.IgnoreColumns{ + { + DB: "db1", + Table: "table2", + Columns: []string{"cola"}, + }, + }, + "", + 0, + true, + map[string]*checkpoints.TidbDBInfo{ + "db1": { + Name: "db1", + Tables: map[string]*checkpoints.TidbTableInfo{ + "table2": { + ID: 1, + DB: "db1", + Name: "table2", + Core: &model.TableInfo{ + Columns: []*model.ColumnInfo{ + { + // colB has the default value + Name: model.NewCIStr("colB"), + DefaultIsExpr: true, + }, + }, + }, + }, + }, + }, + }, + &mydump.MDTableMeta{ + DB: "db1", + Name: "table2", + DataFiles: []mydump.FileInfo{ + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + Type: mydump.SourceTypeCSV, + }, + }, + { + FileMeta: mydump.SourceFileMeta{ + FileSize: 1 * units.TiB, + Path: case2File, + // This type will make the check failed. + // but it's the second file for table. + // so it's unreachable so this case will success. + Type: mydump.SourceTypeIgnore, + }, + }, + }, + }, + }, } for _, ca := range cases { diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index ecc22ead593fe..3b4c681037317 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -850,8 +850,17 @@ func (tr *TableRestore) importKV( engineID int32, ) error { task := closedEngine.Logger().Begin(zap.InfoLevel, "import and cleanup engine") - - err := closedEngine.Import(ctx) + regionSplitSize := int64(rc.cfg.TikvImporter.RegionSplitSize) + if regionSplitSize == 0 && rc.taskMgr != nil { + regionSplitSize = int64(config.SplitRegionSize) + rc.taskMgr.CheckTasksExclusively(ctx, func(tasks []taskMeta) ([]taskMeta, error) { + if len(tasks) > 0 { + regionSplitSize = int64(config.SplitRegionSize) * int64(utils.MinInt(len(tasks), config.MaxSplitRegionSizeRatio)) + } + return nil, nil + }) + } + err := closedEngine.Import(ctx, regionSplitSize) rc.saveStatusCheckpoint(tr.tableName, engineID, err, checkpoints.CheckpointStatusImported) // Also cleanup engine when encountered ErrDuplicateDetected, since all duplicates kv pairs are recorded. if err == nil { diff --git a/br/pkg/mock/backend.go b/br/pkg/mock/backend.go index 43fc2c2af6395..6c9dc25598794 100644 --- a/br/pkg/mock/backend.go +++ b/br/pkg/mock/backend.go @@ -182,17 +182,17 @@ func (mr *MockBackendMockRecorder) FlushEngine(arg0, arg1 interface{}) *gomock.C } // ImportEngine mocks base method -func (m *MockBackend) ImportEngine(arg0 context.Context, arg1 uuid.UUID) error { +func (m *MockBackend) ImportEngine(arg0 context.Context, arg1 uuid.UUID, arg2 int64) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ImportEngine", arg0, arg1) + ret := m.ctrl.Call(m, "ImportEngine", arg0, arg1, arg2) ret0, _ := ret[0].(error) return ret0 } // ImportEngine indicates an expected call of ImportEngine -func (mr *MockBackendMockRecorder) ImportEngine(arg0, arg1 interface{}) *gomock.Call { +func (mr *MockBackendMockRecorder) ImportEngine(arg0, arg1, arg2 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ImportEngine", reflect.TypeOf((*MockBackend)(nil).ImportEngine), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ImportEngine", reflect.TypeOf((*MockBackend)(nil).ImportEngine), arg0, arg1, arg2) } // LocalWriter mocks base method diff --git a/br/pkg/restore/db_test.go b/br/pkg/restore/db_test.go index 6add4e85cd09f..6f82d4e0fc84f 100644 --- a/br/pkg/restore/db_test.go +++ b/br/pkg/restore/db_test.go @@ -77,8 +77,8 @@ func (s *testRestoreSchemaSuite) TestRestoreAutoIncID(c *C) { DB: dbInfo, } // Get the next AutoIncID - idAlloc := autoid.NewAllocator(s.mock.Storage, dbInfo.ID, false, autoid.RowIDAllocType) - globalAutoID, err := idAlloc.NextGlobalAutoID(table.Info.ID) + idAlloc := autoid.NewAllocator(s.mock.Storage, dbInfo.ID, table.Info.ID, false, autoid.RowIDAllocType) + globalAutoID, err := idAlloc.NextGlobalAutoID() c.Assert(err, IsNil, Commentf("Error allocate next auto id")) c.Assert(autoIncID, Equals, uint64(globalAutoID)) // Alter AutoIncID to the next AutoIncID + 100 diff --git a/br/pkg/restore/split_client.go b/br/pkg/restore/split_client.go index 10cbd24159624..3e619a6cab750 100755 --- a/br/pkg/restore/split_client.go +++ b/br/pkg/restore/split_client.go @@ -305,7 +305,7 @@ func (c *pdClient) sendSplitRegionRequest( return nil, multierr.Append(splitErrors, err) } if resp.RegionError != nil { - log.Error("fail to split region", + log.Warn("fail to split region", logutil.Region(regionInfo.Region), zap.Stringer("regionErr", resp.RegionError)) splitErrors = multierr.Append(splitErrors, diff --git a/circle.yml b/circle.yml deleted file mode 100644 index 621621981ff83..0000000000000 --- a/circle.yml +++ /dev/null @@ -1,12 +0,0 @@ -version: 2 - -jobs: - build: - docker: - - image: golang:1.16 - working_directory: /go/src/github.com/pingcap/tidb - steps: - - checkout - - run: - name: "Build & Test" - command: make dev diff --git a/ddl/column.go b/ddl/column.go index 8bd0eb6406be6..a45fdb8c623e2 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1652,7 +1652,7 @@ func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, if err != nil { return errors.Trace(err) } - err = autoRandAlloc.Rebase(tblInfo.ID, nextAutoIncID, false) + err = autoRandAlloc.Rebase(nextAutoIncID, false) if err != nil { return errors.Trace(err) } diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index a9ece8079aeb2..91d3ecfc273a9 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -459,7 +459,7 @@ func getCurrentTable(d *ddl, schemaID, tableID int64) (table.Table, error) { if err != nil { return nil, errors.Trace(err) } - alloc := autoid.NewAllocator(d.store, schemaID, false, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(d.store, schemaID, tblInfo.ID, false, autoid.RowIDAllocType) tbl, err := table.TableFromMeta(autoid.NewAllocators(alloc), tblInfo) if err != nil { return nil, errors.Trace(err) diff --git a/ddl/ddl.go b/ddl/ddl.go index bad34f76e3cbb..7f9f0102120d9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -115,6 +115,8 @@ type DDL interface { CreateSequence(ctx sessionctx.Context, stmt *ast.CreateSequenceStmt) error DropSequence(ctx sessionctx.Context, tableIdent ast.Ident, ifExists bool) (err error) AlterSequence(ctx sessionctx.Context, stmt *ast.AlterSequenceStmt) error + CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) error + DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) error // CreateSchemaWithInfo creates a database (schema) given its database info. // diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 127d2f9c78c18..731d96164c216 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -57,6 +57,7 @@ import ( "github.com/pingcap/tidb/util/domainutil" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/placementpolicy" "github.com/pingcap/tidb/util/set" "go.uber.org/zap" ) @@ -2225,7 +2226,7 @@ func checkCharsetAndCollation(cs string, co string) error { func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, newEnd int64, tp autoid.AllocatorType) error { allocs := autoid.NewAllocatorsFromTblInfo(d.store, schemaID, tbInfo) if alloc := allocs.Get(tp); alloc != nil { - err := alloc.Rebase(tbInfo.ID, newEnd, false) + err := alloc.Rebase(newEnd, false) if err != nil { return errors.Trace(err) } @@ -2683,7 +2684,7 @@ func adjustNewBaseToNextGlobalID(ctx sessionctx.Context, t table.Table, tp autoi if alloc == nil { return newBase, nil } - autoID, err := alloc.NextGlobalAutoID(t.Meta().ID) + autoID, err := alloc.NextGlobalAutoID() if err != nil { return newBase, errors.Trace(err) } @@ -6078,7 +6079,6 @@ func (d *ddl) AlterTableAttributes(ctx sessionctx.Context, ident ast.Ident, spec } return ErrInvalidAttributesSpec.GenWithStackByArgs(err) } - rule.Reset(meta.ID, schema.Name.L, meta.Name.L) job := &model.Job{ @@ -6126,7 +6126,6 @@ func (d *ddl) AlterTablePartitionAttributes(ctx sessionctx.Context, ident ast.Id } return ErrInvalidAttributesSpec.GenWithStackByArgs(sb.String(), err) } - rule.Reset(partitionID, schema.Name.L, meta.Name.L, spec.PartitionNames[0].L) job := &model.Job{ @@ -6146,3 +6145,93 @@ func (d *ddl) AlterTablePartitionAttributes(ctx sessionctx.Context, ident ast.Id err = d.callHookOnChanged(err) return errors.Trace(err) } + +func buildPolicyInfo(stmt *ast.CreatePlacementPolicyStmt) (*placementpolicy.PolicyInfo, error) { + policyInfo := &placementpolicy.PolicyInfo{} + policyInfo.Name = stmt.PolicyName + for _, opt := range stmt.PlacementOptions { + switch opt.Tp { + case ast.PlacementOptionPrimaryRegion: + policyInfo.PrimaryRegion = opt.StrValue + case ast.PlacementOptionRegions: + policyInfo.Regions = opt.StrValue + case ast.PlacementOptionFollowerCount: + policyInfo.Followers = opt.UintValue + case ast.PlacementOptionVoterCount: + policyInfo.Voters = opt.UintValue + case ast.PlacementOptionLearnerCount: + policyInfo.Learners = opt.UintValue + case ast.PlacementOptionSchedule: + policyInfo.Schedule = opt.StrValue + case ast.PlacementOptionConstraints: + policyInfo.Constraints = opt.StrValue + case ast.PlacementOptionLearnerConstraints: + policyInfo.LearnerConstraints = opt.StrValue + case ast.PlacementOptionFollowerConstraints: + policyInfo.FollowerConstraints = opt.StrValue + case ast.PlacementOptionVoterConstraints: + policyInfo.VoterConstraints = opt.StrValue + case ast.PlacementOptionLeaderConstraints: + policyInfo.LeaderConstraints = opt.StrValue + default: + return nil, errors.Trace(errors.New("unknown placement policy option")) + } + } + return policyInfo, nil +} + +func (d *ddl) CreatePlacementPolicy(ctx sessionctx.Context, stmt *ast.CreatePlacementPolicyStmt) (err error) { + policyName := stmt.PolicyName + is := d.GetInfoSchemaWithInterceptor(ctx) + // Check policy existence. + _, ok := is.PolicyByName(policyName) + if ok { + err = infoschema.ErrPlacementPolicyExists.GenWithStackByArgs(policyName) + if stmt.IfNotExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + // Auto fill the policyID when it is inserted. + policyInfo, err := buildPolicyInfo(stmt) + if err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + SchemaName: policyInfo.Name.L, + Type: model.ActionCreatePlacementPolicy, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{policyInfo}, + } + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + +func (d *ddl) DropPlacementPolicy(ctx sessionctx.Context, stmt *ast.DropPlacementPolicyStmt) (err error) { + policyName := stmt.PolicyName + is := d.GetInfoSchemaWithInterceptor(ctx) + // Check policy existence. + policy, ok := is.PolicyByName(policyName) + if !ok { + err = infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs(policyName) + if stmt.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + return err + } + + job := &model.Job{ + SchemaID: policy.ID, + SchemaName: policy.Name.L, + Type: model.ActionDropPlacementPolicy, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{policyName}, + } + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index bef94b4edf91c..b380887cc1c52 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -834,6 +834,10 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onAlterTableAttributes(t, job) case model.ActionAlterTablePartitionAttributes: ver, err = onAlterTablePartitionAttributes(t, job) + case model.ActionCreatePlacementPolicy: + ver, err = onCreatePlacementPolicy(d, t, job) + case model.ActionDropPlacementPolicy: + ver, err = onDropPlacementPolicy(t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/label/rule.go b/ddl/label/rule.go index b1529f598f5e6..2a2e8961d1b29 100644 --- a/ddl/label/rule.go +++ b/ddl/label/rule.go @@ -56,6 +56,10 @@ func NewRule() *Rule { // ApplyAttributesSpec will transfer attributes defined in AttributesSpec to the labels. func (r *Rule) ApplyAttributesSpec(spec *ast.AttributesSpec) error { + if spec.Default { + r.Labels = []Label{} + return nil + } // construct a string list attrBytes := []byte("[" + spec.Attributes + "]") attributes := []string{} @@ -91,7 +95,9 @@ func (r *Rule) Reset(id int64, dbName, tableName string, partName ...string) *Ru } else { r.ID = fmt.Sprintf(TableIDFormat, IDPrefix, dbName, tableName) } - + if len(r.Labels) == 0 { + return r + } var hasDBKey, hasTableKey, hasPartitionKey bool for i := range r.Labels { switch r.Labels[i].Key { diff --git a/ddl/label/rule_test.go b/ddl/label/rule_test.go index ea498d4a9e210..1010040b8a24e 100644 --- a/ddl/label/rule_test.go +++ b/ddl/label/rule_test.go @@ -32,14 +32,30 @@ func (t *testRuleSuite) TestApplyAttributesSpec(c *C) { c.Assert(rule.Labels[1].Key, Equals, "attr2") } +func (t *testRuleSuite) TestDefaultOrEmpty(c *C) { + spec := &ast.AttributesSpec{Attributes: ""} + rule := NewRule() + rule.ApplyAttributesSpec(spec) + rule.Reset(1, "db", "t") + c.Assert(rule.Labels, HasLen, 0) + spec = &ast.AttributesSpec{Default: true} + rule = NewRule() + rule.ApplyAttributesSpec(spec) + rule.Reset(1, "db", "t") + c.Assert(rule.Labels, HasLen, 0) +} + func (t *testRuleSuite) TestReset(c *C) { + spec := &ast.AttributesSpec{Attributes: "attr"} rule := NewRule() + rule.ApplyAttributesSpec(spec) rule.Reset(1, "db1", "t1") c.Assert(rule.ID, Equals, "schema/db1/t1") c.Assert(rule.RuleType, Equals, ruleType) - c.Assert(rule.Labels, HasLen, 2) - c.Assert(rule.Labels[0].Value, Equals, "db1") - c.Assert(rule.Labels[1].Value, Equals, "t1") + c.Assert(rule.Labels, HasLen, 3) + c.Assert(rule.Labels[0].Value, Equals, "true") + c.Assert(rule.Labels[1].Value, Equals, "db1") + c.Assert(rule.Labels[2].Value, Equals, "t1") r := rule.Rule.(map[string]string) c.Assert(r["start_key"], Equals, "7480000000000000ff015f720000000000fa") c.Assert(r["end_key"], Equals, "7480000000000000ff025f720000000000fa") @@ -49,10 +65,11 @@ func (t *testRuleSuite) TestReset(c *C) { 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") + c.Assert(r2.Labels, HasLen, 4) + c.Assert(rule.Labels[0].Value, Equals, "true") + c.Assert(rule.Labels[1].Value, Equals, "db2") + c.Assert(rule.Labels[2].Value, Equals, "t2") + c.Assert(rule.Labels[3].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/placement_policy.go b/ddl/placement_policy.go new file mode 100644 index 0000000000000..fd8e5043bca22 --- /dev/null +++ b/ddl/placement_policy.go @@ -0,0 +1,166 @@ +// 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 ddl + +import ( + "fmt" + + "github.com/pingcap/errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/util/placementpolicy" +) + +func onCreatePlacementPolicy(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + policyInfo := &placementpolicy.PolicyInfo{} + if err := job.DecodeArgs(policyInfo); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + policyInfo.State = model.StateNone + + err := checkPlacementPolicyNotExistAndCancelExistJob(d, t, job, policyInfo) + if err != nil { + return ver, errors.Trace(err) + } + switch policyInfo.State { + case model.StateNone: + // none -> public + policyInfo.State = model.StatePublic + err = t.CreatePolicy(policyInfo) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaID = policyInfo.ID + + ver, err = updateSchemaVersion(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // Finish this job. + job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, nil) + return ver, nil + default: + // We can't enter here. + return ver, ErrInvalidDDLState.GenWithStackByArgs("policy", policyInfo.State) + } +} + +func getPolicyInfo(t *meta.Meta, policyID int64) (*placementpolicy.PolicyInfo, error) { + policy, err := t.GetPolicy(policyID) + if err != nil { + if meta.ErrPolicyNotExists.Equal(err) { + return nil, infoschema.ErrPlacementPolicyNotExists.GenWithStackByArgs( + fmt.Sprintf("(Policy ID %d)", policyID), + ) + } + return nil, err + } + return policy, nil +} + +func checkPlacementPolicyNotExistAndCancelExistJob(d *ddlCtx, t *meta.Meta, job *model.Job, info *placementpolicy.PolicyInfo) error { + currVer, err := t.GetSchemaVersion() + if err != nil { + return err + } + is := d.infoCache.GetLatest() + if is.SchemaMetaVersion() == currVer { + // Use cached policy. + _, ok := is.PolicyByName(info.Name) + if ok { + job.State = model.JobStateCancelled + return infoschema.ErrPlacementPolicyExists.GenWithStackByArgs(info.Name) + } + return nil + } + // Check in meta directly. + policies, err := t.ListPolicies() + if err != nil { + return errors.Trace(err) + } + for _, policy := range policies { + if policy.Name.L == info.Name.L { + job.State = model.JobStateCancelled + return infoschema.ErrPlacementPolicyExists.GenWithStackByArgs(info.Name) + } + } + return nil +} + +func checkPlacementPolicyExistAndCancelNonExistJob(t *meta.Meta, job *model.Job, policyID int64) (*placementpolicy.PolicyInfo, error) { + policy, err := getPolicyInfo(t, policyID) + if err == nil { + return policy, nil + } + if infoschema.ErrPlacementPolicyNotExists.Equal(err) { + job.State = model.JobStateCancelled + } + return nil, err +} + +func onDropPlacementPolicy(t *meta.Meta, job *model.Job) (ver int64, _ error) { + policyInfo, err := checkPlacementPolicyExistAndCancelNonExistJob(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + switch policyInfo.State { + case model.StatePublic: + // public -> write only + policyInfo.State = model.StateWriteOnly + err = t.UpdatePolicy(policyInfo) + if err != nil { + return ver, errors.Trace(err) + } + ver, err = updateSchemaVersion(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // Update the job state when all affairs done. + job.SchemaState = model.StateWriteOnly + case model.StateWriteOnly: + // write only -> delete only + policyInfo.State = model.StateDeleteOnly + err = t.UpdatePolicy(policyInfo) + if err != nil { + return ver, errors.Trace(err) + } + ver, err = updateSchemaVersion(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // Update the job state when all affairs done. + job.SchemaState = model.StateDeleteOnly + case model.StateDeleteOnly: + policyInfo.State = model.StateNone + if err = t.DropPolicy(policyInfo.ID); err != nil { + return ver, errors.Trace(err) + } + ver, err = updateSchemaVersion(t, job) + if err != nil { + return ver, errors.Trace(err) + } + // TODO: Reset all the policy reference, (modify meta & notify pd) + // If any partitions currently use this policy, they will be converted to the policy used by the table + // they belong to. If any databases use this policy, they will be converted to the default placement_policy policy. + + // Finish this job. By now policy don't consider the binlog sync. + job.FinishDBJob(model.JobStateDone, model.StateNone, ver, nil) + default: + err = ErrInvalidDDLState.GenWithStackByArgs("policy", policyInfo.State) + } + return ver, errors.Trace(err) +} diff --git a/ddl/placement_policy_test.go b/ddl/placement_policy_test.go new file mode 100644 index 0000000000000..9836cc0bb63a4 --- /dev/null +++ b/ddl/placement_policy_test.go @@ -0,0 +1,136 @@ +// 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 ddl_test + +import ( + "context" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/domain" + mysql "github.com/pingcap/tidb/errno" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/placementpolicy" + "github.com/pingcap/tidb/util/testkit" +) + +func (s *testDBSuite6) TestPlacementPolicy(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop placement policy if exists x") + + originalHook := s.dom.DDL().GetHook() + defer s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) + + hook := &ddl.TestDDLCallback{} + var policyID int64 + hook.OnJobUpdatedExported = func(job *model.Job) { + if policyID != 0 { + return + } + // job.SchemaID will be assigned when the policy is created. + if job.SchemaName == "x" && job.Type == model.ActionCreatePlacementPolicy && job.SchemaID != 0 { + policyID = job.SchemaID + return + } + } + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + + tk.MustExec("create placement policy x " + + "PRIMARY_REGION=\"cn-east-1\" " + + "REGIONS=\"cn-east-1,cn-east-2\" " + + "LEARNERS=1 " + + "LEARNER_CONSTRAINTS=\"[+region=cn-west-1]\" " + + "VOTERS=3 " + + "VOTER_CONSTRAINTS=\"[+disk=ssd]\"") + + checkFunc := func(policyInfo *placementpolicy.PolicyInfo) { + c.Assert(policyInfo.ID != 0, Equals, true) + c.Assert(policyInfo.Name.L, Equals, "x") + c.Assert(policyInfo.PrimaryRegion, Equals, "cn-east-1") + c.Assert(policyInfo.Regions, Equals, "cn-east-1,cn-east-2") + c.Assert(policyInfo.Followers, Equals, uint64(0)) + c.Assert(policyInfo.FollowerConstraints, Equals, "") + c.Assert(policyInfo.Voters, Equals, uint64(3)) + c.Assert(policyInfo.VoterConstraints, Equals, "[+disk=ssd]") + c.Assert(policyInfo.Learners, Equals, uint64(1)) + c.Assert(policyInfo.LearnerConstraints, Equals, "[+region=cn-west-1]") + c.Assert(policyInfo.State, Equals, model.StatePublic) + c.Assert(policyInfo.Schedule, Equals, "") + } + + // Check the policy is correctly reloaded in the information schema. + po := testGetPolicyByNameFromIS(c, tk.Se, "x") + checkFunc(po) + + // Check the policy is correctly written in the kv meta. + po = testGetPolicyByIDFromMeta(c, s.store, policyID) + checkFunc(po) + + tk.MustGetErrCode("create placement policy x "+ + "PRIMARY_REGION=\"cn-east-1\" "+ + "REGIONS=\"cn-east-1,cn-east-2\" ", mysql.ErrPlacementPolicyExists) + + tk.MustGetErrCode("create placement policy X "+ + "PRIMARY_REGION=\"cn-east-1\" "+ + "REGIONS=\"cn-east-1,cn-east-2\" ", mysql.ErrPlacementPolicyExists) + + tk.MustGetErrCode("create placement policy `X` "+ + "PRIMARY_REGION=\"cn-east-1\" "+ + "REGIONS=\"cn-east-1,cn-east-2\" ", mysql.ErrPlacementPolicyExists) + + tk.MustExec("create placement policy if not exists X " + + "PRIMARY_REGION=\"cn-east-1\" " + + "REGIONS=\"cn-east-1,cn-east-2\" ") + tk.MustQuery("show warnings").Check(testkit.Rows("Note 8238 Placement policy 'X' already exists")) + + tk.MustExec("drop placement policy x") + tk.MustGetErrCode("drop placement policy x", mysql.ErrPlacementPolicyNotExists) + tk.MustExec("drop placement policy if exists x") + tk.MustQuery("show warnings").Check(testkit.Rows("Note 8239 Unknown placement policy 'x'")) + + // TODO: privilege check & constraint syntax check. +} + +func testGetPolicyByIDFromMeta(c *C, store kv.Storage, policyID int64) *placementpolicy.PolicyInfo { + var ( + policyInfo *placementpolicy.PolicyInfo + err error + ) + err1 := kv.RunInNewTxn(context.Background(), store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + policyInfo, err = t.GetPolicy(policyID) + if err != nil { + return err + } + return nil + }) + c.Assert(err1, IsNil) + c.Assert(policyInfo, NotNil) + return policyInfo +} + +func testGetPolicyByNameFromIS(c *C, ctx sessionctx.Context, policy string) *placementpolicy.PolicyInfo { + dom := domain.GetDomain(ctx) + // Make sure the table schema is the new schema. + err := dom.Reload() + c.Assert(err, IsNil) + po, ok := dom.InfoSchema().PolicyByName(model.NewCIStr(policy)) + c.Assert(ok, Equals, true) + return po +} diff --git a/ddl/table.go b/ddl/table.go index 36ea2a6174b00..179280f78fdf7 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -627,9 +627,9 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.Al // The next value to allocate is `newBase`. newEnd := newBase - 1 if force { - err = alloc.ForceRebase(tblInfo.ID, newEnd) + err = alloc.ForceRebase(newEnd) } else { - err = alloc.Rebase(tblInfo.ID, newEnd, false) + err = alloc.Rebase(newEnd, false) } if err != nil { job.State = model.JobStateCancelled @@ -710,7 +710,7 @@ func verifyNoOverflowShardBits(s *sessionPool, tbl table.Table, shardRowIDBits u defer s.put(ctx) // Check next global max auto ID first. - autoIncID, err := tbl.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID) + autoIncID, err := tbl.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID() if err != nil { return errors.Trace(err) } @@ -1163,7 +1163,7 @@ func onRepairTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) func onAlterTableAttributes(t *meta.Meta, job *model.Job) (ver int64, err error) { rule := label.NewRule() - err = job.DecodeArgs(&rule) + err = job.DecodeArgs(rule) if err != nil { job.State = model.JobStateCancelled return 0, errors.Trace(err) @@ -1174,7 +1174,12 @@ func onAlterTableAttributes(t *meta.Meta, job *model.Job) (ver int64, err error) return 0, err } - err = infosync.PutLabelRule(context.TODO(), rule) + if len(rule.Labels) == 0 { + patch := label.NewRulePatch(nil, []string{rule.ID}) + err = infosync.UpdateLabelRules(context.TODO(), patch) + } else { + err = infosync.PutLabelRule(context.TODO(), rule) + } if err != nil { job.State = model.JobStateCancelled return 0, errors.Wrapf(err, "failed to notify PD label rule") @@ -1191,7 +1196,7 @@ func onAlterTableAttributes(t *meta.Meta, job *model.Job) (ver int64, err error) func onAlterTablePartitionAttributes(t *meta.Meta, job *model.Job) (ver int64, err error) { var partitionID int64 rule := label.NewRule() - err = job.DecodeArgs(&partitionID, &rule) + err = job.DecodeArgs(&partitionID, rule) if err != nil { job.State = model.JobStateCancelled return 0, errors.Trace(err) @@ -1207,7 +1212,12 @@ func onAlterTablePartitionAttributes(t *meta.Meta, job *model.Job) (ver int64, e return 0, errors.Trace(table.ErrUnknownPartition.GenWithStackByArgs("drop?", tblInfo.Name.O)) } - err = infosync.PutLabelRule(context.TODO(), rule) + if len(rule.Labels) == 0 { + patch := label.NewRulePatch(nil, []string{rule.ID}) + err = infosync.UpdateLabelRules(context.TODO(), patch) + } else { + err = infosync.PutLabelRule(context.TODO(), rule) + } if err != nil { job.State = model.JobStateCancelled return 0, errors.Wrapf(err, "failed to notify PD region label") diff --git a/ddl/table_test.go b/ddl/table_test.go index 107af86bd10ff..10c868ef5f030 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -339,7 +339,7 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error) if tblInfo == nil { return nil, errors.New("table not found") } - alloc := autoid.NewAllocator(d.store, schemaID, false, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(d.store, schemaID, tblInfo.ID, false, autoid.RowIDAllocType) tbl, err := table.TableFromMeta(autoid.NewAllocators(alloc), tblInfo) if err != nil { return nil, errors.Trace(err) diff --git a/docs/design/2020-06-24-placement-rules-in-sql.md b/docs/design/2020-06-24-placement-rules-in-sql.md index 6003a9ed68870..37f0dd67e0379 100644 --- a/docs/design/2020-06-24-placement-rules-in-sql.md +++ b/docs/design/2020-06-24-placement-rules-in-sql.md @@ -1,14 +1,47 @@ # Defining placement rules in SQL -- Author(s): [djshow832](https://github.com/djshow832) (Ming Zhang) -- Last updated: 2020-06-24 -- Discussion at: https://docs.google.com/document/d/18Kdhi90dv33muF9k_VAIccNLeGf-DdQyUc8JlWF9Gok - -## Motivation - -TiDB supports placement rules, which can define the placement of data in a more flexible and more granular way. But it only provides configuration files to define them, and it’s complicated. - -This article proposes an approach to configure placement rules through DDL statements. TiDB server parses the statements and notify PD to perform the change. In this way, usability can be improved. +- Author(s): [djshow832](https://github.com/djshow832) (Ming Zhang), [morgo](https://github.com/morgo) (Morgan Tocker) +- Last updated: 2021-07-26 +- Discussion PR: https://github.com/pingcap/tidb/pull/26221 +- Tracking Issue: https://github.com/pingcap/tidb/issues/18030 +- Original Document (Chinese): https://docs.google.com/document/d/18Kdhi90dv33muF9k_VAIccNLeGf-DdQyUc8JlWF9Gok + +## Table of Contents + +* [Introduction](#introduction) +* [Motivation or Background](#motivation-or-background) +* [Detailed Design](#detailed-design) + * [New Syntax Overview](#new-syntax-overview) + * [Updates to Existing Syntax](#updates-to-existing-syntax) + * [Placement Rules Syntax](#placement-rules-syntax) + * [Additional Semantics](#additional-semantics) + * [Privilege management](#privilege-management) +* [Implementation](#implementation) + * [Storing Placement Policies](#storing-placement-policies) + * [Storage Consistency](#storage-consistency) + * [Querying Placement Rules](#querying-placement-rules) + * [Building placement rules](#building-placement-rules) + * [Rule priorities](#rule-priorities) +* [Examples](#examples) + * [Optimization: Follower read in every region](#optimization-follower-read-in-every-region) + * [Optimization: Latest data on SSD](#optimization-latest-data-on-ssd) + * [Optimization: Multi-tenancy / control of shared resources](#optimization-multi-tenancy--control-of-shared-resources) + * [Compliance: User data needs geographic split](#compliance-user-data-needs-geographic-split) +* [Impacts & Risks](#impacts--risks) +* [Investigation & Alternatives](#investigation--alternatives) + * [Known Limitations](#known-limitations) +* [Unresolved Questions](#unresolved-questions) + * [Compliance Requirements](#compliance-requirements) + * [Behaviors](#behaviors) +* [Changelog](#changelog) + +## Introduction + +TiDB currently supports placement rules, which can define the placement of data in a more flexible and more granular way. But the current usage requires configuration files to manage them, and for end-users this can be complicated. + +This document proposes an approach to configure placement rules through DDL statements. Usability is improved because the TiDB server parses the statements and notifies PD to perform the change. + +## Motivation or Background The scenarios of defining placement rules in SQL include: @@ -20,401 +53,436 @@ The scenarios of defining placement rules in SQL include: - Increase the replica count of more important data - Separate irrelevant data into different stores to improve availability -## Define placement rules +These scenarios usually fit into one of the following categories: -There are 3 kinds of operations on the placement: +1. An optimization or availability use case (such as improving data access locality) +2. A compliance use case (such as ensuring data resides within a geographic region) -* ADD: Add more replicas for one role. -* ALTER: Override the replica configuration for one role. -* DROP: Remove the replica configuration for one role. +This proposal makes some intentional decisions so that both categories of use-cases can be handled correctly. It improves upon the earlier proposal by reducing the risk of misconfiguration which can affect compliance use cases. -They’re all achieved by executing `ALTER TABLE` statements. +## Detailed Design -### Add placement rules +### New Syntax Overview -Adding new replicas can be done by one or more `ADD PLACEMENT POLICY` clauses: - -```sql -ALTER TABLE table_name - ADD PLACEMENT POLICY CONSTRAINTS=constraints ROLE=role REPLICAS=replicas, - ... -``` +There are two ways to specify placement rules: -This statement indicates TiDB to add replicas for all data of table `table_name`, including indexes. +1. By assigning placement directly on a database, table or partition (direct assignment) +2. By creating a new `PLACEMENT POLICY` and then applying the placement policy to a database, table or partition (placement policy) -`ADD PLACEMENT POLICY` is just a part of alter options, just like `ADD COLUMN` or `ADD CONSTRAINT`. +Using a `PLACEMENT POLICY` will be recommended for compliance requirements, since it can allow administrators to better keep track of usage. This can be seen as similar to how complex environments will use `ROLES` for management instead of directly assigning privileges to users. -To define multiple roles at once, multiple `ADD PLACEMENT POLICY` clauses can appear in a single `ALTER TABLE` statement, even for the same Raft role. For example: +Both syntaxes are considered [`table_option`](https://dev.mysql.com/doc/refman/8.0/en/alter-table.html)s, and available in both `CREATE TABLE` and `ALTER TABLE` contexts. -```sql -ALTER TABLE table_name - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=leader REPLICAS=1, - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=follower REPLICAS=1 - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=gz]" ROLE=follower REPLICAS=1; -``` +The two methods **are mutually exclusive**. Specifying both in a `CREATE`/`ALTER` statement will result in an error. Specifying a `PLACEMENT POLICY` on a table with direct assignment options will clear those options. -This statement indicates PD to schedule the leader to `sh`, add one follower to `sh` and one to `gz`. Note that as the leader can be only one, the first clause doesn't actually add a replica, so this statement adds 2 replicas. +#### Direct Assignment -`ADD PLACEMENT POLICY` also supports adding TiFlash replicas for a table, as statement `ALTER TABLE table_name SET TIFLASH REPLICA count` does. For example: +Creating a new table with directly assigned constraints. The leader is in `us-east-1` region, the followers are in `us-east-1` and `us-east-2`: ```sql -ALTER TABLE table_name - ADD PLACEMENT POLICY CONSTRAINTS="[+engine=tiflash]" ROLE=learner REPLICAS=1; +CREATE TABLE t1 ( + id INT NOT NULL PRIMARY KEY, + b VARCHAR(100) +) PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2"; ``` -The only way to judge whether it’s adding a TiFlash replica is to check the label. If it contains `engine=tiflash`, then it’s adding or removing a TiFlash replica. This logic is conventional in PD for now. +In this context, "REGION" and "REGIONS" are syntactic sugar which map to the label `region`. The following labels have special reserved words (the plural is used in contexts such as followers where multiple is possible): +- `host` and `hosts`: expected to be the same physical machine or hypervisor. +- `rack` and `racks`: similar to host; a group of machines that are physically close together and may suffer from many of the same failures. +- `zone` and `zones`: similar to an AWS zone; much larger degree of blast radius isolation from a rack, but still vulnerable to issues such as a natural disaster. +- `region` and `regions`: expected to be distributed far enough apart that there is isolation from disasters. -Placement rules must conform to Raft constraints. For example, an error should be reported when executing this statement: +To use additional labels not in this list, see "Advanced Placement" below. + +#### Explicit Placement Syntax + +Creating a new `PLACEMENT POLICY`: ```sql -ALTER TABLE test - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=leader REPLICAS=2; +CREATE PLACEMENT POLICY `standardplacement` PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" ``` -There can only be one leader, so `REPLICAS` must be 1 or omitted. But for other roles, `REPLICAS` must be specified. - -Besides, at most one role can be defined on the same object. If multiple rules are added on the same role, they will be combined to one rule. For example: +Creating a new table with the `PLACEMENT POLICY` assigned: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=voter REPLICAS=2, - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=2; +CREATE TABLE t1 ( + id INT NOT NULL PRIMARY KEY, + b VARCHAR(100) +) PLACEMENT POLICY=`standardplacement`; ``` -The same role `voter` is defined in 2 different rules, each of which adds 2 replicas. So it is equivalent to: +Adding `PLACEMENT POLICY` to an existing table: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY CONSTRAINTS="{+zone=sh:2,+zone=bj:2}" ROLE=voter REPLICAS=4; +CREATE TABLE t2 ( + id INT NOT NULL PRIMARY KEY, + b VARCHAR(100) +); +ALTER TABLE t2 PLACEMENT POLICY=`standardplacement`; ``` -Note that as there may already exist 3 replicas by default, so it will be 7 replicas after executing this statement. So `ADD PLACEMENT POLICY` can be taken as a shortcut for adding replicas to a defined role. In the example above, it can be replaced by `ALTER PLACEMENT POLICY`. +Behavior notes: -More details of `CONSTRAINTS` option is described in the "Constraints Configuration" section. +- `CREATE` or `ALTER` and specifying a `PLACEMENT POLICY` that does not exist results in an error: placement policy 'x' is not defined (see "Skipping Policy Validation" below) +- Placement policies are globally unique names. Thus, a policy named `companyplacementpolicy` can apply to the db `test` as well as `userdb`. The namespace does not overlap with other DB objects. +- Placement Policy names are case insensitive, and follow the same rules as tables/other identifiers for length (64 chars) and special characters. +- The full placement policy can be seen with `SHOW CREATE PLACEMENT POLICY x`. This is useful for shorthand usage by DBAs, and consistent with other database objects. +- It is possible to update the definition of a placement policy with `ALTER PLACEMENT POLICY x LEADER_CONSTRAINTS="[+region=us-east-1]" FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,+region=us-east-2:1}";` This is modeled on the statement `ALTER VIEW` (where the view needs to be redefined). When `ALTER PLACEMENT POLICY x` is executed, all tables that use this placement policy will need to be updated in PD. +- The statement `DROP PLACEMENT POLICY` should execute without error. If any partitions currently use this policy, they will be converted to the policy used by the table they belong to. If any tables use this policy, they will be converted to the policy used by the database they belong to. If any databases use this policy, they will be converted to the default placement policy. This is modeled on the behavior of dropping a `ROLE` that might be assigned to users. +- The statement `RENAME PLACEMENT POLICY x TO y` renames a placement policy. The `SHOW CREATE TABLE` output of all databases, tables and partitions that used this placement policy should be updated to the new name. +- You can not use **both** a placement policy and direct assignment. If you alter specify both in a `CREATE TABLE` or `ALTER TABLE` an error will be returned. If you specify a `PLACEMENT POLICY` in an `ALTER TABLE` statement, it will unset other placement options ({FOLLOWERS,VOTERS,LEARNERS}=N, {FOLLOWER,VOTER,LEARNER}_CONSTRAINTS, CONSTRAINTS, PRIMARY_REGION, REGIONS, SCHEDULE). -`ADD PLACEMENT POLICY` is implemented by adding one or more placement rules in PD. The statement must wait until the PD returns a message. It can be cancelled by executing `ADMIN CANCEL DDL JOBS` statement. +#### Advanced Placement -### Alter placement rules - -Altering current placement rules can be done by one or more `ALTER PLACEMENT POLICY` clauses: +The syntax `PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2"` is the recommended syntax for users, but it only works for supported labels. +Consider the case where a user wants to allocate placement based on the label `disk`. Using constraints is required: ```sql -ALTER TABLE table_name - ALTER PLACEMENT POLICY CONSTRAINTS=constraints ROLE=role REPLICAS=replicas, - ... +ALTER PLACEMENT POLICY `standardplacement` CONSTRAINTS="[+disk=ssd]"; ``` -This statement indicates TiDB to overwrite the current placement rule with the same `role`. It affects all data of table `table_name`, including indices. - -Assuming table `test` has 3 replicas by default, the default placement rule is equivalent to: +The following two placement policies are considered equal: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY ROLE=voter REPLICAS=3; +CREATE PLACEMENT POLICY `standardplacement1` PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4; +CREATE PLACEMENT POLICY `standardplacement2` LEADER_CONSTRAINTS="[+region=us-east-1]" FOLLOWERS_CONSTRAINTS="[+region=us-east-1,+region=us-east-2]" FOLLOWERS=4; ``` -`CONSTRAINTS` is omitted here, because there is no label constraints on voters. - -Since at most one rule can be defined for each role, `ALTER PLACEMENT POLICY` will replace the existing rule with the same role. For example: +When the constraints is specified as a dictionary (`{}`) numeric counts for each region must be specified and `FOLLOWERS=n` is disallowed. The special `+any` constraint permits additional followers to be added with no constraints: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=voter REPLICAS=2, - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=2, - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=voter REPLICAS=3, - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3; +ALTER PLACEMENT POLICY `standardplacement3` LEADER_CONSTRAINTS="[+region=us-east-1]" FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,+region=us-east-2:1,+region=us-west-1:1,+any:1}"; ``` -As all the rules are defined on the same role `voter`, the first 3 rules will be overwritten by the last one. So it is equivalent to: +The placement policy above has 4 followers: +- 1 each in the regions us-east-1, us-east-2 and us-west-1 +- 1 that has no constraints and may reside in any location (including the previously specified regions) -```sql -ALTER TABLE test - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3; -``` +Behavior notes: -To add a prohibiting constraint to all the placement rules can be only achieved by overwriting all the rules. For example, assuming the original placement rules are: +* Advanced placement is available in the context of `CREATE|ALTER PLACEMENT POLICY`, `CREATE|ALTER DATABASE` and `CREATE|ALTER TABLE`. i.e. the usage of all placement syntax is expected to be the same in all contexts. +* It is possible to set `CONSTRAINTS`, `LEADER_CONSTRAINTS`, `FOLLOWER_CONSTRAINTS`, `LEARNER_CONSTRAINTS` and `VOTER_CONSTRAINTS`. Assuming that both `CONSTRAINTS` and `FOLLOWER_CONSTRAINTS` are specified, the conditions are "AND"ed together. +* See "Constraints configuration" below for a full set of rules and syntax for constraints. -```sql -ALTER TABLE test - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3; - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=follower REPLICAS=2; -``` +#### Metadata commands -To prohibit all replicas from being placed on zone `gz`, then both the 2 rules should be overwritten: +Besides `SHOW CREATE PLACEMENT POLICY x` and `SHOW CREATE TABLE t1` it should be possible to summarize all placement for a database system. This is beneficial for compliance scenarios. -```sql -ALTER TABLE test - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj,-zone=gz]" ROLE=voter REPLICAS=3; - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh,-zone=gz]" ROLE=follower REPLICAS=2; -``` +##### information_schema.placement_rules -If no rule on a specified role is defined, `ALTER PLACEMENT POLICY` can be used to replace `ADD PLACEMENT POLICY`. In this way, it's more convenient to add replicas because users needn't check the existence of such a rule. For example, assuming the original placement rule is: +A new system table `information_schema.placement_rules` is added to view all explicit placement rules. An explicit rule is one that has been defined by the user and does not use inheritance rules, such as how partitions will use the same rules as the table they belong to. -```sql -ALTER TABLE test - ADD PLACEMENT POLICY ROLE=voter REPLICAS=3; -``` +The table contains columns such as: -It's fine to execute this statement: +* `rule_definition`: the placement policy definition (could be `PLACEMENT POLICY=x`, syntactic sugar variant or full list of constraints) +* `followers`: the number of followers +* `learners`: the number of learners +* `voters`: the number of voters +* `schema_name`: the schema this applies to. +* `table_name`: the table this applies to. +* `partition_name`: the partition this applies to (NULL if not applicable) +* `scheduling state`: the scheduling state of the placement rule. + +The system table is a virtual table, which doesn’t persist data. When querying the table, TiDB queries PD and integrates the result in a table format. That also means the metadata is stored on PD instead of TiKV. + +##### SHOW PLACEMENT + +The `information_schema.placement_rules` table only contains stored placement rules, and users cannot query the effective rule of one object from it. + +For example, table `t` has two partitions `p0` and `p1`, and a placement rule is added on `t`. If the user wants to query the working rule of `p0`, he will find no placement rule is defined for `p0` through the system table. Based on the inheritance rules for partitioned tables the user needs to query the placement rule on `t`. This procedure is annoying. + +To simplify the procedure, a `SHOW PLACEMENT` statement is provided to summarize the effective rules for one specified object. + +The statement is in such a format: ```sql -ALTER TABLE test - ALTER PLACEMENT POLICY ROLE=follower REPLICAS=1; +SHOW PLACEMENT FOR [{DATABASE | SCHEMA} schema_name] [TABLE table_name [PARTITION partition_name]]; ``` -It's equivalent to: +Examples: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY ROLE=follower REPLICAS=1; +SHOW PLACEMENT; ++----------------------------+----------------------------------------------------------------------+------------------+ +| target | placement | scheduling_state | ++----------------------------+----------------------------------------------------------------------+------------------+ +| POLICY system | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4 | SCHEDULED | +| POLICY default | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | +| DATABASE test | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | +| TABLE test.t1 | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | +| TABLE test.t1 PARTITION p1 | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | INPROGRESS | ++----------------------------+----------------------------------------------------------------------+------------------+ +5 rows in set (0.00 sec) + +SHOW PLACEMENT LIKE 'POLICY%'; ++----------------------------+----------------------------------------------------------------------+------------------+ +| target | placement | scheduling_state | ++----------------------------+----------------------------------------------------------------------+------------------+ +| POLICY system | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4 | SCHEDULED | +| POLICY default | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | ++----------------------------+----------------------------------------------------------------------+------------------+ +2 rows in set (0.00 sec) + +SHOW PLACEMENT FOR DATABASE test; ++----------------------------+----------------------------------------------------------------------+------------------+ +| target | placement | scheduling_state | ++----------------------------+----------------------------------------------------------------------+------------------+ +| DATABASE test | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | +| TABLE test.t1 | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | SCHEDULED | +| TABLE test.t1 PARTITION p1 | PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" | INPROGRESS | ++----------------------------+----------------------------------------------------------------------+------------------+ +3 rows in set (0.00 sec) + ``` -Similarly, `ALTER PLACEMENT POLICY` statements must wait until the PD returns a message. It is implemented by overwriting the current placement rule with a new one. -### Drop placement rules +TiDB will automatically find the effective rule based on the rule priorities. -Dropping the placement rule on a specified role can be achieved by a `DROP PLACEMENT POLICY` clause: +This statement outputs at most 1 line. For example, when querying a table, only the placement rule defined on the table itself is shown, and the partitions in it will not be shown. -```sql -ALTER TABLE table_name - DROP PLACEMENT POLICY ROLE=role, - ... -``` +The output of this statement contains these fields: -In the statement, only `ROLE` option is needed. It only drops the placement rule on `role`. The rule can be either defined on the object itself or inherited from its parent. For example, if a rule on table `t` is inherited from its database, it can also be dropped through this way. +* Target: The object queried. It can be a database, table, partition, or index. + * For policies, it is shown as the policy name. + * For database, it is shown in the format `DATABASE database_name` + * For table, it is shown in the format `TABLE database_name.table_name` + * For partition, it is shown in the format `TABLE database_name.table_name PARTITION partition_name` +* Placement: An equivalent `ALTER` statement on `target` that defines the placement rule. +* Scheduling state: The scheduling progress from the PD aspect. -Dropping placement rules should also conform to Raft constraints. That is, there must be a leader after dropping. For example, if the original placement rule is: +For finding the current use of a placement policy, the following syntax can be used: ```sql -ALTER TABLE table_name - ALTER PLACEMENT POLICY ROLE=voter REPLICAS=3; +SHOW PLACEMENT LIKE 'POLICY standardpol%'; ``` -It will report an error when executing following statement: +This will match for `PLACEMENT POLICY` names such as `standardpolicy`. + +### Updates to Existing Syntax + +#### CREATE DATABASE / ALTER DATABASE + +The semantics of a `PLACEMENT POLICY` on a database/schema should be similar to the [default character set attribute](https://dev.mysql.com/doc/refman/8.0/en/charset-database.html). For example: ```sql -ALTER TABLE table_name - DROP PLACEMENT POLICY ROLE=voter; +CREATE DATABASE mydb [DEFAULT] PLACEMENT POLICY=`companystandardpolicy`; +CREATE TABLE mydb.t1 (a INT); +ALTER DATABASE mydb [DEFAULT] PLACEMENT POLICY=`companynewpolicy`; +CREATE TABLE mydb.t2 (a INT); +CREATE TABLE mydb.t3 (a INT) PLACEMENT POLICY=`companystandardpolicy`; ``` -No leader is left after dropping all the voters, so it's illegal. +* The tables t1 and t3 are created with the policy `companystandardpolicy` and the table t2 is created with `companynewpolicy`. +* The `DATABASE` default only affects tables when they are created and there is no explicit placement policy defined. +* Thus, the inheritance rules only apply when tables are being created, and if the database policy changes this will not update the table values. This differs slightly for table partitions. +* The statement `SHOW CREATE DATABASE` is also available, and will show the [DEFAULT] PLACEMENT POLICY in TiDB feature specific comments (/*T![placement] DEFAULT PLACEMENT POLICY x */) -As leader must exist, it's not allowed to drop all the placement rules. Besides, if there are less than 2 followers left after dropping, a warning will be reported. +#### SHOW CREATE TABLE -However, resetting all the rules on an object may be useful. "Resetting" means to drop the placement rules defined on the object itself, and let the object follow all the rules of its parent. +The output of `SHOW CREATE TABLE` should describe any placement options that are either explicitly specified, or inherited from the default placement from `CREATE DATABASE` / `ALTER DATABASE`. This should be escaped in TiDB feature-specific comment syntax. i.e. -There is no shortcut to reset all the rules. It may help, but it makes the system more complicated. It will be reconsidered when it's really needed. +```sql +use test; +CREATE TABLE t1 (a int); +SHOW CREATE TABLE t1; +--> +CREATE TABLE `t1` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin; + +CREATE TABLE t2 (a int) PLACEMENT POLICY='acdc'; +SHOW CREATE TABLE t2; +--> +CREATE TABLE `t2` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`acdc` */; + +ALTER DATABASE test DEFAULT PLACEMENT POLICY=`acdc`; +CREATE TABLE t3 (a int); +SHOW CREATE TABLE t3; +--> +CREATE TABLE `t3` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PLACEMENT POLICY=`acdc` */; + +ALTER TABLE t3 PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2,us-west-1,us-west-2"; +SHOW CREATE TABLE t3; +--> +CREATE TABLE `t3` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![placement] PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2,us-west-1,us-west-2" */; -Placement rules of indices and partitions can also be dropped in a similar grammar. The statement must wait until PD returns a message. +``` + +This helps ensure the highest level of compatibility between both TiDB versions and MySQL. + +### Placement Rules Syntax -### Constraints configuration +#### Constraints configuration -`CONSTRAINTS` option in the `ADD PLACEMENT POLICY` or `ALTER PLACEMENT POLICY` clauses indicates the label constraints. Data must be placed on the stores whose labels conform to `CONSTRAINTS` constraints. If `CONSTRAINTS` is omitted, it means no label constraint is enforced, thus the replicas can be placed anywhere. +The constraints syntax (described as "Advanced Placement" above) allows placement to be enforced based on labels. If CONSTRAINTS are omitted, it means no label constraint is enforced, thus the replicas can be placed anywhere. -Option `CONSTRAINTS` should be a string and in one of these formats: +`CONSTRAINTS` should be a string and in one of these formats: -- List: `[{+|-}key=value,...]`, e.g. `[+zone=bj,-disk=hdd]` -- Dictionary: `{"{+|-}key=value,...":count,...}`, e.g. `{"+zone=bj,-disk=hdd":1, +zone=sh:2}` +- List: `[{+|-}key=value,...]`, e.g. `[+region=us-east-1,-disk=hdd]` +- Dictionary: `{"{+|-}key=value,...":count,...}`, e.g. `{"+region=us-east-1,-disk=hdd":1, +region=us-east-2:2}` -Prefix `+` indicates that data can only be placed on the stores whose labels contain such labels, and `-` indicates that data can’t be placed on the stores whose labels contain such labels. For example, `+zone=sh,+zone=bj` indicates to place data only in `sh` and `bj` zones. +The prefix `+` indicates that data can only be placed on the stores whose labels contain such labels, and `-` indicates that data can’t be placed on the stores whose labels contain such labels. For example, `+region=us-east-1,+region=us-east-2` indicates to place data only in `us-east-1` and `us-east-2` regions. -`key` here refers to the label name, and `value` is the label value. The label name should have already been defined in the store configurations. For example, assuming a store has following labels: +The `key` here refers to the label name, and `value` is the label value. The label name should have already been defined in the store configurations. For example, assuming a store has following labels: ```sql [server] -labels = "zone=bj,rack=rack0,disk=hdd" +labels = "region=us-east-1,rack=rack0,disk=hdd" ``` -Then `+zone=bj` matches this store while `+disk=ssd` doesn't. +Then `+region=us-east-1` matches this store while `+disk=ssd` doesn't. -In the dictionary format, `count` must be specified, which indicates the number of replicas placed on those stores. When the prefix is `-`, the `count` is still meaningful. +In the dictionary format, `count` must be specified, which indicates a quantity which must match. When the prefix is `-`, the `count` is still meaningful. -For example, `CONSTRAINTS="{+zone=sh:1,-zone=bj:2}"` indicates to place 1 replica in `sh`, 2 replicas in anywhere but `bj`. +For example, `FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,-region=us-east-2:2}"` indicates to place at least 1 follower in `us-east-1`, 2 replicas in anywhere but `us-east-2` (by definition this will be `us-east-1` since there are no other regions available). -In the list format, `count` is not specified. The number of replicas for each constraint is not limited, but the total number of replicas should still conform to the `REPLICAS` option. +In the list format, `count` is not specified. The number of followers for each constraint is not limited, but the total number of instances should still conform to the definition. -For example, `CONSTRAINTS="[+zone=sh,+zone=bj]" REPLICAS=3` indicates to place 3 replicas on either `sh` or `bj`. There may be 2 replicas on `sh` and 1 in `bj`, or 2 in `bj` and 1 in `sh`. It's up to PD. +For example, `FOLLOWER_CONSTRAINTS="[+region=us-east-1,+region=us-east-2]" FOLLOWERS=3` indicates to place 3 followers on either `us-east-1` or `us-east-1`. There may be 2 replicas on `us-east-1` and 1 in `us-east-2`, or 2 in `us-east-2` and 1 in `us-east-1`. It's up to PD (see "Schedule Property" for additional details). Label constraints can be implemented by defining `label_constraints` field in PD placement rule configuration. `+` and `-` correspond to property `op`. Specifically, `+` is equivalent to `in` and `-` is equivalent to `notIn`. -For example, `+zone=sh,+zone=bj,-disk=hdd` is equivalent to: +For example, `+region=us-east-1,+region=us-east-2,-disk=hdd` is equivalent to: ``` "label_constraints": [ - {"key": "zone", "op": "in", "values": ["sh", "bj"]}, + {"key": "region", "op": "in", "values": ["us-east-1", "us-east-2"]}, {"key": "disk", "op": "notIn", "values": ["hdd"]} ] ``` Field `location_labels` in PD placement rule configuration is used to isolate replicas to different zones to improve availability. For now, the global configuration can be used as the default `location_labels` for all placement rules defined in SQL, so it's unnecessary to specify it. -### Role configuration +`PLACEMENT` also supports adding TiFlash replicas for a table, as the statement `ALTER TABLE table_name SET TIFLASH REPLICA count` does. For example: -`ROLE` in the statement defines the Raft role of the replicas. It must be specified in the statement. There are 4 predefined roles: +```sql +ALTER TABLE t1 + LEARNER_CONSTRAINTS="[+engine=tiflash]" LEARNERS=1; +``` -- `leader`. Exactly one `leader` is allowed. -- `follower`. -- `voter`. It includes `leader` and `follower`. -- `learner`. It can be either TiFlash or TiKV. +The only way to judge whether it’s adding a TiFlash replica is to check the label. If it contains `engine=tiflash`, then it’s adding or removing a TiFlash replica. This logic is conventional in PD for now. -If both `voter` and `follower` are defined in the rules, the replicas of `follower` are not included in the replicas of `voter`. For example: +#### Specifying role count + +The roles `FOLLOWERS`, `LEARNERS` and `VOTERS` also support an optional count in *list* format. For example: ```sql -ALTER TABLE test - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=follower REPLICAS=2, - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=voter REPLICAS=2; +CREATE PLACEMENT POLICY `standardplacement1` PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4; +CREATE PLACEMENT POLICY `standardplacement2` LEADER_CONSTRAINTS="[+region=us-east-1]" FOLLOWERS_CONSTRAINTS="[+region=us-east-1,+region=us-east-2]" FOLLOWERS=4; ``` -There are 4 replicas for table `test`, 2 of which are in `sh` and 2 are in `bj`. Leader can only be placed on `sh`. - -`ROLE` in the statement is equivalent to field `role` in PD placement rule configuration. +If the constraints is specified as a dictionary (e.g. `{"+region=us-east-1":1}`) the count is not applicable and an error is returned: -### Replicas configuration +```sql +FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,-region=us-east-2:2}" FOLLOWERS=3 // technically accurate, but an error +FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,-region=us-east-2:2}" FOLLOWERS=2 // an error +``` -`REPLICAS` in the statement indicates the replica count of the specified role. +For dictionary format, the count is inferred by the constraint. The following constraint creates 4 followers: -Rules defined on `leader` can omit `REPLICAS`, because the count of leader is always 1. +```sql +FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,-region=us-east-2:2,+any:1}" +``` -When all the replica counts are specified in the `CONSTRAINTS` option, `REPLICAS` can also be omitted. For example, `CONSTRAINTS="{+zone=bj:2,+zone=sh:1}", ROLE=voter` indicates that the `REPLICAS` is 3. +Explanation: +- 1 follower in `us-east-1` +- 2 followers not in `us-east-2` +- 1 follower in any region (a special label of `+any`) -When both `REPLICAS` and `count` in `CONSTRAINTS` are specified, it indicates that the other replicas can be placed anywhere. For example, in the case `CONSTRAINTS="{+zone=bj:2,+zone=sh:1}", ROLE=voter, REPLICAS=4`, 2 replicas are in `bj` and 1 in `sh`, and the last replica can be anywhere, including `bj` and `sh`. +`+any` changes an earlier proposal where the `FOLLOWERS` count could also be specified. This has been removed to reduce the risk of discrepancies and misconfiguration. See also "Policy Validation" below. -When the `CONSTRAINTS` option doesn't contain `count`, `REPLICAS` must be specified. For example, `CONSTRAINTS="[+zone=bj]" ROLE=follower` is vague, as the count of `follower` can not be inferred. +#### Built-in Placement Policies -`REPLICAS` in the statement is equivalent to field `count` in PD placement rule configuration. +By default every system will have two placement policies, which can be modified via `ALTER PLACEMENT POLICY` but never dropped: -### Key range configuration +* `default`: This policy is used only in the event that a policy has not been specified. +* `system`: This policy is used for internal TiDB system tables. -In PD placement rule configuration, the key range must be specified. Now that `table_name` is specified in the `ALTER TABLE` statement, key range can be inferred. +Some common applications might be to increase the replica count on system or default tables. It is not typically recommended to add constraints to these policies as it will lead to cluster inbalance, but it is possible. -Typically, key format is in such a format: `t_{table_id}_r_{pk_value}`, where `pk_value` may be `_tidb_rowid` in some cases. `table_id` can be inferred from `table_name`, thus key range is `t_{table_id}_` to `t_{table_id+1}_`. +#### Schedule Property -Similarly, key range of partitions and indices can also be inferred. +When using either the syntactic sugar or list format for placement rules, PD is free to schedule followers/leaders/voters wherever it decides. For example: -### Region label configuration +```sql +CREATE PLACEMENT POLICY `standardplacement1` PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4; +CREATE PLACEMENT POLICY `standardplacement2` LEADER_CONSTRAINTS="[+region=us-east-1]" FOLLOWERS_CONSTRAINTS="[+region=us-east-1,+region=us-east-2]" FOLLOWERS=4; +``` -Instead of configuring key ranges, you can also configure region labels in placement rules. PD supports label rules, which indicate the key range of a database / table / partition name. TiDB pushes label rules once the schema changes, so that PD maintains the relationship between database / table /partition names and their corresponding key ranges. +- Are each of the followers split equally in us-east-1 and us-east-2? +- Could the majority of followers be placed in us-east-1? That would ensure fast quorum but reduced fault tolerance. -This is what a label rule may look like: +To address the ambiguity the concept of a "schedule" is introduced, which is the name for a strategy that PD uses to determine where to place instances. For an example: -``` -{ - "id": "db1/tb1", - "labels": [ - { - "key": "database-name", - "value": "db1" - }, - { - "key": "table-name", - "value": "db1/tb1" - } - ], - "match-type": "key-range", - "match": { - "start-key": "7480000000000000ff0a00000000000000f8", - "end-key": "7480000000000000ff0b00000000000000f8" - } -} +```sql +CREATE PLACEMENT POLICY `standardplacement1` PRIMARY_REGION="us-east-1" REGIONS="us-east-1,us-east-2" FOLLOWERS=4 SCHEDULE="EVEN"; ``` -It connects the table name `db1/tb` with the key range. +The following `SCHEDULE` options are available: -Now you need to connect the label with the database / table / partition name in the placement rules. +* `EVEN`: Followers will be balanced based on the value of the "region" label. So for example if `us-east-1` has 15 stores and `us-east-2` has 5, it doesn't matter. 2 stores will be placed in each. This is recommended for increased fault tolerance. +* `MAJORITY_IN_PRIMARY`: As many followers as required to achieve quorum will be placed in the primary region. The remaining followers will be scheduled in the remaining secondary regions. -For example: +#### Key range configuration -``` -{ - "group_id": "group_id", - "id": "id", - "region_label_key": "schema/table-name", - "region_label_value": "db1/tb1", - "role": "leader", - "label_constraints": [ - {"key": "zone", "op": "in", "values": ["sh", "bj"]} - ] -} -``` +In PD placement rule implementation, the key range must be specified. Now that `table_name` is specified in the `ALTER TABLE` statement, key range can be inferred. -Combined with the label rule, PD indirectly knows the key range of `db1/tb1` is marked with the label constraint `{"key": "zone", "op": "in", "values": ["sh", "bj"]}`. +Typically, key format is in such a format: `t{table_id}_r{pk_value}`, where `pk_value` may be `_tidb_rowid` in some cases. `table_id` can be inferred from `table_name`, thus the key range is `t{table_id}_` to `t{table_id+1}_`. -### Database placement +Similarly, the key range of partitions can also be inferred. -Defining placement rules of databases simplifies the procedures when there are many tables. +#### Policy Validation -For example, in a typical multi-tenant scenario, each user has a private database. The dataset in one database is relatively small, and it’s rare to query across databases. In this case, a whole database can be placed in a single region to reduce multi-region latency. +When placement policies are specified, they should be validated for correctness: -For another example, multiple businesses may run on a single TiDB cluster, which can reduce the overhead of maintaining multiple clusters. The resources of multiple businesses need to be isolated to avoid the risk that one business takes too many resources and affects others. +1. The `FOLLOWERS` count should respect raft quorum expectations. The default is `2` (which creates raft groups of 3). If the number is odd, it could lead to split brain scenarios, so a warning should be issued. Warnings should also be issued for a count less than 2 (this might be useful for development environments, so an error is not returned) +2. A policy that is impossible based on the current topology (region=us-east-1 and followers=2, but there is only 1 store in us-east-1) should be a warning. This allows for some transitional topologies. +3. If the constraints are specified as a dictionary, specifying the count (i.e. `FOLLOWERS=n`) is prohibited. +4. Specifying both direct placement rules (`{FOLLOWERS,VOTERS,LEARNERS}=N, {FOLLOWER,VOTER,LEARNER}_CONSTRAINTS, CONSTRAINTS, PRIMARY_REGION, REGIONS, SCHEDULE`) and a `PLACEMENT POLICY` is prohibited. + +#### Skipping Policy Validation -Placement of databases is defined through `ALTER` statements: +It should be possible to skip policy validation. This can be seen as similar to skipping foreign key checks, which is often used by logical dumpers: ```sql -ALTER {DATABASE | SCHEMA} schema_name - {ADD | ALTER} PLACEMENT POLICY ROLE=role CONSTRAINTS=constraints REPLICAS=replicas, - ... - -ALTER {DATABASE | SCHEMA} schema_name - DROP PLACEMENT POLICY ROLE=role, - ... +SET FOREIGN_KEY_CHECKS=0; +SET PLACEMENT_CHECKS=0; + +CREATE TABLE t3 (a int) PLACEMENT POLICY `mycompanypolicy`; ``` -This statement defines placement rules for one database, including all tables in it. +If a table is imported when `PLACEMENT_CHECKS` is `OFF`, and the placement policy does not validate, then the same rules of fallback apply as in the case `DROP PLACEMENT POLICY` (where policy is still in use). -Creating or dropping a table also affects the placement rules. If a placement rule is defined on a database, all tables in this database will automatically apply that rule, including the existing tables and the tables created later. +#### Ambiguous and edge cases -Once the placement rules on a database are changed, the tables should also update their placement rules. Users can overwrite the rules by defining placement rules on the tables. See the section "Rule inheritance" for details. +The following two policies are not identical: -Since key range is not successive in one database, each table in the database corresponds to at least one placement rule, so there may be many placement rules. In either case above, there may be up to millions of tables in one database, which costs lots of time to update the rules and lots of space to store the rules. +```sql +CREATE PLACEMENT POLICY p1 FOLLOWER_CONSTRAINTS="[+region=us-east-1,+region=us-east-2]" FOLLOWERS=2; +CREATE PLACEMENT POLICY p2 FOLLOWER_CONSTRAINTS="{+region=us-east-1:1,-region=us-east-2:1}"; +``` -Another option is to take advantage of the region label, which is described earlier. +This is because p2 explicitly requires a follower count of 1 per region, whereas p1 allows for 2 in any of the above (see "Schedule Property" above for an explanation). -In the example below, it defines multiple label rules for one database. Each label rule corresponds to one table or partition. +This is useful in the case that you want to ensure that `FOLLOWERS=2` exists in any of a list of zones: -``` -{ - "id": "db1/tb1", - "labels": [ - { - "key": "database-name", - "value": "db1" - }, - { - "key": "table-name", - "value": "db1/tb1" - } - ], - "match-type": "key-range", - "match": { - "start-key": "7480000000000000ff0a00000000000000f8", - "end-key": "7480000000000000ff0b00000000000000f8" - } -}, -{ - "id": "db1/tb2", - "labels": [ - { - "key": "database-name", - "value": "db1" - }, - { - "key": "table-name", - "value": "db1/tb2" - } - ], - "match-type": "key-range", - "match": { - "start-key": "7480000000000000ff0c00000000000000f8", - "end-key": "7480000000000000ff0d00000000000000f8" - } -} +```sql +CREATE PLACEMENT POLICY p2 FOLLOWER_CONSTRAINTS="[+region=us-east-1,+region=us-east-2,+region=us-west-1]" FOLLOWERS=2; ``` -Then you need only one placement rule for the database. When you change the placement of the database, you need to update one placement rule. However, when you drop a database, you need to delete multiple label rules plus one placement rule. +### Additional Semantics -### Partition placement +#### Partitioned Tables -Defining placement rules of partitions is useful for Geo-Partitioning. In the cases where data is very relevant to zones, Geo-Partitioning can be applied to reduce multi-region latency. +The key format of a partitioned table is `t{partition_id}_r{pk_value}`. As `partition_id` is part of the key prefix, the key range of a partition is successive. The key range is `t{partition_id}_` to `t{partition_id+1}_`. + +Defining placement rules of partitions is expected to be a common use case and is useful for both reducing multi-region latency and compliance scenarios. Because there are multiple key ranges for the table, multiple rules will be generated and sent to PD. In Geo-Partitioning, the table must be splitted into partitions, and each partition is placed in specific zones. There are some kinds of partition placement: @@ -424,299 +492,332 @@ In Geo-Partitioning, the table must be splitted into partitions, and each partit It’s up to users to choose the right solution. -Placement of partitions is also defined through `ALTER TABLE` statements: +The semantics of partitioning are different to the default database policy. When a table is partitioned, the partitions will by default inherit the same placement policy as the table. This can be overwritten on a per-partition basis: ```sql -ALTER TABLE table_name ALTER PARTITION partition_name - {ADD | ALTER} PLACEMENT POLICY CONSTRAINTS=constraints ROLE=role REPLICAS=replicas, - ... - -ALTER TABLE table_name ALTER PARTITION partition_name - DROP PLACEMENT POLICY ROLE=role, - ... +CREATE TABLE t1 (id INT, name VARCHAR(50), purchased DATE) + PARTITION BY RANGE( YEAR(purchased) ) ( + PARTITION p0 VALUES LESS THAN (2000) PLACEMENT POLICY='storeonhdd', + PARTITION p1 VALUES LESS THAN (2005), + PARTITION p2 VALUES LESS THAN (2010), + PARTITION p3 VALUES LESS THAN (2015), + PARTITION p4 VALUES LESS THAN MAXVALUE PLACEMENT POLICY='storeonfastssd' + ) +PLACEMENT POLICY='companystandardpolicy'; ``` -This statement defines placement rules for one partition, including its local indices. - -The key format of a partitioned table is `t_{partition_id}_r_{pk_value}`. As `partition_id` is part of the key prefix, the key range of a partition is successive. The key range is `t_{partition_id}_` to `t_{partition_id+1}_`. - -Placement rules can also be defined on a partitioned table. Because there are multiple key ranges for the table, multiple rules will be generated and sent to PD. When placement rules are defined both on the table and its partitions, the rule priorities described later should be applied. - -### Unpartitioned index placement - -Defining placement rules of indices is more complicated, because indices can be unpartitioned or partitioned. Each case should be considered separately. - -The index here can be primary index or secondary index. When the key of a clustered index is `_tidb_rowid` rather than the primary key, the primary index is actually an unclustered index. In this case, an index placement statement is applied. - -Expression indices and invisible indices are also supported, as the key format is the same as normal. - -Defining placement of an unpartitioned index is in such a statement: +In this example, partition `p0` uses the policy `storeonhdd`, partition `p4` uses the policy `storeonfastssd` and the remaining partitions use the policy `companystandardpolicy`. Assuming the following `ALTER TABLE` statement is executed, only partitions `p1`-`p3` will be updated: ```sql -ALTER TABLE table_name ALTER INDEX index_name - {ADD | ALTER} PLACEMENT POLICY CONSTRAINTS=constraints ROLE=role REPLICAS=replicas, - ... - -ALTER TABLE table_name ALTER INDEX index_name - DROP PLACEMENT POLICY ROLE=role, - ... +ALTER TABLE t1 PLACEMENT POLICY=`companynewpolicy`; ``` -This key format of an unpartitioned index is `t_{table_id}_i_{index_id}_r_{pk_value}`. The key range can be inferred by `table_id` and `index_id`. +For the syntax: +```sql +ALTER TABLE pt + EXCHANGE PARTITION p + WITH TABLE nt; +``` -### Partitioned index placement +If `nt` has placement rules associated with it, they will be retained when it becomes a partition of table `pt`. However, if no placement rules have been specified, then the rules of the table `pt` will be used. This helps protect against the case that a partition may need to have "default" placement rules, but default does not mean what the table uses (the output of `SHOW CREATE TABLE` would appear ambiguous). When the partition `p` is converted to table `nt`, it will continue to use the rules it had as a partition (either explicitly listed for the partition or the default for the table). -Defining placement rules of an index in one specific partition is in such a statement: +This behavior is inspired by how a `CHARACTER SET` or `COLLATE` attribute applies to a column of a table, and columns will use the character set defined at the table-level by default. -```sql -ALTER TABLE table_name ALTER PARTITION partition_name ALTER INDEX index_name - {ADD | ALTER} PLACEMENT POLICY CONSTRAINTS=constraints ROLE=role REPLICAS=replicas, - ... - -ALTER TABLE table_name ALTER PARTITION partition_name ALTER INDEX index_name - DROP PLACEMENT POLICY ROLE=role, - ... -``` +#### Removing placement from a database, table or partition -The key format of partitioned index is `t_{partition_id}_i_{index_id}_r_{pk_value}`. The key range can be inferred by `partition_id` and `index_id`. +Placement policy can be removed from an object via the following syntax: -When an index is partitioned, defining placement rule of the whole index at once is not supported. It will involve multiple key ranges, and the scenario of its application is rare. +```sql +ALTER DATABASE test [DEFAULT] PLACEMENT POLICY SET DEFAULT; -- standard syntax for ALTER DATABASE +ALTER DATABASE test [DEFAULT] PLACEMENT POLICY=default; -- alternative +ALTER TABLE t1 PLACEMENT POLICY=default; +ALTER TABLE t1 PARTITION partition_name PLACEMENT POLICY=default; +``` -For example, `t` is a partitioned table and `idx` is the index on `t`. It’s not supported to do this: +In this case the default rules will apply to placement, and the output from `SHOW CREATE TABLE t1` should show no placement information. Thus, setting `PLACEMENT POLICY=default` must reset the following `table_options`: +- `FOLLOWERS=n` +- `VOTERS=n` +- `LEARNERS=n` +- `PRIMARY REGION` +- `REGIONS` +- `SCHEDULE` +- `CONSTRAINTS` +- `FOLLOWER_CONSTRAINTS` +- `VOTER_CONSTRAINTS` +- `LEARNER_CONSTRAINTS` +- `PLACEMENT POLICY` + +For a more complex rule using partitions, consider the following example: ```sql -ALTER TABLE `t` ALTER INDEX `idx` - ADD PLACEMENT POLICY ... +ALTER TABLE t1 PARTITION p0 PLACEMENT="acdc"; +--> +CREATE TABLE t1 (id INT, name VARCHAR(50), purchased DATE) + PARTITION BY RANGE( YEAR(purchased) ) ( + PARTITION p0 VALUES LESS THAN (2000) PLACEMENT="acdc", + PARTITION p1 VALUES LESS THAN (2005) + ); + +ALTER TABLE t1 PLACEMENT="xyz"; +--> +CREATE TABLE t1 (id INT, name VARCHAR(50), purchased DATE) + PARTITION BY RANGE( YEAR(purchased) ) ( + PARTITION p0 VALUES LESS THAN (2000) PLACEMENT="acdc", + PARTITION p1 VALUES LESS THAN (2005) + ) PLACEMENT="xyz"; + +ALTER TABLE t1 PARTITION p0 PLACEMENT=DEFAULT; +--> +CREATE TABLE t1 (id INT, name VARCHAR(50), purchased DATE) + PARTITION BY RANGE( YEAR(purchased) ) ( + PARTITION p0 VALUES LESS THAN (2000), + PARTITION p1 VALUES LESS THAN (2005) + ) PLACEMENT="xyz"; + ``` -To alter the placement rule of `idx`, a partition must be specified in the statement. +The behavior above is described as `ALTER TABLE t1 PARTITION p0 PLACEMENT=DEFAULT` resets the placement of the partition `p0` to be inherited from the table `t1`. -Currently, global secondary index on partitioned tables is not supported, so it can be ignored for now. - -### Sequence placement +#### Sequences Sequence is typically used to allocate ID in `INSERT` statements, so the placement of sequences affects the latency of `INSERT` statements. However, sequence is typically used with cache enabled, which means very few requests are sent to sequence. So defining placement rules of sequences is not supported for now. -## DDL management - -Some kinds of DDL on databases also affect placement rules. - -### DDL on tables - -Once a table is created, it follows the placement rule of its database. - -Defining placement rules in a `CREATE TABLE` statement is useful, especially in data sovereignty scenarios. Data sovereignty requires sensitive data to reside within its own national border, which is very serious. So defining placement rules after creating tables is not acceptable. But for now, it's not supported, as it complicates the implementation. - -Once a table is dropped, the placement rules on it cannot be dropped immediately, because the table can be recovered by `FLASHBACK` or `RECOVER` statements before GC collects the data. Related placement rules should be kept temporarily and will be removed after GC lifetime. +#### DDL on tables -Since dropped tables are collected by the GC worker, when the GC worker collects a table, the related placement rules can be removed. - -When it’s time to remove all relevant placement rules, not only those rules defined on the table should be removed, but also the rules defined on its partitions and indices. - -Once a table is truncated, the table id is updated. As its key range is changed, the placement rules should also be updated. - -Since the table can be recovered later by `FLASHBACK` statement, a snapshot of the original placement rules should be saved temporarily. After recovering, the table name is changed, but the table id is the original one, so the snapshot of the original placement rules can be recovered directly. - -For example: +The placement policy is associated with the definition of the table (and visible in `SHOW CREATE TABLE`). Thus, if a table is recovered by `FLASHBACK` or `RECOVER`, it is expected that the previous rules will be restored. -```sql -TRUNCATE TABLE t; +Similarly, `TRUNCATE [TABLE]` does not change the definition of a table. It is expected that as new data is inserted, it will continue to respect placement rules. -ALTER TABLE t - ALTER PLACEMENT POLICY CONSTRAINTS="+zone=sh" ROLE=leader; +#### SHOW DDL jobs -FLASHBACK table t to t1; -``` +Because `CREATE TABLE` and `ALTER TABLE` are DDL, changes to placement are also considered DDL and are visible via `ADMIN SHOW DDL JOBS`. -In this case, the placement rules of `t` is altered by the user just after truncating. Once `t` is flashbacked to `t1`, the placement rules of `t1` should be recovered to the version before `TRUNCATE` instead of the version after `ALTER PLACEMENT POLICY`. However, the procedure is quite complicated and this kind of action is rare, so the placement rules will be recovered to the newest version for now. +The fact that the DDL procedure in TiDB is mature helps to achieve some features of defining placement rules: -DDL on partitions and indices will be discussed below, and other DDL on tables won’t affect placement rules: +- Placement rules are defined in serial as there's only one DDL owner at the same time +- DDL is capable of disaster recovery as the middle states are persistent in TiKV +- DDL is rollbackable as the middle states can transform from one to another +- Updating schema version guarantees all active transactions are based on the same version of placement rules -* Altering columns -* Renaming tables -* Altering charset and collation +The actual "completion" of the DDL job as far as TiDB is concerned is that PD has been notified of the placement rules for all of the affected regions. PD will then asynchronously apply the placement rules to all of the regions, and this progress is not observable via `ADMIN SHOW DDL JOBS`. The progress of scheduling can be observed via `SHOW PLACEMENT` or by reading `information_schema.placement_rules`. -### DDL on partitions +### Privilege management -TiDB supports adding and dropping partitions. +Privilege management is quite straightforward: -Once a partition is added, its placement rule is empty and the partition follows the rule of the table it belongs to. +* `ALTER [DATABASE|TABLE]` statement requires `Alter` privilege +* `CREATE TABLE` statement requires `Create` privilege +* `information_schema.placement_rules` and `SHOW PLACEMENT` only shows the placement rules on the objects that visible to the current user +* `ADMIN SHOW DDL` requires `Super` privilege +* `CREATE PLACEMENT POLICY`, `DROP PLACEMENT POLICY` and `ALTER PLACEMENT POLICY` require `PLACEMENT_ADMIN` (a new dynamic privilege). This is because these objects have global scope. -Once a partition is dropped, it can’t be recovered anymore, so its placement rules can be removed immediately. +## Implementation -Also note that DDL on tables may also effect partitions. It's descibed in the section "DDL on tables". +### Storing Placement Policies -### DDL on indices +Placement policies will be stored in a table in the `mysql` schema. The policy name must be globally unique, but the definition of the table is not described in this proposal. -Once an index is created on an unpartitioned table, the index should follow the rule of the table it belongs to. +Because the implementation is TiDB specific (does not require any compatibility with MySQL), it is up to the implementer to decide. -Once an index is created on a table with partitions, each part of the index should follow the rule of the partition it belongs to. +### Storage Consistency -Once an index is dropped, it can’t be recovered anymore, so its placement rules can be removed immediately. +PD uses placement rules to schedule data, so a replica of placement rules for _tables and partitions_ must be persistent in PD. However, because the rules are also considered part of the table definition, the placement rules are also persisted in PD. -Altering primary index is the same with altering secondary indexes. Because if a primary index can be created or dropped, it must be an unclustered index. +The rules to guarantee consistency between these two sources is as follows: -Other DDL on indices won’t affect placement rules: +- Changes to definition (`CREATE|ALTER TABLE`, `CREATE|ALTER PLACEMENT POLICY`, `CREATE|ALTER DATABASE`) will first be persisted to TiKV. +- The changes will then be applied to PD (and asynchronously apply) -* Renaming index -* Altering the visibility of index +It is safe to automatically retry applying the rules to PD because they are all idempotent in the current design. -### Show DDL jobs +If PD can not be modified, then the changes to TIKV are expected to rollback and the statement returns an error. Thus, TiKV acts as an undo log. -As mentioned before, all statements related to placement rules must wait until PD returns. If the execution is interrupted, the job will be cancelled and the DDL will rollback, just like other DDL jobs. +Because placement rules can also be configured outside of placement rules in SQL, PD should be considered the source of truth. -PD schedules regions asynchronously after it returns the message. TiDB can query the progress of scheduling from PD. The progress is observed by executing `SHOW PLACEMENT POLICY` instead of `ADMIN SHOW DDL JOBS`, because the DDL job finishes once PD returns a message. +### Querying placement rules -Ongoing and finished statements can also be queried through `ADMIN SHOW DDL`, `ADMIN SHOW DDL JOBS`, or other similar statements. +The scenarios where TiDB queries placement rules are as follows: -## View rules +1. The optimizer uses placement rules to decide to route cop requests to TiKV or TiFlash. It's already implemented and the TiFlash information is written into table information, which is stored on TiKV. +2. It will probably be used in locality-aware features in the future, such as follower-read. Follower-read is always used when TiDB wants to read the nearest replica to reduce multi-region latency. In some distributed databases, it’s implemented by labelling data nodes and selecting the nearest replica according to the labels. +3. Local transactions need to know the binding relationship between Raft leader and region, which is also defined by placement rules. +4. Once a rule is defined on a table, all the subsequent partitions added to the table should also inherit the rule. So the `ADD PARTITION` operation should query the rules on the table. The same is true for creating tables and indices. +5. The `SHOW PLACEMENT` statement should output the placement rules correctly. -All placement rules can be queried through statements. +As placement rules will be queried in case 1, 2 and 3, low latency must be guaranteed. As discussed in section "Storing placement rules", PD is the source of truth. To lower the latency, the only way is caching the placement rules in TiDB. -### System table +Since the cache is created, there must be a way to validate it. Different from region cache, placement rules cache can only be validated each time from PD. There are some ways to work around: -A new system table `information_schema.placement_rules` is added to view all placement rules. The table contains such columns: +- Update the schema version once a placement rule is changed, just like other DDL. PD broadcasts the latest schema version to all the TiDB instances, and then TiDB instances fetch the newest placement rules from PD. There will be a slight delay for queries before reading the latest placement rules. The side effect is that more transactions will retry since the schema version is changed. +- TiDB queries placement rules from PD periodly. The delay is controllable but not eliminable. +- Once a placement rule is changed, PD broadcasts it to all the TiDB instances. In this approach, the schema version is not involved, so transactions are not affected. The delay is not eliminable either. -* rule_id -* target ID -* target name -* constraints -* role -* replicas -* scheduling state +All the approaches above will result in a delay. Fortunately, for case 1 and 2 above, delay is acceptable. It doesn’t matter much if the optimizer doesn’t perceive the placement rules changement immediately. The worst result is that the latency is relatively high for a short time. -The system table is a virtual table, which doesn’t persist data. When querying the table, TiDB queries PD and integrates the result in a table format. That also means the metadata is stored on PD instead of TiKV. +For case 3, although delay is acceptable, but all TiDB instances must be always consistent on the placement rules. To achieve this goal, the schema version needs to be updated, thus transactions with old placement rules will fail when committed. -An object may contain multiple placement rules, each of which corresponds to a rule in PD. +For case 4 and 5, delay is not acceptable. Once the placement rules are written successfully, subsequent DDL statements should fetch the latest placement rules to guarantee linearizability. Now that the schema version is changed and the latest placement rules are broadcast to all the TiDB instances immediately, delay is eliminable. -Advantages of building system table include: +Once the schema version is changed, all TiDB instances recognize the object ID and fetch placement rules from PD, rather than TiKV. -* It’s easy for users to filter and aggregate the result -* There’s no need to support a new grammar, and it’s easier to implement +To query the placement rules on a specified object, the object ID should be written to the placement rules, or it can be inferred from other fields. Now that `id` contains the object ID, TiDB can decode the object ID from it. See section "Building placement rules" for details. -### Show placement +### Building placement rules -But there’re a problem here. The system table only contains stored placement rules, and users cannot query the effective rule of one object from it. +There needs a way to map the placement rules in SQL to PD placement rule configuration. Most of the fields are discussed above, so this part focuses on `group_id`, `id`, `start_key` and `end_key`. -For example, table `t` has two partitions `p0` and `p1`, and a placement rule is added on `t`. If the user wants to query the working rule of `p0`, he will find no placement rule is defined for `p0` through the system table. Based on the rule priorities described later, he must query the placement rule on `t`. This procedure is annoying. +`group_id` is used to identify the source of the placement rules, so `group_id` is `tidb`. -To simplify the procedure, a `SHOW PLACEMENT POLICY` statement is provided to query the effective rule for one specified object. +`ALTER PLACEMENT POLICY` and `DROP PLACEMENT POLICY` need to find the rules of a specified object efficiently. It can be achieved by encoding the object ID in `id`. -The statement is in such a format: +However, an object (database, table, partition) may have multiple rules for a single role. For example: ```sql -SHOW PLACEMENT POLICY FOR {DATABASE | SCHEMA} schema_name; -SHOW PLACEMENT POLICY FOR TABLE table_name [PARTITION partition_name]; -SHOW PLACEMENT POLICY FOR INDEX index_name FROM table_name [PARTITION partition_name]; +ALTER TABLE t + VOTER_CONSTRAINTS="{+region=us-east-1:2,+region=us-east-2:1}" VOTERS=3; ``` -TiDB will automatically find the effective rule based on the rule priorities. - -This statement outputs at most 1 line. For example, when querying a table, only the placement rule defined on the table itself is shown, and the partitions and indices in it will not be shown. - -The output of this statement contains these fields: - -* Target: The object queried. It can be a database, table, partition, or index. - * For database, it is shown in the format `DATABASE database_name` - * For table, it is shown in the format `TABLE database_name.table_name` - * For partition, it is shown in the format `TABLE database_name.table_name PARTITION partition_name` - * For index, it is shown in the format `INDEX index_name FROM database_name.table_name` -* Equivalent placement: A equivalent `ALTER` statement on `target` that defines the placement rule. -* Existing placement: All the executed `ALTER` statements that affect the placement of `target`, including the statements on its parent. -* Scheduling state: The scheduling progress from the PD aspect. - -### Show create table - -It’s useful to show rules in `SHOW CREATE TABLE` statement, because users can check the rules easily. - -Since data in TiDB can be imported to MySQL, the placement rules definition must be shown as a MySQL-compatible comment such as `/*T![placement] placement_clause*/`, where `placement_clause` can be recognized by TiDB. That means TiDB needs to support two approaches to define placement rules, one in `CREATE TABLE` and another in `ALTER TABLE`. - -This is complicated, and `ALTER TABLE` is able to satisfy most of the cases, so `SHOW CREATE TABLE` is kept untouched for now. - -## Implementation - -This section focuses on the implemention details of defining placement rules in SQL. - -### Storing placement rules +It needs 2 placement rules for `voter` in the PD placement rule configuration, because each rule can only specify one `count`. To make `id` unique, a unique identifier must be appended to `id`. DDL job ID plus an index in the job is a good choice. -PD uses placement rules to schedule data, so a replica of placement rules must be persistent on the PD. +Take the case above for example, assuming the table ID of `t` is 100, the ID of the DDL job executing this statement is 200, then `id` of the placement rules are `100-200-1` and `100-200-2`. -However, TiDB also uses placement rules in some cases, as discussed in section "Querying placement rules". There are basically 2 ways to achieve this: +The prefix of `id` is in such a format: -- Save the placement rules in table information, which will be duplicated with PD -- Only PD persists the placement rules, while TiDB caches a copy of them +* Database: database id +* Table: table id +* Partition: partition id +* Unpartitioned index: the concatenation of table id and index id, e.g. `100_1` +* Partitioned index: the concatenation of partition id and index id -Before choosing the solution, transactional requirements need to be noticed: +To query all the placement rules for one object, PD looks for all the `id` with a specific prefix. -- Defining placement rules may fail, and users will probably retry it. As retrying `ADD PLACEMENT POLICY` will add more replicas than expected, the atomicity of the opertion needs to be guaranteed. -- `ADD PLACEMENT POLICY` needs to read the original placement rules, combine the 2 rules and then store them to PD, so linearizability should be gauranteed. +As all placement rules are mapped to PD placement rule configurations, `start_key` and `end_key` must be generated for each object. However, databases and partitioned tables have no key ranges, so the only way is to generate a key range with no actual records. -If the placement rules are stored on both TiKV and PD, the approaches to keep atomicity are as follows: +As database IDs are all globally unique, it's fine to replace table ID with database ID in the key range. For example, assuming the database ID is 100, then the string format of its key range is: -- Enforce a 2PC protocol on TiKV and PD. -- Store them on TiKV along with a middle state. If TiKV succeeds, then try PD, otherwise rollback it by the middle state. The DDL procedure guarantees the atomicity even if TiDB is down. +- `start_key`: `t{database_id}_` +- `end_key`: `t{database_id+1}_` -The approaches to keep linearizability are as follows: +It's same for partitioned tables. -- Define placement rules in serial. -- Enforce an exclusive lock on one of the replicas and release it after the whole job finishes. +#### Region label configuration -As a contrast, if the placement rules are stored only on PD, the approaches to keep atomicity are as follows: +Instead of configuring key ranges, you can also configure region labels in placement rules. PD supports label rules, which indicate the key range of a database / table / partition name. TiDB pushes label rules once the schema changes, so that PD maintains the relationship between database / table /partition names and their corresponding key ranges. -- Write all the placement rules in one ETCD transaction. -- Persist a middle state on TiKV before sending to PD. This middle state acts as undo log. +This is what a label rule may look like: -The approaches to keep linearizability are as follows: +``` +{ + "id": "db1/tb1", + "labels": [ + { + "key": "database-name", + "value": "db1" + }, + { + "key": "table-name", + "value": "db1/tb1" + } + ], + "match-type": "key-range", + "match": { + "start-key": "7480000000000000ff0a00000000000000f8", + "end-key": "7480000000000000ff0b00000000000000f8" + } +} +``` -- Define placement rules in serial. -- Enforce an exclusive lock on PD and release it after the job finishes. +It connects the table name `db1/tb1` with the key range. -The comparison shows that both solutions are possible, but storing placement rules only on PD is more practical. To guarantee the transactional characteristics, the easiest way is to write all placement rules in a transaction and define them in serial on the TiDB side. +Now you need to connect the label with the database / table / partition name in the placement rules. -### Querying placement rules +For example: -The scenarios where TiDB queries placement rules are as follows: +``` +{ + "group_id": "group_id", + "id": "id", + "region_label_key": "schema/table-name", + "region_label_value": "db1/tb1", + "role": "leader", + "label_constraints": [ + {"key": "region", "op": "in", "values": ["us-east-1", "us-east-2"]} + ] +} +``` -1. The optimizer uses placement rules to decide to route cop request to TiKV or TiFlash. It's already implemented and the TiFlash information is written into table information, which is stored on TiKV. -2. It will be probably used in locality-aware features in the future, such as follower-read. Follower-read is always used when TiDB wants to read the nearest replica to reduce multi-region latency. In some distributed databases, it’s implemented by labelling data nodes and selecting the nearest replica according to the labels. -3. Local transactions need to know the binding relationship between Raft leader and region, which is also defined by placement rules. -4. Once a rule is defined on a table, all the subsequent partitions added to the table should also inherit the rule. So the `ADD PARTITION` operation should query the rules on the table. The same is true for creating tables and indices. -5. `SHOW PLACEMENT POLICY` statement should output the placement rules correctly. +Combined with the label rule, PD indirectly knows the key range of `db1/tb1` is marked with the label constraint `{"key": "region", "op": "in", "values": ["us-east-1", "us-east-2"]}`. -As placement rules will be queried in case 1, 2 and 3, low latency must be guaranteed. As discussed in section "Storing placement rules", placement rules are only persistent on PD. To lower the latency, the only way is caching the placement rules in TiDB. +#### Database placement -Since the cache is created, there must be a way to validate it. Different from region cache, placement rules cache can only be validated each time from PD. There are some ways to work around: +Defining placement rules of databases simplifies the procedures when there are many tables. -- Update the schema version once a placement rule is changed, just like other DDL. PD broadcasts the latest schema version to all the TiDB instances, and then TiDB instances fetch the newest placement rules from PD. There will be a slight delay for queries before reading the latest placement rules. The side affect is that more transactions will retry since the schema version is changed. -- TiDB queries placement rules from PD periodly. The delay is controllable but not eliminable. -- Once a placement rule is changed, PD broadcasts it to all the TiDB instances. In this approach, schema version is not involved, so transactions are not affected. The delay is not eliminable either. +For example, in a typical multi-tenant scenario, each user has a private database. The dataset in one database is relatively small, and it’s rare to query across databases. In this case, a whole database can be placed in a single region to reduce multi-region latency. -All the approaches above will result in a delay. Fortunately, for case 1 and 2 above, delay is acceptable. It doesn’t matter much if the optimizer doesn’t perceive the placement rules changement immediately. The worst result is that the latency is relatively high for a short time. +For another example, multiple businesses may run on a single TiDB cluster, which can reduce the overhead of maintaining multiple clusters. The resources of multiple businesses need to be isolated to avoid the risk that one business takes too many resources and affects others. -For case 3, although delay is acceptable, but all TiDB instances must be always consistent on the placement rules. To achieve this goal, schema version needs to be updated, thus transactions with old placement rules will fail when committed. +Since key range is not successive in one database, each table in the database corresponds to at least one placement rule, so there may be many placement rules. In either case above, there may be up to millions of tables in one database, which costs lots of time to update the rules and lots of space to store the rules. -For case 4 and 5, delay is not acceptable. Once the placement rules are written successfully, subsequent DDL statements should fetch the latest placement rules to gaurantee linearizability. Now that schema version is changed and the latest placement rules are broadcast to all the TiDB instances immediately, delay is eliminable. +Another option is to take advantage of the region label, which is described earlier. -Once the schema version is changed, all TiDB instances recognize the object ID and fetch placement rules from PD, rather than TiKV. +In the example below, it defines multiple label rules for one database. Each label rule corresponds to one table or partition. -To query the placement rules on a specified object, the object ID should be written to the placement rules, or it can be inferred from other fields. Now that `id` contains the object ID, TiDB can decode the object ID from it. See section "Building placement rules" for details. +``` +{ + "id": "db1/tb1", + "labels": [ + { + "key": "database-name", + "value": "db1" + }, + { + "key": "table-name", + "value": "db1/tb1" + } + ], + "match-type": "key-range", + "match": { + "start-key": "7480000000000000ff0a00000000000000f8", + "end-key": "7480000000000000ff0b00000000000000f8" + } +}, +{ + "id": "db1/tb2", + "labels": [ + { + "key": "database-name", + "value": "db1" + }, + { + "key": "table-name", + "value": "db1/tb2" + } + ], + "match-type": "key-range", + "match": { + "start-key": "7480000000000000ff0c00000000000000f8", + "end-key": "7480000000000000ff0d00000000000000f8" + } +} +``` -### DDL procedures +Then you need only one placement rule for the database. When you change the placement of the database, you need to update one placement rule. However, when you drop a database, you need to delete multiple label rules plus one placement rule. -Defining placement rules is a type of DDL, so it's natural to implement it in a typical DDL procedure. But it differs from other DDL in that it writes to PD instead of TiKV. +### Rule priorities -The fact that the DDL procedure in TiDB is mature helps to achieve some features of defining placement rules: +Tables only inherit rules from databases when they are created, and the value is saved in the meta data. Thus, the rules of priorities are simplified from an earlier version of this proposal (and are more inline with how character sets are inherited). -- Placement rules are defined in serial as there's only one DDL owner at the same time -- DDL is capable of disaster recovery as the middle states are persistent in TiKV -- DDL is rollbackable as the middle states can transform from one to another -- Updating schema version guarantees all active transactions are based on the same version of placement rules +The only rules are that indexes and partitions inherit the rules of tables. Partitions can explicitly overwrite the placement policy, but indexes currently do not allow placement policy to be defined (this simplification was made intentionally since there is not a clear use case until global secondary indexes are introduced). -### Rule priorities +Thus the priority is: -When several rules are defined for one record, the most granular rule is chosen for this record. More specifically, the rule priority is: index > partition > table > database > default. +``` +db --> table (Copied from db on create if placement not explicitly specified for the table) +unpartitioned table --> index +partitioned table --> partition (can be overwritten) --> index +``` For example: @@ -739,121 +840,198 @@ Specifically, `index` is in such a format: In such a way, the most granular rule always works. -### Rule inheritance -In some cases, creating a new object doesn't need to store its placement rules: +## Examples -- Creating a database -- Creating an index on an unpartitioned table -- Creating an index on a partition +### Optimization: Follower read in every region -In the last two cases, the key range of the index is included in the key range of the table or partition it belongs to. PD will guarantee the rule priorities described above. +This optimization is straight forward: +```sql +CREATE PLACEMENT POLICY local_stale_reads FOLLOWER_CONSTRAINTS="{+us-east-1:1,+us-east-2:1,+us-west-1:1,+us-west-2:1}"; +CREATE TABLE t (a int, b int) PLACEMENT POLICY=`local_stale_reads`; +``` -But in other cases, creating a new object needs to store its placement rules: +### Optimization: Latest data on SSD -- Creating a table in a database -- Creating a partition in a table +This optimization uses labels to define the storage type: -The placement rules of databases and partitioned tables don't actually work on PD, because the key ranges don't include any records. They are stored on PD and only serve for querying when new objects are created in them. +```sql +CREATE PLACEMENT POLICY storeonfastssd CONSTRAINTS="[+disk=ssd]"; +CREATE PLACEMENT POLICY storeonhdd CONSTRAINTS="[+disk=hdd]"; + +CREATE TABLE t1 (id INT, name VARCHAR(50), purchased DATE) + PARTITION BY RANGE( YEAR(purchased) ) ( + PARTITION p0 VALUES LESS THAN (2000) PLACEMENT POLICY='storeonhdd', + PARTITION p1 VALUES LESS THAN (2005), + PARTITION p2 VALUES LESS THAN (2010), + PARTITION p3 VALUES LESS THAN (2015), + PARTITION p4 VALUES LESS THAN MAXVALUE PLACEMENT POLICY='storeonfastssd' + ) +PLACEMENT POLICY='companystandardpolicy'; +``` -For example, when defining a placement rule on database `db`, the key range of this rule is empty. When a new table `t` is created in `db`, TiDB queries the placement rules of `db` and copies them to table `t`, but the new key range corresponds to table `t`. +### Optimization: Multi-tenancy / control of shared resources -Once the placement rules on a database or a partitioned table are changed, the inherited placement rules are also updated, but others are kept. +This example is similar to latest data on SSD. The customer has a large TiDB Cluster with several workloads that are running on it. They might want to reduce the blast radius of individual users impacting each-other, and potentially improve QoS. -Consider such a scenario: +Assuming a `schema` per tenant, it is easy to create a set of "resource pools". Each pool is a label, which contains a set of tikv-servers (with sufficient capacity, and nodes to provide high availability still): ```sql -ALTER DATABASE db - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=sh]" ROLE=voter REPLICAS=3; - -CREATE TABLE db.t1(id int); - -CREATE TABLE db.t2(id int); - -ALTER TABLE db.t2 - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=follower REPLICAS=1; - -ALTER DATABASE db - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3, - ADD PLACEMENT POLICY CONSTRAINTS="[-zone=sh]" ROLE=follower; +CREATE PLACEMENT POLICY poola CONSTRAINTS="[+pool=poola]"; +CREATE PLACEMENT POLICY poolb CONSTRAINTS="[+pool=poolb]"; +CREATE PLACEMENT POLICY poolc CONSTRAINTS="[+pool=poolc]"; + +ALTER DATABASE workload1 PLACEMENT POLICY=`poola`; +/* for each existing table (new ones will not require this) */ +ALTER TABLE workload1.t1 PLACEMENT POLICY=`poola`; + +CREATE DATABASE workload2 PLACEMENT POLICY=`poolb`; +CREATE DATABASE workload3 PLACEMENT POLICY=`poolb`; +CREATE DATABASE workload4 PLACEMENT POLICY=`poolb`; +CREATE DATABASE workload5 PLACEMENT POLICY=`poolb`; +CREATE DATABASE workload6 PLACEMENT POLICY=`poolc`; ``` -The final placement rules of `t1` and `t2` will be: +### Compliance: User data needs geographic split -```sql -ALTER TABLE db.t1 - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3, - ADD PLACEMENT POLICY CONSTRAINTS="[-zone=sh]" ROLE=follower; +This example has limitations based on the current implementation. Consider the following `users` table: -ALTER TABLE db.t2 - ALTER PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3, - ADD PLACEMENT POLICY CONSTRAINTS="[+zone=bj]" ROLE=follower REPLICAS=1; +```sql +CREATE TABLE users ( + id INT NOT NULL auto_increment, + username VARCHAR(64) NOT NULL, + email VARCHAR(64) NOT NULL, + dateofbirth DATE NOT NULL, + country VARCHAR(10) NOT NULL, + PRIMARY KEY (id), + UNIQUE (username) +); ``` -Because all the placement rules on `t1` are inherited from `db`, they will keep the same with `db` all the time. The placement rule `CONSTRAINTS="[+zone=bj]" ROLE=follower REPLICAS=1` is private for `t2`, so it will be kept after the changement of `db`. But the other rule `CONSTRAINTS="[+zone=bj]" ROLE=voter REPLICAS=3` is still inherited from `db`. +We may want to ensure that users that have users in the EU store their data in a specific location. On the surface this looks straight forward: -To achieve this goal, the placement rules should be marked with the source where they come from. - -### Building placement rules - -There needs a way to map the placement rules in SQL to PD placement rule configuration. Most of the fields are discussed above, so this part focuses on `group_id`, `id`, `start_key` and `end_key`. +```sql +CREATE TABLE users ( + id INT NOT NULL auto_increment, + username VARCHAR(64) NOT NULL, + email VARCHAR(64) NOT NULL, + dateofbirth DATE NOT NULL, + country VARCHAR(10) NOT NULL, + PRIMARY KEY (id), + UNIQUE (username) +) PARTITION BY LIST COLUMNS (country) ( + PARTITION pEurope VALUES IN ('DE', 'FR', 'GB') PLACEMENT POLICY='europe', + PARTITION pOther VALUES IN ('US', 'CA', 'MX') +); +``` -`group_id` is used to identify the source of the placement rules, so `group_id` is `tidb`. +However, the definition is not valid. The unique index on `username` can not be enforced, because there are no global indexes for partitioned tables. -`ALTER PLACEMENT POLICY` and `DROP PLACEMENT POLICY` need to find the rules of a specified object efficiently. It can be achieved by encoding the object ID in `id`. +In the future we will need to be able to define the index for `username` as a global index, and allow it to have different placement rules where it can be read from all regions. -However, an object may have multiple rules for a single role. For example: +This example also demonstrates that this specification only provides partition level placement (and not row-level or column level security). The workaround for the user will require splitting the table: ```sql -ALTER TABLE t - ALTER PLACEMENT POLICY CONSTRAINTS="{+zone=bj:2,+zone=sh:1}" ROLE=voter; +CREATE TABLE users ( + id INT NOT NULL auto_increment PRIMARY KEY, + /* public details */ +); + +CREATE TABLE user_details ( + user_id INT NOT NULL, + /* private columns */ + /* partition this table */ +); ``` -It needs 2 placement rules for `voter` in the PD placement rule configuration, because each rule can only specify one `count`. To make `id` unique, a unique identifier must be appended to `id`. DDL job ID plus an index in the job is a good choice. +Assuming that global indexes can be added to the TiDB server, this use-case can be improved. But for correct execution the server will also require the following functionality: +- The ability to mark global indexes as invisible at run-time if the `PLACEMENT POLICY` does not permit them to be read. +- The ability to mark the clustered index as invisible. i.e. in the case that there is a global unique index on `username` it may be permitted to be read, but a read of the `username` from the clustered index might need to be disabled. -Take the case above for example, assuming the table ID of `t` is 100, the ID of the DDL job executing this statement is 200, then `id` of the placement rules are `100-200-1` and `100-200-2`. +## Impacts & Risks -The prefix of `id` is in such a format: +1. The largest risk is designing a set of SQL statements that are sufficiently flexible to handle major use cases, but not too flexible that misconfiguration is likely when the user has compliance requirements. The following design decisions have been made to mitigate this risk: + - The DDL statement to `ALTER TABLE t1 ADD PLACEMENT` has been removed from the proposal. + - The DDL statement `CREATE PLACEMENT POLICY` has been added (allowing common configurations to be saved). + - Configuring placement rules for indexes is no longer supported (we can add it once global indexes are added). + - The inheritance rules have been simplified to match character set/collation inheritance. +2. There is a risk that we do not fully understand compliance requirements (see unresolved questions), or that the substantial effort required to achieve compliance with DDL, internal SQL and DML. +3. Related to (2), there is risk that the implementation of compliance requirements has a significant burden on multiple teams, including ecosystem tools (Backup, CDC, DM). Even tools such as dumpling should ideally be able to backup placement rules in logical form. +4. The compliance use-case may depend on global secondary indexes for many scenarios (see "Compliance: User data needs geographic split"), but they are not currently supported. +4. There is some risk that a common use case can not be expressed in `CONSTRAINT` syntax, leading to complicated scenarios where users still need to express placement by using PD directly. Ideally, we can recommend users use SQL rules exclusively. +5. Many other features in TiDB are in development, some of which may influence placement rules. Clustered index affects the key format of primary index, but fortunately the prefix of key range is untouched. Global secondary index largely affects the placement rules of partitioned tables. -* Database: database id -* Table: table id -* Partition: partition id -* Unpartitioned index: the concatenation of table id and index id, e.g. `100_1` -* Partitioned index: the concatenation of partition id and index id +## Investigation & Alternatives -To query all the placement rules for one object, PD looks for all the `id` with a specific prefix. +For investigation, we looked at the implementation of placement rules in various databases (CockroachDB, Yugabyte, OceanBase). -As all placement rules are mapped to PD placement rule configurations, `start_key` and `end_key` must be generated for each object. However, databases and partitioned tables have no key ranges, so the only way is to generate a key range with no actual records. +The idea of using a `PLACEMENT POLICY` was inspired by how OceanBase has Placement Groups, which are then applied to tables. But the usage as proposed here is optional, which allows for more flexibility for casual cases. The idea of using a Placement Group can also be seen as similar to using a "tablespace" in a traditional database, but it's not completely the same since the choice is less binary (constraints allow the placement of roles for leaders, followers, voters, learners). -As database IDs are all globally unique, it's fine to replace table ID with database ID in the key range. For example, assuming the database ID is 100, then the string format of its key range is: +CockroachDB does not look to have something directly comparable to `PLACEMENT POLICY`, but it does have the ability to specify "replication zones" for "system ranges" such as default, meta, liveness, system, timeseries. Before dropping `ALTER TABLE t1 ADD PLACEMENT` from this proposal, it was investigated the CockroachDB does not support this syntax, presumably for simplification and minimising similar risks of misconfiguration. -- `start_key`: `t_{database_id}_` -- `end_key`: `t_{database_id+1}_` +### Known Limitations -It's same for partitioned tables. +- In this proposal, placement policies are global-level and not specific to a database. This simplifies the configuration, but makes it restrictive for multi-tenant scenarios where a schema-per-tenant is provided. This is because creating or modifying placement policies requires a privilege which is cluster scoped (`PLACEMENT_ADMIN`). The limitation is that "tenants" will be able to `CREATE TABLE (..) PLACEMENT POLICY=x`, but they will not be able to `CREATE PLACEMENT POLICY x` or `ALTER PLACEMENT POLICY x` +- Complex scenarios may exist where there is not a column in the current table which can be used to partition the table into different regions, but instead there is a column which is a foreign key to another table from which this information can be determined. In this scenario, the user will be required to "denormalize" the usage and move the parent_id into the child table so geo-partitioning is possible. +- Because direct assignment and `PLACEMENT POLICY` are mutually exclusive, it results in some scenarios where users that just want to make one small change on a placement policy need to create a new policy. This is intentional to limit the risk of misconfiguration. +- The name `REGION` is ambigous, since we are using it for placement as well as to refer to a chunk of data. We could avoid region here, but the problem is it is the most used term for a geographic location of a data center. We recommend instead calling both region but in documentation refering to them as "data regions" and "placement regions". -### Future plans +## Unresolved Questions -Many other features in TiDB are in development, some of which may influence placement rules. +### Compliance Requirements -Clustered index affects the key format of primary index. Fortunately, the prefix of key range is untouched. +For compliance use-cases, it is clear that data at rest should reside within a geographic region. What is not clear is which (if any) circumstances data in transit is permitted to leave. There are several known issues which will need to be resolved: -Global secondary index largely affect the placement rules of partitioned tables. The key range of one global secondary index is not successive, so if it's necessary to define placement rules on the index, multiple rules should be generated in the PD. But for now, there's no such scenario. +* **Backup**: The `BACKUP` SQL command (and br) accept a single location such as `s3://bucket-location/my-backup` to centralize the backup. This centralization likely violates compliance requirements (assuming there are >=2 requirements that conflict on a single cluster). Backing up segments of data individually is both an inconsistent backup, and likely to result in misconfiguration which violates compliance rules. Users have a reasonable expectation of backup integration with compliance placement rules. +* **CDC**: Similar to `BACKUP`, it should not be possible to subscribe to changes for data in a different jurisdiction. +* **DDL**: The current implementation of DDL uses a centralized DDL owner which reads data from relevant tables and performs operations such as building indexes. The _read_ operation may violate compliance rules. +* **Internal SQL**: Similar to DDL, several centralized background tasks, such as updating histograms/statistics need to be able to _read_ the data. +* **DML**: It is not yet known which restrictions need to be placed on user queries. For example, if a poorly written user-query does not clearly target the `USA` partition when reading user data, should it generate an error because the `EUROPE` partition needs to be read in order for the semantics of the query to be correct? This may cause problems in development and integration environments if the restrictions can not be distilled into environments with smaller topologies. +* **Routing**: When there is a data center for Europe, and a Data center for the USA, and the data is partitioned with requirements that DML from the USA can not read data in Europe, how is that enforced? Is it configured in such a way that the tidb-servers in the USA can not route to the tikv-servers in Europe? If this is the case, then it means the European servers can not hold non-user compliance data that the USA might need to read. If it is not the case, then there might be some sort of key management/crypto scheme to control access to sensitive data. -## Privilege management +### Behaviors -Privilege management is quite straightforward: +#### Syntax for Restricted Access (Compliance Case) -* `ALTER` statement requires `Alter` privilege -* `information_schema.placement_rules` and `SHOW PLACEMENT POLICY` only shows the placement rules on the objects that visible to the current user -* `ADMIN SHOW DDL` requires `Super` privilege +Assume that if the example is logically like the "Compliance: User data needs geographic split" case: -## Ecosystem tools +```sql +CREATE TABLE users ( + id INT NOT NULL auto_increment, + username VARCHAR(64) NOT NULL, + email VARCHAR(64) NOT NULL, + dateofbirth DATE NOT NULL, + country VARCHAR(10) NOT NULL, + PRIMARY KEY (id), + UNIQUE (username) +) PARTITION BY LIST COLUMNS (country) ( + PARTITION pEurope VALUES IN ('DE', 'FR', 'GB') PLACEMENT POLICY='europe', + PARTITION pOther VALUES IN ('US', 'CA', 'MX') +); +``` -Many tools are based on binlog or metadata. For example, TiDB-binlog is based on binlog, while Lightning and Dumpling are based on metadata. Placement rules need to be compatible with these tools. +What does `SHOW CREATE PLACEMENT POLICY europe` look like? -If the downstream is not TiDB, no change needs to be made. But even if it is TiDB, TiKV nodes may have a different geographical topology, which means the labels of TiKV nodes may be different. In this case, placement rules can not be enforced on them. +I assume that it is something like: -Based on this consideration, placement rules need not to be exported to binlog or metadata. This is applicable for all tools, including TiCDC and BR. +```sql +CREATE PLACEMENT POLICY europe CONSTRAINTS="+region=eu-west-1" RESTRICTED; +``` -However, there may be also cases where users want exactly the same placement rules as the upstream, and altering placement rules manually is very annoying. It will be considered in the future if there’s a need. +This specific semantic will be the hardest to implement because of the other dependencies in the server. + +## Changelog + +* 2021-07-26: + - Converted proposal to use the new template for technical designs. + - Removed the syntax `ALTER TABLE t1 ADD PLACEMENT POLICY` due to ambiguity in some cases, and risk of misconfiguration for compliance cases. + - Added `CREATE PLACEMENT POLICY` syntax. + - Renamed `ALTER TABLE t1 ALTER PLACEMENT POLICY` to `ALTER TABLE t1 PLACEMENT` to bring syntax inline with other atomic changes, such as `ALTER TABLE t1 CHARACTER SET x`. The usage of `PLACEMENT POLICY` now refers to a placement policy defined from `CREATE PLACEMENT POLICY` (other commands like `SHOW PLACEMENT POLICY` are also updated to `SHOW PLACEMENT`). + - Remove index as a placement option (we can add it again once global indexes for temporary tables exist, but it is not strictly required for an MVP). + - Made implementation in `CREATE TABLE` and `SHOW CREATE TABLE` required, to support the compliance use-case. + - Changed `ALTER TABLE ALTER PARTITION p0` to `ALTER TABLE PARTITION p0` + - Added short-hand syntactic sugar for constraints to handle default cases. + - Changed it so that you can no longer specify multiple constraints. + - Use defaults for `count` of each role, and `ROLE_CONSTRAINTS` syntax. + - Added `SCHEDULE` property + - Removed further ambiguous cases such as count when using dictionary syntax. diff --git a/domain/infosync/info.go b/domain/infosync/info.go index e885bbeb175e4..dbf62e276d210 100644 --- a/domain/infosync/info.go +++ b/domain/infosync/info.go @@ -87,16 +87,17 @@ var ErrPrometheusAddrIsNotSet = dbterror.ClassDomain.NewStd(errno.ErrPrometheusA // InfoSyncer stores server info to etcd when the tidb-server starts and delete when tidb-server shuts down. type InfoSyncer struct { - etcdCli *clientv3.Client - info *ServerInfo - serverInfoPath string - minStartTS uint64 - minStartTSPath string - manager util2.SessionManager - session *concurrency.Session - topologySession *concurrency.Session - prometheusAddr string - modifyTime time.Time + etcdCli *clientv3.Client + info *ServerInfo + serverInfoPath string + minStartTS uint64 + minStartTSPath string + manager util2.SessionManager + session *concurrency.Session + topologySession *concurrency.Session + prometheusAddr string + modifyTime time.Time + labelRuleManager LabelRuleManager } // ServerInfo is server static information. @@ -175,6 +176,11 @@ func GlobalInfoSyncerInit(ctx context.Context, id string, serverIDGetter func() if err != nil { return nil, err } + if etcdCli != nil { + is.labelRuleManager = initLabelRuleManager(etcdCli.Endpoints()) + } else { + is.labelRuleManager = initLabelRuleManager([]string{}) + } setGlobalInfoSyncer(is) return is, nil } @@ -201,6 +207,13 @@ func (is *InfoSyncer) GetSessionManager() util2.SessionManager { return is.manager } +func initLabelRuleManager(addrs []string) LabelRuleManager { + if len(addrs) == 0 { + return &mockLabelManager{labelRules: map[string]*label.Rule{}} + } + return &PDLabelManager{addrs: addrs} +} + // GetServerInfo gets self server static information. func GetServerInfo() (*ServerInfo, error) { is, err := getGlobalInfoSyncer() @@ -817,24 +830,10 @@ func PutLabelRule(ctx context.Context, rule *label.Rule) error { if err != nil { return err } - - if is.etcdCli == nil { + if is.labelRuleManager == nil { return nil } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return errors.Errorf("pd unavailable") - } - - r, err := json.Marshal(rule) - if err != nil { - return err - } - - _, err = doRequest(ctx, addrs, path.Join(pdapi.Config, "region-label", "rule"), "POST", bytes.NewReader(r)) - return err + return is.labelRuleManager.PutLabelRule(ctx, rule) } // UpdateLabelRules synchronizes the label rule to PD. @@ -847,24 +846,10 @@ func UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { if err != nil { return err } - - if is.etcdCli == nil { + if is.labelRuleManager == nil { return nil } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return errors.Errorf("pd unavailable") - } - - r, err := json.Marshal(patch) - if err != nil { - return err - } - - _, err = doRequest(ctx, addrs, path.Join(pdapi.Config, "region-label", "rules"), "PATCH", bytes.NewReader(r)) - return err + return is.labelRuleManager.UpdateLabelRules(ctx, patch) } // GetAllLabelRules gets all label rules from PD. @@ -873,24 +858,10 @@ func GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) { if err != nil { return nil, err } - - if is.etcdCli == nil { - return nil, err - } - - addrs := is.etcdCli.Endpoints() - - if len(addrs) == 0 { - return nil, errors.Errorf("pd unavailable") - } - - rules := []*label.Rule{} - res, err := doRequest(ctx, addrs, path.Join(pdapi.Config, "region-label", "rules"), "GET", nil) - - if err == nil && res != nil { - err = json.Unmarshal(res, &rules) + if is.labelRuleManager == nil { + return nil, nil } - return rules, err + return is.labelRuleManager.GetAllLabelRules(ctx) } // GetLabelRules gets the label rules according to the given IDs from PD. @@ -903,27 +874,8 @@ func GetLabelRules(ctx context.Context, ruleIDs []string) ([]*label.Rule, error) if err != nil { return nil, err } - - if is.etcdCli == nil { + if is.labelRuleManager == 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 + return is.labelRuleManager.GetLabelRules(ctx, ruleIDs) } diff --git a/domain/infosync/label_manager.go b/domain/infosync/label_manager.go new file mode 100644 index 0000000000000..bf6ba634fdf24 --- /dev/null +++ b/domain/infosync/label_manager.go @@ -0,0 +1,155 @@ +// 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 infosync + +import ( + "bytes" + "context" + "encoding/json" + "path" + "sync" + + "github.com/pingcap/tidb/ddl/label" + "github.com/pingcap/tidb/util/pdapi" +) + +// LabelRuleManager manages label rules +type LabelRuleManager interface { + PutLabelRule(ctx context.Context, rule *label.Rule) error + UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error + GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) + GetLabelRules(ctx context.Context, ruleIDs []string) ([]*label.Rule, error) +} + +// PDLabelManager manages rules with pd +type PDLabelManager struct { + addrs []string +} + +// PutLabelRule implements PutLabelRule +func (lm *PDLabelManager) PutLabelRule(ctx context.Context, rule *label.Rule) error { + r, err := json.Marshal(rule) + if err != nil { + return err + } + _, err = doRequest(ctx, lm.addrs, path.Join(pdapi.Config, "region-label", "rule"), "POST", bytes.NewReader(r)) + return err +} + +// UpdateLabelRules implements UpdateLabelRules +func (lm *PDLabelManager) UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { + r, err := json.Marshal(patch) + if err != nil { + return err + } + + _, err = doRequest(ctx, lm.addrs, path.Join(pdapi.Config, "region-label", "rules"), "PATCH", bytes.NewReader(r)) + return err +} + +// GetAllLabelRules implements GetAllLabelRules +func (lm *PDLabelManager) GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) { + var rules []*label.Rule + res, err := doRequest(ctx, lm.addrs, path.Join(pdapi.Config, "region-label", "rules"), "GET", nil) + + if err == nil && res != nil { + err = json.Unmarshal(res, &rules) + } + return rules, err +} + +// GetLabelRules implements GetLabelRules +func (lm *PDLabelManager) GetLabelRules(ctx context.Context, ruleIDs []string) ([]*label.Rule, error) { + ids, err := json.Marshal(ruleIDs) + if err != nil { + return nil, err + } + + rules := []*label.Rule{} + res, err := doRequest(ctx, lm.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 +} + +type mockLabelManager struct { + sync.RWMutex + labelRules map[string]*label.Rule +} + +// PutLabelRule implements PutLabelRule +func (mm *mockLabelManager) PutLabelRule(ctx context.Context, rule *label.Rule) error { + mm.Lock() + defer mm.Unlock() + if rule == nil { + return nil + } + mm.labelRules[rule.ID] = rule + return nil +} + +// UpdateLabelRules implements UpdateLabelRules +func (mm *mockLabelManager) UpdateLabelRules(ctx context.Context, patch *label.RulePatch) error { + mm.Lock() + defer mm.Unlock() + if patch == nil { + return nil + } + for _, p := range patch.DeleteRules { + delete(mm.labelRules, p) + } + for _, p := range patch.SetRules { + if p == nil { + continue + } + mm.labelRules[p.ID] = p + } + return nil +} + +// mockLabelManager implements GetAllLabelRules +func (mm *mockLabelManager) GetAllLabelRules(ctx context.Context) ([]*label.Rule, error) { + mm.RLock() + defer mm.RUnlock() + r := make([]*label.Rule, 0, len(mm.labelRules)) + for _, labelRule := range mm.labelRules { + if labelRule == nil { + continue + } + r = append(r, labelRule) + } + return r, nil +} + +// mockLabelManager implements GetLabelRules +func (mm *mockLabelManager) GetLabelRules(ctx context.Context, ruleIDs []string) ([]*label.Rule, error) { + mm.RLock() + defer mm.RUnlock() + r := make([]*label.Rule, 0, len(ruleIDs)) + for _, ruleID := range ruleIDs { + for _, labelRule := range mm.labelRules { + if labelRule.ID == ruleID { + if labelRule == nil { + continue + } + r = append(r, labelRule) + break + } + } + } + return r, nil +} diff --git a/errno/errname.go b/errno/errname.go index 025caa8f0e4dd..5ee1e470357bf 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1055,7 +1055,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ 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), + ErrPlacementPolicyExists: mysql.Message("Placement policy '%-.192s' already exists", nil), ErrPlacementPolicyNotExists: mysql.Message("Unknown placement policy '%-.192s'", nil), // TiKV/PD errors. diff --git a/errors.toml b/errors.toml index a6143d74bcd2a..c9dfc9c1eda91 100644 --- a/errors.toml +++ b/errors.toml @@ -1063,7 +1063,7 @@ DDL reorg element does not exist ["meta:8238"] error = ''' -Can't create placement policy '%-.192s'; policy exists +Placement policy '%-.192s' already exists ''' ["meta:8239"] @@ -1616,6 +1616,16 @@ error = ''' Table '%s' was locked in %s by %v ''' +["schema:8238"] +error = ''' +Placement policy '%-.192s' already exists +''' + +["schema:8239"] +error = ''' +Unknown placement policy '%-.192s' +''' + ["session:8002"] error = ''' [%d] can not retry select for update statement diff --git a/executor/brie_test.go b/executor/brie_test.go index 87ae456d35b4d..05657b82117c8 100644 --- a/executor/brie_test.go +++ b/executor/brie_test.go @@ -96,7 +96,7 @@ func (s *testBRIESuite) TestFetchShowBRIE(c *C) { p.SetParserConfig(parser.ParserConfig{EnableWindowFunction: true, EnableStrictDoubleTypeCheck: true}) stmt, err := p.ParseOneStmt("show backups", "", "") c.Assert(err, IsNil) - plan, _, err := core.BuildLogicalPlan(ctx, sctx, stmt, infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable(), core.MockView()})) + plan, _, err := core.BuildLogicalPlanForTest(ctx, sctx, stmt, infoschema.MockInfoSchema([]*model.TableInfo{core.MockSignedTable(), core.MockUnsignedTable(), core.MockView()})) c.Assert(err, IsNil) schema := plan.Schema() diff --git a/executor/ddl.go b/executor/ddl.go index e8b28b8e3e876..e242fc0600238 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -233,6 +233,10 @@ func (e *DDLExec) Next(ctx context.Context, req *chunk.Chunk) (err error) { err = e.executeDropSequence(x) case *ast.AlterSequenceStmt: err = e.executeAlterSequence(x) + case *ast.CreatePlacementPolicyStmt: + err = e.executeCreatePlacementPolicy(x) + case *ast.DropPlacementPolicyStmt: + err = e.executeDropPlacementPolicy(x) } if err != nil { // If the owner return ErrTableNotExists error when running this DDL, it may be caused by schema changed, @@ -903,3 +907,10 @@ func (e *DDLExec) executeCreateSequence(s *ast.CreateSequenceStmt) error { func (e *DDLExec) executeAlterSequence(s *ast.AlterSequenceStmt) error { return domain.GetDomain(e.ctx).DDL().AlterSequence(e.ctx, s) } + +func (e *DDLExec) executeCreatePlacementPolicy(s *ast.CreatePlacementPolicyStmt) error { + return domain.GetDomain(e.ctx).DDL().CreatePlacementPolicy(e.ctx, s) +} +func (e *DDLExec) executeDropPlacementPolicy(s *ast.DropPlacementPolicyStmt) error { + return domain.GetDomain(e.ctx).DDL().DropPlacementPolicy(e.ctx, s) +} diff --git a/executor/executor.go b/executor/executor.go index f7e828cc3600a..3a1fa3beb9c48 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -345,7 +345,7 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { allocators := tbl.Allocators(e.ctx) for _, alloc := range allocators { - nextGlobalID, err := alloc.NextGlobalAutoID(tblMeta.ID) + nextGlobalID, err := alloc.NextGlobalAutoID() if err != nil { return err } diff --git a/executor/executor_test.go b/executor/executor_test.go index fa917a1845cfb..a0613ee111600 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -364,6 +364,7 @@ func (s *testSuiteP1) TestShow(c *C) { "SYSTEM_VARIABLES_ADMIN Server Admin ", "ROLE_ADMIN Server Admin ", "CONNECTION_ADMIN Server Admin ", + "DASHBOARD_CLIENT Server Admin ", "RESTRICTED_TABLES_ADMIN Server Admin ", "RESTRICTED_STATUS_ADMIN Server Admin ", "RESTRICTED_VARIABLES_ADMIN Server Admin ", @@ -3837,7 +3838,7 @@ func (s *testSuite) TestCheckIndex(c *C) { c.Assert(err, IsNil) tbInfo := tbl.Meta() - alloc := autoid.NewAllocator(s.store, dbInfo.ID, false, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(s.store, dbInfo.ID, tbInfo.ID, false, autoid.RowIDAllocType) tb, err := tables.TableFromMeta(autoid.NewAllocators(alloc), tbInfo) c.Assert(err, IsNil) diff --git a/executor/insert_common.go b/executor/insert_common.go index 492dfdc836273..0ac00a9973067 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -915,7 +915,7 @@ func (e *InsertValues) allocAutoRandomID(ctx context.Context, fieldType *types.F tableInfo := e.Table.Meta() increment := e.ctx.GetSessionVars().AutoIncrementIncrement offset := e.ctx.GetSessionVars().AutoIncrementOffset - _, autoRandomID, err := alloc.Alloc(ctx, tableInfo.ID, 1, int64(increment), int64(offset)) + _, autoRandomID, err := alloc.Alloc(ctx, 1, int64(increment), int64(offset)) if err != nil { return 0, err } @@ -946,7 +946,7 @@ func (e *InsertValues) rebaseAutoRandomID(recordID int64, fieldType *types.Field layout := autoid.NewShardIDLayout(fieldType, tableInfo.AutoRandomBits) autoRandomID := layout.IncrementalMask() & recordID - return alloc.Rebase(tableInfo.ID, autoRandomID, true) + return alloc.Rebase(autoRandomID, true) } func (e *InsertValues) adjustImplicitRowID(ctx context.Context, d types.Datum, hasValue bool, c *table.Column) (types.Datum, error) { @@ -1000,7 +1000,7 @@ func (e *InsertValues) rebaseImplicitRowID(recordID int64) error { layout := autoid.NewShardIDLayout(types.NewFieldType(mysql.TypeLonglong), tableInfo.ShardRowIDBits) newTiDBRowIDBase := layout.IncrementalMask() & recordID - return alloc.Rebase(tableInfo.ID, newTiDBRowIDBase, true) + return alloc.Rebase(newTiDBRowIDBase, true) } func (e *InsertValues) handleWarning(err error) { diff --git a/executor/show.go b/executor/show.go index 427c538ee90f7..5d7f721a4dd4b 100644 --- a/executor/show.go +++ b/executor/show.go @@ -1020,7 +1020,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T incrementAllocator := allocators.Get(autoid.RowIDAllocType) if hasAutoIncID && incrementAllocator != nil { - autoIncID, err := incrementAllocator.NextGlobalAutoID(tableInfo.ID) + autoIncID, err := incrementAllocator.NextGlobalAutoID() if err != nil { return errors.Trace(err) } @@ -1037,7 +1037,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T randomAllocator := allocators.Get(autoid.AutoRandomType) if randomAllocator != nil { - autoRandID, err := randomAllocator.NextGlobalAutoID(tableInfo.ID) + autoRandID, err := randomAllocator.NextGlobalAutoID() if err != nil { return errors.Trace(err) } @@ -1423,7 +1423,9 @@ func (e *ShowExec) fetchShowGrants() error { // The input is a "SHOW GRANTS" statement with no users *or* SHOW GRANTS FOR CURRENT_USER() // In these cases we include the active roles for showing privileges. e.User = &auth.UserIdentity{Username: vars.User.AuthUsername, Hostname: vars.User.AuthHostname} - e.Roles = vars.ActiveRoles + if len(e.Roles) == 0 { + e.Roles = vars.ActiveRoles + } } else { userName := vars.User.AuthUsername hostName := vars.User.AuthHostname @@ -1434,14 +1436,14 @@ func (e *ShowExec) fetchShowGrants() error { return ErrDBaccessDenied.GenWithStackByArgs(userName, hostName, mysql.SystemDB) } } - // This is for the syntax SHOW GRANTS FOR x USING role - for _, r := range e.Roles { - if r.Hostname == "" { - r.Hostname = "%" - } - if !checker.FindEdge(e.ctx, r, e.User) { - return ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String()) - } + } + // This is for the syntax SHOW GRANTS FOR x USING role + for _, r := range e.Roles { + if r.Hostname == "" { + r.Hostname = "%" + } + if !checker.FindEdge(e.ctx, r, e.User) { + return ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String()) } } gs, err := checker.ShowGrants(e.ctx, e.User, e.Roles) diff --git a/executor/write.go b/executor/write.go index d64969286769c..00dab9ff5270d 100644 --- a/executor/write.go +++ b/executor/write.go @@ -237,7 +237,7 @@ func rebaseAutoRandomValue(sctx sessionctx.Context, t table.Table, newData *type layout := autoid.NewShardIDLayout(&col.FieldType, tableInfo.AutoRandomBits) // Set bits except incremental_bits to zero. recordID = recordID & (1< base will cause alloc rebase to offset-1. @@ -271,120 +269,118 @@ func TestUnsignedAutoid(t *testing.T) { }) require.NoError(t, err) - alloc := autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(store, 1, 1, true, autoid.RowIDAllocType) require.NotNil(t, alloc) ctx := context.Background() - globalAutoID, err := alloc.NextGlobalAutoID(1) + globalAutoID, err := alloc.NextGlobalAutoID() require.NoError(t, err) require.Equal(t, int64(1), globalAutoID) - _, id, err := alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err := alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(1), id) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(2), id) - _, _, err = alloc.Alloc(ctx, 0, 1, 1, 1) - require.Error(t, err) - globalAutoID, err = alloc.NextGlobalAutoID(1) + globalAutoID, err = alloc.NextGlobalAutoID() require.NoError(t, err) require.Equal(t, autoid.GetStep()+1, globalAutoID) // rebase - err = alloc.Rebase(1, int64(1), true) + err = alloc.Rebase(int64(1), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(3), id) - err = alloc.Rebase(1, int64(3), true) + err = alloc.Rebase(int64(3), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(4), id) - err = alloc.Rebase(1, int64(10), true) + err = alloc.Rebase(int64(10), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(11), id) - err = alloc.Rebase(1, int64(3010), true) + err = alloc.Rebase(int64(3010), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(3011), id) - alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc = autoid.NewAllocator(store, 1, 1, true, autoid.RowIDAllocType) require.NotNil(t, alloc) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, autoid.GetStep()+1, id) - alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc = autoid.NewAllocator(store, 1, 2, true, autoid.RowIDAllocType) require.NotNil(t, alloc) - err = alloc.Rebase(2, int64(1), false) + err = alloc.Rebase(int64(1), false) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 2, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(2), id) - alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc = autoid.NewAllocator(store, 1, 3, true, autoid.RowIDAllocType) require.NotNil(t, alloc) - err = alloc.Rebase(3, int64(3210), false) + err = alloc.Rebase(int64(3210), false) require.NoError(t, err) - alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc = autoid.NewAllocator(store, 1, 3, true, autoid.RowIDAllocType) require.NotNil(t, alloc) - err = alloc.Rebase(3, int64(3000), false) + err = alloc.Rebase(int64(3000), false) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 3, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(3211), id) - err = alloc.Rebase(3, int64(6543), false) + err = alloc.Rebase(int64(6543), false) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 3, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(6544), id) // Test the MaxUint64 is the upper bound of `alloc` func but not `rebase`. var n uint64 = math.MaxUint64 - 1 un := int64(n) - err = alloc.Rebase(3, un, true) + err = alloc.Rebase(un, true) require.NoError(t, err) - _, _, err = alloc.Alloc(ctx, 3, 1, 1, 1) + _, _, err = alloc.Alloc(ctx, 1, 1, 1) require.Error(t, err) un = int64(n + 1) - err = alloc.Rebase(3, un, true) + err = alloc.Rebase(un, true) require.NoError(t, err) // alloc N for unsigned - alloc = autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + alloc = autoid.NewAllocator(store, 1, 4, true, autoid.RowIDAllocType) require.NotNil(t, alloc) - globalAutoID, err = alloc.NextGlobalAutoID(4) + globalAutoID, err = alloc.NextGlobalAutoID() require.NoError(t, err) require.Equal(t, int64(1), globalAutoID) - min, max, err := alloc.Alloc(ctx, 4, 2, 1, 1) + min, max, err := alloc.Alloc(ctx, 2, 1, 1) require.NoError(t, err) require.Equal(t, int64(2), max-min) require.Equal(t, int64(1), min+1) require.Equal(t, int64(2), max) - err = alloc.Rebase(4, int64(500), true) + err = alloc.Rebase(int64(500), true) require.NoError(t, err) - min, max, err = alloc.Alloc(ctx, 4, 2, 1, 1) + min, max, err = alloc.Alloc(ctx, 2, 1, 1) require.NoError(t, err) require.Equal(t, int64(2), max-min) require.Equal(t, int64(501), min+1) require.Equal(t, int64(502), max) lastRemainOne := alloc.End() - err = alloc.Rebase(4, alloc.End()-2, false) + err = alloc.Rebase(alloc.End()-2, false) require.NoError(t, err) - min, max, err = alloc.Alloc(ctx, 4, 5, 1, 1) + min, max, err = alloc.Alloc(ctx, 5, 1, 1) require.NoError(t, err) require.Equal(t, int64(5), max-min) require.Greater(t, min+1, lastRemainOne) // Test increment & offset for unsigned. Using AutoRandomType to avoid valid range check for increment and offset. - alloc = autoid.NewAllocator(store, 1, true, autoid.AutoRandomType) + alloc = autoid.NewAllocator(store, 1, 5, true, autoid.AutoRandomType) require.NotNil(t, alloc) require.NoError(t, err) require.Equal(t, int64(1), globalAutoID) @@ -393,7 +389,7 @@ func TestUnsignedAutoid(t *testing.T) { n = math.MaxUint64 - 100 offset := int64(n) - min, max, err = alloc.Alloc(ctx, 5, 2, increment, offset) + min, max, err = alloc.Alloc(ctx, 2, increment, offset) require.NoError(t, err) require.Equal(t, uint64(math.MaxUint64-101), uint64(min)) require.Equal(t, uint64(math.MaxUint64-98), uint64(max)) @@ -437,9 +433,9 @@ func TestConcurrentAlloc(t *testing.T) { allocIDs := func() { ctx := context.Background() - alloc := autoid.NewAllocator(store, dbID, false, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(store, dbID, tblID, false, autoid.RowIDAllocType) for j := 0; j < int(autoid.GetStep())+5; j++ { - _, id, err1 := alloc.Alloc(ctx, tblID, 1, 1, 1) + _, id, err1 := alloc.Alloc(ctx, 1, 1, 1) if err1 != nil { errCh <- err1 break @@ -456,7 +452,7 @@ func TestConcurrentAlloc(t *testing.T) { // test Alloc N N := rand.Uint64() % 100 - min, max, err1 := alloc.Alloc(ctx, tblID, N, 1, 1) + min, max, err1 := alloc.Alloc(ctx, N, 1, 1) if err1 != nil { errCh <- err1 break @@ -519,13 +515,13 @@ func TestRollbackAlloc(t *testing.T) { injectConf := new(kv.InjectionConfig) injectConf.SetCommitError(errors.New("injected")) injectedStore := kv.NewInjectedStore(store, injectConf) - alloc := autoid.NewAllocator(injectedStore, 1, false, autoid.RowIDAllocType) - _, _, err = alloc.Alloc(ctx, 2, 1, 1, 1) + alloc := autoid.NewAllocator(injectedStore, 1, 2, false, autoid.RowIDAllocType) + _, _, err = alloc.Alloc(ctx, 1, 1, 1) require.Error(t, err) require.Equal(t, int64(0), alloc.Base()) require.Equal(t, int64(0), alloc.End()) - err = alloc.Rebase(2, 100, true) + err = alloc.Rebase(100, true) require.Error(t, err) require.Equal(t, int64(0), alloc.Base()) require.Equal(t, int64(0), alloc.End()) @@ -569,29 +565,31 @@ func TestAllocComputationIssue(t *testing.T) { require.NoError(t, err) // Since the test here is applicable to any type of allocators, autoid.RowIDAllocType is chosen. - unsignedAlloc := autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) - require.NotNil(t, unsignedAlloc) - signedAlloc := autoid.NewAllocator(store, 1, false, autoid.RowIDAllocType) - require.NotNil(t, signedAlloc) + unsignedAlloc1 := autoid.NewAllocator(store, 1, 1, true, autoid.RowIDAllocType) + require.NotNil(t, unsignedAlloc1) + signedAlloc1 := autoid.NewAllocator(store, 1, 1, false, autoid.RowIDAllocType) + require.NotNil(t, signedAlloc1) + signedAlloc2 := autoid.NewAllocator(store, 1, 2, false, autoid.RowIDAllocType) + require.NotNil(t, signedAlloc2) // the next valid two value must be 13 & 16, batch size = 6. - err = unsignedAlloc.Rebase(1, 10, false) + err = unsignedAlloc1.Rebase(10, false) require.NoError(t, err) // the next valid two value must be 10 & 13, batch size = 6. - err = signedAlloc.Rebase(2, 7, false) + err = signedAlloc2.Rebase(7, false) require.NoError(t, err) // Simulate the rest cache is not enough for next batch, assuming 10 & 13, batch size = 4. - autoid.TestModifyBaseAndEndInjection(unsignedAlloc, 9, 9) + autoid.TestModifyBaseAndEndInjection(unsignedAlloc1, 9, 9) // Simulate the rest cache is not enough for next batch, assuming 10 & 13, batch size = 4. - autoid.TestModifyBaseAndEndInjection(signedAlloc, 4, 6) + autoid.TestModifyBaseAndEndInjection(signedAlloc1, 4, 6) ctx := context.Background() // Here will recompute the new allocator batch size base on new base = 10, which will get 6. - min, max, err := unsignedAlloc.Alloc(ctx, 1, 2, 3, 1) + min, max, err := unsignedAlloc1.Alloc(ctx, 2, 3, 1) require.NoError(t, err) require.Equal(t, int64(10), min) require.Equal(t, int64(16), max) - min, max, err = signedAlloc.Alloc(ctx, 2, 2, 3, 1) + min, max, err = signedAlloc2.Alloc(ctx, 2, 3, 1) require.NoError(t, err) require.Equal(t, int64(7), min) require.Equal(t, int64(13), max) diff --git a/meta/autoid/bench_test.go b/meta/autoid/bench_test.go index 24c71870399b7..77ad7ac4824af 100644 --- a/meta/autoid/bench_test.go +++ b/meta/autoid/bench_test.go @@ -57,10 +57,10 @@ func BenchmarkAllocator_Alloc(b *testing.B) { return } ctx := context.Background() - alloc := autoid.NewAllocator(store, 1, false, autoid.RowIDAllocType) + alloc := autoid.NewAllocator(store, 1, 2, false, autoid.RowIDAllocType) b.StartTimer() for i := 0; i < b.N; i++ { - _, _, err := alloc.Alloc(ctx, 2, 1, 1, 1) + _, _, err := alloc.Alloc(ctx, 1, 1, 1) if err != nil { b.Fatal(err) } @@ -108,10 +108,10 @@ func BenchmarkAllocator_SequenceAlloc(b *testing.B) { if err != nil { return } - alloc := autoid.NewSequenceAllocator(store, 1, seq) + alloc := autoid.NewSequenceAllocator(store, 1, 1, seq) b.StartTimer() for i := 0; i < b.N; i++ { - _, _, _, err := alloc.AllocSeqCache(1) + _, _, _, err := alloc.AllocSeqCache() if err != nil { fmt.Println("err") } diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go index 597a564bef2d8..1a9af524959b2 100644 --- a/meta/autoid/memid.go +++ b/meta/autoid/memid.go @@ -64,11 +64,11 @@ func (alloc *inMemoryAllocator) GetType() AllocatorType { } // NextGlobalAutoID implements autoid.Allocator NextGlobalAutoID interface. -func (alloc *inMemoryAllocator) NextGlobalAutoID(tableID int64) (int64, error) { +func (alloc *inMemoryAllocator) NextGlobalAutoID() (int64, error) { return alloc.base, nil } -func (alloc *inMemoryAllocator) Alloc(ctx context.Context, tableID int64, n uint64, increment, offset int64) (int64, int64, error) { +func (alloc *inMemoryAllocator) Alloc(ctx context.Context, n uint64, increment, offset int64) (int64, int64, error) { if n == 0 { return 0, 0, nil } @@ -86,7 +86,7 @@ func (alloc *inMemoryAllocator) Alloc(ctx context.Context, tableID int64, n uint // Rebase implements autoid.Allocator Rebase interface. // The requiredBase is the minimum base value after Rebase. // The real base may be greater than the required base. -func (alloc *inMemoryAllocator) Rebase(tableID, requiredBase int64, allocIDs bool) error { +func (alloc *inMemoryAllocator) Rebase(requiredBase int64, allocIDs bool) error { if alloc.isUnsigned { if uint64(requiredBase) > uint64(alloc.base) { alloc.base = requiredBase @@ -100,7 +100,7 @@ func (alloc *inMemoryAllocator) Rebase(tableID, requiredBase int64, allocIDs boo } // ForceRebase implements autoid.Allocator ForceRebase interface. -func (alloc *inMemoryAllocator) ForceRebase(tableID, requiredBase int64) error { +func (alloc *inMemoryAllocator) ForceRebase(requiredBase int64) error { alloc.base = requiredBase return nil } @@ -143,10 +143,10 @@ func (alloc *inMemoryAllocator) alloc4Unsigned(n uint64, increment, offset int64 return min, alloc.base, nil } -func (alloc *inMemoryAllocator) AllocSeqCache(tableID int64) (int64, int64, int64, error) { +func (alloc *inMemoryAllocator) AllocSeqCache() (int64, int64, int64, error) { return 0, 0, 0, errNotImplemented.GenWithStackByArgs() } -func (alloc *inMemoryAllocator) RebaseSeq(tableID, requiredBase int64) (int64, bool, error) { +func (alloc *inMemoryAllocator) RebaseSeq(requiredBase int64) (int64, bool, error) { return 0, false, errNotImplemented.GenWithStackByArgs() } diff --git a/meta/autoid/memid_test.go b/meta/autoid/memid_test.go index 700bfc5fd5a23..15dd436f98ed1 100644 --- a/meta/autoid/memid_test.go +++ b/meta/autoid/memid_test.go @@ -49,47 +49,47 @@ func TestInMemoryAlloc(t *testing.T) { // alloc 1 ctx := context.Background() - _, id, err := alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err := alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(1), id) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(2), id) // alloc N - _, id, err = alloc.Alloc(ctx, 1, 10, 1, 1) + _, id, err = alloc.Alloc(ctx, 10, 1, 1) require.NoError(t, err) require.Equal(t, int64(12), id) // increment > N - _, id, err = alloc.Alloc(ctx, 1, 1, 10, 1) + _, id, err = alloc.Alloc(ctx, 1, 10, 1) require.NoError(t, err) require.Equal(t, int64(21), id) // offset - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 30) + _, id, err = alloc.Alloc(ctx, 1, 1, 30) require.NoError(t, err) require.Equal(t, int64(30), id) // rebase - err = alloc.Rebase(1, int64(40), true) + err = alloc.Rebase(int64(40), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(41), id) - err = alloc.Rebase(1, int64(10), true) + err = alloc.Rebase(int64(10), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(42), id) // maxInt64 - err = alloc.Rebase(1, int64(math.MaxInt64-2), true) + err = alloc.Rebase(int64(math.MaxInt64-2), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(math.MaxInt64-1), id) - _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, _, err = alloc.Alloc(ctx, 1, 1, 1) require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) // test unsigned @@ -98,11 +98,11 @@ func TestInMemoryAlloc(t *testing.T) { require.NotNil(t, alloc) var n uint64 = math.MaxUint64 - 2 - err = alloc.Rebase(1, int64(n), true) + err = alloc.Rebase(int64(n), true) require.NoError(t, err) - _, id, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, id, err = alloc.Alloc(ctx, 1, 1, 1) require.NoError(t, err) require.Equal(t, int64(n+1), id) - _, _, err = alloc.Alloc(ctx, 1, 1, 1, 1) + _, _, err = alloc.Alloc(ctx, 1, 1, 1) require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) } diff --git a/meta/autoid/seq_autoid_test.go b/meta/autoid/seq_autoid_test.go index 8cef4d21a171f..af46a70303d7e 100644 --- a/meta/autoid/seq_autoid_test.go +++ b/meta/autoid/seq_autoid_test.go @@ -64,11 +64,11 @@ func TestSequenceAutoid(t *testing.T) { }) require.NoError(t, err) - alloc := autoid.NewSequenceAllocator(store, 1, seq) + alloc := autoid.NewSequenceAllocator(store, 1, 1, seq) require.NotNil(t, alloc) // allocate sequence cache. - base, end, round, err := alloc.AllocSeqCache(1) + base, end, round, err := alloc.AllocSeqCache() require.NoError(t, err) require.Equal(t, int64(0), base) require.Equal(t, int64(5), end) @@ -95,7 +95,7 @@ func TestSequenceAutoid(t *testing.T) { require.True(t, ok) require.Equal(t, int64(5), nextVal) - base, end, round, err = alloc.AllocSeqCache(1) + base, end, round, err = alloc.AllocSeqCache() require.NoError(t, err) require.Equal(t, int64(5), base) require.Equal(t, int64(10), end) @@ -120,7 +120,7 @@ func TestSequenceAutoid(t *testing.T) { // the rest in cache in not enough for next value. require.False(t, ok) - base, end, round, err = alloc.AllocSeqCache(1) + base, end, round, err = alloc.AllocSeqCache() require.NoError(t, err) require.Equal(t, int64(-11), base) require.Equal(t, int64(-6), end) @@ -199,9 +199,9 @@ func TestConcurrentAllocSequence(t *testing.T) { errCh := make(chan error, count) allocSequence := func() { - alloc := autoid.NewSequenceAllocator(store, 2, seq) + alloc := autoid.NewSequenceAllocator(store, 2, 2, seq) for j := 0; j < 3; j++ { - base, end, _, err1 := alloc.AllocSeqCache(2) + base, end, _, err1 := alloc.AllocSeqCache() if err1 != nil { errCh <- err1 break diff --git a/meta/meta.go b/meta/meta.go index e4872e7d08740..05f0f011dea18 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -490,6 +490,19 @@ func (m *Meta) RestartSequenceValue(dbID int64, tableInfo *model.TableInfo, seqV return errors.Trace(m.txn.HSet(m.dbKey(dbID), m.sequenceKey(tableInfo.ID), []byte(strconv.FormatInt(seqValue, 10)))) } +// DropPolicy drops the specified policy. +func (m *Meta) DropPolicy(policyID int64) error { + // Check if policy exists. + policyKey := m.policyKey(policyID) + if err := m.txn.HClear(policyKey); err != nil { + return errors.Trace(err) + } + if err := m.txn.HDel(mPolicies, policyKey); err != nil { + return errors.Trace(err) + } + return nil +} + // DropDatabase drops whole database. func (m *Meta) DropDatabase(dbID int64) error { // Check if db exists. @@ -669,9 +682,13 @@ func (m *Meta) ListPolicies() ([]*placement.PolicyInfo, error) { 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 { + if err != nil { return nil, errors.Trace(err) } + if value == nil { + return nil, ErrPolicyNotExists.GenWithStack("policy id : %d doesn't exist", policyID) + } + value, err = detachMagicByte(value) if err != nil { return nil, errors.Trace(err) diff --git a/planner/cascades/optimize_test.go b/planner/cascades/optimize_test.go index 50bb7d9a18ba1..beb4a6ee93ac0 100644 --- a/planner/cascades/optimize_test.go +++ b/planner/cascades/optimize_test.go @@ -39,7 +39,7 @@ func TestImplGroupZeroCost(t *testing.T) { stmt, err := p.ParseOneStmt("select t1.a, t2.a from t as t1 left join t as t2 on t1.a = t2.a where t1.a < 1.0", "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) @@ -64,7 +64,7 @@ func TestInitGroupSchema(t *testing.T) { stmt, err := p.ParseOneStmt("select a from t", "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) @@ -87,7 +87,7 @@ func TestFillGroupStats(t *testing.T) { stmt, err := p.ParseOneStmt("select * from t t1 join t t2 on t1.a = t2.a", "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) @@ -119,7 +119,7 @@ func TestPreparePossibleProperties(t *testing.T) { stmt, err := p.ParseOneStmt("select f, sum(a) from t group by f", "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) @@ -214,7 +214,7 @@ func TestAppliedRuleSet(t *testing.T) { stmt, err := p.ParseOneStmt("select 1", "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) diff --git a/planner/cascades/stringer_test.go b/planner/cascades/stringer_test.go index dbde4e86ce736..8bd1dab264b15 100644 --- a/planner/cascades/stringer_test.go +++ b/planner/cascades/stringer_test.go @@ -58,7 +58,7 @@ func TestGroupStringer(t *testing.T) { stmt, err := p.ParseOneStmt(sql, "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) diff --git a/planner/cascades/transformation_rules_test.go b/planner/cascades/transformation_rules_test.go index ee5d93bebcd8f..bf6e9948ad95c 100644 --- a/planner/cascades/transformation_rules_test.go +++ b/planner/cascades/transformation_rules_test.go @@ -39,7 +39,7 @@ func testGroupToString(t *testing.T, input []string, output []struct { stmt, err := p.ParseOneStmt(sql, "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) @@ -89,7 +89,7 @@ func TestAggPushDownGather(t *testing.T) { stmt, err := p.ParseOneStmt(sql, "", "") require.NoError(t, err) - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt, is) require.NoError(t, err) logic, ok := plan.(plannercore.LogicalPlan) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index e8e6d0862817c..a6c12579f0120 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -4307,3 +4307,31 @@ func (s *testIntegrationSerialSuite) TestTemporaryTableForCte(c *C) { 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")) } + +func (s *testIntegrationSuite) TestGroupBySetVar(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1(c1 int);") + tk.MustExec("insert into t1 values(1), (2), (3), (4), (5), (6);") + rows := tk.MustQuery("select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum;") + rows.Check(testkit.Rows("0 2", "1 2", "2 2")) + + tk.MustExec("create table ta(a int, b int);") + tk.MustExec("set sql_mode='';") + + var input []string + var output []struct { + SQL string + Plan []string + } + s.testData.GetTestCases(c, &input, &output) + for i, tt := range input { + res := tk.MustQuery("explain format = 'brief' " + tt) + s.testData.OnRecord(func() { + output[i].SQL = tt + output[i].Plan = s.testData.ConvertRowsToStrings(res.Rows()) + }) + res.Check(testkit.Rows(output[i].Plan...)) + } +} diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index ce4e64c286c19..f99090ce5439a 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -82,7 +82,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { comment := Commentf("for %s", ca) stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -109,7 +109,7 @@ func (s *testPlanSuite) TestJoinPredicatePushDown(c *C) { comment := Commentf("for %s", ca) stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -148,7 +148,7 @@ func (s *testPlanSuite) TestOuterWherePredicatePushDown(c *C) { comment := Commentf("for %s", ca) stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -193,7 +193,7 @@ func (s *testPlanSuite) TestSimplifyOuterJoin(c *C) { comment := Commentf("for %s", ca) stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -233,7 +233,7 @@ func (s *testPlanSuite) TestAntiSemiJoinConstFalse(c *C) { comment := Commentf("for %s", ca.sql) stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -260,7 +260,7 @@ func (s *testPlanSuite) TestDeriveNotNullConds(c *C) { comment := Commentf("for %s", ca) stmt, err := s.ParseOneStmt(ca, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagDecorrelate, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -288,7 +288,7 @@ func (s *testPlanSuite) TestExtraPKNotNullFlag(c *C) { comment := Commentf("for %s", sql) stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) ds := p.(*LogicalProjection).children[0].(*LogicalAggregation).children[0].(*DataSource) c.Assert(ds.Columns[2].Name.L, Equals, "_tidb_rowid") @@ -309,7 +309,7 @@ func buildLogicPlan4GroupBy(s *testPlanSuite, c *C, sql string) (Plan, error) { stmt.(*ast.SelectStmt).From.TableRefs.Left.(*ast.TableSource).Source.(*ast.TableName).TableInfo = mockedTableInfo - p, _, err := BuildLogicalPlan(context.Background(), s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) return p, err } @@ -367,7 +367,7 @@ func (s *testPlanSuite) TestDupRandJoinCondsPushDown(c *C) { comment := Commentf("for %s", sql) stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(context.Background(), s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -435,7 +435,7 @@ func (s *testPlanSuite) TestTablePartition(c *C) { s.testData.OnRecord(func() { }) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, isChoices[ca.IsIdx]) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, isChoices[ca.IsIdx]) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain|flagPredicatePushDown|flagPartitionProcessor, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -460,7 +460,7 @@ func (s *testPlanSuite) TestSubquery(c *C) { err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagDecorrelate|flagPrunColumns|flagPrunColumnsAgain, lp) @@ -486,7 +486,7 @@ func (s *testPlanSuite) TestPlanBuilder(c *C) { s.ctx.GetSessionVars().SetHashJoinConcurrency(1) err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) if lp, ok := p.(LogicalPlan); ok { p, err = logicalOptimize(context.TODO(), flagPrunColumns|flagPrunColumnsAgain, lp) @@ -510,7 +510,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -534,7 +534,7 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagPushDownAgg, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -560,7 +560,7 @@ func (s *testPlanSuite) TestColumnPruning(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) lp, err := logicalOptimize(ctx, flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -588,7 +588,7 @@ func (s *testPlanSuite) TestSortByItemsPruning(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) lp, err := logicalOptimize(ctx, flagEliminateProjection|flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -614,7 +614,7 @@ func (s *testPlanSuite) TestProjectionEliminator(c *C) { stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagPrunColumns|flagPrunColumnsAgain|flagEliminateProjection, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -851,7 +851,7 @@ func (s *testPlanSuite) TestValidate(c *C) { c.Assert(err, IsNil, comment) err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil) - _, _, err = BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) if tt.err == nil { c.Assert(err, IsNil, comment) } else { @@ -902,7 +902,7 @@ func (s *testPlanSuite) TestUniqueKeyInfo(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) lp, err := logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagBuildKeyInfo, p.(LogicalPlan)) c.Assert(err, IsNil) @@ -924,7 +924,7 @@ func (s *testPlanSuite) TestAggPrune(c *C) { stmt, err := s.ParseOneStmt(tt, "", "") c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil) p, err = logicalOptimize(context.TODO(), flagPredicatePushDown|flagPrunColumns|flagPrunColumnsAgain|flagBuildKeyInfo|flagEliminateAgg|flagEliminateProjection, p.(LogicalPlan)) @@ -1491,7 +1491,7 @@ func (s *testPlanSuite) TestNameResolver(c *C) { c.Assert(err, IsNil, comment) s.ctx.GetSessionVars().SetHashJoinConcurrency(1) - _, _, err = BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + _, _, err = BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) if t.err == "" { c.Check(err, IsNil) } else { @@ -1922,7 +1922,7 @@ func (s *testPlanSuite) TestResolvingCorrelatedAggregate(c *C) { c.Assert(err, IsNil, comment) err = Preprocess(s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is})) c.Assert(err, IsNil, comment) - p, _, err := BuildLogicalPlan(ctx, s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(ctx, s.ctx, stmt, s.is) c.Assert(err, IsNil, comment) p, err = logicalOptimize(context.TODO(), flagBuildKeyInfo|flagEliminateProjection|flagPrunColumns|flagPrunColumnsAgain, p.(LogicalPlan)) c.Assert(err, IsNil, comment) @@ -1981,7 +1981,7 @@ func (s *testPlanSuite) TestWindowLogicalPlanAmbiguous(c *C) { for i := 0; i < iterations; i++ { stmt, err := s.ParseOneStmt(sql, "", "") c.Assert(err, IsNil) - p, _, err := BuildLogicalPlan(context.Background(), s.ctx, stmt, s.is) + p, _, err := BuildLogicalPlanForTest(context.Background(), s.ctx, stmt, s.is) c.Assert(err, IsNil) if planString == "" { planString = ToString(p) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d406248ebfac9..001add7c5021b 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -88,11 +88,11 @@ type logicalOptRule interface { name() string } -// BuildLogicalPlan used to build logical plan from ast.Node. -func BuildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (Plan, types.NameSlice, error) { +// BuildLogicalPlanForTest builds a logical plan for testing purpose from ast.Node. +func BuildLogicalPlanForTest(ctx context.Context, sctx sessionctx.Context, node ast.Node, infoSchema infoschema.InfoSchema) (Plan, types.NameSlice, error) { sctx.GetSessionVars().PlanID = 0 sctx.GetSessionVars().PlanColumnID = 0 - builder, _ := NewPlanBuilder().Init(sctx, is, &utilhint.BlockHintProcessor{}) + builder, _ := NewPlanBuilder().Init(sctx, infoSchema, &utilhint.BlockHintProcessor{}) p, err := builder.Build(ctx, node) if err != nil { return nil, nil, err diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 4ad2e6f260c6a..0dc84b07be4a0 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -428,22 +428,41 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan) (_ LogicalPlan, e } else if proj, ok1 := child.(*LogicalProjection); ok1 { // TODO: This optimization is not always reasonable. We have not supported pushing projection to kv layer yet, // so we must do this optimization. - for i, gbyItem := range agg.GroupByItems { - agg.GroupByItems[i] = expression.ColumnSubstitute(gbyItem, proj.schema, proj.Exprs) + noSideEffects := true + newGbyItems := make([]expression.Expression, 0, len(agg.GroupByItems)) + for _, gbyItem := range agg.GroupByItems { + newGbyItems = append(newGbyItems, expression.ColumnSubstitute(gbyItem, proj.schema, proj.Exprs)) + if ExprsHasSideEffects(newGbyItems) { + noSideEffects = false + break + } + } + newAggFuncsArgs := make([][]expression.Expression, 0, len(agg.AggFuncs)) + if noSideEffects { + for _, aggFunc := range agg.AggFuncs { + newArgs := make([]expression.Expression, 0, len(aggFunc.Args)) + for _, arg := range aggFunc.Args { + newArgs = append(newArgs, expression.ColumnSubstitute(arg, proj.schema, proj.Exprs)) + } + if ExprsHasSideEffects(newArgs) { + noSideEffects = false + break + } + newAggFuncsArgs = append(newAggFuncsArgs, newArgs) + } } - for _, aggFunc := range agg.AggFuncs { - newArgs := make([]expression.Expression, 0, len(aggFunc.Args)) - for _, arg := range aggFunc.Args { - newArgs = append(newArgs, expression.ColumnSubstitute(arg, proj.schema, proj.Exprs)) + if noSideEffects { + agg.GroupByItems = newGbyItems + for i, aggFunc := range agg.AggFuncs { + aggFunc.Args = newAggFuncsArgs[i] } - aggFunc.Args = newArgs + projChild := proj.children[0] + agg.SetChildren(projChild) + // When the origin plan tree is `Aggregation->Projection->Union All->X`, we need to merge 'Aggregation' and 'Projection' first. + // And then push the new 'Aggregation' below the 'Union All' . + // The final plan tree should be 'Aggregation->Union All->Aggregation->X'. + child = projChild } - projChild := proj.children[0] - agg.SetChildren(projChild) - // When the origin plan tree is `Aggregation->Projection->Union All->X`, we need to merge 'Aggregation' and 'Projection' first. - // And then push the new 'Aggregation' below the 'Union All' . - // The final plan tree should be 'Aggregation->Union All->Aggregation->X'. - child = projChild } if union, ok1 := child.(*LogicalUnionAll); ok1 && p.SCtx().GetSessionVars().AllowAggPushDown { err := a.tryAggPushDownForUnion(union, agg) diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 77154520bcffe..46ecf884d55e2 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -96,7 +96,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) if la.AggFuncs[i].Name != ast.AggFuncFirstRow { allFirstRow = false } - if !used[i] { + if !used[i] && !ExprsHasSideEffects(la.AggFuncs[i].Args) { la.schema.Columns = append(la.schema.Columns[:i], la.schema.Columns[i+1:]...) la.AggFuncs = append(la.AggFuncs[:i], la.AggFuncs[i+1:]...) } else if la.AggFuncs[i].Name != ast.AggFuncFirstRow { @@ -137,7 +137,7 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column) if len(la.GroupByItems) > 0 { for i := len(la.GroupByItems) - 1; i >= 0; i-- { cols := expression.ExtractColumns(la.GroupByItems[i]) - if len(cols) == 0 { + if len(cols) == 0 && !exprHasSetVarOrSleep(la.GroupByItems[i]) { la.GroupByItems = append(la.GroupByItems[:i], la.GroupByItems[i+1:]...) } else { selfUsedCols = append(selfUsedCols, cols...) diff --git a/planner/core/testdata/integration_suite_in.json b/planner/core/testdata/integration_suite_in.json index 10dfddf8c8f9a..a8b601d99faa4 100644 --- a/planner/core/testdata/integration_suite_in.json +++ b/planner/core/testdata/integration_suite_in.json @@ -355,5 +355,18 @@ "cases": [ "select * from t use index (idx_b) where b = 2 limit 1" ] + }, + { + "name": "TestGroupBySetVar", + "cases": [ + "select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum;", + // TODO: fix these two cases + "select @n:=@n+1 as e from ta group by e", + "select @n:=@n+a as e from ta group by e", + "select * from (select @n:=@n+1 as e from ta) tt group by e", + "select * from (select @n:=@n+a as e from ta) tt group by e", + "select a from ta group by @n:=@n+1", + "select a from ta group by @n:=@n+a" + ] } ] diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 261b35bc71898..9d8c313a49921 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1897,5 +1897,81 @@ ] } ] + }, + { + "Name": "TestGroupBySetVar", + "Cases": [ + { + "SQL": "select floor(dt.rn/2) rownum, count(c1) from (select @rownum := @rownum + 1 rn, c1 from (select @rownum := -1) drn, t1) dt group by floor(dt.rn/2) order by rownum;", + "Plan": [ + "Sort 1.00 root Column#6", + "└─Projection 1.00 root floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#6, Column#5", + " └─HashAgg 1.00 root group by:Column#13, funcs:count(Column#11)->Column#5, funcs:firstrow(Column#12)->Column#4", + " └─Projection 10000.00 root test.t1.c1, Column#4, floor(div(cast(Column#4, decimal(20,0) BINARY), 2))->Column#13", + " └─Projection 10000.00 root setvar(rownum, plus(getvar(rownum), 1))->Column#4, test.t1.c1", + " └─HashJoin 10000.00 root CARTESIAN inner join", + " ├─Projection(Build) 1.00 root setvar(rownum, -1)->Column#1", + " │ └─TableDual 1.00 root rows:1", + " └─TableReader(Probe) 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select @n:=@n+1 as e from ta group by e", + "Plan": [ + "Projection 1.00 root setvar(n, plus(getvar(n), 1))->Column#4", + "└─HashAgg 1.00 root group by:Column#8, funcs:firstrow(1)->Column#7", + " └─Projection 10000.00 root setvar(n, plus(cast(getvar(n), double BINARY), 1))->Column#8", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select @n:=@n+a as e from ta group by e", + "Plan": [ + "Projection 8000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", + "└─HashAgg 8000.00 root group by:Column#7, funcs:firstrow(Column#6)->test.ta.a", + " └─Projection 10000.00 root test.ta.a, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#7", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select @n:=@n+1 as e from ta) tt group by e", + "Plan": [ + "HashAgg 1.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4", + "└─Projection 10000.00 root setvar(n, plus(getvar(n), 1))->Column#4", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select * from (select @n:=@n+a as e from ta) tt group by e", + "Plan": [ + "HashAgg 8000.00 root group by:Column#4, funcs:firstrow(Column#4)->Column#4", + "└─Projection 10000.00 root setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#4", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select a from ta group by @n:=@n+1", + "Plan": [ + "HashAgg 1.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", + "└─Projection 10000.00 root test.ta.a, setvar(n, plus(getvar(n), 1))->Column#5", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + }, + { + "SQL": "select a from ta group by @n:=@n+a", + "Plan": [ + "HashAgg 8000.00 root group by:Column#5, funcs:firstrow(Column#4)->test.ta.a", + "└─Projection 10000.00 root test.ta.a, setvar(n, plus(getvar(n), cast(test.ta.a, double BINARY)))->Column#5", + " └─TableReader 10000.00 root data:TableFullScan", + " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" + ] + } + ] } ] diff --git a/planner/memo/group_test.go b/planner/memo/group_test.go index 73ac5f4e351d1..0c21e48044fcf 100644 --- a/planner/memo/group_test.go +++ b/planner/memo/group_test.go @@ -104,7 +104,7 @@ func TestGroupFingerPrint(t *testing.T) { is := infoschema.MockInfoSchema([]*model.TableInfo{plannercore.MockSignedTable()}) ctx := plannercore.MockContext() - plan, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt1, is) + plan, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt1, is) require.NoError(t, err) logic1, ok := plan.(plannercore.LogicalPlan) require.True(t, ok) @@ -250,7 +250,7 @@ func TestBuildKeyInfo(t *testing.T) { // case 1: primary key has constant constraint stmt1, err := p.ParseOneStmt("select a from t where a = 10", "", "") require.NoError(t, err) - p1, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt1, is) + p1, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt1, is) require.NoError(t, err) logic1, ok := p1.(plannercore.LogicalPlan) require.True(t, ok) @@ -262,7 +262,7 @@ func TestBuildKeyInfo(t *testing.T) { // case 2: group by column is key stmt2, err := p.ParseOneStmt("select b, sum(a) from t group by b", "", "") require.NoError(t, err) - p2, _, err := plannercore.BuildLogicalPlan(context.Background(), ctx, stmt2, is) + p2, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), ctx, stmt2, is) require.NoError(t, err) logic2, ok := p2.(plannercore.LogicalPlan) require.True(t, ok) diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 1c554f9ee55a8..0ac652b2f01c0 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -46,6 +46,7 @@ var dynamicPrivs = []string{ "SYSTEM_VARIABLES_ADMIN", "ROLE_ADMIN", "CONNECTION_ADMIN", + "DASHBOARD_CLIENT", // Can login to the TiDB-Dashboard. "RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled "RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled. "RESTRICTED_VARIABLES_ADMIN", // Can see all variables when SEM is enabled diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 3dfa6769c601a..a8b9f0c54a7a7 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" + "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/privilege" @@ -2086,6 +2087,50 @@ func TestGrantReferences(t *testing.T) { tk.MustExec("DROP SCHEMA reftestdb") } +func TestDashboardClientDynamicPriv(t *testing.T) { + t.Parallel() + store, clean := newStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("CREATE ROLE dc_r1") + tk.MustExec("CREATE USER dc_u1") + tk.MustExec("GRANT dc_r1 TO dc_u1") + tk.MustExec("SET DEFAULT ROLE dc_r1 TO dc_u1") + + tk1 := testkit.NewTestKit(t, store) + tk1.Session().Auth(&auth.UserIdentity{ + Username: "dc_u1", + Hostname: "localhost", + }, nil, nil) + tk1.MustQuery("SHOW GRANTS FOR CURRENT_USER()").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'dc_u1'@'%'", + "GRANT 'dc_r1'@'%' TO 'dc_u1'@'%'", + )) + tk.MustExec("GRANT DASHBOARD_CLIENT ON *.* TO dc_r1") + tk1.MustQuery("SHOW GRANTS FOR CURRENT_USER()").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'dc_u1'@'%'", + "GRANT 'dc_r1'@'%' TO 'dc_u1'@'%'", + "GRANT DASHBOARD_CLIENT ON *.* TO 'dc_u1'@'%'", + )) + tk.MustExec("REVOKE DASHBOARD_CLIENT ON *.* FROM dc_r1") + tk1.MustQuery("SHOW GRANTS FOR CURRENT_USER()").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'dc_u1'@'%'", + "GRANT 'dc_r1'@'%' TO 'dc_u1'@'%'", + )) + tk.MustExec("GRANT DASHBOARD_CLIENT ON *.* TO dc_u1") + tk1.MustQuery("SHOW GRANTS FOR CURRENT_USER()").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'dc_u1'@'%'", + "GRANT 'dc_r1'@'%' TO 'dc_u1'@'%'", + "GRANT DASHBOARD_CLIENT ON *.* TO 'dc_u1'@'%'", + )) + tk.MustExec("REVOKE DASHBOARD_CLIENT ON *.* FROM dc_u1") + tk1.MustQuery("SHOW GRANTS FOR CURRENT_USER()").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'dc_u1'@'%'", + "GRANT 'dc_r1'@'%' TO 'dc_u1'@'%'", + )) +} + // https://github.com/pingcap/tidb/issues/27213 func TestShowGrantsWithRolesAndDynamicPrivs(t *testing.T) { t.Parallel() @@ -2200,3 +2245,57 @@ func TestGrantLockTables(t *testing.T) { tk.MustExec("DROP USER lock_tables_user") tk.MustExec("DROP DATABASE lock_tables_db") } + +// https://github.com/pingcap/tidb/issues/27560 +func TestShowGrantsForCurrentUserUsingRole(t *testing.T) { + t.Parallel() + + store, clean := newStore(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("DROP USER IF EXISTS joe, engineering, notgranted, otherrole, delete_stuff_privilege") + tk.MustExec("CREATE USER joe;") + tk.MustExec("CREATE ROLE engineering;") + tk.MustExec("CREATE ROLE admins;") + tk.MustExec("CREATE ROLE notgranted;") + tk.MustExec("CREATE ROLE otherrole;") + tk.MustExec("GRANT INSERT ON test.* TO engineering;") + tk.MustExec("GRANT DELETE ON test.* TO admins;") + tk.MustExec("GRANT SELECT on test.* to joe;") + tk.MustExec("GRANT engineering TO joe;") + tk.MustExec("GRANT admins TO joe;") + tk.MustExec("SET DEFAULT ROLE admins TO joe;") + tk.MustExec("GRANT otherrole TO joe;") + tk.MustExec("GRANT UPDATE ON role.* TO otherrole;") + tk.MustExec("GRANT SELECT ON mysql.user TO otherrole;") + tk.MustExec("CREATE ROLE delete_stuff_privilege;") + tk.MustExec("GRANT DELETE ON mysql.user TO delete_stuff_privilege;") + tk.MustExec("GRANT delete_stuff_privilege TO otherrole;") + + tk.Session().Auth(&auth.UserIdentity{ + Username: "joe", + Hostname: "%", + }, nil, nil) + + err := tk.QueryToErr("SHOW GRANTS FOR CURRENT_USER() USING notgranted") + require.Error(t, err) + require.True(t, terror.ErrorEqual(err, executor.ErrRoleNotGranted)) + + tk.MustQuery("SHOW GRANTS FOR current_user() USING otherrole;").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'joe'@'%'", + "GRANT UPDATE ON role.* TO 'joe'@'%'", + "GRANT SELECT ON test.* TO 'joe'@'%'", + "GRANT DELETE ON mysql.user TO 'joe'@'%'", + "GRANT 'admins'@'%', 'engineering'@'%', 'otherrole'@'%' TO 'joe'@'%'", + )) + tk.MustQuery("SHOW GRANTS FOR joe USING otherrole;").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'joe'@'%'", + "GRANT UPDATE ON role.* TO 'joe'@'%'", + "GRANT SELECT ON test.* TO 'joe'@'%'", + "GRANT DELETE ON mysql.user TO 'joe'@'%'", + "GRANT 'admins'@'%', 'engineering'@'%', 'otherrole'@'%' TO 'joe'@'%'", + )) + +} diff --git a/sessionctx/context_test.go b/sessionctx/context_test.go index 073ee65b76d89..97cf9c63b79bc 100644 --- a/sessionctx/context_test.go +++ b/sessionctx/context_test.go @@ -18,13 +18,9 @@ import ( "fmt" "testing" - . "github.com/pingcap/check" + "github.com/stretchr/testify/require" ) -func TestT(t *testing.T) { - TestingT(t) -} - func TestBasicCtxTypeToString(t *testing.T) { tests := []struct { key fmt.Stringer @@ -36,8 +32,6 @@ func TestBasicCtxTypeToString(t *testing.T) { {basicCtxType(9), "unknown"}, } for _, tt := range tests { - if tt.key.String() != tt.v { - t.Fatalf("want %s but got %s", tt.v, tt.key.String()) - } + require.Equal(t, tt.key.String(), tt.v) } } diff --git a/sessionctx/main_test.go b/sessionctx/main_test.go new file mode 100644 index 0000000000000..89eeb11becb85 --- /dev/null +++ b/sessionctx/main_test.go @@ -0,0 +1,27 @@ +// 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 sessionctx + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + goleak.VerifyTestMain(m) +} diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index 1e4ac7508d5bd..c6ef9cf132519 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -307,7 +307,7 @@ func (s *testStatsSuite) TestSelectivity(c *C) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, comment) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, tt.exprs)) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) @@ -584,7 +584,7 @@ func BenchmarkSelectivity(b *testing.B) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, comment) - p, _, err := plannercore.BuildLogicalPlan(context.Background(), sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(context.Background(), sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, expr %s", err, exprs)) file, err := os.Create("cpu.profile") @@ -848,7 +848,7 @@ func (s *testStatsSuite) TestDNFCondSelectivity(c *C) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) c.Assert(err, IsNil, Commentf("error %v, for sql %s", err, tt)) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) c.Assert(err, IsNil, Commentf("error %v, for building plan, sql %s", err, tt)) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) diff --git a/table/table.go b/table/table.go index a3440b01f9b4f..5471f912e72cb 100644 --- a/table/table.go +++ b/table/table.go @@ -205,7 +205,7 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte } increment := sctx.GetSessionVars().AutoIncrementIncrement offset := sctx.GetSessionVars().AutoIncrementOffset - _, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, t.Meta().ID, uint64(1), int64(increment), int64(offset)) + _, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(1), int64(increment), int64(offset)) if err != nil { return 0, err } @@ -217,7 +217,7 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) (firstID int64, increment int64, err error) { increment = int64(sctx.GetSessionVars().AutoIncrementIncrement) offset := int64(sctx.GetSessionVars().AutoIncrementOffset) - min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, t.Meta().ID, uint64(N), increment, offset) + min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(N), increment, offset) if err != nil { return min, max, err } diff --git a/table/tables/tables.go b/table/tables/tables.go index da32fcfbcf06c..dc4a7594d5924 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1397,7 +1397,7 @@ func AllocHandle(ctx context.Context, sctx sessionctx.Context, t table.Table) (k func allocHandleIDs(ctx context.Context, sctx sessionctx.Context, t table.Table, n uint64) (int64, int64, error) { meta := t.Meta() - base, maxID, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, meta.ID, n, 1, 1) + base, maxID, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, n, 1, 1) if err != nil { return 0, 0, err } @@ -1469,7 +1469,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { // RebaseAutoID implements table.Table RebaseAutoID interface. // Both auto-increment and auto-random can use this function to do rebase on explicit newBase value (without shadow bits). func (t *TableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { - return t.Allocators(ctx).Get(tp).Rebase(t.tableID, newBase, isSetStep) + return t.Allocators(ctx).Get(tp).Rebase(newBase, isSetStep) } // Type implements table.Table Type interface. @@ -1643,7 +1643,7 @@ func (t *TableCommon) GetSequenceNextVal(ctx interface{}, dbName, seqName string return err1 } var base, end, round int64 - base, end, round, err1 = sequenceAlloc.AllocSeqCache(t.tableID) + base, end, round, err1 = sequenceAlloc.AllocSeqCache() if err1 != nil { return err1 } @@ -1719,7 +1719,7 @@ func (t *TableCommon) SetSequenceVal(ctx interface{}, newVal int64, dbName, seqN if err != nil { return 0, false, err } - res, alreadySatisfied, err := sequenceAlloc.RebaseSeq(t.tableID, newVal) + res, alreadySatisfied, err := sequenceAlloc.RebaseSeq(newVal) if err != nil { return 0, false, err } diff --git a/tablecodec/main_test.go b/tablecodec/main_test.go new file mode 100644 index 0000000000000..75ea2dc757133 --- /dev/null +++ b/tablecodec/main_test.go @@ -0,0 +1,27 @@ +// 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 tablecodec + +import ( + "testing" + + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" +) + +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + goleak.VerifyTestMain(m) +} diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 88f001ef5b173..af440cdf555f4 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -20,7 +20,6 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -29,30 +28,22 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" - "github.com/pingcap/tidb/util/testleak" + "github.com/stretchr/testify/require" ) -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = SerialSuites(&testTableCodecSuite{}) - -type testTableCodecSuite struct{} - // TestTableCodec tests some functions in package tablecodec // TODO: add more tests. -func (s *testTableCodecSuite) TestTableCodec(c *C) { - defer testleak.AfterTest(c)() +func TestTableCodec(t *testing.T) { + t.Parallel() key := EncodeRowKey(1, codec.EncodeInt(nil, 2)) h, err := DecodeRowKey(key) - c.Assert(err, IsNil) - c.Assert(h.IntValue(), Equals, int64(2)) + require.NoError(t, err) + require.Equal(t, int64(2), h.IntValue()) key = EncodeRowKeyWithHandle(1, kv.IntHandle(2)) h, err = DecodeRowKey(key) - c.Assert(err, IsNil) - c.Assert(h.IntValue(), Equals, int64(2)) + require.NoError(t, err) + require.Equal(t, int64(2), h.IntValue()) } // column is a structure used for test @@ -61,9 +52,8 @@ type column struct { tp *types.FieldType } -func (s *testTableCodecSuite) TestRowCodec(c *C) { - defer testleak.AfterTest(c)() - +func TestRowCodec(t *testing.T) { + t.Parallel() c1 := &column{id: 1, tp: types.NewFieldType(mysql.TypeLonglong)} c2 := &column{id: 2, tp: types.NewFieldType(mysql.TypeVarchar)} c3 := &column{id: 3, tp: types.NewFieldType(mysql.TypeNewDecimal)} @@ -87,8 +77,8 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { rd := rowcodec.Encoder{Enable: true} sc := &stmtctx.StatementContext{TimeZone: time.Local} bs, err := EncodeRow(sc, row, colIDs, nil, nil, &rd) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) // Decode colMap := make(map[int64]*types.FieldType, len(row)) @@ -96,148 +86,149 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) { colMap[col.id] = col.tp } r, err := DecodeRowToDatumMap(bs, colMap, time.UTC) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r, HasLen, len(row)) + require.NoError(t, err) + require.NotNil(t, r) + require.Len(t, r, len(row)) // Compare decoded row and original row for i, col := range cols { v, ok := r[col.id] - c.Assert(ok, IsTrue) + require.True(t, ok) equal, err1 := v.CompareDatum(sc, &row[i]) - c.Assert(err1, IsNil) - c.Assert(equal, Equals, 0, Commentf("expect: %v, got %v", row[i], v)) + require.NoError(t, err1) + require.Equalf(t, 0, equal, "expect: %v, got %v", row[i], v) } // colMap may contains more columns than encoded row. // colMap[4] = types.NewFieldType(mysql.TypeFloat) r, err = DecodeRowToDatumMap(bs, colMap, time.UTC) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r, HasLen, len(row)) + require.NoError(t, err) + require.NotNil(t, r) + require.Len(t, r, len(row)) for i, col := range cols { v, ok := r[col.id] - c.Assert(ok, IsTrue) + require.True(t, ok) equal, err1 := v.CompareDatum(sc, &row[i]) - c.Assert(err1, IsNil) - c.Assert(equal, Equals, 0) + require.NoError(t, err1) + require.Equal(t, 0, equal) } // colMap may contains less columns than encoded row. delete(colMap, 3) delete(colMap, 4) r, err = DecodeRowToDatumMap(bs, colMap, time.UTC) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r, HasLen, len(row)-2) + require.NoError(t, err) + require.NotNil(t, r) + require.Len(t, r, len(row)-2) for i, col := range cols { if i > 1 { break } v, ok := r[col.id] - c.Assert(ok, IsTrue) + require.True(t, ok) equal, err1 := v.CompareDatum(sc, &row[i]) - c.Assert(err1, IsNil) - c.Assert(equal, Equals, 0) + require.NoError(t, err1) + require.Equal(t, 0, equal) } // Make sure empty row return not nil value. bs, err = EncodeOldRow(sc, []types.Datum{}, []int64{}, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, HasLen, 1) + require.NoError(t, err) + require.Len(t, bs, 1) r, err = DecodeRowToDatumMap(bs, colMap, time.UTC) - c.Assert(err, IsNil) - c.Assert(len(r), Equals, 0) + require.NoError(t, err) + require.Len(t, r, 0) } -func (s *testTableCodecSuite) TestDecodeColumnValue(c *C) { +func TestDecodeColumnValue(t *testing.T) { + t.Parallel() sc := &stmtctx.StatementContext{TimeZone: time.Local} // test timestamp d := types.NewTimeDatum(types.NewTime(types.FromGoTime(time.Now()), mysql.TypeTimestamp, types.DefaultFsp)) bs, err := EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID - c.Assert(err, IsNil) + require.NoError(t, err) tp := types.NewFieldType(mysql.TypeTimestamp) d1, err := DecodeColumnValue(bs, tp, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err := d1.CompareDatum(sc, &d) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, cmp) // test set elems := []string{"a", "b", "c", "d", "e"} e, _ := types.ParseSetValue(elems, uint64(1)) d = types.NewMysqlSetDatum(e, "") bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID - c.Assert(err, IsNil) + require.NoError(t, err) tp = types.NewFieldType(mysql.TypeSet) tp.Elems = elems d1, err = DecodeColumnValue(bs, tp, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err = d1.CompareDatum(sc, &d) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, cmp) // test bit d = types.NewMysqlBitDatum(types.NewBinaryLiteralFromUint(3223600, 3)) bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID - c.Assert(err, IsNil) + require.NoError(t, err) tp = types.NewFieldType(mysql.TypeBit) tp.Flen = 24 d1, err = DecodeColumnValue(bs, tp, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err = d1.CompareDatum(sc, &d) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, cmp) // test empty enum d = types.NewMysqlEnumDatum(types.Enum{}) bs, err = EncodeOldRow(sc, []types.Datum{d}, []int64{1}, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) _, bs, err = codec.CutOne(bs) // ignore colID - c.Assert(err, IsNil) + require.NoError(t, err) tp = types.NewFieldType(mysql.TypeEnum) d1, err = DecodeColumnValue(bs, tp, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err = d1.CompareDatum(sc, &d) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, cmp) } -func (s *testTableCodecSuite) TestUnflattenDatums(c *C) { +func TestUnflattenDatums(t *testing.T) { + t.Parallel() sc := &stmtctx.StatementContext{TimeZone: time.UTC} input := types.MakeDatums(int64(1)) tps := []*types.FieldType{types.NewFieldType(mysql.TypeLonglong)} output, err := UnflattenDatums(input, tps, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err := input[0].CompareDatum(sc, &output[0]) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) + require.NoError(t, err) + require.Equal(t, 0, cmp) input = []types.Datum{types.NewCollationStringDatum("aaa", "utf8mb4_unicode_ci", 0)} tps = []*types.FieldType{types.NewFieldType(mysql.TypeBlob)} tps[0].Collate = "utf8mb4_unicode_ci" output, err = UnflattenDatums(input, tps, sc.TimeZone) - c.Assert(err, IsNil) + require.NoError(t, err) cmp, err = input[0].CompareDatum(sc, &output[0]) - c.Assert(err, IsNil) - c.Assert(cmp, Equals, 0) - c.Assert(output[0].Collation(), Equals, "utf8mb4_unicode_ci") + require.NoError(t, err) + require.Equal(t, 0, cmp) + require.Equal(t, "utf8mb4_unicode_ci", output[0].Collation()) } -func (s *testTableCodecSuite) TestTimeCodec(c *C) { - defer testleak.AfterTest(c)() - +func TestTimeCodec(t *testing.T) { + t.Parallel() c1 := &column{id: 1, tp: types.NewFieldType(mysql.TypeLonglong)} c2 := &column{id: 2, tp: types.NewFieldType(mysql.TypeVarchar)} c3 := &column{id: 3, tp: types.NewFieldType(mysql.TypeTimestamp)} @@ -250,10 +241,10 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { row[1] = types.NewBytesDatum([]byte("abc")) ts, err := types.ParseTimestamp(&stmtctx.StatementContext{TimeZone: time.UTC}, "2016-06-23 11:30:45") - c.Assert(err, IsNil) + require.NoError(t, err) row[2] = types.NewDatum(ts) du, err := types.ParseDuration(nil, "12:59:59.999999", 6) - c.Assert(err, IsNil) + require.NoError(t, err) row[3] = types.NewDatum(du) // Encode @@ -264,8 +255,8 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { rd := rowcodec.Encoder{Enable: true} sc := &stmtctx.StatementContext{TimeZone: time.UTC} bs, err := EncodeRow(sc, row, colIDs, nil, nil, &rd) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) // Decode colMap := make(map[int64]*types.FieldType, colLen) @@ -273,22 +264,21 @@ func (s *testTableCodecSuite) TestTimeCodec(c *C) { colMap[col.id] = col.tp } r, err := DecodeRowToDatumMap(bs, colMap, time.UTC) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r, HasLen, colLen) + require.NoError(t, err) + require.NotNil(t, r) + require.Len(t, r, colLen) // Compare decoded row and original row for i, col := range cols { v, ok := r[col.id] - c.Assert(ok, IsTrue) + require.True(t, ok) equal, err1 := v.CompareDatum(sc, &row[i]) - c.Assert(err1, IsNil) - c.Assert(equal, Equals, 0) + require.Nil(t, err1) + require.Equal(t, 0, equal) } } -func (s *testTableCodecSuite) TestCutRow(c *C) { - defer testleak.AfterTest(c)() - +func TestCutRow(t *testing.T) { + t.Parallel() var err error c1 := &column{id: 1, tp: types.NewFieldType(mysql.TypeLonglong)} c2 := &column{id: 2, tp: types.NewFieldType(mysql.TypeVarchar)} @@ -303,19 +293,19 @@ func (s *testTableCodecSuite) TestCutRow(c *C) { sc := &stmtctx.StatementContext{TimeZone: time.UTC} data := make([][]byte, 3) data[0], err = EncodeValue(sc, nil, row[0]) - c.Assert(err, IsNil) + require.NoError(t, err) data[1], err = EncodeValue(sc, nil, row[1]) - c.Assert(err, IsNil) + require.NoError(t, err) data[2], err = EncodeValue(sc, nil, row[2]) - c.Assert(err, IsNil) + require.NoError(t, err) // Encode colIDs := make([]int64, 0, 3) for _, col := range cols { colIDs = append(colIDs, col.id) } bs, err := EncodeOldRow(sc, row, colIDs, nil, nil) - c.Assert(err, IsNil) - c.Assert(bs, NotNil) + require.NoError(t, err) + require.NotNil(t, bs) // Decode colMap := make(map[int64]int, 3) @@ -323,145 +313,151 @@ func (s *testTableCodecSuite) TestCutRow(c *C) { colMap[col.id] = i } r, err := CutRowNew(bs, colMap) - c.Assert(err, IsNil) - c.Assert(r, NotNil) - c.Assert(r, HasLen, 3) + require.NoError(t, err) + require.NotNil(t, r) + require.Len(t, r, 3) // Compare cut row and original row for i := range colIDs { - c.Assert(r[i], DeepEquals, data[i]) + require.Equal(t, data[i], r[i]) } bs = []byte{codec.NilFlag} r, err = CutRowNew(bs, colMap) - c.Assert(err, IsNil) - c.Assert(r, IsNil) + require.NoError(t, err) + require.Nil(t, r) bs = nil r, err = CutRowNew(bs, colMap) - c.Assert(err, IsNil) - c.Assert(r, IsNil) + require.NoError(t, err) + require.Nil(t, r) } -func (s *testTableCodecSuite) TestCutKeyNew(c *C) { +func TestCutKeyNew(t *testing.T) { + t.Parallel() values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)} handle := types.NewIntDatum(100) values = append(values, handle) sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) - c.Assert(err, IsNil) + require.NoError(t, err) tableID := int64(4) indexID := int64(5) indexKey := EncodeIndexSeekKey(tableID, indexID, encodedValue) valuesBytes, handleBytes, err := CutIndexKeyNew(indexKey, 3) - c.Assert(err, IsNil) + require.NoError(t, err) for i := 0; i < 3; i++ { valueBytes := valuesBytes[i] var val types.Datum _, val, _ = codec.DecodeOne(valueBytes) - c.Assert(val, DeepEquals, values[i]) + require.Equal(t, values[i], val) } _, handleVal, _ := codec.DecodeOne(handleBytes) - c.Assert(handleVal, DeepEquals, types.NewIntDatum(100)) + require.Equal(t, types.NewIntDatum(100), handleVal) } -func (s *testTableCodecSuite) TestCutKey(c *C) { +func TestCutKey(t *testing.T) { + t.Parallel() colIDs := []int64{1, 2, 3} values := []types.Datum{types.NewIntDatum(1), types.NewBytesDatum([]byte("abc")), types.NewFloat64Datum(5.5)} handle := types.NewIntDatum(100) values = append(values, handle) sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) - c.Assert(err, IsNil) + require.NoError(t, err) tableID := int64(4) indexID := int64(5) indexKey := EncodeIndexSeekKey(tableID, indexID, encodedValue) valuesMap, handleBytes, err := CutIndexKey(indexKey, colIDs) - c.Assert(err, IsNil) + require.NoError(t, err) for i, colID := range colIDs { valueBytes := valuesMap[colID] var val types.Datum _, val, _ = codec.DecodeOne(valueBytes) - c.Assert(val, DeepEquals, values[i]) + require.Equal(t, values[i], val) } _, handleVal, _ := codec.DecodeOne(handleBytes) - c.Assert(handleVal, DeepEquals, types.NewIntDatum(100)) + require.Equal(t, types.NewIntDatum(100), handleVal) } -func (s *testTableCodecSuite) TestDecodeBadDecical(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal", `return(true)`), IsNil) +func TestDecodeBadDecical(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal", `return(true)`)) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal"), IsNil) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal")) }() dec := types.NewDecFromStringForTest("0.111") b, err := codec.EncodeDecimal(nil, dec, 0, 0) - c.Assert(err, IsNil) + require.NoError(t, err) // Expect no panic. _, _, err = codec.DecodeOne(b) - c.Assert(err, NotNil) + require.Error(t, err) } -func (s *testTableCodecSuite) TestIndexKey(c *C) { +func TestIndexKey(t *testing.T) { + t.Parallel() tableID := int64(4) indexID := int64(5) indexKey := EncodeIndexSeekKey(tableID, indexID, []byte{}) tTableID, tIndexID, isRecordKey, err := DecodeKeyHead(indexKey) - c.Assert(err, IsNil) - c.Assert(tTableID, Equals, tableID) - c.Assert(tIndexID, Equals, indexID) - c.Assert(isRecordKey, IsFalse) + require.NoError(t, err) + require.Equal(t, tableID, tTableID) + require.Equal(t, indexID, tIndexID) + require.False(t, isRecordKey) } -func (s *testTableCodecSuite) TestRecordKey(c *C) { +func TestRecordKey(t *testing.T) { + t.Parallel() tableID := int64(55) tableKey := EncodeRowKeyWithHandle(tableID, kv.IntHandle(math.MaxUint32)) tTableID, _, isRecordKey, err := DecodeKeyHead(tableKey) - c.Assert(err, IsNil) - c.Assert(tTableID, Equals, tableID) - c.Assert(isRecordKey, IsTrue) + require.NoError(t, err) + require.Equal(t, tableID, tTableID) + require.True(t, isRecordKey) encodedHandle := codec.EncodeInt(nil, math.MaxUint32) rowKey := EncodeRowKey(tableID, encodedHandle) - c.Assert([]byte(tableKey), BytesEquals, []byte(rowKey)) + require.Equal(t, []byte(rowKey), []byte(tableKey)) tTableID, handle, err := DecodeRecordKey(rowKey) - c.Assert(err, IsNil) - c.Assert(tTableID, Equals, tableID) - c.Assert(handle.IntValue(), Equals, int64(math.MaxUint32)) + require.NoError(t, err) + require.Equal(t, tableID, tTableID) + require.Equal(t, int64(math.MaxUint32), handle.IntValue()) recordPrefix := GenTableRecordPrefix(tableID) rowKey = EncodeRecordKey(recordPrefix, kv.IntHandle(math.MaxUint32)) - c.Assert([]byte(tableKey), BytesEquals, []byte(rowKey)) + require.Equal(t, []byte(rowKey), []byte(tableKey)) _, _, err = DecodeRecordKey(nil) - c.Assert(err, NotNil) + require.Error(t, err) _, _, err = DecodeRecordKey([]byte("abcdefghijklmnopqrstuvwxyz")) - c.Assert(err, NotNil) - c.Assert(DecodeTableID(nil), Equals, int64(0)) + require.Error(t, err) + require.Equal(t, int64(0), DecodeTableID(nil)) } -func (s *testTableCodecSuite) TestPrefix(c *C) { +func TestPrefix(t *testing.T) { + t.Parallel() const tableID int64 = 66 key := EncodeTablePrefix(tableID) tTableID := DecodeTableID(key) - c.Assert(tTableID, Equals, tableID) + require.Equal(t, tableID, tTableID) - c.Assert(TablePrefix(), BytesEquals, tablePrefix) + require.Equal(t, tablePrefix, TablePrefix()) tablePrefix1 := GenTablePrefix(tableID) - c.Assert([]byte(tablePrefix1), BytesEquals, []byte(key)) + require.Equal(t, []byte(key), []byte(tablePrefix1)) indexPrefix := EncodeTableIndexPrefix(tableID, math.MaxUint32) tTableID, indexID, isRecordKey, err := DecodeKeyHead(indexPrefix) - c.Assert(err, IsNil) - c.Assert(tTableID, Equals, tableID) - c.Assert(indexID, Equals, int64(math.MaxUint32)) - c.Assert(isRecordKey, IsFalse) + require.NoError(t, err) + require.Equal(t, tableID, tTableID) + require.Equal(t, int64(math.MaxUint32), indexID) + require.False(t, isRecordKey) prefixKey := GenTableIndexPrefix(tableID) - c.Assert(DecodeTableID(prefixKey), Equals, tableID) + require.Equal(t, tableID, DecodeTableID(prefixKey)) - c.Assert(TruncateToRowKeyLen(append(indexPrefix, "xyz"...)), HasLen, RecordRowKeyLen) - c.Assert(TruncateToRowKeyLen(key), HasLen, len(key)) + require.Len(t, TruncateToRowKeyLen(append(indexPrefix, "xyz"...)), RecordRowKeyLen) + require.Len(t, TruncateToRowKeyLen(key), len(key)) } -func (s *testTableCodecSuite) TestDecodeIndexKey(c *C) { +func TestDecodeIndexKey(t *testing.T) { + t.Parallel() tableID := int64(4) indexID := int64(5) values := []types.Datum{ @@ -485,44 +481,47 @@ func (s *testTableCodecSuite) TestDecodeIndexKey(c *C) { } sc := &stmtctx.StatementContext{TimeZone: time.UTC} encodedValue, err := codec.EncodeKey(sc, nil, values...) - c.Assert(err, IsNil) + require.NoError(t, err) indexKey := EncodeIndexSeekKey(tableID, indexID, encodedValue) decodeTableID, decodeIndexID, decodeValues, err := DecodeIndexKey(indexKey) - c.Assert(err, IsNil) - c.Assert(decodeTableID, Equals, tableID) - c.Assert(decodeIndexID, Equals, indexID) - c.Assert(decodeValues, DeepEquals, valueStrs) + require.NoError(t, err) + require.Equal(t, tableID, decodeTableID) + require.Equal(t, indexID, decodeIndexID) + require.Equal(t, valueStrs, decodeValues) } -func (s *testTableCodecSuite) TestCutPrefix(c *C) { +func TestCutPrefix(t *testing.T) { + t.Parallel() key := EncodeTableIndexPrefix(42, 666) res := CutRowKeyPrefix(key) - c.Assert(res, BytesEquals, []byte{0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2, 0x9a}) + require.Equal(t, []byte{0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x2, 0x9a}, res) res = CutIndexPrefix(key) - c.Assert(res, BytesEquals, []byte{}) + require.Equal(t, []byte{}, res) } -func (s *testTableCodecSuite) TestRange(c *C) { +func TestRange(t *testing.T) { + t.Parallel() s1, e1 := GetTableHandleKeyRange(22) s2, e2 := GetTableHandleKeyRange(23) - c.Assert(s1, Less, e1) - c.Assert(e1, Less, s2) - c.Assert(s2, Less, e2) + require.Less(t, string(s1), string(e1)) + require.Less(t, string(e1), string(s2)) + require.Less(t, string(s2), string(e2)) s1, e1 = GetTableIndexKeyRange(42, 666) s2, e2 = GetTableIndexKeyRange(42, 667) - c.Assert(s1, Less, e1) - c.Assert(e1, Less, s2) - c.Assert(s2, Less, e2) + require.Less(t, string(s1), string(e1)) + require.Less(t, string(e1), string(s2)) + require.Less(t, string(s2), string(e2)) } -func (s *testTableCodecSuite) TestDecodeAutoIDMeta(c *C) { +func TestDecodeAutoIDMeta(t *testing.T) { + t.Parallel() keyBytes := []byte{0x6d, 0x44, 0x42, 0x3a, 0x35, 0x36, 0x0, 0x0, 0x0, 0xfc, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x68, 0x54, 0x49, 0x44, 0x3a, 0x31, 0x30, 0x38, 0x0, 0xfe} key, field, err := DecodeMetaKey(keyBytes) - c.Assert(err, IsNil) - c.Assert(string(key), Equals, "DB:56") - c.Assert(string(field), Equals, "TID:108") + require.NoError(t, err) + require.Equal(t, "DB:56", string(key)) + require.Equal(t, "TID:108", string(field)) } func BenchmarkHasTablePrefix(b *testing.B) { @@ -563,7 +562,8 @@ func BenchmarkEncodeValue(b *testing.B) { } } -func (s *testTableCodecSuite) TestError(c *C) { +func TestError(t *testing.T) { + t.Parallel() kvErrs := []*terror.Error{ errInvalidKey, errInvalidRecordKey, @@ -571,12 +571,14 @@ func (s *testTableCodecSuite) TestError(c *C) { } for _, err := range kvErrs { code := terror.ToSQLError(err).Code - c.Assert(code != mysql.ErrUnknown && code == uint16(err.Code()), IsTrue, Commentf("err: %v", err)) + require.NotEqual(t, code, mysql.ErrUnknown) + require.Equal(t, code, uint16(err.Code())) } } -func (s *testTableCodecSuite) TestUntouchedIndexKValue(c *C) { +func TestUntouchedIndexKValue(t *testing.T) { + t.Parallel() untouchedIndexKey := []byte("t00000001_i000000001") untouchedIndexValue := []byte{0, 0, 0, 0, 0, 0, 0, 1, 49} - c.Assert(IsUntouchedIndexKValue(untouchedIndexKey, untouchedIndexValue), IsTrue) + require.True(t, IsUntouchedIndexKValue(untouchedIndexKey, untouchedIndexValue)) } diff --git a/types/json/binary_functions.go b/types/json/binary_functions.go index 897cc3e5d6e9a..d7c9763d09c4c 100644 --- a/types/json/binary_functions.go +++ b/types/json/binary_functions.go @@ -644,7 +644,7 @@ func (bm *binaryModifier) rebuildTo(buf []byte) ([]byte, TypeCode) { // floatEpsilon is the acceptable error quantity when comparing two float numbers. const floatEpsilon = 1.e-8 -// compareFloat64 returns an integer comparing the float64 x to y, +// compareFloat64PrecisionLoss returns an integer comparing the float64 x to y, // allowing precision loss. func compareFloat64PrecisionLoss(x, y float64) int { if x-y < floatEpsilon && y-x < floatEpsilon { @@ -1101,10 +1101,10 @@ func (bj BinaryJSON) Search(containType string, search string, escape byte, path } -// extractCallbackFn: the type of CALLBACK function for extractToCallback +// extractCallbackFn the type of CALLBACK function for extractToCallback type extractCallbackFn func(fullpath PathExpression, bj BinaryJSON) (stop bool, err error) -// extractToCallback: callback alternative of extractTo +// extractToCallback callback alternative of extractTo // would be more effective when walk through the whole JSON is unnecessary // NOTICE: path [0] & [*] for JSON object other than array is INVALID, which is different from extractTo. func (bj BinaryJSON) extractToCallback(pathExpr PathExpression, callbackFn extractCallbackFn, fullpath PathExpression) (stop bool, err error) { diff --git a/util/placementpolicy/policy.go b/util/placementpolicy/policy.go index 4768e76ed43aa..c08b03ecd493f 100644 --- a/util/placementpolicy/policy.go +++ b/util/placementpolicy/policy.go @@ -29,6 +29,7 @@ type PolicyInfo struct { Voters uint64 `json:"voters"` Schedule string `json:"schedule"` Constraints string `json:"constraints"` + LeaderConstraints string `json:"leader_constraints"` LearnerConstraints string `json:"learner_constraints"` FollowerConstraints string `json:"follower_constraints"` VoterConstraints string `json:"voter_constraints"` diff --git a/util/ranger/points.go b/util/ranger/points.go index 672af5d8bcc94..26f821f87286c 100644 --- a/util/ranger/points.go +++ b/util/ranger/points.go @@ -571,7 +571,20 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool) dt.SetString(dt.GetString(), colCollate) } if expr.GetArgs()[0].GetType().Tp == mysql.TypeEnum { - dt, err = dt.ConvertTo(r.sc, expr.GetArgs()[0].GetType()) + switch dt.Kind() { + case types.KindString, types.KindBytes, types.KindBinaryLiteral: + // Can't use ConvertTo directly, since we shouldn't convert numerical string to Enum in select stmt. + targetType := expr.GetArgs()[0].GetType() + enum, parseErr := types.ParseEnumName(targetType.Elems, dt.GetString(), targetType.Collate) + if parseErr == nil { + dt.SetMysqlEnum(enum, targetType.Collate) + } else { + err = parseErr + } + default: + dt, err = dt.ConvertTo(r.sc, expr.GetArgs()[0].GetType()) + } + if err != nil { // in (..., an impossible value (not valid enum), ...), the range is empty, so skip it. continue diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index fd71d23028590..cd2d42572e5f5 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -286,7 +286,7 @@ func TestTableRange(t *testing.T) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) conds := make([]expression.Expression, len(selection.Conditions)) @@ -632,7 +632,7 @@ create table t( ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -825,7 +825,7 @@ create table t( ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -1190,7 +1190,7 @@ func TestColumnRange(t *testing.T) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) sel := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) ds, ok := sel.Children()[0].(*plannercore.DataSource) @@ -1615,7 +1615,7 @@ func TestIndexRangeForYear(t *testing.T) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() @@ -1688,7 +1688,7 @@ func TestPrefixIndexRangeScan(t *testing.T) { ret := &plannercore.PreprocessorReturn{} err = plannercore.Preprocess(sctx, stmts[0], plannercore.WithPreprocessorReturn(ret)) require.NoError(t, err) - p, _, err := plannercore.BuildLogicalPlan(ctx, sctx, stmts[0], ret.InfoSchema) + p, _, err := plannercore.BuildLogicalPlanForTest(ctx, sctx, stmts[0], ret.InfoSchema) require.NoError(t, err) selection := p.(plannercore.LogicalPlan).Children()[0].(*plannercore.LogicalSelection) tbl := selection.Children()[0].(*plannercore.DataSource).TableInfo() diff --git a/util/rowcodec/export_test.go b/util/rowcodec/main_test.go similarity index 87% rename from util/rowcodec/export_test.go rename to util/rowcodec/main_test.go index 47cd7b73eee6f..7a54747ceaf72 100644 --- a/util/rowcodec/export_test.go +++ b/util/rowcodec/main_test.go @@ -1,4 +1,4 @@ -// Copyright 2019 PingCAP, Inc. +// 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. @@ -15,11 +15,20 @@ package rowcodec import ( + "testing" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/testbridge" + "go.uber.org/goleak" ) +func TestMain(m *testing.M) { + testbridge.WorkaroundGoCheckFlags() + goleak.VerifyTestMain(m) +} + // EncodeFromOldRow encodes a row from an old-format row. // this method will be used in test. func EncodeFromOldRow(encoder *Encoder, sc *stmtctx.StatementContext, oldRow, buf []byte) ([]byte, error) { diff --git a/util/rowcodec/rowcodec_test.go b/util/rowcodec/rowcodec_test.go index c7ff304705721..741c3ec7b3f44 100644 --- a/util/rowcodec/rowcodec_test.go +++ b/util/rowcodec/rowcodec_test.go @@ -20,7 +20,6 @@ import ( "testing" "time" - . "github.com/pingcap/check" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -31,33 +30,28 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/rowcodec" + "github.com/stretchr/testify/require" ) -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testSuite{}) - -type testSuite struct{} - type testData struct { id int64 ft *types.FieldType - dt types.Datum - bt types.Datum + input types.Datum + output types.Datum def *types.Datum handle bool } -func (s *testSuite) TestEncodeLargeSmallReuseBug(c *C) { +func TestEncodeLargeSmallReuseBug(t *testing.T) { + t.Parallel() + // reuse one rowcodec.Encoder. var encoder rowcodec.Encoder colFt := types.NewFieldType(mysql.TypeString) largeColID := int64(300) b, err := encoder.Encode(&stmtctx.StatementContext{}, []int64{largeColID}, []types.Datum{types.NewBytesDatum([]byte(""))}, nil) - c.Assert(err, IsNil) + require.NoError(t, err) bDecoder := rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{ { @@ -67,12 +61,12 @@ func (s *testSuite) TestEncodeLargeSmallReuseBug(c *C) { }, }, nil) _, err = bDecoder.DecodeToDatumMap(b, nil) - c.Assert(err, IsNil) + require.NoError(t, err) colFt = types.NewFieldType(mysql.TypeLonglong) smallColID := int64(1) b, err = encoder.Encode(&stmtctx.StatementContext{}, []int64{smallColID}, []types.Datum{types.NewIntDatum(2)}, nil) - c.Assert(err, IsNil) + require.NoError(t, err) bDecoder = rowcodec.NewDatumMapDecoder([]rowcodec.ColInfo{ { @@ -82,172 +76,183 @@ func (s *testSuite) TestEncodeLargeSmallReuseBug(c *C) { }, }, nil) m, err := bDecoder.DecodeToDatumMap(b, nil) - c.Assert(err, IsNil) + require.NoError(t, err) + v := m[smallColID] - c.Assert(v.GetInt64(), Equals, int64(2)) + require.Equal(t, int64(2), v.GetInt64()) } -func (s *testSuite) TestDecodeRowWithHandle(c *C) { +func TestDecodeRowWithHandle(t *testing.T) { + t.Parallel() + handleID := int64(-1) handleValue := int64(10000) - encodeAndDecodeHandle := func(c *C, testData []testData) { - // transform test data into input. - colIDs := make([]int64, 0, len(testData)) - dts := make([]types.Datum, 0, len(testData)) - fts := make([]*types.FieldType, 0, len(testData)) - cols := make([]rowcodec.ColInfo, 0, len(testData)) - handleColFtMap := make(map[int64]*types.FieldType) - for i := range testData { - t := testData[i] - if t.handle { - handleColFtMap[handleID] = t.ft - } else { - colIDs = append(colIDs, t.id) - dts = append(dts, t.dt) - } - fts = append(fts, t.ft) - cols = append(cols, rowcodec.ColInfo{ - ID: t.id, - IsPKHandle: t.handle, - Ft: t.ft, - }) - } - - // test encode input. - var encoder rowcodec.Encoder - sc := new(stmtctx.StatementContext) - sc.TimeZone = time.UTC - newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) - - // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, nil) - c.Assert(err, IsNil) - dm, err = tablecodec.DecodeHandleToDatumMap(kv.IntHandle(handleValue), - []int64{handleID}, handleColFtMap, sc.TimeZone, dm) - c.Assert(err, IsNil) - for _, t := range testData { - d, exists := dm[t.id] - c.Assert(exists, IsTrue) - c.Assert(d, DeepEquals, t.dt) - } - - // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) - chk := chunk.New(fts, 1, 1) - err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(handleValue), chk) - c.Assert(err, IsNil) - chkRow := chk.GetRow(0) - cdt := chkRow.GetDatumRow(fts) - for i, t := range testData { - d := cdt[i] - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else { - c.Assert(d, DeepEquals, t.bt) - } - } - - // decode to old row bytes. - colOffset := make(map[int64]int) - for i, t := range testData { - colOffset[t.id] = i - } - bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) - oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(handleValue), newRow, nil) - c.Assert(err, IsNil) - for i, t := range testData { - remain, d, err := codec.DecodeOne(oldRow[i]) - c.Assert(err, IsNil) - c.Assert(len(remain), Equals, 0) - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else { - c.Assert(d, DeepEquals, t.bt) - } - } - } - - // encode & decode signed int. - testDataSigned := []testData{ + tests := []struct { + name string + testData []testData + }{ { - handleID, - types.NewFieldType(mysql.TypeLonglong), - types.NewIntDatum(handleValue), - types.NewIntDatum(handleValue), - nil, - true, + "signed int", + []testData{ + { + handleID, + types.NewFieldType(mysql.TypeLonglong), + types.NewIntDatum(handleValue), + types.NewIntDatum(handleValue), + nil, + true, + }, + { + 10, + types.NewFieldType(mysql.TypeLonglong), + types.NewIntDatum(1), + types.NewIntDatum(1), + nil, + false, + }, + }, }, { - 10, - types.NewFieldType(mysql.TypeLonglong), - types.NewIntDatum(1), - types.NewIntDatum(1), - nil, - false, + "unsigned int", + []testData{ + { + handleID, + withUnsigned(types.NewFieldType(mysql.TypeLonglong)), + types.NewUintDatum(uint64(handleValue)), + types.NewUintDatum(uint64(handleValue)), // decode as bytes will uint if unsigned. + nil, + true, + }, + { + 10, + types.NewFieldType(mysql.TypeLonglong), + types.NewIntDatum(1), + types.NewIntDatum(1), + nil, + false, + }, + }, }, } - encodeAndDecodeHandle(c, testDataSigned) - // encode & decode unsigned int. - testDataUnsigned := []testData{ - { - handleID, - withUnsigned(types.NewFieldType(mysql.TypeLonglong)), - types.NewUintDatum(uint64(handleValue)), - types.NewUintDatum(uint64(handleValue)), // decode as bytes will uint if unsigned. - nil, - true, - }, - { - 10, - types.NewFieldType(mysql.TypeLonglong), - types.NewIntDatum(1), - types.NewIntDatum(1), - nil, - false, - }, + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + td := test.testData + + // transform test data into input. + colIDs := make([]int64, 0, len(td)) + dts := make([]types.Datum, 0, len(td)) + fts := make([]*types.FieldType, 0, len(td)) + cols := make([]rowcodec.ColInfo, 0, len(td)) + handleColFtMap := make(map[int64]*types.FieldType) + for _, d := range td { + if d.handle { + handleColFtMap[handleID] = d.ft + } else { + colIDs = append(colIDs, d.id) + dts = append(dts, d.input) + } + fts = append(fts, d.ft) + cols = append(cols, rowcodec.ColInfo{ + ID: d.id, + IsPKHandle: d.handle, + Ft: d.ft, + }) + } + + // test encode input. + var encoder rowcodec.Encoder + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + require.NoError(t, err) + + // decode to datum map. + mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, nil) + require.NoError(t, err) + + dm, err = tablecodec.DecodeHandleToDatumMap(kv.IntHandle(handleValue), []int64{handleID}, handleColFtMap, sc.TimeZone, dm) + require.NoError(t, err) + + for _, d := range td { + dat, exists := dm[d.id] + require.True(t, exists) + require.Equal(t, d.input, dat) + } + + // decode to chunk. + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + chk := chunk.New(fts, 1, 1) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(handleValue), chk) + require.NoError(t, err) + + chkRow := chk.GetRow(0) + cdt := chkRow.GetDatumRow(fts) + for i, d := range td { + dat := cdt[i] + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else { + require.Equal(t, d.output, dat) + } + } + + // decode to old row bytes. + colOffset := make(map[int64]int) + for i, t := range td { + colOffset[t.id] = i + } + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(handleValue), newRow, nil) + require.NoError(t, err) + + for i, d := range td { + remain, dat, err := codec.DecodeOne(oldRow[i]) + require.NoError(t, err) + require.Len(t, remain, 0) + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else { + require.Equal(t, d.output, dat) + } + } + }) } - encodeAndDecodeHandle(c, testDataUnsigned) } -func (s *testSuite) TestEncodeKindNullDatum(c *C) { +func TestEncodeKindNullDatum(t *testing.T) { + t.Parallel() + var encoder rowcodec.Encoder sc := new(stmtctx.StatementContext) sc.TimeZone = time.UTC - colIDs := []int64{ - 1, - 2, - } + colIDs := []int64{1, 2} + var nilDt types.Datum nilDt.SetNull() dts := []types.Datum{nilDt, types.NewIntDatum(2)} ft := types.NewFieldType(mysql.TypeLonglong) fts := []*types.FieldType{ft, ft} newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) + require.NoError(t, err) - cols := []rowcodec.ColInfo{{ - ID: 1, - Ft: ft, - }, - { - ID: 2, - Ft: ft, - }} + cols := []rowcodec.ColInfo{{ID: 1, Ft: ft}, {ID: 2, Ft: ft}} cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) + require.NoError(t, err) + chkRow := chk.GetRow(0) cdt := chkRow.GetDatumRow(fts) - c.Assert(cdt[0].IsNull(), Equals, true) - c.Assert(cdt[1].GetInt64(), Equals, int64(2)) + require.True(t, cdt[0].IsNull()) + require.Equal(t, int64(2), cdt[1].GetInt64()) } -func (s *testSuite) TestDecodeDecimalFspNotMatch(c *C) { +func TestDecodeDecimalFspNotMatch(t *testing.T) { + t.Parallel() + var encoder rowcodec.Encoder sc := new(stmtctx.StatementContext) sc.TimeZone = time.UTC @@ -260,7 +265,7 @@ func (s *testSuite) TestDecodeDecimalFspNotMatch(c *C) { ft.Decimal = 4 fts := []*types.FieldType{ft} newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) + require.NoError(t, err) // decode to chunk. ft = types.NewFieldType(mysql.TypeNewDecimal) @@ -273,17 +278,20 @@ func (s *testSuite) TestDecodeDecimalFspNotMatch(c *C) { cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) chk := chunk.New(fts, 1, 1) err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) + require.NoError(t, err) + chkRow := chk.GetRow(0) cdt := chkRow.GetDatumRow(fts) dec = withFrac(3)(withLen(6)(types.NewDecimalDatum(types.NewDecFromStringForTest("11.990")))) - c.Assert(cdt[0].GetMysqlDecimal().String(), DeepEquals, dec.GetMysqlDecimal().String()) + require.Equal(t, dec.GetMysqlDecimal().String(), cdt[0].GetMysqlDecimal().String()) } -func (s *testSuite) TestTypesNewRowCodec(c *C) { +func TestTypesNewRowCodec(t *testing.T) { + t.Parallel() + getJSONDatum := func(value string) types.Datum { j, err := json.ParseBinaryFromString(value) - c.Assert(err, IsNil) + require.NoError(t, err) var d types.Datum d.SetMysqlJSON(j) return d @@ -294,85 +302,12 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { return d } getTime := func(value string) types.Time { - t, err := types.ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, value, mysql.TypeTimestamp, 6) - c.Assert(err, IsNil) - return t - } - - var encoder rowcodec.Encoder - encodeAndDecode := func(c *C, testData []testData) { - // transform test data into input. - colIDs := make([]int64, 0, len(testData)) - dts := make([]types.Datum, 0, len(testData)) - fts := make([]*types.FieldType, 0, len(testData)) - cols := make([]rowcodec.ColInfo, 0, len(testData)) - for i := range testData { - t := testData[i] - colIDs = append(colIDs, t.id) - dts = append(dts, t.dt) - fts = append(fts, t.ft) - cols = append(cols, rowcodec.ColInfo{ - ID: t.id, - IsPKHandle: t.handle, - Ft: t.ft, - }) - } - - // test encode input. - sc := new(stmtctx.StatementContext) - sc.TimeZone = time.UTC - newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) - - // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, nil) - c.Assert(err, IsNil) - for _, t := range testData { - d, exists := dm[t.id] - c.Assert(exists, IsTrue) - c.Assert(d, DeepEquals, t.dt) - } - - // decode to chunk. - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) - chk := chunk.New(fts, 1, 1) - err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) - chkRow := chk.GetRow(0) - cdt := chkRow.GetDatumRow(fts) - for i, t := range testData { - d := cdt[i] - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else { - c.Assert(d, DeepEquals, t.dt) - } - } - - // decode to old row bytes. - colOffset := make(map[int64]int) - for i, t := range testData { - colOffset[t.id] = i - } - bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) - oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) - c.Assert(err, IsNil) - for i, t := range testData { - remain, d, err := codec.DecodeOne(oldRow[i]) - c.Assert(err, IsNil) - c.Assert(len(remain), Equals, 0) - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else if d.Kind() == types.KindBytes { - c.Assert(d.GetBytes(), DeepEquals, t.bt.GetBytes()) - } else { - c.Assert(d, DeepEquals, t.bt) - } - } + d, err := types.ParseTime(&stmtctx.StatementContext{TimeZone: time.UTC}, value, mysql.TypeTimestamp, 6) + require.NoError(t, err) + return d } - testData := []testData{ + smallTestDataList := []testData{ { 1, types.NewFieldType(mysql.TypeLonglong), @@ -519,128 +454,118 @@ func (s *testSuite) TestTypesNewRowCodec(c *C) { }, } - // test small - encodeAndDecode(c, testData) + largeColIDTestDataList := make([]testData, len(smallTestDataList)) + copy(largeColIDTestDataList, smallTestDataList) + largeColIDTestDataList[0].id = 300 - // test large colID - testData[0].id = 300 - encodeAndDecode(c, testData) - testData[0].id = 1 + largeTestDataList := make([]testData, len(smallTestDataList)) + copy(largeTestDataList, smallTestDataList) + largeTestDataList[3].input = types.NewStringDatum(strings.Repeat("a", math.MaxUint16+1)) + largeTestDataList[3].output = types.NewStringDatum(strings.Repeat("a", math.MaxUint16+1)) - // test large data - testData[3].dt = types.NewStringDatum(strings.Repeat("a", math.MaxUint16+1)) - testData[3].bt = types.NewStringDatum(strings.Repeat("a", math.MaxUint16+1)) - encodeAndDecode(c, testData) -} + var encoder rowcodec.Encoder -func (s *testSuite) TestNilAndDefault(c *C) { - encodeAndDecode := func(c *C, testData []testData) { - // transform test data into input. - colIDs := make([]int64, 0, len(testData)) - dts := make([]types.Datum, 0, len(testData)) - cols := make([]rowcodec.ColInfo, 0, len(testData)) - fts := make([]*types.FieldType, 0, len(testData)) - for i := range testData { - t := testData[i] - if t.def == nil { - colIDs = append(colIDs, t.id) - dts = append(dts, t.dt) - } - fts = append(fts, t.ft) - cols = append(cols, rowcodec.ColInfo{ - ID: t.id, - IsPKHandle: t.handle, - Ft: t.ft, - }) - } - ddf := func(i int, chk *chunk.Chunk) error { - t := testData[i] - if t.def == nil { - chk.AppendNull(i) - return nil - } - chk.AppendDatum(i, t.def) - return nil - } - bdf := func(i int) ([]byte, error) { - t := testData[i] - if t.def == nil { - return nil, nil - } - return getOldDatumByte(*t.def), nil - } - // test encode input. - var encoder rowcodec.Encoder - sc := new(stmtctx.StatementContext) - sc.TimeZone = time.UTC - newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) - - // decode to datum map. - mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) - dm, err := mDecoder.DecodeToDatumMap(newRow, nil) - c.Assert(err, IsNil) - for _, t := range testData { - d, exists := dm[t.id] - if t.def != nil { - // for datum should not fill default value. - c.Assert(exists, IsFalse) - } else { - c.Assert(exists, IsTrue) - c.Assert(d, DeepEquals, t.bt) + tests := []struct { + name string + testData []testData + }{ + { + "small", + smallTestDataList, + }, + { + "largeColID", + largeColIDTestDataList, + }, + { + "largeData", + largeTestDataList, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + td := test.testData + + // transform test data into input. + colIDs := make([]int64, 0, len(td)) + dts := make([]types.Datum, 0, len(td)) + fts := make([]*types.FieldType, 0, len(td)) + cols := make([]rowcodec.ColInfo, 0, len(td)) + for _, d := range td { + colIDs = append(colIDs, d.id) + dts = append(dts, d.input) + fts = append(fts, d.ft) + cols = append(cols, rowcodec.ColInfo{ + ID: d.id, + IsPKHandle: d.handle, + Ft: d.ft, + }) } - } - // decode to chunk. - chk := chunk.New(fts, 1, 1) - cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, ddf, sc.TimeZone) - err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) - chkRow := chk.GetRow(0) - cdt := chkRow.GetDatumRow(fts) - for i, t := range testData { - d := cdt[i] - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else { - c.Assert(d, DeepEquals, t.bt) + // test encode input. + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + require.NoError(t, err) + + // decode to datum map. + mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, nil) + require.NoError(t, err) + + for _, d := range td { + dat, exists := dm[d.id] + require.True(t, exists) + require.Equal(t, d.input, dat) } - } - chk = chunk.New(fts, 1, 1) - cDecoder = rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) - err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) - chkRow = chk.GetRow(0) - cdt = chkRow.GetDatumRow(fts) - for i := range testData { - if i == 0 { - continue + // decode to chunk. + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + chk := chunk.New(fts, 1, 1) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + require.NoError(t, err) + + chkRow := chk.GetRow(0) + cdt := chkRow.GetDatumRow(fts) + for i, d := range td { + dat := cdt[i] + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else { + require.Equal(t, d.input, dat) + } } - d := cdt[i] - c.Assert(d.IsNull(), Equals, true) - } - // decode to old row bytes. - colOffset := make(map[int64]int) - for i, t := range testData { - colOffset[t.id] = i - } - bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, bdf, sc.TimeZone) - oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) - c.Assert(err, IsNil) - for i, t := range testData { - remain, d, err := codec.DecodeOne(oldRow[i]) - c.Assert(err, IsNil) - c.Assert(len(remain), Equals, 0) - if d.Kind() == types.KindMysqlDecimal { - c.Assert(d.GetMysqlDecimal(), DeepEquals, t.bt.GetMysqlDecimal()) - } else { - c.Assert(d, DeepEquals, t.bt) + // decode to old row bytes. + colOffset := make(map[int64]int) + for i, t := range td { + colOffset[t.id] = i } - } + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, nil) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) + require.NoError(t, err) + + for i, d := range td { + remain, dat, err := codec.DecodeOne(oldRow[i]) + require.NoError(t, err) + require.Len(t, remain, 0) + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else if dat.Kind() == types.KindBytes { + require.Equal(t, d.output.GetBytes(), dat.GetBytes()) + } else { + require.Equal(t, d.output, dat) + } + } + }) } - dtNilData := []testData{ +} + +func TestNilAndDefault(t *testing.T) { + t.Parallel() + + td := []testData{ { 1, types.NewFieldType(mysql.TypeLonglong), @@ -658,48 +583,122 @@ func (s *testSuite) TestNilAndDefault(c *C) { false, }, } - encodeAndDecode(c, dtNilData) -} -func (s *testSuite) TestVarintCompatibility(c *C) { - encodeAndDecodeByte := func(c *C, testData []testData) { - // transform test data into input. - colIDs := make([]int64, 0, len(testData)) - dts := make([]types.Datum, 0, len(testData)) - cols := make([]rowcodec.ColInfo, 0, len(testData)) - for i := range testData { - t := testData[i] - colIDs = append(colIDs, t.id) - dts = append(dts, t.dt) - cols = append(cols, rowcodec.ColInfo{ - ID: t.id, - IsPKHandle: t.handle, - Ft: t.ft, - }) + // transform test data into input. + colIDs := make([]int64, 0, len(td)) + dts := make([]types.Datum, 0, len(td)) + cols := make([]rowcodec.ColInfo, 0, len(td)) + fts := make([]*types.FieldType, 0, len(td)) + for i := range td { + d := td[i] + if d.def == nil { + colIDs = append(colIDs, d.id) + dts = append(dts, d.input) } + fts = append(fts, d.ft) + cols = append(cols, rowcodec.ColInfo{ + ID: d.id, + IsPKHandle: d.handle, + Ft: d.ft, + }) + } + ddf := func(i int, chk *chunk.Chunk) error { + d := td[i] + if d.def == nil { + chk.AppendNull(i) + return nil + } + chk.AppendDatum(i, d.def) + return nil + } + bdf := func(i int) ([]byte, error) { + d := td[i] + if d.def == nil { + return nil, nil + } + return getOldDatumByte(*d.def), nil + } - // test encode input. - var encoder rowcodec.Encoder - sc := new(stmtctx.StatementContext) - sc.TimeZone = time.UTC - newRow, err := encoder.Encode(sc, colIDs, dts, nil) - c.Assert(err, IsNil) - decoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, sc.TimeZone) - // decode to old row bytes. - colOffset := make(map[int64]int) - for i, t := range testData { - colOffset[t.id] = i + // test encode input. + var encoder rowcodec.Encoder + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + require.NoError(t, err) + + // decode to datum map. + mDecoder := rowcodec.NewDatumMapDecoder(cols, sc.TimeZone) + dm, err := mDecoder.DecodeToDatumMap(newRow, nil) + require.NoError(t, err) + + for _, d := range td { + dat, exists := dm[d.id] + if d.def != nil { + // for datum should not fill default value. + require.False(t, exists) + } else { + require.True(t, exists) + require.Equal(t, d.output, dat) } - oldRow, err := decoder.DecodeToBytes(colOffset, kv.IntHandle(1), newRow, nil) - c.Assert(err, IsNil) - for i, t := range testData { - oldVarint, err := tablecodec.EncodeValue(nil, nil, t.bt) // tablecodec will encode as varint/varuint - c.Assert(err, IsNil) - c.Assert(oldVarint, DeepEquals, oldRow[i]) + } + + // decode to chunk. + chk := chunk.New(fts, 1, 1) + cDecoder := rowcodec.NewChunkDecoder(cols, []int64{-1}, ddf, sc.TimeZone) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + require.NoError(t, err) + + chkRow := chk.GetRow(0) + cdt := chkRow.GetDatumRow(fts) + for i, d := range td { + dat := cdt[i] + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else { + require.Equal(t, d.output, dat) } } - testDataValue := []testData{ + chk = chunk.New(fts, 1, 1) + cDecoder = rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, sc.TimeZone) + err = cDecoder.DecodeToChunk(newRow, kv.IntHandle(-1), chk) + require.NoError(t, err) + + chkRow = chk.GetRow(0) + cdt = chkRow.GetDatumRow(fts) + for i := range td { + if i == 0 { + continue + } + require.True(t, cdt[i].IsNull()) + } + + // decode to old row bytes. + colOffset := make(map[int64]int) + for i, t := range td { + colOffset[t.id] = i + } + bDecoder := rowcodec.NewByteDecoder(cols, []int64{-1}, bdf, sc.TimeZone) + oldRow, err := bDecoder.DecodeToBytes(colOffset, kv.IntHandle(-1), newRow, nil) + require.NoError(t, err) + + for i, d := range td { + remain, dat, err := codec.DecodeOne(oldRow[i]) + require.NoError(t, err) + require.Len(t, remain, 0) + + if dat.Kind() == types.KindMysqlDecimal { + require.Equal(t, d.output.GetMysqlDecimal(), dat.GetMysqlDecimal()) + } else { + require.Equal(t, d.output, dat) + } + } +} + +func TestVarintCompatibility(t *testing.T) { + t.Parallel() + + td := []testData{ { 1, types.NewFieldType(mysql.TypeLonglong), @@ -717,10 +716,47 @@ func (s *testSuite) TestVarintCompatibility(c *C) { false, }, } - encodeAndDecodeByte(c, testDataValue) + + // transform test data into input. + colIDs := make([]int64, 0, len(td)) + dts := make([]types.Datum, 0, len(td)) + cols := make([]rowcodec.ColInfo, 0, len(td)) + for _, d := range td { + colIDs = append(colIDs, d.id) + dts = append(dts, d.input) + cols = append(cols, rowcodec.ColInfo{ + ID: d.id, + IsPKHandle: d.handle, + Ft: d.ft, + }) + } + + // test encode input. + var encoder rowcodec.Encoder + sc := new(stmtctx.StatementContext) + sc.TimeZone = time.UTC + newRow, err := encoder.Encode(sc, colIDs, dts, nil) + require.NoError(t, err) + + decoder := rowcodec.NewByteDecoder(cols, []int64{-1}, nil, sc.TimeZone) + // decode to old row bytes. + colOffset := make(map[int64]int) + for i, t := range td { + colOffset[t.id] = i + } + oldRow, err := decoder.DecodeToBytes(colOffset, kv.IntHandle(1), newRow, nil) + require.NoError(t, err) + + for i, d := range td { + oldVarint, err := tablecodec.EncodeValue(nil, nil, d.output) // tablecodec will encode as varint/varuint + require.NoError(t, err) + require.Equal(t, oldRow[i], oldVarint) + } } -func (s *testSuite) TestCodecUtil(c *C) { +func TestCodecUtil(t *testing.T) { + t.Parallel() + colIDs := []int64{1, 2, 3, 4} tps := make([]*types.FieldType, 4) for i := 0; i < 3; i++ { @@ -729,15 +765,16 @@ func (s *testSuite) TestCodecUtil(c *C) { tps[3] = types.NewFieldType(mysql.TypeNull) sc := new(stmtctx.StatementContext) oldRow, err := tablecodec.EncodeOldRow(sc, types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) - c.Check(err, IsNil) + require.NoError(t, err) + var ( rb rowcodec.Encoder newRow []byte ) newRow, err = rowcodec.EncodeFromOldRow(&rb, nil, oldRow, nil) - c.Assert(err, IsNil) - c.Assert(rowcodec.IsNewFormat(newRow), IsTrue) - c.Assert(rowcodec.IsNewFormat(oldRow), IsFalse) + require.NoError(t, err) + require.True(t, rowcodec.IsNewFormat(newRow)) + require.False(t, rowcodec.IsNewFormat(oldRow)) // test stringer for decoder. var cols = make([]rowcodec.ColInfo, 0, len(tps)) @@ -751,25 +788,27 @@ func (s *testSuite) TestCodecUtil(c *C) { d := rowcodec.NewDecoder(cols, []int64{-1}, nil) // test ColumnIsNull - isNil, err := d.ColumnIsNull(newRow, 4, nil) - c.Assert(err, IsNil) - c.Assert(isNil, IsTrue) - isNil, err = d.ColumnIsNull(newRow, 1, nil) - c.Assert(err, IsNil) - c.Assert(isNil, IsFalse) - isNil, err = d.ColumnIsNull(newRow, 5, nil) - c.Assert(err, IsNil) - c.Assert(isNil, IsTrue) - isNil, err = d.ColumnIsNull(newRow, 5, []byte{1}) - c.Assert(err, IsNil) - c.Assert(isNil, IsFalse) + isNull, err := d.ColumnIsNull(newRow, 4, nil) + require.NoError(t, err) + require.True(t, isNull) + isNull, err = d.ColumnIsNull(newRow, 1, nil) + require.NoError(t, err) + require.False(t, isNull) + isNull, err = d.ColumnIsNull(newRow, 5, nil) + require.NoError(t, err) + require.True(t, isNull) + isNull, err = d.ColumnIsNull(newRow, 5, []byte{1}) + require.NoError(t, err) + require.False(t, isNull) // test isRowKey - c.Assert(rowcodec.IsRowKey([]byte{'b', 't'}), IsFalse) - c.Assert(rowcodec.IsRowKey([]byte{'t', 'r'}), IsFalse) + require.False(t, rowcodec.IsRowKey([]byte{'b', 't'})) + require.False(t, rowcodec.IsRowKey([]byte{'t', 'r'})) } -func (s *testSuite) TestOldRowCodec(c *C) { +func TestOldRowCodec(t *testing.T) { + t.Parallel() + colIDs := []int64{1, 2, 3, 4} tps := make([]*types.FieldType, 4) for i := 0; i < 3; i++ { @@ -778,14 +817,15 @@ func (s *testSuite) TestOldRowCodec(c *C) { tps[3] = types.NewFieldType(mysql.TypeNull) sc := new(stmtctx.StatementContext) oldRow, err := tablecodec.EncodeOldRow(sc, types.MakeDatums(1, 2, 3, nil), colIDs, nil, nil) - c.Check(err, IsNil) + require.NoError(t, err) var ( rb rowcodec.Encoder newRow []byte ) newRow, err = rowcodec.EncodeFromOldRow(&rb, nil, oldRow, nil) - c.Check(err, IsNil) + require.NoError(t, err) + cols := make([]rowcodec.ColInfo, len(tps)) for i, tp := range tps { cols[i] = rowcodec.ColInfo{ @@ -796,14 +836,16 @@ func (s *testSuite) TestOldRowCodec(c *C) { rd := rowcodec.NewChunkDecoder(cols, []int64{-1}, nil, time.Local) chk := chunk.NewChunkWithCapacity(tps, 1) err = rd.DecodeToChunk(newRow, kv.IntHandle(-1), chk) - c.Assert(err, IsNil) + require.NoError(t, err) row := chk.GetRow(0) for i := 0; i < 3; i++ { - c.Assert(row.GetInt64(i), Equals, int64(i)+1) + require.Equal(t, int64(i+1), row.GetInt64(i)) } } -func (s *testSuite) Test65535Bug(c *C) { +func Test65535Bug(t *testing.T) { + t.Parallel() + colIds := []int64{1} tps := make([]*types.FieldType, 1) tps[0] = types.NewFieldType(mysql.TypeString) @@ -811,7 +853,7 @@ func (s *testSuite) Test65535Bug(c *C) { text65535 := strings.Repeat("a", 65535) encode := rowcodec.Encoder{} bd, err := encode.Encode(sc, colIds, []types.Datum{types.NewStringDatum(text65535)}, nil) - c.Check(err, IsNil) + require.NoError(t, err) cols := make([]rowcodec.ColInfo, 1) cols[0] = rowcodec.ColInfo{ @@ -820,9 +862,10 @@ func (s *testSuite) Test65535Bug(c *C) { } dc := rowcodec.NewDatumMapDecoder(cols, nil) result, err := dc.DecodeToDatumMap(bd, nil) - c.Check(err, IsNil) + require.NoError(t, err) + rs := result[1] - c.Check(rs.GetString(), Equals, text65535) + require.Equal(t, text65535, rs.GetString()) } var (