diff --git a/br/pkg/lightning/backend/backend.go b/br/pkg/lightning/backend/backend.go index e6c14728ae9aa..e090cc053dc37 100644 --- a/br/pkg/lightning/backend/backend.go +++ b/br/pkg/lightning/backend/backend.go @@ -152,7 +152,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, regionSplitSize int64) error + ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize, regionSplitKeys int64) error CleanupEngine(ctx context.Context, engineUUID uuid.UUID) error @@ -315,7 +315,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, regionSplitSize int64) error { +func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID, regionSplitSize, regionSplitKeys int64) error { // DO NOT call be.abstract.CloseEngine()! The engine should still be writable after // calling UnsafeImportAndReset(). closedEngine := ClosedEngine{ @@ -325,7 +325,7 @@ func (be Backend) UnsafeImportAndReset(ctx context.Context, engineUUID uuid.UUID uuid: engineUUID, }, } - if err := closedEngine.Import(ctx, regionSplitSize); err != nil { + if err := closedEngine.Import(ctx, regionSplitSize, regionSplitKeys); err != nil { return err } return be.abstract.ResetEngine(ctx, engineUUID) @@ -445,12 +445,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, regionSplitSize int64) error { +func (engine *ClosedEngine) Import(ctx context.Context, regionSplitSize, regionSplitKeys 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, regionSplitSize) + err = engine.backend.ImportEngine(ctx, engine.uuid, regionSplitSize, regionSplitKeys) 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 b42ce1815b70b..d388e74533833 100644 --- a/br/pkg/lightning/backend/backend_test.go +++ b/br/pkg/lightning/backend/backend_test.go @@ -54,7 +54,7 @@ func TestOpenCloseImportCleanUpEngine(t *testing.T) { Return(nil). After(openCall) importCall := s.mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID, gomock.Any()). + ImportEngine(ctx, engineUUID, gomock.Any(), gomock.Any()). Return(nil). After(closeCall) s.mockBackend.EXPECT(). @@ -66,7 +66,7 @@ func TestOpenCloseImportCleanUpEngine(t *testing.T) { require.NoError(t, err) closedEngine, err := engine.Close(ctx, nil) require.NoError(t, err) - err = closedEngine.Import(ctx, 1) + err = closedEngine.Import(ctx, 1, 1) require.NoError(t, err) err = closedEngine.Cleanup(ctx) require.NoError(t, err) @@ -250,12 +250,12 @@ func TestImportFailedNoRetry(t *testing.T) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any(), gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any(), gomock.Any()). Return(errors.Annotate(context.Canceled, "fake unrecoverable import error")) closedEngine, err := s.backend.UnsafeCloseEngine(ctx, nil, "`db`.`table`", 1) require.NoError(t, err) - err = closedEngine.Import(ctx, 1) + err = closedEngine.Import(ctx, 1, 1) require.Error(t, err) require.Regexp(t, "^fake unrecoverable import error", err.Error()) } @@ -268,14 +268,14 @@ func TestImportFailedWithRetry(t *testing.T) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any(), gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any(), gomock.Any()). Return(errors.Annotate(driver.ErrBadConn, "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) require.NoError(t, err) - err = closedEngine.Import(ctx, 1) + err = closedEngine.Import(ctx, 1, 1) require.Error(t, err) require.Contains(t, err.Error(), "fake recoverable import error") } @@ -288,16 +288,16 @@ func TestImportFailedRecovered(t *testing.T) { s.mockBackend.EXPECT().CloseEngine(ctx, nil, gomock.Any()).Return(nil) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any(), gomock.Any()). + ImportEngine(ctx, gomock.Any(), gomock.Any(), gomock.Any()). Return(gmysql.ErrInvalidConn) s.mockBackend.EXPECT(). - ImportEngine(ctx, gomock.Any(), gomock.Any()). + ImportEngine(ctx, gomock.Any(), 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) require.NoError(t, err) - err = closedEngine.Import(ctx, 1) + err = closedEngine.Import(ctx, 1, 1) require.NoError(t, err) } diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index be55fc7eb9cef..ed140ce1c0ef8 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -88,10 +88,6 @@ const ( gRPCKeepAliveTimeout = 5 * time.Minute gRPCBackOffMaxDelay = 10 * time.Minute - // See: https://github.com/tikv/tikv/blob/e030a0aae9622f3774df89c62f21b2171a72a69e/etc/config-template.toml#L360 - // lower the max-key-count to avoid tikv trigger region auto split - regionMaxKeyCount = 1_280_000 - defaultRegionSplitSize = 96 * units.MiB // The max ranges count in a batch to split and scatter. maxBatchSplitRanges = 4096 @@ -823,7 +819,7 @@ func (local *local) WriteToTiKV( // if region-split-size <= 96MiB, we bump the threshold a bit to avoid too many retry split // because the range-properties is not 100% accurate regionMaxSize := regionSplitSize - if regionSplitSize <= defaultRegionSplitSize { + if regionSplitSize <= int64(config.SplitRegionSize) { regionMaxSize = regionSplitSize * 4 / 3 } @@ -1328,7 +1324,7 @@ func (local *local) writeAndIngestByRanges(ctx context.Context, engine *Engine, return allErr } -func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { +func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize, regionSplitKeys int64) error { lf := local.lockEngine(engineUUID, importMutexStateImport) if lf == nil { // skip if engine not exist. See the comment of `CloseEngine` for more detail. @@ -1342,9 +1338,16 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi 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)) + kvRegionSplitSize, kvRegionSplitKeys, err := getRegionSplitSizeKeys(ctx, local.pdCtl.GetPDClient(), local.tls) + if err == nil { + if kvRegionSplitSize > regionSplitSize { + regionSplitSize = kvRegionSplitSize + } + if kvRegionSplitKeys > regionSplitKeys { + regionSplitKeys = kvRegionSplitKeys + } + } else { + log.L().Warn("fail to get region split keys and size", zap.Error(err)) } // split sorted file into range by 96MB size per file @@ -1842,3 +1845,41 @@ func (local *local) EngineFileSizes() (res []backend.EngineFileSize) { }) return } + +func getSplitConfFromStore(ctx context.Context, host string, tls *common.TLS) (int64, int64, error) { + var ( + nested struct { + Coprocessor struct { + RegionSplitSize string `json:"region-split-size"` + RegionSplitKeys int64 `json:"region-split-keys"` + } `json:"coprocessor"` + } + ) + if err := tls.WithHost(host).GetJSON(ctx, "/config", &nested); err != nil { + return 0, 0, errors.Trace(err) + } + splitSize, err := units.FromHumanSize(nested.Coprocessor.RegionSplitSize) + if err != nil { + return 0, 0, errors.Trace(err) + } + + return splitSize, nested.Coprocessor.RegionSplitKeys, nil +} + +func getRegionSplitSizeKeys(ctx context.Context, cli pd.Client, tls *common.TLS) (int64, int64, error) { + stores, err := cli.GetAllStores(ctx, pd.WithExcludeTombstone()) + if err != nil { + return 0, 0, err + } + for _, store := range stores { + if store.StatusAddress == "" || version.IsTiFlash(store) { + continue + } + regionSplitSize, regionSplitKeys, err := getSplitConfFromStore(ctx, store.StatusAddress, tls) + if err == nil { + return regionSplitSize, regionSplitKeys, nil + } + log.L().Warn("get region split size and keys failed", zap.Error(err), zap.String("store", store.StatusAddress)) + } + return 0, 0, errors.New("get region split size and keys failed") +} diff --git a/br/pkg/lightning/backend/noop/noop.go b/br/pkg/lightning/backend/noop/noop.go index 430c4c5a83e8c..2ac3e2b346dbb 100644 --- a/br/pkg/lightning/backend/noop/noop.go +++ b/br/pkg/lightning/backend/noop/noop.go @@ -79,7 +79,7 @@ func (b noopBackend) CloseEngine(ctx context.Context, cfg *backend.EngineConfig, return nil } -func (b noopBackend) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize int64) error { +func (b noopBackend) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regionSplitSize, regionSplitKeys int64) error { return nil } diff --git a/br/pkg/lightning/backend/tidb/tidb.go b/br/pkg/lightning/backend/tidb/tidb.go index fe93c84c249db..9ae70e7afef1d 100644 --- a/br/pkg/lightning/backend/tidb/tidb.go +++ b/br/pkg/lightning/backend/tidb/tidb.go @@ -432,7 +432,7 @@ func (be *tidbBackend) ResolveDuplicateRows(ctx context.Context, tbl table.Table return nil } -func (be *tidbBackend) ImportEngine(context.Context, uuid.UUID, int64) error { +func (be *tidbBackend) ImportEngine(context.Context, uuid.UUID, int64, int64) error { return nil } diff --git a/br/pkg/lightning/config/config.go b/br/pkg/lightning/config/config.go index 1955e11c7e067..fee2aaf29deb2 100644 --- a/br/pkg/lightning/config/config.go +++ b/br/pkg/lightning/config/config.go @@ -523,6 +523,7 @@ type TikvImporter struct { MaxKVPairs int `toml:"max-kv-pairs" json:"max-kv-pairs"` SendKVPairs int `toml:"send-kv-pairs" json:"send-kv-pairs"` RegionSplitSize ByteSize `toml:"region-split-size" json:"region-split-size"` + RegionSplitKeys int `toml:"region-split-keys" json:"region-split-keys"` SortedKVDir string `toml:"sorted-kv-dir" json:"sorted-kv-dir"` DiskQuota ByteSize `toml:"disk-quota" json:"disk-quota"` RangeConcurrency int `toml:"range-concurrency" json:"range-concurrency"` diff --git a/br/pkg/lightning/config/const.go b/br/pkg/lightning/config/const.go index bf807f2fe759a..23a38ac41117d 100644 --- a/br/pkg/lightning/config/const.go +++ b/br/pkg/lightning/config/const.go @@ -20,9 +20,12 @@ import ( const ( // mydumper - ReadBlockSize ByteSize = 64 * units.KiB - MaxRegionSize ByteSize = 256 * units.MiB + ReadBlockSize ByteSize = 64 * units.KiB + MaxRegionSize ByteSize = 256 * units.MiB + // See: https://github.com/tikv/tikv/blob/e030a0aae9622f3774df89c62f21b2171a72a69e/etc/config-template.toml#L360 + // lower the max-key-count to avoid tikv trigger region auto split SplitRegionSize ByteSize = 96 * units.MiB + SplitRegionKeys int = 1_280_000 MaxSplitRegionSizeRatio int = 10 BufferSizeScale = 5 diff --git a/br/pkg/lightning/restore/restore.go b/br/pkg/lightning/restore/restore.go index 464402549803e..c776510ae3c9d 100644 --- a/br/pkg/lightning/restore/restore.go +++ b/br/pkg/lightning/restore/restore.go @@ -1807,7 +1807,7 @@ func (rc *Controller) enforceDiskQuota(ctx context.Context) { var importErr error for _, engine := range largeEngines { // 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 { + if err := rc.backend.UnsafeImportAndReset(ctx, engine, int64(config.SplitRegionSize)*int64(config.MaxSplitRegionSizeRatio), int64(config.SplitRegionKeys)*int64(config.MaxSplitRegionSizeRatio)); err != nil { importErr = multierr.Append(importErr, err) } } diff --git a/br/pkg/lightning/restore/table_restore.go b/br/pkg/lightning/restore/table_restore.go index 48057c0f17607..feedb2d47681e 100644 --- a/br/pkg/lightning/restore/table_restore.go +++ b/br/pkg/lightning/restore/table_restore.go @@ -921,6 +921,8 @@ func (tr *TableRestore) importKV( ) error { task := closedEngine.Logger().Begin(zap.InfoLevel, "import and cleanup engine") regionSplitSize := int64(rc.cfg.TikvImporter.RegionSplitSize) + regionSplitKeys := int64(rc.cfg.TikvImporter.RegionSplitKeys) + if regionSplitSize == 0 && rc.taskMgr != nil { regionSplitSize = int64(config.SplitRegionSize) if err := rc.taskMgr.CheckTasksExclusively(ctx, func(tasks []taskMeta) ([]taskMeta, error) { @@ -932,7 +934,14 @@ func (tr *TableRestore) importKV( return errors.Trace(err) } } - err := closedEngine.Import(ctx, regionSplitSize) + if regionSplitKeys == 0 { + if regionSplitSize > int64(config.SplitRegionSize) { + regionSplitKeys = int64(float64(regionSplitSize) / float64(config.SplitRegionSize) * float64(config.SplitRegionKeys)) + } else { + regionSplitKeys = int64(config.SplitRegionKeys) + } + } + err := closedEngine.Import(ctx, regionSplitSize, regionSplitKeys) saveCpErr := rc.saveStatusCheckpoint(ctx, tr.tableName, engineID, err, checkpoints.CheckpointStatusImported) // Don't clean up when save checkpoint failed, because we will verifyLocalFile and import engine again after restart. if err == nil && saveCpErr == nil { diff --git a/br/pkg/lightning/restore/table_restore_test.go b/br/pkg/lightning/restore/table_restore_test.go index 5114ab98d3187..2083a51f98746 100644 --- a/br/pkg/lightning/restore/table_restore_test.go +++ b/br/pkg/lightning/restore/table_restore_test.go @@ -831,7 +831,7 @@ func (s *tableRestoreSuite) TestImportKVSuccess() { CloseEngine(ctx, nil, engineUUID). Return(nil) mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID, gomock.Any()). + ImportEngine(ctx, engineUUID, gomock.Any(), gomock.Any()). Return(nil) mockBackend.EXPECT(). CleanupEngine(ctx, engineUUID). @@ -866,7 +866,7 @@ func (s *tableRestoreSuite) TestImportKVFailure() { CloseEngine(ctx, nil, engineUUID). Return(nil) mockBackend.EXPECT(). - ImportEngine(ctx, engineUUID, gomock.Any()). + ImportEngine(ctx, engineUUID, gomock.Any(), gomock.Any()). Return(errors.Annotate(context.Canceled, "fake import error")) closedEngine, err := importer.UnsafeCloseEngineWithUUID(ctx, nil, "tag", engineUUID) diff --git a/br/pkg/mock/backend.go b/br/pkg/mock/backend.go index ee8016f3d2921..7eba5180694ac 100644 --- a/br/pkg/mock/backend.go +++ b/br/pkg/mock/backend.go @@ -185,17 +185,17 @@ func (mr *MockBackendMockRecorder) FlushEngine(arg0, arg1 interface{}) *gomock.C } // ImportEngine mocks base method. -func (m *MockBackend) ImportEngine(arg0 context.Context, arg1 uuid.UUID, arg2 int64) error { +func (m *MockBackend) ImportEngine(arg0 context.Context, arg1 uuid.UUID, arg2, arg3 int64) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ImportEngine", arg0, arg1, arg2) + ret := m.ctrl.Call(m, "ImportEngine", arg0, arg1, arg2, arg3) ret0, _ := ret[0].(error) return ret0 } // ImportEngine indicates an expected call of ImportEngine. -func (mr *MockBackendMockRecorder) ImportEngine(arg0, arg1, arg2 interface{}) *gomock.Call { +func (mr *MockBackendMockRecorder) ImportEngine(arg0, arg1, arg2, arg3 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ImportEngine", reflect.TypeOf((*MockBackend)(nil).ImportEngine), arg0, arg1, arg2) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ImportEngine", reflect.TypeOf((*MockBackend)(nil).ImportEngine), arg0, arg1, arg2, arg3) } // LocalWriter mocks base method. diff --git a/dumpling/export/metrics.go b/dumpling/export/metrics.go index cb62caf041228..2a812ea2a06b2 100644 --- a/dumpling/export/metrics.go +++ b/dumpling/export/metrics.go @@ -68,7 +68,7 @@ func InitMetricsVector(labels prometheus.Labels) { Namespace: "dumpling", Subsystem: "write", Name: "receive_chunk_duration_time", - Help: "Bucketed histogram of write time (s) of files", + Help: "Bucketed histogram of receiving time (s) of chunks", Buckets: prometheus.ExponentialBuckets(0.00005, 2, 20), }, labelNames) errorCount = prometheus.NewCounterVec( diff --git a/executor/set_test.go b/executor/set_test.go index cdacc31b905a7..08e2a8b8b53b3 100644 --- a/executor/set_test.go +++ b/executor/set_test.go @@ -594,7 +594,7 @@ func TestSetVar(t *testing.T) { tk.MustQuery("show global variables like 'tidb_ignore_prepared_cache_close_stmt'").Check(testkit.Rows("tidb_ignore_prepared_cache_close_stmt OFF")) // test for tidb_enable_new_cost_interface - tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("0")) // default value is 0 + tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("1")) // default value is 1 tk.MustExec("set global tidb_enable_new_cost_interface=1") tk.MustQuery("select @@global.tidb_enable_new_cost_interface").Check(testkit.Rows("1")) tk.MustQuery("show global variables like 'tidb_enable_new_cost_interface'").Check(testkit.Rows()) // hidden diff --git a/executor/show_stats.go b/executor/show_stats.go index 1f8f6fef0048a..417e285573d75 100644 --- a/executor/show_stats.go +++ b/executor/show_stats.go @@ -426,15 +426,10 @@ func (e *ShowExec) fetchShowStatsHealthy() { } func (e *ShowExec) appendTableForStatsHealthy(dbName, tblName, partitionName string, statsTbl *statistics.Table) { - if statsTbl.Pseudo { + healthy, ok := statsTbl.GetStatsHealthy() + if !ok { return } - var healthy int64 - if statsTbl.ModifyCount < statsTbl.Count { - healthy = int64((1.0 - float64(statsTbl.ModifyCount)/float64(statsTbl.Count)) * 100.0) - } else if statsTbl.ModifyCount == 0 { - healthy = 100 - } e.appendRow([]interface{}{ dbName, tblName, diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 313e51a808929..ff88b30483d2e 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -11575,31 +11575,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiDB managing stats cache by lru", + "description": "TiDB table stats healthy distribution", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { "h": 7, "w": 8, "x": 16, - "y": 89 + "y": 170 }, "hiddenSeries": false, - "id": 246, + "id": 233, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, + "avg": false, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -11609,7 +11607,6 @@ "alertThreshold": true }, "percentage": false, - "pluginVersion": "7.5.11", "pointradius": 5, "points": false, "renderer": "flot", @@ -11620,7 +11617,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tidb_statistics_stats_cache_lru_op{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "expr": "avg(tidb_statistics_stats_healthy{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (type)", "format": "time_series", "interval": "", "intervalFactor": 2, @@ -11633,7 +11630,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Stats Cache LRU OPS", + "title": "Stats Healthy Distribution", "tooltip": { "shared": true, "sort": 0, @@ -11649,7 +11646,6 @@ }, "yaxes": [ { - "$$hashKey": "object:90", "format": "short", "label": null, "logBase": 1, @@ -11658,7 +11654,6 @@ "show": true }, { - "$$hashKey": "object:91", "format": "short", "label": null, "logBase": 1, @@ -11692,10 +11687,10 @@ "h": 7, "w": 12, "x": 0, - "y": 96 + "y": 177 }, "hiddenSeries": false, - "id": 247, + "id": 234, "legend": { "alignAsTable": true, "avg": false, @@ -11786,6 +11781,109 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiDB managing stats cache by lru", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 177 + }, + "hiddenSeries": false, + "id": 235, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_statistics_stats_cache_lru_op{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\"}[1m])) by (type)", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "refId": "A", + "step": 30 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Stats Cache LRU OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:90", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:91", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/metrics/metrics.go b/metrics/metrics.go index b668db426169d..886e5ab81aab9 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -178,6 +178,7 @@ func RegisterMetrics() { prometheus.MustRegister(MemoryUsage) prometheus.MustRegister(StatsCacheLRUCounter) prometheus.MustRegister(StatsCacheLRUGauge) + prometheus.MustRegister(StatsHealthyGauge) tikvmetrics.InitMetrics(TiDB, TiKVClient) tikvmetrics.RegisterMetrics() diff --git a/metrics/stats.go b/metrics/stats.go index 9c54831579f9b..76bd1ec7a936b 100644 --- a/metrics/stats.go +++ b/metrics/stats.go @@ -143,4 +143,11 @@ var ( Name: "stats_cache_lru_val", Help: "gauge of stats cache lru value", }, []string{LblType}) + + StatsHealthyGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Namespace: "tidb", + Subsystem: "statistics", + Name: "stats_healthy", + Help: "Gauge of stats healthy", + }, []string{LblType}) ) diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index cdeb7c5b78233..c720879d15ed4 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -1821,6 +1821,8 @@ func (s *LogicalTableScan) GetPhysicalScan(schema *expression.Schema, stats *pro physicalTableID: ds.physicalTableID, Ranges: s.Ranges, AccessCondition: s.AccessConds, + tblCols: ds.TblCols, + tblColHists: ds.TblColHists, }.Init(s.ctx, s.blockOffset) ts.stats = stats ts.SetSchema(schema.Clone()) @@ -1850,6 +1852,8 @@ func (s *LogicalIndexScan) GetPhysicalIndexScan(schema *expression.Schema, stats dataSourceSchema: ds.schema, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, + tblColHists: ds.TblColHists, + pkIsHandleCol: ds.getPKIsHandleCol(), }.Init(ds.ctx, ds.blockOffset) is.stats = stats is.initSchema(s.FullIdxCols, s.IsDoubleRead) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 1c4b1675b0869..00e9cab055693 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -5398,24 +5398,24 @@ func TestIndexJoinCost(t *testing.T) { `│ └─Selection_17 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_16 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─TableReader_8(Probe) 1.00 3.88 root data:TableRangeScan_7`, - ` └─TableRangeScan_7 1.00 0.00 cop[tikv] table:t_inner_pk range: decided by [test.t_outer.a], keep order:false, stats:pseudo`)) + ` └─TableRangeScan_7 1.00 30.00 cop[tikv] table:t_inner_pk range: decided by [test.t_outer.a], keep order:false, stats:pseudo`)) tk.MustQuery(`explain format=verbose select /*+ TIDB_INLJ(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a`).Check(testkit.Rows( // IndexJoin with inner IndexScan `IndexJoin_10 12487.50 235192.19 root inner join, inner:IndexReader_9, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, `├─TableReader_20(Build) 9990.00 36412.58 root data:Selection_19`, `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_9(Probe) 1.25 5.89 root index:Selection_8`, - ` └─Selection_8 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) + ` └─Selection_8 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_7 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) tk.MustQuery(`explain format=verbose select /*+ TIDB_INLJ(t_outer, t_inner_idx) */ * from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a`).Check(testkit.Rows( // IndexJoin with inner IndexLookup `IndexJoin_11 12487.50 531469.38 root inner join, inner:IndexLookUp_10, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, `├─TableReader_23(Build) 9990.00 36412.58 root data:Selection_22`, `│ └─Selection_22 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_21 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexLookUp_10(Probe) 1.25 35.55 root `, - ` ├─Selection_9(Build) 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` │ └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`, - ` └─TableRowIDScan_8(Probe) 1.25 0.00 cop[tikv] table:t_inner_idx keep order:false, stats:pseudo`)) + ` ├─Selection_9(Build) 1.25 75.08 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` │ └─IndexRangeScan_7 1.25 71.32 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`, + ` └─TableRowIDScan_8(Probe) 1.25 71.25 cop[tikv] table:t_inner_idx keep order:false, stats:pseudo`)) tk.MustQuery("explain format=verbose select /*+ inl_hash_join(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a").Check(testkit.Rows( `IndexHashJoin_12 12487.50 235192.19 root inner join, inner:IndexReader_9, outer key:test.t_outer.a, inner key:test.t_inner_idx.a, equal cond:eq(test.t_outer.a, test.t_inner_idx.a)`, @@ -5423,16 +5423,16 @@ func TestIndexJoinCost(t *testing.T) { `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_9(Probe) 1.25 5.89 root index:Selection_8`, - ` └─Selection_8 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_7 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) + ` └─Selection_8 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_7 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:false, stats:pseudo`)) tk.MustQuery("explain format=verbose select /*+ inl_merge_join(t_outer, t_inner_idx) */ t_inner_idx.a from t_outer, t_inner_idx where t_outer.a=t_inner_idx.a").Check(testkit.Rows( `IndexMergeJoin_17 12487.50 229210.68 root inner join, inner:IndexReader_15, outer key:test.t_outer.a, inner key:test.t_inner_idx.a`, `├─TableReader_20(Build) 9990.00 36412.58 root data:Selection_19`, `│ └─Selection_19 9990.00 465000.00 cop[tikv] not(isnull(test.t_outer.a))`, `│ └─TableFullScan_18 10000.00 435000.00 cop[tikv] table:t_outer keep order:false, stats:pseudo`, `└─IndexReader_15(Probe) 1.25 5.89 root index:Selection_14`, - ` └─Selection_14 1.25 0.00 cop[tikv] not(isnull(test.t_inner_idx.a))`, - ` └─IndexRangeScan_13 1.25 0.00 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:true, stats:pseudo`)) + ` └─Selection_14 1.25 58.18 cop[tikv] not(isnull(test.t_inner_idx.a))`, + ` └─IndexRangeScan_13 1.25 54.43 cop[tikv] table:t_inner_idx, index:a(a) range: decided by [eq(test.t_inner_idx.a, test.t_outer.a)], keep order:true, stats:pseudo`)) } func TestHeuristicIndexSelection(t *testing.T) { diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index f3f200872b93e..fdb3bca061701 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1723,7 +1723,7 @@ "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", "Plan": [ "Selection_6 0.01 8.60 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", - "└─Point_Get_5 1.00 0.00 root table:t2, index:idx_a(a) " + "└─Point_Get_5 1.00 5.60 root table:t2, index:idx_a(a) " ], "Warnings": [ "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" @@ -1780,7 +1780,7 @@ "Plan": [ "Delete_4 N/A N/A root N/A", "└─Selection_7 2.00 9.80 root in(test.t1.g, 3, 4)", - " └─Point_Get_6 1.00 0.00 root table:t1, index:f(f) " + " └─Point_Get_6 1.00 6.80 root table:t1, index:f(f) " ], "Warnings": [ "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" @@ -1816,9 +1816,9 @@ "SQL": "select * from t where a > 1 order by f", "Plan": [ "IndexLookUp_14 3333.33 136747.00 root ", - "├─Selection_13(Build) 3333.33 0.00 cop[tikv] gt(test.t.a, 1)", + "├─Selection_13(Build) 3333.33 585000.00 cop[tikv] gt(test.t.a, 1)", "│ └─IndexFullScan_11 10000.00 555000.00 cop[tikv] table:t, index:f(f) keep order:true, stats:pseudo", - "└─TableRowIDScan_12(Probe) 3333.33 555000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_12(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [{test.t.f asc}], TaskTp: rootTask}" @@ -1829,7 +1829,7 @@ "Plan": [ "IndexLookUp_10 3333.33 86674.83 root ", "├─IndexRangeScan_8(Build) 3333.33 185000.00 cop[tikv] table:t, index:f(f) range:(1,+inf], keep order:false, stats:pseudo", - "└─TableRowIDScan_9(Probe) 3333.33 185000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_9(Probe) 3333.33 370000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f,f_g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1850,8 +1850,8 @@ "Plan": [ "IndexLookUp_15 3.33 206.74 root ", "├─IndexRangeScan_12(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - "└─Selection_14(Probe) 3.33 0.00 cop[tikv] gt(test.t.f, 3)", - " └─TableRowIDScan_13 10.00 570.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─Selection_14(Probe) 3.33 1140.00 cop[tikv] gt(test.t.f, 3)", + " └─TableRowIDScan_13 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,f_g,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1863,7 +1863,7 @@ "Sort_5 10.00 353.68 root test.t.f", "└─IndexLookUp_13 10.00 230.01 root ", " ├─IndexRangeScan_11(Build) 10.00 570.00 cop[tikv] table:t, index:g(g) range:[5,5], keep order:false, stats:pseudo", - " └─TableRowIDScan_12(Probe) 10.00 570.00 cop[tikv] table:t keep order:false, stats:pseudo" + " └─TableRowIDScan_12(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,g] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -1873,9 +1873,9 @@ "SQL": "select * from t where d = 3 order by c, e", "Plan": [ "IndexLookUp_15 10.00 57222.78 root ", - "├─Selection_14(Build) 10.00 0.00 cop[tikv] eq(test.t.d, 3)", + "├─Selection_14(Build) 10.00 855000.00 cop[tikv] eq(test.t.d, 3)", "│ └─IndexFullScan_12 10000.00 825000.00 cop[tikv] table:t, index:c_d_e(c, d, e) keep order:true, stats:pseudo", - "└─TableRowIDScan_13(Probe) 10.00 825000.00 cop[tikv] table:t keep order:false, stats:pseudo" + "└─TableRowIDScan_13(Probe) 10.00 1110.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 [t,c_d_e] remain after pruning paths for t given Prop{SortItems: [{test.t.c asc} {test.t.e asc}], TaskTp: rootTask}" @@ -1932,7 +1932,7 @@ "Plan": [ "IndexLookUp_7 3.00 57.91 root ", "├─IndexRangeScan_5(Build) 3.00 171.00 cop[tikv] table:t, index:idx_b(b) range:(5,+inf], keep order:false", - "└─TableRowIDScan_6(Probe) 3.00 171.00 cop[tikv] table:t keep order:false" + "└─TableRowIDScan_6(Probe) 3.00 108.00 cop[tikv] table:t keep order:false" ], "Warnings": [ "Note 1105 [idx_b] remain after pruning paths for t given Prop{SortItems: [], TaskTp: rootTask}" @@ -2355,7 +2355,7 @@ { "SQL": "explain format = 'verbose' select * from t3 order by a", "Plan": [ - "Sort_4 3.00 45.85 root test.t3.a", + "Sort_4 3.00 26.05 root test.t3.a", "└─TableReader_8 3.00 11.78 root data:TableFullScan_7", " └─TableFullScan_7 3.00 108.00 cop[tikv] table:t3 keep order:false" ] @@ -2373,7 +2373,7 @@ "Plan": [ "TopN_7 1.00 13.22 root test.t3.a, offset:0, count:1", "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.a, offset:0, count:1", + " └─TopN_15 1.00 117.00 cop[tikv] test.t3.a, offset:0, count:1", " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" ] }, @@ -2382,7 +2382,7 @@ "Plan": [ "TopN_7 1.00 13.22 root test.t3.b, offset:0, count:1", "└─TableReader_16 1.00 10.22 root data:TopN_15", - " └─TopN_15 1.00 0.00 cop[tikv] test.t3.b, offset:0, count:1", + " └─TopN_15 1.00 117.00 cop[tikv] test.t3.b, offset:0, count:1", " └─TableFullScan_14 3.00 108.00 cop[tikv] table:t3 keep order:false" ] }, @@ -2394,7 +2394,7 @@ " └─Projection_22 3.00 76.80 mpp[tiflash] Column#4", " └─HashAgg_8 3.00 57.00 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", " └─ExchangeReceiver_21 3.00 48.00 mpp[tiflash] ", - " └─ExchangeSender_20 3.00 48.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─ExchangeSender_20 3.00 45.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", " └─TableFullScan_19 3.00 45.00 mpp[tiflash] table:t2 keep order:false" ] }, @@ -2444,7 +2444,7 @@ " └─ExchangeSender_45 3.00 195.38 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_42 3.00 195.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_21(Build) 3.00 57.00 mpp[tiflash] ", - " │ └─ExchangeSender_20 3.00 57.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─ExchangeSender_20 3.00 54.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_19 3.00 54.00 mpp[tiflash] not(isnull(test.t1.a))", " │ └─TableFullScan_18 3.00 45.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_23(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", @@ -2462,7 +2462,7 @@ " └─ExchangeSender_38 3.00 204.38 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin_29 3.00 204.38 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─ExchangeReceiver_35(Build) 3.00 66.00 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 66.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─ExchangeSender_34 3.00 63.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection_33 3.00 63.00 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " │ └─TableFullScan_32 3.00 54.00 mpp[tiflash] table:t1 keep order:false", " └─Selection_37(Probe) 3.00 54.00 mpp[tiflash] not(isnull(test.t2.a))", diff --git a/session/session_legacy_test.go b/session/session_legacy_test.go index 7fca381118e72..b2ccb156754c2 100644 --- a/session/session_legacy_test.go +++ b/session/session_legacy_test.go @@ -20,20 +20,14 @@ import ( "fmt" "net" "strconv" - "strings" "sync" - "sync/atomic" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" - "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" - "github.com/pingcap/tidb/meta/autoid" - "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -45,8 +39,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/tikv/client-go/v2/testutils" @@ -738,120 +730,6 @@ func (s *testSessionSuite) TestGetSysVariables(c *C) { c.Assert(err.Error(), Equals, "[variable:1238]Variable 'last_insert_id' is a SESSION variable") } -func (s *testSessionSuite) TestRetryResetStmtCtx(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table retrytxn (a int unique, b int)") - tk.MustExec("insert retrytxn values (1, 1)") - tk.MustExec("set @@tidb_disable_txn_auto_retry = 0") - tk.MustExec("begin") - tk.MustExec("update retrytxn set b = b + 1 where a = 1") - - // Make retryable error. - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("update retrytxn set b = b + 1 where a = 1") - - err := tk.Se.CommitTxn(context.TODO()) - c.Assert(err, IsNil) - c.Assert(tk.Se.AffectedRows(), Equals, uint64(1)) -} - -func (s *testSessionSuite) TestRetryCleanTxn(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table retrytxn (a int unique, b int)") - tk.MustExec("insert retrytxn values (1, 1)") - tk.MustExec("begin") - tk.MustExec("update retrytxn set b = b + 1 where a = 1") - - // Make retryable error. - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("update retrytxn set b = b + 1 where a = 1") - - // Hijack retry history, add a statement that returns error. - history := session.GetHistory(tk.Se) - stmtNode, err := parser.New().ParseOneStmt("insert retrytxn values (2, 'a')", "", "") - c.Assert(err, IsNil) - compiler := executor.Compiler{Ctx: tk.Se} - stmt, _ := compiler.Compile(context.TODO(), stmtNode) - executor.ResetContextOfStmt(tk.Se, stmtNode) - history.Add(stmt, tk.Se.GetSessionVars().StmtCtx) - _, err = tk.Exec("commit") - c.Assert(err, NotNil) - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) - c.Assert(txn.Valid(), IsFalse) - c.Assert(tk.Se.GetSessionVars().InTxn(), IsFalse) -} - -func (s *testSessionSuite) TestReadOnlyNotInHistory(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table history (a int)") - tk.MustExec("insert history values (1), (2), (3)") - tk.MustExec("set @@autocommit = 0") - tk.MustExec("set tidb_disable_txn_auto_retry = 0") - tk.MustQuery("select * from history") - history := session.GetHistory(tk.Se) - c.Assert(history.Count(), Equals, 0) - - tk.MustExec("insert history values (4)") - tk.MustExec("insert history values (5)") - c.Assert(history.Count(), Equals, 2) - tk.MustExec("commit") - tk.MustQuery("select * from history") - history = session.GetHistory(tk.Se) - c.Assert(history.Count(), Equals, 0) -} - -func (s *testSessionSuite) TestRetryUnion(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table history (a int)") - tk.MustExec("insert history values (1), (2), (3)") - tk.MustExec("set @@autocommit = 0") - tk.MustExec("set tidb_disable_txn_auto_retry = 0") - // UNION should't be in retry history. - tk.MustQuery("(select * from history) union (select * from history)") - history := session.GetHistory(tk.Se) - c.Assert(history.Count(), Equals, 0) - tk.MustQuery("(select * from history for update) union (select * from history)") - tk.MustExec("update history set a = a + 1") - history = session.GetHistory(tk.Se) - c.Assert(history.Count(), Equals, 2) - - // Make retryable error. - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("update history set a = a + 1") - - _, err := tk.Exec("commit") - c.Assert(err, ErrorMatches, ".*can not retry select for update statement") -} - -func (s *testSessionSuite) TestRetryShow(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set @@autocommit = 0") - tk.MustExec("set tidb_disable_txn_auto_retry = 0") - // UNION should't be in retry history. - tk.MustQuery("show variables") - tk.MustQuery("show databases") - history := session.GetHistory(tk.Se) - c.Assert(history.Count(), Equals, 0) -} - -func (s *testSessionSuite) TestNoRetryForCurrentTxn(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table history (a int)") - tk.MustExec("insert history values (1)") - - // Firstly, disable retry. - tk.MustExec("set tidb_disable_txn_auto_retry = 1") - tk.MustExec("begin") - tk.MustExec("update history set a = 2") - // Enable retry now. - tk.MustExec("set tidb_disable_txn_auto_retry = 0") - - tk1.MustExec("update history set a = 3") - c.Assert(tk.ExecToErr("commit"), NotNil) -} - // TestInTrans . See https://dev.mysql.com/doc/internals/en/status-flags.html func (s *testSessionSuite) TestInTrans(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) @@ -896,31 +774,6 @@ func (s *testSessionSuite) TestInTrans(c *C) { c.Assert(txn.Valid(), IsFalse) } -func (s *testSessionSuite) TestRetryPreparedStmt(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk2 := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("drop table if exists t") - txn, err := tk.Se.Txn(true) - c.Assert(kv.ErrInvalidTxn.Equal(err), IsTrue) - c.Assert(txn.Valid(), IsFalse) - tk.MustExec("create table t (c1 int, c2 int, c3 int)") - tk.MustExec("insert t values (11, 2, 3)") - - tk1.MustExec("set @@tidb_disable_txn_auto_retry = 0") - tk1.MustExec("begin") - tk1.MustExec("update t set c2=? where c1=11;", 21) - - tk2.MustExec("begin") - tk2.MustExec("update t set c2=? where c1=11", 22) - tk2.MustExec("commit") - - tk1.MustExec("commit") - - tk.MustQuery("select c2 from t where c1=11").Check(testkit.Rows("21")) -} - func (s *testSessionSuite) TestSession(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("ROLLBACK;") @@ -977,199 +830,6 @@ func (s *testSessionSuite) TestLastInsertID(c *C) { c.Assert(lastInsertID+2, Equals, currLastInsertID) } -func (s *testSessionSuite) TestAutoIncrementID(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)") - tk.MustExec("insert t values ()") - tk.MustExec("insert t values ()") - tk.MustExec("insert t values ()") - tk.MustExec("drop table if exists t;") - tk.MustExec("create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)") - tk.MustExec("insert t values ()") - lastID := tk.Se.LastInsertID() - c.Assert(lastID, Less, uint64(4)) - tk.MustExec("insert t () values ()") - c.Assert(tk.Se.LastInsertID(), Greater, lastID) - lastID = tk.Se.LastInsertID() - tk.MustExec("insert t values (100)") - c.Assert(tk.Se.LastInsertID(), Equals, uint64(100)) - - // If the auto_increment column value is given, it uses the value of the latest row. - tk.MustExec("insert t values (120), (112)") - c.Assert(tk.Se.LastInsertID(), Equals, uint64(112)) - - // The last_insert_id function only use last auto-generated id. - tk.MustQuery("select last_insert_id()").Check(testkit.Rows(fmt.Sprint(lastID))) - - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (i tinyint unsigned not null auto_increment, primary key (i));") - tk.MustExec("insert into t set i = 254;") - tk.MustExec("insert t values ()") - - // The last insert ID doesn't care about primary key, it is set even if its a normal index column. - tk.MustExec("create table autoid (id int auto_increment, index (id))") - tk.MustExec("insert autoid values ()") - c.Assert(tk.Se.LastInsertID(), Greater, uint64(0)) - tk.MustExec("insert autoid values (100)") - c.Assert(tk.Se.LastInsertID(), Equals, uint64(100)) - - tk.MustQuery("select last_insert_id(20)").Check(testkit.Rows(fmt.Sprint(20))) - tk.MustQuery("select last_insert_id()").Check(testkit.Rows(fmt.Sprint(20))) - - // Corner cases for unsigned bigint auto_increment Columns. - tk.MustExec("drop table if exists autoid") - tk.MustExec("create table autoid(`auto_inc_id` bigint(20) UNSIGNED NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") - tk.MustExec("insert into autoid values(9223372036854775808);") - tk.MustExec("insert into autoid values();") - tk.MustExec("insert into autoid values();") - tk.MustQuery("select * from autoid").Check(testkit.Rows("9223372036854775808", "9223372036854775810", "9223372036854775812")) - // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. - // Using the MaxUint64 and MaxInt64 as the autoID upper limit like MySQL will cause _tidb_rowid allocation fail here. - _, err := tk.Exec("insert into autoid values(18446744073709551614)") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - _, err = tk.Exec("insert into autoid values()") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - // FixMe: MySQL works fine with the this sql. - _, err = tk.Exec("insert into autoid values(18446744073709551615)") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - - tk.MustExec("drop table if exists autoid") - tk.MustExec("create table autoid(`auto_inc_id` bigint(20) UNSIGNED NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") - tk.MustExec("insert into autoid values()") - tk.MustQuery("select * from autoid").Check(testkit.Rows("1")) - tk.MustExec("insert into autoid values(5000)") - tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) - _, err = tk.Exec("update autoid set auto_inc_id = 8000") - c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) - tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000")) - tk.MustExec("insert into autoid values()") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000", "9001")) - - // Corner cases for signed bigint auto_increment Columns. - tk.MustExec("drop table if exists autoid") - tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") - // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. - // Using the MaxUint64 and MaxInt64 as autoID upper limit like MySQL will cause insert fail if the values is - // 9223372036854775806. Because _tidb_rowid will be allocated 9223372036854775807 at same time. - tk.MustExec("insert into autoid values(9223372036854775805);") - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) - _, err = tk.Exec("insert into autoid values();") - c.Assert(terror.ErrorEqual(err, autoid.ErrAutoincReadFailed), IsTrue) - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) - tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index(auto_inc_id)").Check(testkit.Rows("9223372036854775805 9223372036854775806")) - - tk.MustExec("drop table if exists autoid") - tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") - tk.MustExec("insert into autoid values()") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1")) - tk.MustExec("insert into autoid values(5000)") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) - _, err = tk.Exec("update autoid set auto_inc_id = 8000") - c.Assert(terror.ErrorEqual(err, kv.ErrKeyExists), IsTrue) - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) - tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000")) - tk.MustExec("insert into autoid values()") - tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000", "9001")) -} - -func (s *testSessionSuite) TestAutoIncrementWithRetry(c *C) { - // test for https://github.com/pingcap/tidb/issues/827 - - tk := testkit.NewTestKitWithInit(c, s.store) - tk1 := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("set @@tidb_disable_txn_auto_retry = 0") - tk.MustExec("create table t (c2 int, c1 int not null auto_increment, PRIMARY KEY (c1))") - tk.MustExec("insert into t (c2) values (1), (2), (3), (4), (5)") - - // insert values - lastInsertID := tk.Se.LastInsertID() - tk.MustExec("begin") - tk.MustExec("insert into t (c2) values (11), (12), (13)") - tk.MustQuery("select c1 from t where c2 = 11").Check(testkit.Rows("6")) - tk.MustExec("update t set c2 = 33 where c2 = 1") - - tk1.MustExec("update t set c2 = 22 where c2 = 1") - - tk.MustExec("commit") - - tk.MustQuery("select c1 from t where c2 = 11").Check(testkit.Rows("6")) - currLastInsertID := tk.Se.GetSessionVars().StmtCtx.PrevLastInsertID - c.Assert(lastInsertID+5, Equals, currLastInsertID) - - // insert set - lastInsertID = currLastInsertID - tk.MustExec("begin") - tk.MustExec("insert into t set c2 = 31") - tk.MustQuery("select c1 from t where c2 = 31").Check(testkit.Rows("9")) - tk.MustExec("update t set c2 = 44 where c2 = 2") - - tk1.MustExec("update t set c2 = 55 where c2 = 2") - - tk.MustExec("commit") - - tk.MustQuery("select c1 from t where c2 = 31").Check(testkit.Rows("9")) - currLastInsertID = tk.Se.GetSessionVars().StmtCtx.PrevLastInsertID - c.Assert(lastInsertID+3, Equals, currLastInsertID) - - // replace - lastInsertID = currLastInsertID - tk.MustExec("begin") - tk.MustExec("insert into t (c2) values (21), (22), (23)") - tk.MustQuery("select c1 from t where c2 = 21").Check(testkit.Rows("10")) - tk.MustExec("update t set c2 = 66 where c2 = 3") - - tk1.MustExec("update t set c2 = 77 where c2 = 3") - - tk.MustExec("commit") - - tk.MustQuery("select c1 from t where c2 = 21").Check(testkit.Rows("10")) - currLastInsertID = tk.Se.GetSessionVars().StmtCtx.PrevLastInsertID - c.Assert(lastInsertID+1, Equals, currLastInsertID) - - // update - lastInsertID = currLastInsertID - tk.MustExec("begin") - tk.MustExec("insert into t set c2 = 41") - tk.MustExec("update t set c1 = 0 where c2 = 41") - tk.MustQuery("select c1 from t where c2 = 41").Check(testkit.Rows("0")) - tk.MustExec("update t set c2 = 88 where c2 = 4") - - tk1.MustExec("update t set c2 = 99 where c2 = 4") - - tk.MustExec("commit") - - tk.MustQuery("select c1 from t where c2 = 41").Check(testkit.Rows("0")) - currLastInsertID = tk.Se.GetSessionVars().StmtCtx.PrevLastInsertID - c.Assert(lastInsertID+3, Equals, currLastInsertID) - - // prepare - lastInsertID = currLastInsertID - tk.MustExec("begin") - tk.MustExec("prepare stmt from 'insert into t (c2) values (?)'") - tk.MustExec("set @v1=100") - tk.MustExec("set @v2=200") - tk.MustExec("set @v3=300") - tk.MustExec("execute stmt using @v1") - tk.MustExec("execute stmt using @v2") - tk.MustExec("execute stmt using @v3") - tk.MustExec("deallocate prepare stmt") - tk.MustQuery("select c1 from t where c2 = 12").Check(testkit.Rows("7")) - tk.MustExec("update t set c2 = 111 where c2 = 5") - - tk1.MustExec("update t set c2 = 222 where c2 = 5") - - tk.MustExec("commit") - - tk.MustQuery("select c1 from t where c2 = 12").Check(testkit.Rows("7")) - currLastInsertID = tk.Se.GetSessionVars().StmtCtx.PrevLastInsertID - c.Assert(lastInsertID+3, Equals, currLastInsertID) -} - func (s *testSessionSuite) TestBinaryReadOnly(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t (i int key)") @@ -1508,45 +1168,6 @@ func (s *testSessionSuite3) TestISColumns(c *C) { tk.MustQuery("SELECT CHARACTER_SET_NAME FROM INFORMATION_SCHEMA.CHARACTER_SETS WHERE CHARACTER_SET_NAME = 'utf8mb4'").Check(testkit.Rows("utf8mb4")) } -func (s *testSessionSuite2) TestRetry(c *C) { - // For https://github.com/pingcap/tidb/issues/571 - tk := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("begin") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t (c int)") - tk.MustExec("insert t values (1), (2), (3)") - tk.MustExec("commit") - - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk2 := testkit.NewTestKitWithInit(c, s.store) - tk3 := testkit.NewTestKitWithInit(c, s.store) - tk3.MustExec("SET SESSION autocommit=0;") - tk1.MustExec("set @@tidb_disable_txn_auto_retry = 0") - tk2.MustExec("set @@tidb_disable_txn_auto_retry = 0") - tk3.MustExec("set @@tidb_disable_txn_auto_retry = 0") - - var wg util.WaitGroupWrapper - wg.Run(func() { - for i := 0; i < 30; i++ { - tk1.MustExec("update t set c = 1;") - } - }) - wg.Run(func() { - for i := 0; i < 30; i++ { - tk2.MustExec("update t set c = ?;", 1) - } - }) - wg.Run(func() { - for i := 0; i < 30; i++ { - tk3.MustExec("begin") - tk3.MustExec("update t set c = 1;") - tk3.MustExec("commit") - } - }) - wg.Wait() -} - func (s *testSessionSuite3) TestMultiStmts(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("drop table if exists t1; create table t1(id int ); insert into t1 values (1);") @@ -1919,709 +1540,3 @@ func (s *testSessionSuite2) TestStatementErrorInTransaction(c *C) { tk.MustExec("rollback") tk.MustQuery("select * from test where a = 1 and b = 11").Check(testkit.Rows()) } - -func (s *testSessionSuite3) TestCastTimeToDate(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set time_zone = '-8:00'") - date := time.Now().In(time.FixedZone("", -8*int(time.Hour/time.Second))) - tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format("2006-01-02"))) - - tk.MustExec("set time_zone = '+08:00'") - date = time.Now().In(time.FixedZone("", 8*int(time.Hour/time.Second))) - tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format("2006-01-02"))) -} - -func (s *testSessionSuite) TestSetGlobalTZ(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set time_zone = '+08:00'") - tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - - tk.MustExec("set global time_zone = '+00:00'") - - tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) - - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) -} - -func (s *testSessionSuite2) TestRollbackOnCompileError(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table t (a int)") - tk.MustExec("insert t values (1)") - - tk2 := testkit.NewTestKitWithInit(c, s.store) - tk2.MustQuery("select * from t").Check(testkit.Rows("1")) - - tk.MustExec("rename table t to t2") - - var meetErr bool - for i := 0; i < 100; i++ { - _, err := tk2.Exec("insert t values (1)") - if err != nil { - meetErr = true - break - } - } - c.Assert(meetErr, IsTrue) - tk.MustExec("rename table t2 to t") - var recoverErr bool - for i := 0; i < 100; i++ { - _, err := tk2.Exec("insert t values (1)") - if err == nil { - recoverErr = true - break - } - } - c.Assert(recoverErr, IsTrue) -} - -func (s *testSessionSuite3) TestSetTransactionIsolationOneShot(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table t (k int, v int)") - tk.MustExec("insert t values (1, 42)") - tk.MustExec("set tx_isolation = 'read-committed'") - tk.MustQuery("select @@tx_isolation").Check(testkit.Rows("READ-COMMITTED")) - tk.MustExec("set tx_isolation = 'repeatable-read'") - tk.MustExec("set transaction isolation level read committed") - tk.MustQuery("select @@tx_isolation_one_shot").Check(testkit.Rows("READ-COMMITTED")) - tk.MustQuery("select @@tx_isolation").Check(testkit.Rows("REPEATABLE-READ")) - - // Check isolation level is set to read committed. - ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, kv.SI) - }) - tk.Se.Execute(ctx, "select * from t where k = 1") - - // Check it just take effect for one time. - ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { - c.Assert(req.IsolationLevel, Equals, kv.SI) - }) - tk.Se.Execute(ctx, "select * from t where k = 1") - - // Can't change isolation level when it's inside a transaction. - tk.MustExec("begin") - _, err := tk.Se.Execute(ctx, "set transaction isolation level read committed") - c.Assert(err, NotNil) -} - -func (s *testSessionSuite2) TestDBUserNameLength(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table if not exists t (a int)") - // Test user name length can be longer than 16. - tk.MustExec(`CREATE USER 'abcddfjakldfjaldddds'@'%' identified by ''`) - tk.MustExec(`grant all privileges on test.* to 'abcddfjakldfjaldddds'@'%'`) - tk.MustExec(`grant all privileges on test.t to 'abcddfjakldfjaldddds'@'%'`) -} - -func (s *testSessionSuite2) TestHostLengthMax(c *C) { - host1 := strings.Repeat("a", 65) - host2 := strings.Repeat("a", 256) - - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec(fmt.Sprintf(`CREATE USER 'abcddfjakldfjaldddds'@'%s'`, host1)) - - err := tk.ExecToErr(fmt.Sprintf(`CREATE USER 'abcddfjakldfjaldddds'@'%s'`, host2)) - c.Assert(err.Error(), Equals, "[ddl:1470]String 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long for host name (should be no longer than 255)") -} - -func (s *testSessionSuite2) TestCommitRetryCount(c *C) { - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk2 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("create table no_retry (id int)") - tk1.MustExec("insert into no_retry values (1)") - tk1.MustExec("set @@tidb_retry_limit = 0") - - tk1.MustExec("begin") - tk1.MustExec("update no_retry set id = 2") - - tk2.MustExec("begin") - tk2.MustExec("update no_retry set id = 3") - tk2.MustExec("commit") - - // No auto retry because retry limit is set to 0. - _, err := tk1.Se.Execute(context.Background(), "commit") - c.Assert(err, NotNil) -} - -func (s *testSessionSuite3) TestEnablePartition(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("set tidb_enable_table_partition=off") - tk.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition OFF")) - - tk.MustExec("set global tidb_enable_table_partition = on") - - tk.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition OFF")) - tk.MustQuery("show global variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) - - tk.MustExec("set tidb_enable_list_partition=off") - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) - tk.MustExec("set global tidb_enable_list_partition=on") - tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) - - tk.MustExec("set tidb_enable_list_partition=1") - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - - tk.MustExec("set tidb_enable_list_partition=on") - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - - tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - tk.MustExec("set global tidb_enable_list_partition=off") - tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - tk.MustExec("set tidb_enable_list_partition=off") - tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) - - tk.MustExec("set global tidb_enable_list_partition=on") - tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) - tk1.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) -} - -func (s *testSessionSuite2) TestUpdatePrivilege(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists t1, t2;") - tk.MustExec("create table t1 (id int);") - tk.MustExec("create table t2 (id int);") - tk.MustExec("insert into t1 values (1);") - tk.MustExec("insert into t2 values (2);") - tk.MustExec("create user xxx;") - tk.MustExec("grant all on test.t1 to xxx;") - tk.MustExec("grant select on test.t2 to xxx;") - - tk1 := testkit.NewTestKitWithInit(c, s.store) - c.Assert(tk1.Se.Auth(&auth.UserIdentity{Username: "xxx", Hostname: "localhost"}, - []byte(""), - []byte("")), IsTrue) - - _, err := tk1.Exec("update t2 set id = 666 where id = 1;") - c.Assert(err, NotNil) - c.Assert(strings.Contains(err.Error(), "privilege check"), IsTrue) - - // Cover a bug that t1 and t2 both require update privilege. - // In fact, the privlege check for t1 should be update, and for t2 should be select. - _, err = tk1.Exec("update t1,t2 set t1.id = t2.id;") - c.Assert(err, IsNil) - - // Fix issue 8911 - tk.MustExec("create database weperk") - tk.MustExec("use weperk") - tk.MustExec("create table tb_wehub_server (id int, active_count int, used_count int)") - tk.MustExec("create user 'weperk'") - tk.MustExec("grant all privileges on weperk.* to 'weperk'@'%'") - c.Assert(tk1.Se.Auth(&auth.UserIdentity{Username: "weperk", Hostname: "%"}, - []byte(""), []byte("")), IsTrue) - tk1.MustExec("use weperk") - tk1.MustExec("update tb_wehub_server a set a.active_count=a.active_count+1,a.used_count=a.used_count+1 where id=1") - - tk.MustExec("create database service") - tk.MustExec("create database report") - tk.MustExec(`CREATE TABLE service.t1 ( - id int(11) DEFAULT NULL, - a bigint(20) NOT NULL, - b text DEFAULT NULL, - PRIMARY KEY (a) -)`) - tk.MustExec(`CREATE TABLE report.t2 ( - a bigint(20) DEFAULT NULL, - c bigint(20) NOT NULL -)`) - tk.MustExec("grant all privileges on service.* to weperk") - tk.MustExec("grant all privileges on report.* to weperk") - tk1.Se.GetSessionVars().CurrentDB = "" - tk1.MustExec(`update service.t1 s, -report.t2 t -set s.a = t.a -WHERE -s.a = t.a -and t.c >= 1 and t.c <= 10000 -and s.b !='xx';`) - - // Fix issue 10028 - tk.MustExec("create database ap") - tk.MustExec("create database tp") - tk.MustExec("grant all privileges on ap.* to xxx") - tk.MustExec("grant select on tp.* to xxx") - tk.MustExec("create table tp.record( id int,name varchar(128),age int)") - tk.MustExec("insert into tp.record (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18)") - tk.MustExec("create table ap.record( id int,name varchar(128),age int)") - tk.MustExec("insert into ap.record(id) values(1)") - c.Assert(tk1.Se.Auth(&auth.UserIdentity{Username: "xxx", Hostname: "localhost"}, - []byte(""), - []byte("")), IsTrue) - _, err2 := tk1.Exec("update ap.record t inner join tp.record tt on t.id=tt.id set t.name=tt.name") - c.Assert(err2, IsNil) -} - -func (s *testSessionSuite2) TestTxnGoString(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists gostr;") - tk.MustExec("create table gostr (id int);") - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) - str1 := fmt.Sprintf("%#v", txn) - c.Assert(str1, Equals, "Txn{state=invalid}") - tk.MustExec("begin") - txn, err = tk.Se.Txn(false) - c.Assert(err, IsNil) - c.Assert(fmt.Sprintf("%#v", txn), Equals, fmt.Sprintf("Txn{state=valid, txnStartTS=%d}", txn.StartTS())) - - tk.MustExec("insert into gostr values (1)") - c.Assert(fmt.Sprintf("%#v", txn), Equals, fmt.Sprintf("Txn{state=valid, txnStartTS=%d}", txn.StartTS())) - - tk.MustExec("rollback") - c.Assert(fmt.Sprintf("%#v", txn), Equals, "Txn{state=invalid}") -} - -func (s *testSessionSuite3) TestMaxExeucteTime(c *C) { - var err error - tk := testkit.NewTestKit(c, s.store) - tk.Se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - - tk.MustExec("use test") - tk.MustExec("create table MaxExecTime( id int,name varchar(128),age int);") - tk.MustExec("begin") - tk.MustExec("insert into MaxExecTime (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18);") - - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) MAX_EXECUTION_TIME(500) */ * FROM MaxExecTime;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(500)") - c.Assert(tk.Se.GetSessionVars().StmtCtx.HasMaxExecutionTime, Equals, true) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MaxExecutionTime, Equals, uint64(500)) - - tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) - tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) - tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) */ * FROM MaxExecTime;") - - tk.MustExec("set @@global.MAX_EXECUTION_TIME = 300;") - tk.MustQuery("select * FROM MaxExecTime;") - - tk.MustExec("set @@MAX_EXECUTION_TIME = 150;") - tk.MustQuery("select * FROM MaxExecTime;") - - tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("300")) - tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("150")) - - tk.MustExec("set @@global.MAX_EXECUTION_TIME = 0;") - tk.MustExec("set @@MAX_EXECUTION_TIME = 0;") - tk.MustExec("commit") - tk.MustExec("drop table if exists MaxExecTime;") -} - -func (s *testSessionSuite2) TestGrantViewRelated(c *C) { - tkRoot := testkit.NewTestKitWithInit(c, s.store) - tkUser := testkit.NewTestKitWithInit(c, s.store) - - tkRoot.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) - - tkRoot.MustExec("create table if not exists t (a int)") - tkRoot.MustExec("create view v_version29 as select * from t") - tkRoot.MustExec("create user 'u_version29'@'%'") - tkRoot.MustExec("grant select on t to u_version29@'%'") - - tkUser.Se.Auth(&auth.UserIdentity{Username: "u_version29", Hostname: "localhost", CurrentUser: true, AuthUsername: "u_version29", AuthHostname: "%"}, nil, []byte("012345678901234567890")) - - tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) - err := tkUser.ExecToErr("select * from test.v_version29;") - c.Assert(err, NotNil) - tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) - err = tkUser.ExecToErr("create view v_version29_c as select * from t;") - c.Assert(err, NotNil) - - tkRoot.MustExec(`grant show view, select on v_version29 to 'u_version29'@'%'`) - tkRoot.MustQuery("select table_priv from mysql.tables_priv where host='%' and db='test' and user='u_version29' and table_name='v_version29'").Check(testkit.Rows("Select,Show View")) - - tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) - tkUser.MustQuery("show create view v_version29;") - err = tkUser.ExecToErr("create view v_version29_c as select * from v_version29;") - c.Assert(err, NotNil) - - tkRoot.MustExec("create view v_version29_c as select * from v_version29;") - tkRoot.MustExec(`grant create view on v_version29_c to 'u_version29'@'%'`) // Can't grant privilege on a non-exist table/view. - tkRoot.MustQuery("select table_priv from mysql.tables_priv where host='%' and db='test' and user='u_version29' and table_name='v_version29_c'").Check(testkit.Rows("Create View")) - tkRoot.MustExec("drop view v_version29_c") - - tkRoot.MustExec(`grant select on v_version29 to 'u_version29'@'%'`) - tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) - tkUser.MustExec("create view v_version29_c as select * from v_version29;") -} - -func (s *testSessionSuite3) TestLoadClientInteractive(c *C) { - var ( - err error - connectionID uint64 - ) - tk := testkit.NewTestKit(c, s.store) - tk.Se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - id := atomic.AddUint64(&connectionID, 1) - tk.Se.SetConnectionID(id) - tk.Se.GetSessionVars().ClientCapability = tk.Se.GetSessionVars().ClientCapability | mysql.ClientInteractive - tk.MustQuery("select @@wait_timeout").Check(testkit.Rows("28800")) -} - -func (s *testSessionSuite2) TestReplicaRead(c *C) { - var err error - tk := testkit.NewTestKit(c, s.store) - tk.Se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) - tk.MustExec("set @@tidb_replica_read = 'follower';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) - tk.MustExec("set @@tidb_replica_read = 'leader';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) -} - -func (s *testSessionSuite3) TestIsolationRead(c *C) { - var err error - tk := testkit.NewTestKit(c, s.store) - tk.Se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - c.Assert(len(tk.Se.GetSessionVars().GetIsolationReadEngines()), Equals, 3) - tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") - engines := tk.Se.GetSessionVars().GetIsolationReadEngines() - c.Assert(len(engines), Equals, 1) - _, hasTiFlash := engines[kv.TiFlash] - _, hasTiKV := engines[kv.TiKV] - c.Assert(hasTiFlash, Equals, true) - c.Assert(hasTiKV, Equals, false) -} - -func (s *testSessionSuite2) TestStmtHints(c *C) { - var err error - tk := testkit.NewTestKit(c, s.store) - tk.Se, err = session.CreateSession4Test(s.store) - c.Assert(err, IsNil) - - // Test MEMORY_QUOTA hint - tk.MustExec("select /*+ MEMORY_QUOTA(1 MB) */ 1;") - val := int64(1) * 1024 * 1024 - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - tk.MustExec("select /*+ MEMORY_QUOTA(1 GB) */ 1;") - val = int64(1) * 1024 * 1024 * 1024 - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - tk.MustExec("select /*+ MEMORY_QUOTA(1 GB), MEMORY_QUOTA(1 MB) */ 1;") - val = int64(1) * 1024 * 1024 - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - tk.MustExec("select /*+ MEMORY_QUOTA(0 GB) */ 1;") - val = int64(0) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "Setting the MEMORY_QUOTA to 0 means no memory limit") - - tk.MustExec("use test") - tk.MustExec("create table t1(a int);") - tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 (a) values (1);") - val = int64(1) * 1024 * 1024 - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - - tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 select /*+ MEMORY_QUOTA(3 MB) */ * from t1;") - val = int64(1) * 1024 * 1024 - c.Assert(tk.Se.GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val), IsTrue) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "[util:3126]Hint MEMORY_QUOTA(`3145728`) is ignored as conflicting/duplicated.") - - // Test NO_INDEX_MERGE hint - tk.Se.GetSessionVars().SetEnableIndexMerge(true) - tk.MustExec("select /*+ NO_INDEX_MERGE() */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.NoIndexMergeHint, IsTrue) - tk.MustExec("select /*+ NO_INDEX_MERGE(), NO_INDEX_MERGE() */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetEnableIndexMerge(), IsTrue) - - // Test STRAIGHT_JOIN hint - tk.MustExec("select /*+ straight_join() */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.StraightJoinOrder, IsTrue) - tk.MustExec("select /*+ straight_join(), straight_join() */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - - // Test USE_TOJA hint - tk.Se.GetSessionVars().SetAllowInSubqToJoinAndAgg(true) - tk.MustExec("select /*+ USE_TOJA(false) */ 1;") - c.Assert(tk.Se.GetSessionVars().GetAllowInSubqToJoinAndAgg(), IsFalse) - tk.Se.GetSessionVars().SetAllowInSubqToJoinAndAgg(false) - tk.MustExec("select /*+ USE_TOJA(true) */ 1;") - c.Assert(tk.Se.GetSessionVars().GetAllowInSubqToJoinAndAgg(), IsTrue) - tk.MustExec("select /*+ USE_TOJA(false), USE_TOJA(true) */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetAllowInSubqToJoinAndAgg(), IsTrue) - - // Test USE_CASCADES hint - tk.Se.GetSessionVars().SetEnableCascadesPlanner(true) - tk.MustExec("select /*+ USE_CASCADES(false) */ 1;") - c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsFalse) - tk.Se.GetSessionVars().SetEnableCascadesPlanner(false) - tk.MustExec("select /*+ USE_CASCADES(true) */ 1;") - c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsTrue) - tk.MustExec("select /*+ USE_CASCADES(false), USE_CASCADES(true) */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings()[0].Err.Error(), Equals, "USE_CASCADES() is defined more than once, only the last definition takes effect: USE_CASCADES(true)") - c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsTrue) - - // Test READ_CONSISTENT_REPLICA hint - tk.Se.GetSessionVars().SetReplicaRead(kv.ReplicaReadLeader) - tk.MustExec("select /*+ READ_CONSISTENT_REPLICA() */ 1;") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) - tk.MustExec("select /*+ READ_CONSISTENT_REPLICA(), READ_CONSISTENT_REPLICA() */ 1;") - c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) -} - -func (s *testSessionSuite3) TestPessimisticLockOnPartition(c *C) { - // This test checks that 'select ... for update' locks the partition instead of the table. - // Cover a bug that table ID is used to encode the lock key mistakenly. - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec(`create table if not exists forupdate_on_partition ( - age int not null primary key, - nickname varchar(20) not null, - gender int not null default 0, - first_name varchar(30) not null default '', - last_name varchar(20) not null default '', - full_name varchar(60) as (concat(first_name, ' ', last_name)), - index idx_nickname (nickname) -) partition by range (age) ( - partition child values less than (18), - partition young values less than (30), - partition middle values less than (50), - partition old values less than (123) -);`) - tk.MustExec("insert into forupdate_on_partition (`age`, `nickname`) values (25, 'cosven');") - - tk1 := testkit.NewTestKit(c, s.store) - tk1.MustExec("use test") - - tk.MustExec("begin pessimistic") - tk.MustQuery("select * from forupdate_on_partition where age=25 for update").Check(testkit.Rows("25 cosven 0 ")) - tk1.MustExec("begin pessimistic") - - ch := make(chan int32, 5) - go func() { - tk1.MustExec("update forupdate_on_partition set first_name='sw' where age=25") - ch <- 0 - tk1.MustExec("commit") - ch <- 0 - }() - - // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. - time.Sleep(50 * time.Millisecond) - ch <- 1 - - tk.MustExec("commit") - // tk1 should be blocked until tk commit, check the order. - c.Assert(<-ch, Equals, int32(1)) - c.Assert(<-ch, Equals, int32(0)) - <-ch // wait for goroutine to quit. - - // Once again... - // This time, test for the update-update conflict. - tk.MustExec("begin pessimistic") - tk.MustExec("update forupdate_on_partition set first_name='sw' where age=25") - tk1.MustExec("begin pessimistic") - - go func() { - tk1.MustExec("update forupdate_on_partition set first_name = 'xxx' where age=25") - ch <- 0 - tk1.MustExec("commit") - ch <- 0 - }() - - // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. - time.Sleep(50 * time.Millisecond) - ch <- 1 - - tk.MustExec("commit") - // tk1 should be blocked until tk commit, check the order. - c.Assert(<-ch, Equals, int32(1)) - c.Assert(<-ch, Equals, int32(0)) - <-ch // wait for goroutine to quit. -} - -func (s *testSessionSuite2) TestPerStmtTaskID(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("create table task_id (v int)") - - tk.MustExec("begin") - tk.MustExec("select * from task_id where v > 10") - taskID1 := tk.Se.GetSessionVars().StmtCtx.TaskID - tk.MustExec("select * from task_id where v < 5") - taskID2 := tk.Se.GetSessionVars().StmtCtx.TaskID - tk.MustExec("commit") - - c.Assert(taskID1 != taskID2, IsTrue) -} - -func (s *testSessionSuite2) TestSetEnableRateLimitAction(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - // assert default value - result := tk.MustQuery("select @@tidb_enable_rate_limit_action;") - result.Check(testkit.Rows("1")) - tk.MustExec("use test") - tk.MustExec("create table tmp123(id int)") - tk.MustQuery("select * from tmp123;") - haveRateLimitAction := false - action := tk.Se.GetSessionVars().StmtCtx.MemTracker.GetFallbackForTest(false) - for ; action != nil; action = action.GetFallback() { - if action.GetPriority() == memory.DefRateLimitPriority { - haveRateLimitAction = true - break - } - } - c.Assert(haveRateLimitAction, IsTrue) - - // assert set sys variable - tk.MustExec("set global tidb_enable_rate_limit_action= '0';") - tk.Se.Close() - - se, err := session.CreateSession4Test(s.store) - c.Check(err, IsNil) - tk.Se = se - result = tk.MustQuery("select @@tidb_enable_rate_limit_action;") - result.Check(testkit.Rows("0")) - - haveRateLimitAction = false - action = tk.Se.GetSessionVars().StmtCtx.MemTracker.GetFallbackForTest(false) - for ; action != nil; action = action.GetFallback() { - if action.GetPriority() == memory.DefRateLimitPriority { - haveRateLimitAction = true - break - } - } - c.Assert(haveRateLimitAction, IsFalse) -} - -func (s *testSessionSuite2) TestIssue19127(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("drop table if exists issue19127") - tk.MustExec("create table issue19127 (c_int int, c_str varchar(40), primary key (c_int, c_str) ) partition by hash (c_int) partitions 4;") - tk.MustExec("insert into issue19127 values (9, 'angry williams'), (10, 'thirsty hugle');") - tk.Exec("update issue19127 set c_int = c_int + 10, c_str = 'adoring stonebraker' where c_int in (10, 9);") - c.Assert(tk.Se.AffectedRows(), Equals, uint64(2)) -} - -func (s *testSessionSuite2) TestMemoryUsageAlarmVariable(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - - tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=1") - tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("1")) - tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=0") - tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0")) - tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=0.7") - tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0.7")) - tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=1.1") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_memory_usage_alarm_ratio value: '1.1'")) - tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("1")) - - tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=-1") - tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_memory_usage_alarm_ratio value: '-1'")) - tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0")) - - tk.MustExec("set @@session.tidb_memory_usage_alarm_ratio=0.8") - tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_memory_usage_alarm_ratio will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) -} - -func (s *testSessionSuite2) TestSelectLockInShare(c *C) { - tk1 := testkit.NewTestKitWithInit(c, s.store) - tk1.MustExec("DROP TABLE IF EXISTS t_sel_in_share") - tk1.MustExec("CREATE TABLE t_sel_in_share (id int DEFAULT NULL)") - tk1.MustExec("insert into t_sel_in_share values (11)") - err := tk1.ExecToErr("select * from t_sel_in_share lock in share mode") - c.Assert(err, NotNil) - tk1.MustExec("set @@tidb_enable_noop_functions = 1") - tk1.MustQuery("select * from t_sel_in_share lock in share mode").Check(testkit.Rows("11")) - tk1.MustExec("DROP TABLE t_sel_in_share") -} - -func (s *testSessionSuite) TestReadDMLBatchSize(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("set global tidb_dml_batch_size=1000") - se, err := session.CreateSession(s.store) - c.Assert(err, IsNil) - // `select 1` to load the global variables. - _, _ = se.Execute(context.TODO(), "select 1") - c.Assert(se.GetSessionVars().DMLBatchSize, Equals, 1000) -} - -func (s *testSessionSuite) TestInTxnPSProtoPointGet(c *C) { - ctx := context.Background() - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - tk.MustExec("create table t1(c1 int primary key, c2 int, c3 int)") - tk.MustExec("insert into t1 values(1, 10, 100)") - - // Generate the ps statement and make the prepared plan cached for point get. - id, _, _, err := tk.Se.PrepareStmt("select c1, c2 from t1 where c1 = ?") - c.Assert(err, IsNil) - idForUpdate, _, _, err := tk.Se.PrepareStmt("select c1, c2 from t1 where c1 = ? for update") - c.Assert(err, IsNil) - params := []types.Datum{types.NewDatum(1)} - rs, err := tk.Se.ExecutePreparedStmt(ctx, id, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - - // Query again the cached plan will be used. - rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - - // Start a transaction, now the in txn flag will be added to the session vars. - _, err = tk.Se.Execute(ctx, "start transaction") - c.Assert(err, IsNil) - rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - txn, err := tk.Se.Txn(false) - c.Assert(err, IsNil) - c.Assert(txn.Valid(), IsTrue) - rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 10")) - txn, err = tk.Se.Txn(false) - c.Assert(err, IsNil) - c.Assert(txn.Valid(), IsTrue) - _, err = tk.Se.Execute(ctx, "update t1 set c2 = c2 + 1") - c.Assert(err, IsNil) - // Check the read result after in-transaction update. - rs, err = tk.Se.ExecutePreparedStmt(ctx, id, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 11")) - rs, err = tk.Se.ExecutePreparedStmt(ctx, idForUpdate, params) - c.Assert(err, IsNil) - tk.ResultSetToResult(rs, Commentf("%v", rs)).Check(testkit.Rows("1 11")) - txn, err = tk.Se.Txn(false) - c.Assert(err, IsNil) - c.Assert(txn.Valid(), IsTrue) - tk.MustExec("commit") -} - -func (s *testSessionSuite) TestAuthPluginForUser(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("CREATE USER 'tapfu1' IDENTIFIED WITH mysql_native_password BY 'tapfu1'") - plugin, err := tk.Se.AuthPluginForUser(&auth.UserIdentity{Username: "tapfu1", Hostname: `%`}) - c.Assert(err, IsNil) - c.Assert(plugin, Equals, "mysql_native_password") - - tk.MustExec("CREATE USER 'tapfu2' IDENTIFIED WITH mysql_native_password") - plugin, err = tk.Se.AuthPluginForUser(&auth.UserIdentity{Username: "tapfu2", Hostname: `%`}) - c.Assert(err, IsNil) - c.Assert(plugin, Equals, "") - - tk.MustExec("CREATE USER 'tapfu3' IDENTIFIED WITH caching_sha2_password BY 'tapfu3'") - plugin, err = tk.Se.AuthPluginForUser(&auth.UserIdentity{Username: "tapfu3", Hostname: `%`}) - c.Assert(err, IsNil) - c.Assert(plugin, Equals, "caching_sha2_password") - - tk.MustExec("CREATE USER 'tapfu4' IDENTIFIED WITH caching_sha2_password") - plugin, err = tk.Se.AuthPluginForUser(&auth.UserIdentity{Username: "tapfu4", Hostname: `%`}) - c.Assert(err, IsNil) - c.Assert(plugin, Equals, "") -} diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 68207dc0588b0..ca9b52493d53e 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1455,7 +1455,7 @@ var defaultSysVars = []*SysVar{ return nil }, }, - {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNewCostInterface, Value: BoolToOnOff(false), Hidden: true, Type: TypeBool, + {Scope: ScopeGlobal | ScopeSession, Name: TiDBEnableNewCostInterface, Value: BoolToOnOff(true), Hidden: true, Type: TypeBool, SetSession: func(vars *SessionVars, s string) error { vars.EnableNewCostInterface = TiDBOptOn(s) return nil diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 1770a1a15fa7b..cc99a7cfcfcbd 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -418,6 +418,17 @@ func (h *Handle) InitStats(is infoschema.InfoSchema) (err error) { } cache.FreshMemUsage() h.updateStatsCache(cache) + v := h.statsCache.Load() + if v == nil { + return nil + } + healthyChange := &statsHealthyChange{} + for _, tbl := range v.(statsCache).Values() { + if healthy, ok := tbl.GetStatsHealthy(); ok { + healthyChange.add(healthy) + } + } + healthyChange.apply() return nil } diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 88ea07d7ae144..eeb774d016926 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" @@ -44,6 +45,7 @@ import ( "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" + "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" atomic2 "go.uber.org/atomic" "go.uber.org/zap" @@ -240,6 +242,49 @@ func DurationToTS(d time.Duration) uint64 { return oracle.ComposeTS(d.Nanoseconds()/int64(time.Millisecond), 0) } +var statsHealthyGauges = []prometheus.Gauge{ + metrics.StatsHealthyGauge.WithLabelValues("[0,50)"), + metrics.StatsHealthyGauge.WithLabelValues("[50,80)"), + metrics.StatsHealthyGauge.WithLabelValues("[80,100)"), + metrics.StatsHealthyGauge.WithLabelValues("[100,100]"), +} + +type statsHealthyChange struct { + bucketDelta [4]int +} + +func (c *statsHealthyChange) update(add bool, statsHealthy int64) { + var idx int + if statsHealthy < 50 { + idx = 0 + } else if statsHealthy < 80 { + idx = 1 + } else if statsHealthy < 100 { + idx = 2 + } else { + idx = 3 + } + if add { + c.bucketDelta[idx] += 1 + } else { + c.bucketDelta[idx] -= 1 + } +} + +func (c *statsHealthyChange) drop(statsHealthy int64) { + c.update(false, statsHealthy) +} + +func (c *statsHealthyChange) add(statsHealthy int64) { + c.update(true, statsHealthy) +} + +func (c *statsHealthyChange) apply() { + for i, val := range c.bucketDelta { + statsHealthyGauges[i].Add(float64(val)) + } +} + // Update reads stats meta from store and updates the stats map. func (h *Handle) Update(is infoschema.InfoSchema, opts ...TableStatsOpt) error { oldCache := h.statsCache.Load().(statsCache) @@ -260,6 +305,7 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...TableStatsOpt) error { if err != nil { return errors.Trace(err) } + healthyChange := &statsHealthyChange{} option := &tableStatsOption{} for _, opt := range opts { opt(option) @@ -279,7 +325,8 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...TableStatsOpt) error { continue } tableInfo := table.Meta() - if oldTbl, ok := oldCache.Get(physicalID); ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { + oldTbl, ok := oldCache.Get(physicalID) + if ok && oldTbl.Version >= version && tableInfo.UpdateTS == oldTbl.TblInfoUpdateTS { continue } tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, false, 0) @@ -288,6 +335,9 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...TableStatsOpt) error { logutil.BgLogger().Error("[stats] error occurred when read table stats", zap.String("table", tableInfo.Name.O), zap.Error(err)) continue } + if oldHealthy, ok := oldTbl.GetStatsHealthy(); ok { + healthyChange.drop(oldHealthy) + } if tbl == nil { oldCache.Del(physicalID) continue @@ -302,8 +352,15 @@ func (h *Handle) Update(is infoschema.InfoSchema, opts ...TableStatsOpt) error { } else { oldCache.Put(physicalID, tbl) } + oldCache.Put(physicalID, tbl) + if newHealthy, ok := tbl.GetStatsHealthy(); ok { + healthyChange.add(newHealthy) + } + } + updated := h.updateStatsCache(oldCache.update(nil, nil, lastVersion)) + if updated { + healthyChange.apply() } - h.updateStatsCache(oldCache.update(nil, nil, lastVersion)) return nil } diff --git a/statistics/table.go b/statistics/table.go index b75dc179e92f3..86a21cd956442 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -317,6 +317,21 @@ func (t *Table) GetColRowCount() float64 { return -1 } +// GetStatsHealthy calculates stats healthy if the table stats is not pseudo. +// If the table stats is pseudo, it returns 0, false, otherwise it returns stats healthy, ture. +func (t *Table) GetStatsHealthy() (int64, bool) { + if t == nil || t.Pseudo { + return 0, false + } + var healthy int64 + if t.ModifyCount < t.Count { + healthy = int64((1.0 - float64(t.ModifyCount)/float64(t.Count)) * 100.0) + } else if t.ModifyCount == 0 { + healthy = 100 + } + return healthy, true +} + type tableColumnID struct { TableID int64 ColumnID int64 diff --git a/tests/realtikvtest/pessimistictest/pessimistic_test.go b/tests/realtikvtest/pessimistictest/pessimistic_test.go index 9338053d2cb1e..f148edd802be8 100644 --- a/tests/realtikvtest/pessimistictest/pessimistic_test.go +++ b/tests/realtikvtest/pessimistictest/pessimistic_test.go @@ -3129,3 +3129,76 @@ func TestPessimisticAutoCommitTxn(t *testing.T) { explain = fmt.Sprintf("%v", rows[1]) require.Regexp(t, ".*SelectLock.*", explain) } + +func TestPessimisticLockOnPartition(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + // This test checks that 'select ... for update' locks the partition instead of the table. + // Cover a bug that table ID is used to encode the lock key mistakenly. + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table if not exists forupdate_on_partition ( + age int not null primary key, + nickname varchar(20) not null, + gender int not null default 0, + first_name varchar(30) not null default '', + last_name varchar(20) not null default '', + full_name varchar(60) as (concat(first_name, ' ', last_name)), + index idx_nickname (nickname) +) partition by range (age) ( + partition child values less than (18), + partition young values less than (30), + partition middle values less than (50), + partition old values less than (123) +);`) + tk.MustExec("insert into forupdate_on_partition (`age`, `nickname`) values (25, 'cosven');") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from forupdate_on_partition where age=25 for update").Check(testkit.Rows("25 cosven 0 ")) + tk1.MustExec("begin pessimistic") + + ch := make(chan int32, 5) + go func() { + tk1.MustExec("update forupdate_on_partition set first_name='sw' where age=25") + ch <- 0 + tk1.MustExec("commit") + ch <- 0 + }() + + // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. + time.Sleep(50 * time.Millisecond) + ch <- 1 + + tk.MustExec("commit") + // tk1 should be blocked until tk commit, check the order. + require.Equal(t, int32(1), <-ch) + require.Equal(t, int32(0), <-ch) + <-ch // wait for goroutine to quit. + + // Once again... + // This time, test for the update-update conflict. + tk.MustExec("begin pessimistic") + tk.MustExec("update forupdate_on_partition set first_name='sw' where age=25") + tk1.MustExec("begin pessimistic") + + go func() { + tk1.MustExec("update forupdate_on_partition set first_name = 'xxx' where age=25") + ch <- 0 + tk1.MustExec("commit") + ch <- 0 + }() + + // Leave 50ms for tk1 to run, tk1 should be blocked at the update operation. + time.Sleep(50 * time.Millisecond) + ch <- 1 + + tk.MustExec("commit") + // tk1 should be blocked until tk commit, check the order. + require.Equal(t, int32(1), <-ch) + require.Equal(t, int32(0), <-ch) + <-ch // wait for goroutine to quit. +} diff --git a/tests/realtikvtest/sessiontest/retry_test.go b/tests/realtikvtest/sessiontest/retry_test.go new file mode 100644 index 0000000000000..f9cf887a30c3e --- /dev/null +++ b/tests/realtikvtest/sessiontest/retry_test.go @@ -0,0 +1,454 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package sessiontest + +import ( + "context" + "fmt" + "testing" + + "github.com/pingcap/tidb/executor" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/terror" + "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" + "github.com/pingcap/tidb/util" + "github.com/stretchr/testify/require" +) + +func TestRetryShow(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set @@autocommit = 0") + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + // UNION should't be in retry history. + tk.MustQuery("show variables") + tk.MustQuery("show databases") + history := session.GetHistory(tk.Session()) + require.Equal(t, 0, history.Count()) +} + +func TestNoRetryForCurrentTxn(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + tk.MustExec("create table history (a int)") + tk.MustExec("insert history values (1)") + + // Firstly, disable retry. + tk.MustExec("set tidb_disable_txn_auto_retry = 1") + tk.MustExec("begin") + tk.MustExec("update history set a = 2") + // Enable retry now. + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + + tk1.MustExec("update history set a = 3") + require.Error(t, tk.ExecToErr("commit")) +} + +func TestRetryPreparedStmt(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + tk.MustExec("drop table if exists t") + txn, err := tk.Session().Txn(true) + require.True(t, kv.ErrInvalidTxn.Equal(err)) + require.False(t, txn.Valid()) + + tk.MustExec("create table t (c1 int, c2 int, c3 int)") + tk.MustExec("insert t values (11, 2, 3)") + + tk1.MustExec("set @@tidb_disable_txn_auto_retry = 0") + tk1.MustExec("begin") + tk1.MustExec("update t set c2=? where c1=11;", 21) + + tk2.MustExec("begin") + tk2.MustExec("update t set c2=? where c1=11", 22) + tk2.MustExec("commit") + + tk1.MustExec("commit") + + tk.MustQuery("select c2 from t where c1=11").Check(testkit.Rows("21")) +} + +func TestAutoIncrementID(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)") + tk.MustExec("insert t values ()") + tk.MustExec("insert t values ()") + tk.MustExec("insert t values ()") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id BIGINT PRIMARY KEY AUTO_INCREMENT NOT NULL)") + tk.MustExec("insert t values ()") + lastID := tk.Session().LastInsertID() + require.Less(t, lastID, uint64(4)) + + tk.MustExec("insert t () values ()") + require.Greater(t, tk.Session().LastInsertID(), lastID) + lastID = tk.Session().LastInsertID() + tk.MustExec("insert t values (100)") + require.Equal(t, uint64(100), tk.Session().LastInsertID()) + + // If the auto_increment column value is given, it uses the value of the latest row. + tk.MustExec("insert t values (120), (112)") + require.Equal(t, uint64(112), tk.Session().LastInsertID()) + + // The last_insert_id function only use last auto-generated id. + tk.MustQuery("select last_insert_id()").Check(testkit.Rows(fmt.Sprint(lastID))) + + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (i tinyint unsigned not null auto_increment, primary key (i));") + tk.MustExec("insert into t set i = 254;") + tk.MustExec("insert t values ()") + + // The last insert ID doesn't care about primary key, it is set even if its a normal index column. + tk.MustExec("create table autoid (id int auto_increment, index (id))") + tk.MustExec("insert autoid values ()") + require.Greater(t, tk.Session().LastInsertID(), uint64(0)) + tk.MustExec("insert autoid values (100)") + require.Equal(t, uint64(100), tk.Session().LastInsertID()) + + tk.MustQuery("select last_insert_id(20)").Check(testkit.Rows(fmt.Sprint(20))) + tk.MustQuery("select last_insert_id()").Check(testkit.Rows(fmt.Sprint(20))) + + // Corner cases for unsigned bigint auto_increment Columns. + tk.MustExec("drop table if exists autoid") + tk.MustExec("create table autoid(`auto_inc_id` bigint(20) UNSIGNED NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") + tk.MustExec("insert into autoid values(9223372036854775808);") + tk.MustExec("insert into autoid values();") + tk.MustExec("insert into autoid values();") + tk.MustQuery("select * from autoid").Check(testkit.Rows("9223372036854775808", "9223372036854775810", "9223372036854775812")) + // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. + // Using the MaxUint64 and MaxInt64 as the autoID upper limit like MySQL will cause _tidb_rowid allocation fail here. + _, err := tk.Exec("insert into autoid values(18446744073709551614)") + require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) + _, err = tk.Exec("insert into autoid values()") + require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) + // FixMe: MySQL works fine with the this sql. + _, err = tk.Exec("insert into autoid values(18446744073709551615)") + require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) + + tk.MustExec("drop table if exists autoid") + tk.MustExec("create table autoid(`auto_inc_id` bigint(20) UNSIGNED NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1")) + tk.MustExec("insert into autoid values(5000)") + tk.MustQuery("select * from autoid").Check(testkit.Rows("1", "5000")) + _, err = tk.Exec("update autoid set auto_inc_id = 8000") + require.True(t, terror.ErrorEqual(err, kv.ErrKeyExists)) + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) + tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000")) + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000", "9001")) + + // Corner cases for signed bigint auto_increment Columns. + tk.MustExec("drop table if exists autoid") + tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") + // In TiDB : _tidb_rowid will also consume the autoID when the auto_increment column is not the primary key. + // Using the MaxUint64 and MaxInt64 as autoID upper limit like MySQL will cause insert fail if the values is + // 9223372036854775806. Because _tidb_rowid will be allocated 9223372036854775807 at same time. + tk.MustExec("insert into autoid values(9223372036854775805);") + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) + _, err = tk.Exec("insert into autoid values();") + require.True(t, terror.ErrorEqual(err, autoid.ErrAutoincReadFailed)) + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index()").Check(testkit.Rows("9223372036854775805 9223372036854775806")) + tk.MustQuery("select auto_inc_id, _tidb_rowid from autoid use index(auto_inc_id)").Check(testkit.Rows("9223372036854775805 9223372036854775806")) + + tk.MustExec("drop table if exists autoid") + tk.MustExec("create table autoid(`auto_inc_id` bigint(20) NOT NULL AUTO_INCREMENT,UNIQUE KEY `auto_inc_id` (`auto_inc_id`))") + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1")) + tk.MustExec("insert into autoid values(5000)") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) + _, err = tk.Exec("update autoid set auto_inc_id = 8000") + require.True(t, terror.ErrorEqual(err, kv.ErrKeyExists)) + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("1", "5000")) + tk.MustExec("update autoid set auto_inc_id = 9000 where auto_inc_id=1") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000")) + tk.MustExec("insert into autoid values()") + tk.MustQuery("select * from autoid use index()").Check(testkit.Rows("9000", "5000", "9001")) +} + +// test for https://github.com/pingcap/tidb/issues/827 +func TestAutoIncrementWithRetry(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + tk.MustExec("set @@tidb_disable_txn_auto_retry = 0") + tk.MustExec("create table t (c2 int, c1 int not null auto_increment, PRIMARY KEY (c1))") + tk.MustExec("insert into t (c2) values (1), (2), (3), (4), (5)") + + // insert values + lastInsertID := tk.Session().LastInsertID() + tk.MustExec("begin") + tk.MustExec("insert into t (c2) values (11), (12), (13)") + tk.MustQuery("select c1 from t where c2 = 11").Check(testkit.Rows("6")) + tk.MustExec("update t set c2 = 33 where c2 = 1") + + tk1.MustExec("update t set c2 = 22 where c2 = 1") + + tk.MustExec("commit") + + tk.MustQuery("select c1 from t where c2 = 11").Check(testkit.Rows("6")) + currLastInsertID := tk.Session().GetSessionVars().StmtCtx.PrevLastInsertID + require.Equal(t, currLastInsertID, lastInsertID+5) + + // insert set + lastInsertID = currLastInsertID + tk.MustExec("begin") + tk.MustExec("insert into t set c2 = 31") + tk.MustQuery("select c1 from t where c2 = 31").Check(testkit.Rows("9")) + tk.MustExec("update t set c2 = 44 where c2 = 2") + + tk1.MustExec("update t set c2 = 55 where c2 = 2") + + tk.MustExec("commit") + + tk.MustQuery("select c1 from t where c2 = 31").Check(testkit.Rows("9")) + currLastInsertID = tk.Session().GetSessionVars().StmtCtx.PrevLastInsertID + require.Equal(t, currLastInsertID, lastInsertID+3) + + // replace + lastInsertID = currLastInsertID + tk.MustExec("begin") + tk.MustExec("insert into t (c2) values (21), (22), (23)") + tk.MustQuery("select c1 from t where c2 = 21").Check(testkit.Rows("10")) + tk.MustExec("update t set c2 = 66 where c2 = 3") + + tk1.MustExec("update t set c2 = 77 where c2 = 3") + + tk.MustExec("commit") + + tk.MustQuery("select c1 from t where c2 = 21").Check(testkit.Rows("10")) + currLastInsertID = tk.Session().GetSessionVars().StmtCtx.PrevLastInsertID + require.Equal(t, currLastInsertID, lastInsertID+1) + + // update + lastInsertID = currLastInsertID + tk.MustExec("begin") + tk.MustExec("insert into t set c2 = 41") + tk.MustExec("update t set c1 = 0 where c2 = 41") + tk.MustQuery("select c1 from t where c2 = 41").Check(testkit.Rows("0")) + tk.MustExec("update t set c2 = 88 where c2 = 4") + + tk1.MustExec("update t set c2 = 99 where c2 = 4") + + tk.MustExec("commit") + + tk.MustQuery("select c1 from t where c2 = 41").Check(testkit.Rows("0")) + currLastInsertID = tk.Session().GetSessionVars().StmtCtx.PrevLastInsertID + require.Equal(t, currLastInsertID, lastInsertID+3) + + // prepare + lastInsertID = currLastInsertID + tk.MustExec("begin") + tk.MustExec("prepare stmt from 'insert into t (c2) values (?)'") + tk.MustExec("set @v1=100") + tk.MustExec("set @v2=200") + tk.MustExec("set @v3=300") + tk.MustExec("execute stmt using @v1") + tk.MustExec("execute stmt using @v2") + tk.MustExec("execute stmt using @v3") + tk.MustExec("deallocate prepare stmt") + tk.MustQuery("select c1 from t where c2 = 12").Check(testkit.Rows("7")) + tk.MustExec("update t set c2 = 111 where c2 = 5") + + tk1.MustExec("update t set c2 = 222 where c2 = 5") + + tk.MustExec("commit") + + tk.MustQuery("select c1 from t where c2 = 12").Check(testkit.Rows("7")) + currLastInsertID = tk.Session().GetSessionVars().StmtCtx.PrevLastInsertID + require.Equal(t, currLastInsertID, lastInsertID+3) +} + +func TestRetryCleanTxn(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table retrytxn (a int unique, b int)") + tk.MustExec("insert retrytxn values (1, 1)") + tk.MustExec("begin") + tk.MustExec("update retrytxn set b = b + 1 where a = 1") + + // Make retryable error. + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("update retrytxn set b = b + 1 where a = 1") + + // Hijack retry history, add a statement that returns error. + history := session.GetHistory(tk.Session()) + stmtNode, err := parser.New().ParseOneStmt("insert retrytxn values (2, 'a')", "", "") + require.NoError(t, err) + compiler := executor.Compiler{Ctx: tk.Session()} + stmt, _ := compiler.Compile(context.TODO(), stmtNode) + _ = executor.ResetContextOfStmt(tk.Session(), stmtNode) + history.Add(stmt, tk.Session().GetSessionVars().StmtCtx) + _, err = tk.Exec("commit") + require.Error(t, err) + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + require.False(t, txn.Valid()) + require.False(t, tk.Session().GetSessionVars().InTxn()) +} + +func TestRetryUnion(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table history (a int)") + tk.MustExec("insert history values (1), (2), (3)") + tk.MustExec("set @@autocommit = 0") + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + // UNION shouldn't be in retry history. + tk.MustQuery("(select * from history) union (select * from history)") + history := session.GetHistory(tk.Session()) + require.Equal(t, 0, history.Count()) + tk.MustQuery("(select * from history for update) union (select * from history)") + tk.MustExec("update history set a = a + 1") + history = session.GetHistory(tk.Session()) + require.Equal(t, 2, history.Count()) + + // Make retryable error. + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("update history set a = a + 1") + + tk.MustMatchErrMsg("commit", ".*can not retry select for update statement") +} + +func TestRetryResetStmtCtx(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table retrytxn (a int unique, b int)") + tk.MustExec("insert retrytxn values (1, 1)") + tk.MustExec("set @@tidb_disable_txn_auto_retry = 0") + tk.MustExec("begin") + tk.MustExec("update retrytxn set b = b + 1 where a = 1") + + // Make retryable error. + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustExec("update retrytxn set b = b + 1 where a = 1") + + require.NoError(t, tk.Session().CommitTxn(context.TODO())) + require.Equal(t, uint64(1), tk.Session().AffectedRows()) +} + +func TestReadOnlyNotInHistory(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table history (a int)") + tk.MustExec("insert history values (1), (2), (3)") + tk.MustExec("set @@autocommit = 0") + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + tk.MustQuery("select * from history") + history := session.GetHistory(tk.Session()) + require.Equal(t, 0, history.Count()) + + tk.MustExec("insert history values (4)") + tk.MustExec("insert history values (5)") + require.Equal(t, 2, history.Count()) + tk.MustExec("commit") + tk.MustQuery("select * from history") + history = session.GetHistory(tk.Session()) + require.Equal(t, 0, history.Count()) +} + +// For https://github.com/pingcap/tidb/issues/571 +func TestRetry(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("begin") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (c int)") + tk.MustExec("insert t values (1), (2), (3)") + tk.MustExec("commit") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec("use test") + + tk3.MustExec("SET SESSION autocommit=0;") + tk1.MustExec("set @@tidb_disable_txn_auto_retry = 0") + tk2.MustExec("set @@tidb_disable_txn_auto_retry = 0") + tk3.MustExec("set @@tidb_disable_txn_auto_retry = 0") + + var wg util.WaitGroupWrapper + wg.Run(func() { + for i := 0; i < 30; i++ { + tk1.MustExec("update t set c = 1;") + } + }) + wg.Run(func() { + for i := 0; i < 30; i++ { + tk2.MustExec("update t set c = ?;", 1) + } + }) + wg.Run(func() { + for i := 0; i < 30; i++ { + tk3.MustExec("begin") + tk3.MustExec("update t set c = 1;") + tk3.MustExec("commit") + } + }) + wg.Wait() +} diff --git a/tests/realtikvtest/sessiontest/session_test.go b/tests/realtikvtest/sessiontest/session_test.go index 6fe159314c1b7..5da90803d154b 100644 --- a/tests/realtikvtest/sessiontest/session_test.go +++ b/tests/realtikvtest/sessiontest/session_test.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/tests/realtikvtest" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/sqlexec" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" @@ -1276,7 +1277,7 @@ func TestSetTxnScope(t *testing.T) { require.Equal(t, kv.GlobalTxnScope, tk.Session().GetSessionVars().CheckAndGetTxnScope()) // @@tidb_enable_local_txn is off with configuring the zone label. - failpoint.Enable("tikvclient/injectTxnScope", `return("bj")`) + require.NoError(t, failpoint.Enable("tikvclient/injectTxnScope", `return("bj")`)) tk = testkit.NewTestKit(t, store) tk.MustExec("use test") tk.MustQuery("select @@global.tidb_enable_local_txn;").Check(testkit.Rows("0")) @@ -1338,7 +1339,7 @@ func TestDoDDLJobQuit(t *testing.T) { // use isolated store, because in below failpoint we will cancel its context store, err := mockstore.NewMockStore(mockstore.WithStoreType(mockstore.MockTiKV)) require.NoError(t, err) - defer store.Close() + defer func() { require.NoError(t, store.Close()) }() dom, err := session.BootstrapSession(store) require.NoError(t, err) defer dom.Close() @@ -1347,7 +1348,7 @@ func TestDoDDLJobQuit(t *testing.T) { defer se.Close() require.Nil(t, failpoint.Enable("github.com/pingcap/tidb/ddl/storeCloseInLoop", `return`)) - defer failpoint.Disable("github.com/pingcap/tidb/ddl/storeCloseInLoop") + defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/storeCloseInLoop")) }() // this DDL call will enter deadloop before this fix err = dom.DDL().CreateSchema(se, model.NewCIStr("testschema"), nil, nil) @@ -1393,8 +1394,10 @@ func TestCoprocessorOOMAction(t *testing.T) { config.UpdateGlobal(func(conf *config.Config) { conf.OOMAction = config.OOMActionCancel }) - failpoint.Enable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockConsumeAndAssert", `return(true)`) - defer failpoint.Disable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockConsumeAndAssert") + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockConsumeAndAssert", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockConsumeAndAssert")) + }() enableOOM := func(tk *testkit.TestKit, name, sql string) { t.Logf("enable OOM, testcase: %v", name) @@ -1423,7 +1426,7 @@ func TestCoprocessorOOMAction(t *testing.T) { require.Regexp(t, "Out Of Memory Quota.*", err) } - failpoint.Enable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockWaitMax", `return(true)`) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockWaitMax", `return(true)`)) // assert oom action and switch for _, testcase := range testcases { se, err := session.CreateSession4Test(store) @@ -1454,7 +1457,7 @@ func TestCoprocessorOOMAction(t *testing.T) { enableOOM(tk, testcase.name, testcase.sql) se.Close() } - failpoint.Disable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockWaitMax") + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockWaitMax")) // assert oom fallback for _, testcase := range testcases { @@ -1652,3 +1655,557 @@ func TestProcessInfoIssue22068(t *testing.T) { require.Nil(t, pi.Plan) wg.Wait() } + +func TestIssue19127(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists issue19127") + tk.MustExec("create table issue19127 (c_int int, c_str varchar(40), primary key (c_int, c_str) ) partition by hash (c_int) partitions 4;") + tk.MustExec("insert into issue19127 values (9, 'angry williams'), (10, 'thirsty hugle');") + _, _ = tk.Exec("update issue19127 set c_int = c_int + 10, c_str = 'adoring stonebraker' where c_int in (10, 9);") + require.Equal(t, uint64(2), tk.Session().AffectedRows()) +} + +func TestMemoryUsageAlarmVariable(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=1") + tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("1")) + tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=0") + tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0")) + tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=0.7") + tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0.7")) + tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=1.1") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_memory_usage_alarm_ratio value: '1.1'")) + tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("1")) + + tk.MustExec("set @@global.tidb_memory_usage_alarm_ratio=-1") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_memory_usage_alarm_ratio value: '-1'")) + tk.MustQuery("select @@global.tidb_memory_usage_alarm_ratio").Check(testkit.Rows("0")) + + tk.MustExec("set @@session.tidb_memory_usage_alarm_ratio=0.8") + tk.MustQuery(`show warnings`).Check(testkit.Rows(fmt.Sprintf("Warning %d modifying tidb_memory_usage_alarm_ratio will require SET GLOBAL in a future version of TiDB", errno.ErrInstanceScope))) +} + +func TestSelectLockInShare(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("DROP TABLE IF EXISTS t_sel_in_share") + tk.MustExec("CREATE TABLE t_sel_in_share (id int DEFAULT NULL)") + tk.MustExec("insert into t_sel_in_share values (11)") + require.Error(t, tk.ExecToErr("select * from t_sel_in_share lock in share mode")) + tk.MustExec("set @@tidb_enable_noop_functions = 1") + tk.MustQuery("select * from t_sel_in_share lock in share mode").Check(testkit.Rows("11")) + tk.MustExec("DROP TABLE t_sel_in_share") +} + +func TestReadDMLBatchSize(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_dml_batch_size=1000") + se, err := session.CreateSession(store) + require.NoError(t, err) + + // `select 1` to load the global variables. + _, _ = se.Execute(context.TODO(), "select 1") + require.Equal(t, 1000, se.GetSessionVars().DMLBatchSize) +} + +func TestPerStmtTaskID(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table task_id (v int)") + + tk.MustExec("begin") + tk.MustExec("select * from task_id where v > 10") + taskID1 := tk.Session().GetSessionVars().StmtCtx.TaskID + tk.MustExec("select * from task_id where v < 5") + taskID2 := tk.Session().GetSessionVars().StmtCtx.TaskID + tk.MustExec("commit") + + require.NotEqual(t, taskID1, taskID2) +} + +func TestSetEnableRateLimitAction(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // assert default value + result := tk.MustQuery("select @@tidb_enable_rate_limit_action;") + result.Check(testkit.Rows("1")) + tk.MustExec("use test") + tk.MustExec("create table tmp123(id int)") + tk.MustQuery("select * from tmp123;") + haveRateLimitAction := false + action := tk.Session().GetSessionVars().StmtCtx.MemTracker.GetFallbackForTest(false) + for ; action != nil; action = action.GetFallback() { + if action.GetPriority() == memory.DefRateLimitPriority { + haveRateLimitAction = true + break + } + } + require.True(t, haveRateLimitAction) + + // assert set sys variable + tk.MustExec("set global tidb_enable_rate_limit_action= '0';") + tk.Session().Close() + + tk.RefreshSession() + result = tk.MustQuery("select @@tidb_enable_rate_limit_action;") + result.Check(testkit.Rows("0")) + + haveRateLimitAction = false + action = tk.Session().GetSessionVars().StmtCtx.MemTracker.GetFallbackForTest(false) + for ; action != nil; action = action.GetFallback() { + if action.GetPriority() == memory.DefRateLimitPriority { + haveRateLimitAction = true + break + } + } + require.False(t, haveRateLimitAction) +} + +func TestStmtHints(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // Test MEMORY_QUOTA hint + tk.MustExec("select /*+ MEMORY_QUOTA(1 MB) */ 1;") + val := int64(1) * 1024 * 1024 + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + tk.MustExec("select /*+ MEMORY_QUOTA(1 GB) */ 1;") + val = int64(1) * 1024 * 1024 * 1024 + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + tk.MustExec("select /*+ MEMORY_QUOTA(1 GB), MEMORY_QUOTA(1 MB) */ 1;") + val = int64(1) * 1024 * 1024 + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + tk.MustExec("select /*+ MEMORY_QUOTA(0 GB) */ 1;") + val = int64(0) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "Setting the MEMORY_QUOTA to 0 means no memory limit") + + tk.MustExec("use test") + tk.MustExec("create table t1(a int);") + tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 (a) values (1);") + val = int64(1) * 1024 * 1024 + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + + tk.MustExec("insert /*+ MEMORY_QUOTA(1 MB) */ into t1 select /*+ MEMORY_QUOTA(3 MB) */ * from t1;") + val = int64(1) * 1024 * 1024 + require.True(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.CheckBytesLimit(val)) + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "[util:3126]Hint MEMORY_QUOTA(`3145728`) is ignored as conflicting/duplicated.") + + // Test NO_INDEX_MERGE hint + tk.Session().GetSessionVars().SetEnableIndexMerge(true) + tk.MustExec("select /*+ NO_INDEX_MERGE() */ 1;") + require.True(t, tk.Session().GetSessionVars().StmtCtx.NoIndexMergeHint) + tk.MustExec("select /*+ NO_INDEX_MERGE(), NO_INDEX_MERGE() */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.True(t, tk.Session().GetSessionVars().GetEnableIndexMerge()) + + // Test STRAIGHT_JOIN hint + tk.MustExec("select /*+ straight_join() */ 1;") + require.True(t, tk.Session().GetSessionVars().StmtCtx.StraightJoinOrder) + tk.MustExec("select /*+ straight_join(), straight_join() */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + + // Test USE_TOJA hint + tk.Session().GetSessionVars().SetAllowInSubqToJoinAndAgg(true) + tk.MustExec("select /*+ USE_TOJA(false) */ 1;") + require.False(t, tk.Session().GetSessionVars().GetAllowInSubqToJoinAndAgg()) + tk.Session().GetSessionVars().SetAllowInSubqToJoinAndAgg(false) + tk.MustExec("select /*+ USE_TOJA(true) */ 1;") + require.True(t, tk.Session().GetSessionVars().GetAllowInSubqToJoinAndAgg()) + tk.MustExec("select /*+ USE_TOJA(false), USE_TOJA(true) */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.True(t, tk.Session().GetSessionVars().GetAllowInSubqToJoinAndAgg()) + + // Test USE_CASCADES hint + tk.Session().GetSessionVars().SetEnableCascadesPlanner(true) + tk.MustExec("select /*+ USE_CASCADES(false) */ 1;") + require.False(t, tk.Session().GetSessionVars().GetEnableCascadesPlanner()) + tk.Session().GetSessionVars().SetEnableCascadesPlanner(false) + tk.MustExec("select /*+ USE_CASCADES(true) */ 1;") + require.True(t, tk.Session().GetSessionVars().GetEnableCascadesPlanner()) + tk.MustExec("select /*+ USE_CASCADES(false), USE_CASCADES(true) */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "USE_CASCADES() is defined more than once, only the last definition takes effect: USE_CASCADES(true)") + require.True(t, tk.Session().GetSessionVars().GetEnableCascadesPlanner()) + + // Test READ_CONSISTENT_REPLICA hint + tk.Session().GetSessionVars().SetReplicaRead(kv.ReplicaReadLeader) + tk.MustExec("select /*+ READ_CONSISTENT_REPLICA() */ 1;") + require.Equal(t, kv.ReplicaReadFollower, tk.Session().GetSessionVars().GetReplicaRead()) + tk.MustExec("select /*+ READ_CONSISTENT_REPLICA(), READ_CONSISTENT_REPLICA() */ 1;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.Equal(t, kv.ReplicaReadFollower, tk.Session().GetSessionVars().GetReplicaRead()) +} + +func TestMaxExecutionTime(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("use test") + tk.MustExec("create table MaxExecTime( id int,name varchar(128),age int);") + tk.MustExec("begin") + tk.MustExec("insert into MaxExecTime (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18);") + + tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) MAX_EXECUTION_TIME(500) */ * FROM MaxExecTime;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(500)") + require.True(t, tk.Session().GetSessionVars().StmtCtx.HasMaxExecutionTime) + require.Equal(t, uint64(500), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) + + tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) + tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) + tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) */ * FROM MaxExecTime;") + + tk.MustExec("set @@global.MAX_EXECUTION_TIME = 300;") + tk.MustQuery("select * FROM MaxExecTime;") + + tk.MustExec("set @@MAX_EXECUTION_TIME = 150;") + tk.MustQuery("select * FROM MaxExecTime;") + + tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("300")) + tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("150")) + + tk.MustExec("set @@global.MAX_EXECUTION_TIME = 0;") + tk.MustExec("set @@MAX_EXECUTION_TIME = 0;") + tk.MustExec("commit") + tk.MustExec("drop table if exists MaxExecTime;") +} + +func TestGrantViewRelated(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tkRoot := testkit.NewTestKit(t, store) + tkUser := testkit.NewTestKit(t, store) + tkRoot.MustExec("use test") + tkUser.MustExec("use test") + + tkRoot.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + + tkRoot.MustExec("create table if not exists t (a int)") + tkRoot.MustExec("create view v_version29 as select * from t") + tkRoot.MustExec("create user 'u_version29'@'%'") + tkRoot.MustExec("grant select on t to u_version29@'%'") + + tkUser.Session().Auth(&auth.UserIdentity{Username: "u_version29", Hostname: "localhost", CurrentUser: true, AuthUsername: "u_version29", AuthHostname: "%"}, nil, []byte("012345678901234567890")) + + tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) + require.Error(t, tkUser.ExecToErr("select * from test.v_version29;")) + tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) + require.Error(t, tkUser.ExecToErr("create view v_version29_c as select * from t;")) + + tkRoot.MustExec(`grant show view, select on v_version29 to 'u_version29'@'%'`) + tkRoot.MustQuery("select table_priv from mysql.tables_priv where host='%' and db='test' and user='u_version29' and table_name='v_version29'").Check(testkit.Rows("Select,Show View")) + + tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) + tkUser.MustQuery("show create view v_version29;") + require.Error(t, tkUser.ExecToErr("create view v_version29_c as select * from v_version29;")) + + tkRoot.MustExec("create view v_version29_c as select * from v_version29;") + tkRoot.MustExec(`grant create view on v_version29_c to 'u_version29'@'%'`) // Can't grant privilege on a non-exist table/view. + tkRoot.MustQuery("select table_priv from mysql.tables_priv where host='%' and db='test' and user='u_version29' and table_name='v_version29_c'").Check(testkit.Rows("Create View")) + tkRoot.MustExec("drop view v_version29_c") + + tkRoot.MustExec(`grant select on v_version29 to 'u_version29'@'%'`) + tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%")) + tkUser.MustExec("create view v_version29_c as select * from v_version29;") +} + +func TestLoadClientInteractive(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.RefreshSession() + tk.Session().GetSessionVars().ClientCapability = tk.Session().GetSessionVars().ClientCapability | mysql.ClientInteractive + tk.MustQuery("select @@wait_timeout").Check(testkit.Rows("28800")) +} + +func TestReplicaRead(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) + tk.MustExec("set @@tidb_replica_read = 'follower';") + require.Equal(t, kv.ReplicaReadFollower, tk.Session().GetSessionVars().GetReplicaRead()) + tk.MustExec("set @@tidb_replica_read = 'leader';") + require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) +} + +func TestIsolationRead(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + require.Len(t, tk.Session().GetSessionVars().GetIsolationReadEngines(), 3) + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") + engines := tk.Session().GetSessionVars().GetIsolationReadEngines() + require.Len(t, engines, 1) + _, hasTiFlash := engines[kv.TiFlash] + _, hasTiKV := engines[kv.TiKV] + require.True(t, hasTiFlash) + require.False(t, hasTiKV) +} + +func TestUpdatePrivilege(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1, t2;") + tk.MustExec("create table t1 (id int);") + tk.MustExec("create table t2 (id int);") + tk.MustExec("insert into t1 values (1);") + tk.MustExec("insert into t2 values (2);") + tk.MustExec("create user xxx;") + tk.MustExec("grant all on test.t1 to xxx;") + tk.MustExec("grant select on test.t2 to xxx;") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "xxx", Hostname: "localhost"}, []byte(""), []byte(""))) + + tk1.MustMatchErrMsg("update t2 set id = 666 where id = 1;", "privilege check.*") + + // Cover a bug that t1 and t2 both require update privilege. + // In fact, the privlege check for t1 should be update, and for t2 should be select. + tk1.MustExec("update t1,t2 set t1.id = t2.id;") + + // Fix issue 8911 + tk.MustExec("create database weperk") + tk.MustExec("use weperk") + tk.MustExec("create table tb_wehub_server (id int, active_count int, used_count int)") + tk.MustExec("create user 'weperk'") + tk.MustExec("grant all privileges on weperk.* to 'weperk'@'%'") + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "weperk", Hostname: "%"}, []byte(""), []byte(""))) + tk1.MustExec("use weperk") + tk1.MustExec("update tb_wehub_server a set a.active_count=a.active_count+1,a.used_count=a.used_count+1 where id=1") + + tk.MustExec("create database service") + tk.MustExec("create database report") + tk.MustExec(`CREATE TABLE service.t1 ( + id int(11) DEFAULT NULL, + a bigint(20) NOT NULL, + b text DEFAULT NULL, + PRIMARY KEY (a) +)`) + tk.MustExec(`CREATE TABLE report.t2 ( + a bigint(20) DEFAULT NULL, + c bigint(20) NOT NULL +)`) + tk.MustExec("grant all privileges on service.* to weperk") + tk.MustExec("grant all privileges on report.* to weperk") + tk1.Session().GetSessionVars().CurrentDB = "" + tk1.MustExec(`update service.t1 s, +report.t2 t +set s.a = t.a +WHERE +s.a = t.a +and t.c >= 1 and t.c <= 10000 +and s.b !='xx';`) + + // Fix issue 10028 + tk.MustExec("create database ap") + tk.MustExec("create database tp") + tk.MustExec("grant all privileges on ap.* to xxx") + tk.MustExec("grant select on tp.* to xxx") + tk.MustExec("create table tp.record( id int,name varchar(128),age int)") + tk.MustExec("insert into tp.record (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18)") + tk.MustExec("create table ap.record( id int,name varchar(128),age int)") + tk.MustExec("insert into ap.record(id) values(1)") + require.True(t, tk1.Session().Auth(&auth.UserIdentity{Username: "xxx", Hostname: "localhost"}, []byte(""), []byte(""))) + tk1.MustExec("update ap.record t inner join tp.record tt on t.id=tt.id set t.name=tt.name") +} + +func TestDBUserNameLength(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table if not exists t (a int)") + // Test username length can be longer than 16. + tk.MustExec(`CREATE USER 'abcddfjakldfjaldddds'@'%' identified by ''`) + tk.MustExec(`grant all privileges on test.* to 'abcddfjakldfjaldddds'@'%'`) + tk.MustExec(`grant all privileges on test.t to 'abcddfjakldfjaldddds'@'%'`) +} + +func TestHostLengthMax(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + host1 := strings.Repeat("a", 65) + host2 := strings.Repeat("a", 256) + + tk.MustExec(fmt.Sprintf(`CREATE USER 'abcddfjakldfjaldddds'@'%s'`, host1)) + tk.MustGetErrMsg(fmt.Sprintf(`CREATE USER 'abcddfjakldfjaldddds'@'%s'`, host2), "[ddl:1470]String 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long for host name (should be no longer than 255)") +} + +func TestCommitRetryCount(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + + tk1.MustExec("create table no_retry (id int)") + tk1.MustExec("insert into no_retry values (1)") + tk1.MustExec("set @@tidb_retry_limit = 0") + + tk1.MustExec("begin") + tk1.MustExec("update no_retry set id = 2") + + tk2.MustExec("begin") + tk2.MustExec("update no_retry set id = 3") + tk2.MustExec("commit") + + // No auto retry because retry limit is set to 0. + require.Error(t, tk1.ExecToErr("commit")) +} + +func TestEnablePartition(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set tidb_enable_table_partition=off") + tk.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition OFF")) + + tk.MustExec("set global tidb_enable_table_partition = on") + + tk.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition OFF")) + tk.MustQuery("show global variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) + + tk.MustExec("set tidb_enable_list_partition=off") + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) + tk.MustExec("set global tidb_enable_list_partition=on") + tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) + + tk.MustExec("set tidb_enable_list_partition=1") + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + + tk.MustExec("set tidb_enable_list_partition=on") + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + + tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + tk.MustExec("set global tidb_enable_list_partition=off") + tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + tk.MustExec("set tidb_enable_list_partition=off") + tk.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition OFF")) + + tk.MustExec("set global tidb_enable_list_partition=on") + tk.MustQuery("show global variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + tk1.MustQuery("show variables like 'tidb_enable_table_partition'").Check(testkit.Rows("tidb_enable_table_partition ON")) + tk1.MustQuery("show variables like 'tidb_enable_list_partition'").Check(testkit.Rows("tidb_enable_list_partition ON")) +} + +func TestRollbackOnCompileError(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int)") + tk.MustExec("insert t values (1)") + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec("use test") + tk2.MustQuery("select * from t").Check(testkit.Rows("1")) + + tk.MustExec("rename table t to t2") + var meetErr bool + for i := 0; i < 100; i++ { + _, err := tk2.Exec("insert t values (1)") + if err != nil { + meetErr = true + break + } + } + require.True(t, meetErr) + + tk.MustExec("rename table t2 to t") + var recoverErr bool + for i := 0; i < 100; i++ { + _, err := tk2.Exec("insert t values (1)") + if err == nil { + recoverErr = true + break + } + } + require.True(t, recoverErr) +} + +func TestCastTimeToDate(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("set time_zone = '-8:00'") + date := time.Now().In(time.FixedZone("", -8*int(time.Hour/time.Second))) + tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format("2006-01-02"))) + + tk.MustExec("set time_zone = '+08:00'") + date = time.Now().In(time.FixedZone("", 8*int(time.Hour/time.Second))) + tk.MustQuery("select cast(time('12:23:34') as date)").Check(testkit.Rows(date.Format("2006-01-02"))) +} + +func TestSetGlobalTZ(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set time_zone = '+08:00'") + tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) + + tk.MustExec("set global time_zone = '+00:00'") + + tk.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +08:00")) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustQuery("show variables like 'time_zone'").Check(testkit.Rows("time_zone +00:00")) +} diff --git a/tests/realtikvtest/txntest/txn_test.go b/tests/realtikvtest/txntest/txn_test.go new file mode 100644 index 0000000000000..7dc1f69f5f80d --- /dev/null +++ b/tests/realtikvtest/txntest/txn_test.go @@ -0,0 +1,151 @@ +// Copyright 2019 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 txntest + +import ( + "context" + "fmt" + "testing" + + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/tests/realtikvtest" + "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" +) + +func TestInTxnPSProtoPointGet(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(c1 int primary key, c2 int, c3 int)") + tk.MustExec("insert into t1 values(1, 10, 100)") + + ctx := context.Background() + + // Generate the ps statement and make the prepared plan cached for point get. + id, _, _, err := tk.Session().PrepareStmt("select c1, c2 from t1 where c1 = ?") + require.NoError(t, err) + idForUpdate, _, _, err := tk.Session().PrepareStmt("select c1, c2 from t1 where c1 = ? for update") + require.NoError(t, err) + params := []types.Datum{types.NewDatum(1)} + rs, err := tk.Session().ExecutePreparedStmt(ctx, id, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + rs, err = tk.Session().ExecutePreparedStmt(ctx, idForUpdate, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + + // Query again the cached plan will be used. + rs, err = tk.Session().ExecutePreparedStmt(ctx, id, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + rs, err = tk.Session().ExecutePreparedStmt(ctx, idForUpdate, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + + // Start a transaction, now the in txn flag will be added to the session vars. + _, err = tk.Session().Execute(ctx, "start transaction") + require.NoError(t, err) + rs, err = tk.Session().ExecutePreparedStmt(ctx, id, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + require.True(t, txn.Valid()) + rs, err = tk.Session().ExecutePreparedStmt(ctx, idForUpdate, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 10")) + txn, err = tk.Session().Txn(false) + require.NoError(t, err) + require.True(t, txn.Valid()) + _, err = tk.Session().Execute(ctx, "update t1 set c2 = c2 + 1") + require.NoError(t, err) + // Check the read result after in-transaction update. + rs, err = tk.Session().ExecutePreparedStmt(ctx, id, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 11")) + rs, err = tk.Session().ExecutePreparedStmt(ctx, idForUpdate, params) + require.NoError(t, err) + tk.ResultSetToResult(rs, fmt.Sprintf("%v", rs)).Check(testkit.Rows("1 11")) + txn, err = tk.Session().Txn(false) + require.NoError(t, err) + require.True(t, txn.Valid()) + tk.MustExec("commit") +} + +func TestTxnGoString(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists gostr;") + tk.MustExec("create table gostr (id int);") + + txn, err := tk.Session().Txn(false) + require.NoError(t, err) + require.Equal(t, "Txn{state=invalid}", fmt.Sprintf("%#v", txn)) + + tk.MustExec("begin") + txn, err = tk.Session().Txn(false) + require.NoError(t, err) + + require.Equal(t, fmt.Sprintf("Txn{state=valid, txnStartTS=%d}", txn.StartTS()), fmt.Sprintf("%#v", txn)) + + tk.MustExec("insert into gostr values (1)") + require.Equal(t, fmt.Sprintf("Txn{state=valid, txnStartTS=%d}", txn.StartTS()), fmt.Sprintf("%#v", txn)) + + tk.MustExec("rollback") + require.Equal(t, "Txn{state=invalid}", fmt.Sprintf("%#v", txn)) +} + +func TestSetTransactionIsolationOneSho(t *testing.T) { + store, clean := realtikvtest.CreateMockStoreAndSetup(t) + defer clean() + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (k int, v int)") + tk.MustExec("insert t values (1, 42)") + tk.MustExec("set tx_isolation = 'read-committed'") + tk.MustQuery("select @@tx_isolation").Check(testkit.Rows("READ-COMMITTED")) + tk.MustExec("set tx_isolation = 'repeatable-read'") + tk.MustExec("set transaction isolation level read committed") + tk.MustQuery("select @@tx_isolation_one_shot").Check(testkit.Rows("READ-COMMITTED")) + tk.MustQuery("select @@tx_isolation").Check(testkit.Rows("REPEATABLE-READ")) + + // Check isolation level is set to read committed. + ctx := context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { + require.Equal(t, kv.SI, req.IsolationLevel) + }) + _, err := tk.Session().Execute(ctx, "select * from t where k = 1") + require.NoError(t, err) + + // Check it just take effect for one time. + ctx = context.WithValue(context.Background(), "CheckSelectRequestHook", func(req *kv.Request) { + require.Equal(t, kv.SI, req.IsolationLevel) + }) + _, err = tk.Session().Execute(ctx, "select * from t where k = 1") + require.NoError(t, err) + + // Can't change isolation level when it's inside a transaction. + tk.MustExec("begin") + _, err = tk.Session().Execute(ctx, "set transaction isolation level read committed") + require.Error(t, err) +}