diff --git a/DEPS.bzl b/DEPS.bzl index e3430df032e83..5c83488776211 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3373,8 +3373,8 @@ def go_deps(): name = "com_github_pingcap_kvproto", build_file_proto_mode = "disable_global", importpath = "github.com/pingcap/kvproto", - sum = "h1:Qt8xe4CWgA/pPfYLHwCl8Mz0g7Mbnbhx4l0gVf9eH1w=", - version = "v0.0.0-20230216153817-c6df78cc9dea", + sum = "h1:bgLRG7gPJCq6aduA65ZV7xWQBThTcuarBB9VdfAzV4g=", + version = "v0.0.0-20230228041042-1e9aca94bab6", ) go_repository( name = "com_github_pingcap_log", @@ -3984,8 +3984,8 @@ def go_deps(): name = "com_github_stretchr_testify", build_file_proto_mode = "disable_global", importpath = "github.com/stretchr/testify", - sum = "h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk=", - version = "v1.8.1", + sum = "h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8=", + version = "v1.8.2", ) go_repository( name = "com_github_subosito_gotenv", @@ -4101,8 +4101,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:9exwxGqABskTj2pwtTqwziT/nGJ2N3Mh4gFAdP7GsrE=", - version = "v2.0.6-0.20230228091502-e2da5527026f", + sum = "h1:3BQR4RmBxu391t3z/q9h7BjDTS3cuRn8dfgYgMWk57s=", + version = "v2.0.6-0.20230302054057-3f7860f10959", ) go_repository( name = "com_github_tikv_pd", @@ -4116,8 +4116,8 @@ def go_deps(): name = "com_github_tikv_pd_client", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/pd/client", - sum = "h1:wZVfR5IsFod5Lym9lgjAaZ9TFsOqZ1iUaxockPacai8=", - version = "v0.0.0-20230224101107-b2f2a8219a40", + sum = "h1:1fomIvN2iiKT5uZbe2E6uNHZnRzmS6O47D/PJ9BAuPw=", + version = "v0.0.0-20230301094509-c82b237672a0", ) go_repository( name = "com_github_timakin_bodyclose", diff --git a/br/pkg/lightning/backend/local/engine.go b/br/pkg/lightning/backend/local/engine.go index 8e0554a76bed0..7ceb0c7857607 100644 --- a/br/pkg/lightning/backend/local/engine.go +++ b/br/pkg/lightning/backend/local/engine.go @@ -985,7 +985,7 @@ func (e *Engine) newKVIter(ctx context.Context, opts *pebble.IterOptions) Iter { zap.String("table", common.UniqueTable(e.tableInfo.DB, e.tableInfo.Name)), zap.Int64("tableID", e.tableInfo.ID), zap.Stringer("engineUUID", e.UUID)) - return newDupDetectIter(ctx, e.db, e.keyAdapter, opts, e.duplicateDB, logger, e.dupDetectOpt) + return newDupDetectIter(e.db, e.keyAdapter, opts, e.duplicateDB, logger, e.dupDetectOpt) } // getFirstAndLastKey reads the first and last key in range [lowerBound, upperBound) diff --git a/br/pkg/lightning/backend/local/iterator.go b/br/pkg/lightning/backend/local/iterator.go index c55547ea60f66..d0394a158fb1c 100644 --- a/br/pkg/lightning/backend/local/iterator.go +++ b/br/pkg/lightning/backend/local/iterator.go @@ -16,7 +16,6 @@ package local import ( "bytes" - "context" "github.com/cockroachdb/pebble" sst "github.com/pingcap/kvproto/pkg/import_sstpb" @@ -70,7 +69,6 @@ var _ Iter = pebbleIter{} const maxDuplicateBatchSize = 4 << 20 type dupDetectIter struct { - ctx context.Context iter *pebble.Iterator curKey []byte curRawKey []byte @@ -143,7 +141,7 @@ func (d *dupDetectIter) record(rawKey, key, val []byte) { func (d *dupDetectIter) Next() bool { recordFirst := false - for d.err == nil && d.ctx.Err() == nil && d.iter.Next() { + for d.err == nil && d.iter.Next() { d.nextKey, d.err = d.keyAdapter.Decode(d.nextKey[:0], d.iter.Key()) if d.err != nil { return false @@ -168,9 +166,6 @@ func (d *dupDetectIter) Next() bool { } d.record(d.iter.Key(), d.nextKey, d.iter.Value()) } - if d.err == nil { - d.err = d.ctx.Err() - } return false } @@ -204,7 +199,7 @@ func (d *dupDetectIter) OpType() sst.Pair_OP { var _ Iter = &dupDetectIter{} -func newDupDetectIter(ctx context.Context, db *pebble.DB, keyAdapter KeyAdapter, +func newDupDetectIter(db *pebble.DB, keyAdapter KeyAdapter, opts *pebble.IterOptions, dupDB *pebble.DB, logger log.Logger, dupOpt dupDetectOpt) *dupDetectIter { newOpts := &pebble.IterOptions{TableFilter: opts.TableFilter} if len(opts.LowerBound) > 0 { @@ -214,7 +209,6 @@ func newDupDetectIter(ctx context.Context, db *pebble.DB, keyAdapter KeyAdapter, newOpts.UpperBound = keyAdapter.Encode(nil, opts.UpperBound, MinRowID) } return &dupDetectIter{ - ctx: ctx, iter: db.NewIter(newOpts), keyAdapter: keyAdapter, writeBatch: dupDB.NewBatch(), diff --git a/br/pkg/lightning/backend/local/iterator_test.go b/br/pkg/lightning/backend/local/iterator_test.go index ee643bf42ce0b..75560ac012cd3 100644 --- a/br/pkg/lightning/backend/local/iterator_test.go +++ b/br/pkg/lightning/backend/local/iterator_test.go @@ -16,7 +16,6 @@ package local import ( "bytes" - "context" "math/rand" "path/filepath" "sort" @@ -122,7 +121,7 @@ func TestDupDetectIterator(t *testing.T) { dupDB, err := pebble.Open(filepath.Join(storeDir, "duplicates"), &pebble.Options{}) require.NoError(t, err) var iter Iter - iter = newDupDetectIter(context.Background(), db, keyAdapter, &pebble.IterOptions{}, dupDB, log.L(), dupDetectOpt{}) + iter = newDupDetectIter(db, keyAdapter, &pebble.IterOptions{}, dupDB, log.L(), dupDetectOpt{}) sort.Slice(pairs, func(i, j int) bool { key1 := keyAdapter.Encode(nil, pairs[i].Key, pairs[i].RowID) key2 := keyAdapter.Encode(nil, pairs[j].Key, pairs[j].RowID) @@ -217,7 +216,7 @@ func TestDupDetectIterSeek(t *testing.T) { dupDB, err := pebble.Open(filepath.Join(storeDir, "duplicates"), &pebble.Options{}) require.NoError(t, err) - iter := newDupDetectIter(context.Background(), db, keyAdapter, &pebble.IterOptions{}, dupDB, log.L(), dupDetectOpt{}) + iter := newDupDetectIter(db, keyAdapter, &pebble.IterOptions{}, dupDB, log.L(), dupDetectOpt{}) require.True(t, iter.Seek([]byte{1, 2, 3, 1})) require.Equal(t, pairs[1].Val, iter.Value()) diff --git a/br/web/package-lock.json b/br/web/package-lock.json index 39982f501ca13..429590d9174c3 100644 --- a/br/web/package-lock.json +++ b/br/web/package-lock.json @@ -1618,9 +1618,9 @@ } }, "node_modules/loader-utils": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.0.tgz", - "integrity": "sha512-rP4F0h2RaWSvPEkD7BLDFQnvSf+nK+wr3ESUjNTyAGobqrijmW92zc+SO6d4p4B1wh7+B/Jg1mkQe5NYUEHtHQ==", + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.4.tgz", + "integrity": "sha512-xXqpXoINfFhgua9xiqD8fPFHgkoq1mmmpE92WlDbm9rNRd/EbRb+Gqf908T2DMfuHjjJlksiK2RbHVOdD/MqSw==", "dev": true, "dependencies": { "big.js": "^5.2.2", @@ -3989,9 +3989,9 @@ "dev": true }, "loader-utils": { - "version": "2.0.0", - "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.0.tgz", - "integrity": "sha512-rP4F0h2RaWSvPEkD7BLDFQnvSf+nK+wr3ESUjNTyAGobqrijmW92zc+SO6d4p4B1wh7+B/Jg1mkQe5NYUEHtHQ==", + "version": "2.0.4", + "resolved": "https://registry.npmjs.org/loader-utils/-/loader-utils-2.0.4.tgz", + "integrity": "sha512-xXqpXoINfFhgua9xiqD8fPFHgkoq1mmmpE92WlDbm9rNRd/EbRb+Gqf908T2DMfuHjjJlksiK2RbHVOdD/MqSw==", "dev": true, "requires": { "big.js": "^5.2.2", diff --git a/ddl/indexmergetest/merge_test.go b/ddl/indexmergetest/merge_test.go index 65730d6342db4..78d0f4eb2b2b3 100644 --- a/ddl/indexmergetest/merge_test.go +++ b/ddl/indexmergetest/merge_test.go @@ -897,3 +897,44 @@ func TestAddIndexDuplicateAndWriteConflict(t *testing.T) { tk.MustExec("admin check table t;") tk.MustQuery("select * from t;").Check(testkit.Rows("1 1", "2 1")) } + +func TestAddIndexUpdateUntouchedValues(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t(id int primary key, b int, k int);") + tk.MustExec("insert into t values (1, 1, 1);") + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + + d := dom.DDL() + originalCallback := d.GetHook() + defer d.SetHook(originalCallback) + callback := &callback.TestDDLCallback{} + var runDML bool + callback.OnJobRunAfterExported = func(job *model.Job) { + if t.Failed() || runDML { + return + } + switch job.SchemaState { + case model.StateWriteReorganization: + _, err := tk1.Exec("begin;") + assert.NoError(t, err) + _, err = tk1.Exec("update t set k=k+1 where id = 1;") + assert.NoError(t, err) + _, err = tk1.Exec("insert into t values (2, 1, 2);") + // Should not report "invalid temp index value". + assert.NoError(t, err) + _, err = tk1.Exec("commit;") + assert.NoError(t, err) + runDML = true + } + } + d.SetHook(callback) + + tk.MustGetErrCode("alter table t add unique index idx(b);", errno.ErrDupEntry) + tk.MustExec("admin check table t;") + tk.MustQuery("select * from t;").Check(testkit.Rows("1 1 2", "2 1 2")) +} diff --git a/ddl/job_table.go b/ddl/job_table.go index 0ce54414d8f5b..db56b5afa9272 100644 --- a/ddl/job_table.go +++ b/ddl/job_table.go @@ -925,7 +925,7 @@ func GetBackfillJobs(sess *session, tblName, condition string, label string) ([] func RemoveBackfillJob(sess *session, isOneEle bool, backfillJob *BackfillJob) error { sql := "delete from mysql.tidb_background_subtask" if !isOneEle { - sql += fmt.Sprintf(" where task_key like '%s'", backfillJob.keyString()) + sql += fmt.Sprintf(" where task_key = '%s'", backfillJob.keyString()) } else { sql += fmt.Sprintf(" where task_key like '%s'", backfillJob.PrefixKeyString()) } diff --git a/errno/errcode.go b/errno/errcode.go index a165e2f6ded2e..10d84088cd63e 100644 --- a/errno/errcode.go +++ b/errno/errcode.go @@ -1103,9 +1103,12 @@ const ( ErrColumnInChange = 8245 ErrDDLSetting = 8246 ErrIngestFailed = 8247 - ErrResourceGroupExists = 8248 - ErrResourceGroupNotExists = 8249 - ErrResourceGroupSupportDisabled = 8250 + // Resource group errors. + ErrResourceGroupExists = 8248 + ErrResourceGroupNotExists = 8249 + ErrResourceGroupSupportDisabled = 8250 + ErrResourceGroupConfigUnavailable = 8251 + ErrResourceGroupThrottled = 8252 // TiKV/PD/TiFlash errors. ErrPDServerTimeout = 9001 diff --git a/errno/errname.go b/errno/errname.go index b7272a4710e27..7151277e48709 100644 --- a/errno/errname.go +++ b/errno/errname.go @@ -1110,8 +1110,10 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{ ErrResourceGroupExists: mysql.Message("Resource group '%-.192s' already exists", nil), ErrResourceGroupNotExists: mysql.Message("Unknown resource group '%-.192s'", nil), - ErrColumnInChange: mysql.Message("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", nil), - ErrResourceGroupSupportDisabled: mysql.Message("Resource control feature is disabled. Run `SET GLOBAL tidb_enable_resource_control='on'` to enable the feature", nil), + ErrColumnInChange: mysql.Message("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", nil), + ErrResourceGroupSupportDisabled: mysql.Message("Resource control feature is disabled. Run `SET GLOBAL tidb_enable_resource_control='on'` to enable the feature", nil), + ErrResourceGroupConfigUnavailable: mysql.Message("Resource group configuration is unavailable", nil), + ErrResourceGroupThrottled: mysql.Message("Exceeded resource group quota limitation", nil), // TiKV/PD errors. ErrPDServerTimeout: mysql.Message("PD server timeout: %s", nil), diff --git a/errors.toml b/errors.toml index bbc2ff8d78192..07c4ef0a4e4ad 100644 --- a/errors.toml +++ b/errors.toml @@ -2751,6 +2751,21 @@ error = ''' TTL manager has timed out, pessimistic locks may expire, please commit or rollback this transaction ''' +["tikv:8249"] +error = ''' +Unknown resource group '%-.192s' +''' + +["tikv:8251"] +error = ''' +Resource group configuration is unavailable +''' + +["tikv:8252"] +error = ''' +Exceeded resource group quota limitation +''' + ["tikv:9001"] error = ''' PD server timeout: %s diff --git a/executor/delete.go b/executor/delete.go index 3fedb55806364..91a23a7007dd6 100644 --- a/executor/delete.go +++ b/executor/delete.go @@ -228,8 +228,8 @@ func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error { func (e *DeleteExec) removeRowsInTblRowMap(tblRowMap tableRowMapType) error { for id, rowMap := range tblRowMap { var err error - rowMap.Range(func(h kv.Handle, val interface{}) bool { - err = e.removeRow(e.ctx, e.tblID2Table[id], h, val.([]types.Datum)) + rowMap.Range(func(h kv.Handle, val []types.Datum) bool { + err = e.removeRow(e.ctx, e.tblID2Table[id], h, val) return err == nil }) if err != nil { diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 8415169d10c88..1db7150577809 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -768,7 +768,7 @@ func TestIndexMerge4PlanCache(t *testing.T) { ps := []*util.ProcessInfo{tkProcess} tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) res := tk.MustQuery("explain for connection " + strconv.FormatUint(tkProcess.ID, 10)) - require.Len(t, res.Rows(), 6) + require.Len(t, res.Rows(), 7) require.Regexp(t, ".*IndexMerge.*", res.Rows()[1][0]) require.Regexp(t, ".*IndexRangeScan.*", res.Rows()[3][0]) require.Equal(t, "range:(NULL,\"mm\"), (\"mm\",+inf], keep order:false, stats:pseudo", res.Rows()[3][4]) diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 610c15b809bfa..c04fa6ee62fe4 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -998,8 +998,8 @@ func (w *intersectionProcessWorker) doIntersectionPerPartition(ctx context.Conte // We assume the result of intersection is small, so no need to track memory. intersectedMap := make(map[int][]kv.Handle, len(w.handleMapsPerWorker)) for parTblIdx, hMap := range w.handleMapsPerWorker { - hMap.Range(func(h kv.Handle, val interface{}) bool { - if *(val.(*int)) == len(w.indexMerge.partialPlans) { + hMap.Range(func(h kv.Handle, val *int) bool { + if *(val) == len(w.indexMerge.partialPlans) { // Means all partial paths have this handle. intersectedMap[parTblIdx] = append(intersectedMap[parTblIdx], h) } diff --git a/expression/builtin_time.go b/expression/builtin_time.go index d7354ed2c9fbc..a34f9b3611c4b 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -3192,7 +3192,11 @@ func (du *baseDateArithmetical) vecGetDateFromString(b *baseBuiltinFunc, input * } result.SetNull(i, true) } else if b.ctx.GetSessionVars().SQLMode.HasNoZeroDateMode() && (date.Year() == 0 || date.Month() == 0 || date.Day() == 0) { - return handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, dateStr)) + err = handleInvalidTimeError(b.ctx, types.ErrWrongValue.GenWithStackByArgs(types.DateTimeStr, dateStr)) + if err != nil { + return err + } + result.SetNull(i, true) } else { dates[i] = date } diff --git a/expression/integration_test/integration_test.go b/expression/integration_test/integration_test.go index 1d6e5edc25005..e98fc2fdd9660 100644 --- a/expression/integration_test/integration_test.go +++ b/expression/integration_test/integration_test.go @@ -7907,3 +7907,18 @@ func TestIssue40536(t *testing.T) { tk.MustExec("CREATE TABLE `8919f3f4-25be-4a1a-904a-bb5e863d8fc8` (\n `9804d5f2-cbc7-43b7-b241-ea2656dc941a` enum('s951','36d','ua65','49yru','6l2em','4ea','jf2d2','vprsc','3yl7n','hz','ov') DEFAULT '4ea',\n `323cdbcb-0c14-4362-90ab-ea42caaed6a5` year(4) NOT NULL DEFAULT '1983',\n `b9b70f39-1a02-4114-9d7d-fa6259c1b691` time DEFAULT '20:18:04',\n PRIMARY KEY (`323cdbcb-0c14-4362-90ab-ea42caaed6a5`) /*T![clustered_index] CLUSTERED */,\n KEY `a704d6bb-772b-44ea-8cb0-6f7491c1aaa6` (`323cdbcb-0c14-4362-90ab-ea42caaed6a5`,`9804d5f2-cbc7-43b7-b241-ea2656dc941a`)\n) ENGINE=InnoDB DEFAULT CHARSET=ascii COLLATE=ascii_bin;") tk.MustExec("delete from `6bf9e76d-ab44-4031-8a07-418b10741580` where not( `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( select `9804d5f2-cbc7-43b7-b241-ea2656dc941a` from `8919f3f4-25be-4a1a-904a-bb5e863d8fc8` where `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( '1f' ) and `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` in ( '1g' ,'va' ,'uhnh' ) ) ) and not( IsNull( `6bf9e76d-ab44-4031-8a07-418b10741580`.`e0b5f703-6cfe-49b4-bc21-16a6455e43a7` ) );\n") } + +func TestIssue40015(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("CREATE TABLE test ( c1 varchar(20));") + tk.MustExec("INSERT INTO test VALUES (101111),(11100),(101111),(101111);") + tk.MustExec("set tidb_enable_vectorized_expression = true;") + tk.MustQuery("SELECT DATE_ADD(c1, INTERVAL 1 DAY_HOUR) from test;").Sort().Check(testkit.Rows( + "2010-11-11 01:00:00", + "2010-11-11 01:00:00", + "2010-11-11 01:00:00", + "", + )) +} diff --git a/go.mod b/go.mod index 8525d089ad03e..371eb1e6a4c23 100644 --- a/go.mod +++ b/go.mod @@ -71,7 +71,7 @@ require ( github.com/pingcap/errors v0.11.5-0.20221009092201-b66cddb77c32 github.com/pingcap/failpoint v0.0.0-20220801062533-2eaa32854a6c github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059 - github.com/pingcap/kvproto v0.0.0-20230216153817-c6df78cc9dea + github.com/pingcap/kvproto v0.0.0-20230228041042-1e9aca94bab6 github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4 github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e @@ -89,11 +89,11 @@ require ( github.com/spf13/pflag v1.0.5 github.com/spkg/bom v1.0.0 github.com/stathat/consistent v1.0.0 - github.com/stretchr/testify v1.8.1 + github.com/stretchr/testify v1.8.2 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.6-0.20230228091502-e2da5527026f - github.com/tikv/pd/client v0.0.0-20230224101107-b2f2a8219a40 + github.com/tikv/client-go/v2 v2.0.6-0.20230302054057-3f7860f10959 + github.com/tikv/pd/client v0.0.0-20230301094509-c82b237672a0 github.com/timakin/bodyclose v0.0.0-20221125081123-e39cf3fc478e github.com/twmb/murmur3 v1.1.6 github.com/uber/jaeger-client-go v2.22.1+incompatible diff --git a/go.sum b/go.sum index 6392cd0212c2c..2ac05292f9983 100644 --- a/go.sum +++ b/go.sum @@ -772,8 +772,8 @@ github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059/go.mod h1:fMRU1BA1y+r89 github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN8dIUmo4Be2+pMRb6f55i+UIYrluu2E= github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= -github.com/pingcap/kvproto v0.0.0-20230216153817-c6df78cc9dea h1:Qt8xe4CWgA/pPfYLHwCl8Mz0g7Mbnbhx4l0gVf9eH1w= -github.com/pingcap/kvproto v0.0.0-20230216153817-c6df78cc9dea/go.mod h1:+on3Lfk/fb1lXkud3XvskJumhSIEEgN2TTbMObUlrxE= +github.com/pingcap/kvproto v0.0.0-20230228041042-1e9aca94bab6 h1:bgLRG7gPJCq6aduA65ZV7xWQBThTcuarBB9VdfAzV4g= +github.com/pingcap/kvproto v0.0.0-20230228041042-1e9aca94bab6/go.mod h1:KUrW1FGoznGMMTssYBu0czfAhn6vQcIrHyZoSC6T990= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM= github.com/pingcap/log v1.1.0/go.mod h1:DWQW5jICDR7UJh4HtxXSM20Churx4CQL0fwL/SoOSA4= @@ -924,8 +924,9 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/subosito/gotenv v1.2.0/go.mod h1:N0PQaV/YGNqwC0u51sEeR/aUtSLEXKX9iv69rRypqCw= github.com/tdakkota/asciicheck v0.1.1 h1:PKzG7JUTUmVspQTDqtkX9eSiLGossXTybutHwTXuO0A= github.com/tdakkota/asciicheck v0.1.1/go.mod h1:yHp0ai0Z9gUljN3o0xMhYJnH/IcvkdTBOX2fmJ93JEM= @@ -937,10 +938,10 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJf github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4= github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM= -github.com/tikv/client-go/v2 v2.0.6-0.20230228091502-e2da5527026f h1:9exwxGqABskTj2pwtTqwziT/nGJ2N3Mh4gFAdP7GsrE= -github.com/tikv/client-go/v2 v2.0.6-0.20230228091502-e2da5527026f/go.mod h1:a2N4+WwNsicV1E2+KQFg4CDqeoxvOyphZXmldX/TXDU= -github.com/tikv/pd/client v0.0.0-20230224101107-b2f2a8219a40 h1:wZVfR5IsFod5Lym9lgjAaZ9TFsOqZ1iUaxockPacai8= -github.com/tikv/pd/client v0.0.0-20230224101107-b2f2a8219a40/go.mod h1:j94ECCo0drzB/GsMFaUjESjLXNvIbed5PWJXcd8eggo= +github.com/tikv/client-go/v2 v2.0.6-0.20230302054057-3f7860f10959 h1:3BQR4RmBxu391t3z/q9h7BjDTS3cuRn8dfgYgMWk57s= +github.com/tikv/client-go/v2 v2.0.6-0.20230302054057-3f7860f10959/go.mod h1:HdCAbFaUCsjI4n5vlCJ0rGpMfIHoD1o6UlA1rwD9u1o= +github.com/tikv/pd/client v0.0.0-20230301094509-c82b237672a0 h1:1fomIvN2iiKT5uZbe2E6uNHZnRzmS6O47D/PJ9BAuPw= +github.com/tikv/pd/client v0.0.0-20230301094509-c82b237672a0/go.mod h1:4wjAY2NoMn4wx5+hZrEhrSGBs3jvKb+lxfUt+thHFQ4= github.com/timakin/bodyclose v0.0.0-20221125081123-e39cf3fc478e h1:MV6KaVu/hzByHP0UvJ4HcMGE/8a6A4Rggc/0wx2AvJo= github.com/timakin/bodyclose v0.0.0-20221125081123-e39cf3fc478e/go.mod h1:27bSVNWSBOHm+qRp1T9qzaIpsWEP6TbUnei/43HK+PQ= github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs= diff --git a/kv/key.go b/kv/key.go index 3c243a06ed2eb..d7f2e483ad2c8 100644 --- a/kv/key.go +++ b/kv/key.go @@ -506,7 +506,7 @@ func (m *MemAwareHandleMap[V]) Set(h Handle, val V) int64 { } // Range iterates the MemAwareHandleMap with fn, the fn returns true to continue, returns false to stop. -func (m *MemAwareHandleMap[V]) Range(fn func(h Handle, val interface{}) bool) { +func (m *MemAwareHandleMap[V]) Range(fn func(h Handle, val V) bool) { for h, val := range m.ints.M { if !fn(IntHandle(h), val) { return diff --git a/parser/go.mod b/parser/go.mod index bd63a46fe8f0c..6f9093644b4bf 100644 --- a/parser/go.mod +++ b/parser/go.mod @@ -7,7 +7,7 @@ require ( github.com/go-sql-driver/mysql v1.7.0 github.com/pingcap/errors v0.11.5-0.20210425183316-da1aaba5fb63 github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7 - github.com/stretchr/testify v1.8.0 + github.com/stretchr/testify v1.8.2 go.uber.org/goleak v1.1.11 go.uber.org/zap v1.24.0 golang.org/x/exp v0.0.0-20220428152302-39d4317da171 diff --git a/parser/go.sum b/parser/go.sum index 470e2c8276361..ba43ea4ddc824 100644 --- a/parser/go.sum +++ b/parser/go.sum @@ -31,12 +31,14 @@ github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0 h1:OdAsTTz6O github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= -github.com/stretchr/testify v1.8.0 h1:pSgiaMZlXftHpm5L7V1+rVB+AZJydKsMxsQBIJw4PKk= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/yuin/goldmark v1.3.5/go.mod h1:mwnBkeHKe2W/ZEtQ+71ViKU8L12m81fl3OWwC1Zlc8k= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= diff --git a/planner/core/indexmerge_path.go b/planner/core/indexmerge_path.go index 4cefaa813052e..b6f86100e81af 100644 --- a/planner/core/indexmerge_path.go +++ b/planner/core/indexmerge_path.go @@ -350,6 +350,11 @@ func (ds *DataSource) buildIndexMergeOrPath( path.TableFilters = nil } } + + // Keep this filter as a part of table filters for safety if it has any parameter. + if expression.MaybeOverOptimized4PlanCache(ds.ctx, filters[current:current+1]) { + shouldKeepCurrentFilter = true + } if shouldKeepCurrentFilter { indexMergePath.TableFilters = append(indexMergePath.TableFilters, filters[current]) } @@ -437,6 +442,11 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access } } + // Keep these partial filters as a part of table filters for safety if there is any parameter. + if expression.MaybeOverOptimized4PlanCache(ds.ctx, partialFilters) { + dedupedFinalFilters = append(dedupedFinalFilters, partialFilters...) + } + // 3. Estimate the row count after partial paths. sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, partialFilters, nil) if err != nil { diff --git a/planner/core/plan_cache_test.go b/planner/core/plan_cache_test.go index 2cb4e0c2ad8c3..85d357c2e61dd 100644 --- a/planner/core/plan_cache_test.go +++ b/planner/core/plan_cache_test.go @@ -1100,6 +1100,26 @@ func TestPlanCacheWithSubquery(t *testing.T) { } } +func TestIssue41828(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`CREATE TABLE IDT_MULTI15840STROBJSTROBJ ( + COL1 enum('aa', 'zzz') DEFAULT NULL, + COL2 smallint(6) DEFAULT NULL, + COL3 date DEFAULT NULL, + KEY U_M_COL4 (COL1,COL2), + KEY U_M_COL5 (COL3,COL2))`) + + tk.MustExec(`INSERT INTO IDT_MULTI15840STROBJSTROBJ VALUES ('zzz',1047,'6115-06-05'),('zzz',-23221,'4250-09-03'),('zzz',27138,'1568-07-30'),('zzz',-30903,'6753-08-21'),('zzz',-26875,'6117-10-10')`) + tk.MustExec(`prepare stmt from 'select * from IDT_MULTI15840STROBJSTROBJ where col3 <=> ? or col1 in (?, ?, ?) and col2 not between ? and ?'`) + tk.MustExec(`set @a="0051-12-23", @b="none", @c="none", @d="none", @e=-32757, @f=-32757`) + tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f`).Check(testkit.Rows()) + tk.MustQuery(`show warnings`).Check(testkit.Rows(`Warning 1105 skip plan-cache: IndexMerge plan with full-scan is un-cacheable`)) + tk.MustExec(`set @a="9795-01-10", @b="aa", @c="aa", @d="aa", @e=31928, @f=31928`) + tk.MustQuery(`execute stmt using @a,@b,@c,@d,@e,@f`).Check(testkit.Rows()) +} + func TestPlanCacheSubquerySPMEffective(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/plan_cacheable_checker.go b/planner/core/plan_cacheable_checker.go index b201ca7e006a3..e1cc11a06c0a6 100644 --- a/planner/core/plan_cacheable_checker.go +++ b/planner/core/plan_cacheable_checker.go @@ -376,11 +376,12 @@ func isPlanCacheable(sctx sessionctx.Context, p Plan, paramNum, limitParamNum in if limitParamNum != 0 && !sctx.GetSessionVars().EnablePlanCacheForParamLimit { return false, "skip plan-cache: the switch 'tidb_enable_plan_cache_for_param_limit' is off" } - return isPhysicalPlanCacheable(sctx, pp, paramNum, limitParamNum) + return isPhysicalPlanCacheable(sctx, pp, paramNum, limitParamNum, false) } // isPhysicalPlanCacheable returns whether this physical plan is cacheable and return the reason if not. -func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, limitParamNum int) (cacheable bool, reason string) { +func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, limitParamNum int, underIndexMerge bool) (cacheable bool, reason string) { + var subPlans []PhysicalPlan switch x := p.(type) { case *PhysicalTableDual: if paramNum > 0 { @@ -398,12 +399,23 @@ func isPhysicalPlanCacheable(sctx sessionctx.Context, p PhysicalPlan, paramNum, if x.AccessMVIndex { return false, "skip plan-cache: the plan with IndexMerge accessing Multi-Valued Index is un-cacheable" } + underIndexMerge = true + subPlans = append(subPlans, x.partialPlans...) + case *PhysicalIndexScan: + if underIndexMerge && x.isFullScan() { + return false, "skip plan-cache: IndexMerge plan with full-scan is un-cacheable" + } + case *PhysicalTableScan: + if underIndexMerge && x.isFullScan() { + return false, "skip plan-cache: IndexMerge plan with full-scan is un-cacheable" + } case *PhysicalApply: return false, "skip plan-cache: PhysicalApply plan is un-cacheable" } - for _, c := range p.Children() { - if cacheable, reason = isPhysicalPlanCacheable(sctx, c, paramNum, limitParamNum); !cacheable { + subPlans = append(subPlans, p.Children()...) + for _, c := range subPlans { + if cacheable, reason = isPhysicalPlanCacheable(sctx, c, paramNum, limitParamNum, underIndexMerge); !cacheable { return cacheable, reason } } diff --git a/resourcemanager/scheduler/cpu_scheduler.go b/resourcemanager/scheduler/cpu_scheduler.go index 14338d80683d4..7b0f487916876 100644 --- a/resourcemanager/scheduler/cpu_scheduler.go +++ b/resourcemanager/scheduler/cpu_scheduler.go @@ -34,10 +34,14 @@ func (*CPUScheduler) Tune(_ util.Component, pool util.GoroutinePool) Command { if time.Since(pool.LastTunerTs()) < util.MinSchedulerInterval.Load() { return Hold } - if cpu.GetCPUUsage() < 0.5 { + value, unsupported := cpu.GetCPUUsage() + if unsupported { + return Hold + } + if value < 0.5 { return Overclock } - if cpu.GetCPUUsage() > 0.7 { + if value > 0.7 { return Downclock } return Hold diff --git a/store/driver/error/BUILD.bazel b/store/driver/error/BUILD.bazel index 233f441f200db..d7bff2d85a136 100644 --- a/store/driver/error/BUILD.bazel +++ b/store/driver/error/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "//util/dbterror", "@com_github_pingcap_errors//:errors", "@com_github_tikv_client_go_v2//error", + "@com_github_tikv_pd_client//errs", ], ) diff --git a/store/driver/error/error.go b/store/driver/error/error.go index 4be6e3628c5cc..7716628bdbc6e 100644 --- a/store/driver/error/error.go +++ b/store/driver/error/error.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/util/dbterror" tikverr "github.com/tikv/client-go/v2/error" + pderr "github.com/tikv/pd/client/errs" ) // tikv error instance @@ -53,6 +54,12 @@ var ( ErrPDServerTimeout = dbterror.ClassTiKV.NewStd(errno.ErrPDServerTimeout) // ErrRegionUnavailable is the error when region is not available. ErrRegionUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrRegionUnavailable) + // ErrResourceGroupNotExists is the error when resource group does not exist. + ErrResourceGroupNotExists = dbterror.ClassTiKV.NewStd(errno.ErrResourceGroupNotExists) + // ErrResourceGroupConfigUnavailable is the error when resource group configuration is unavailable. + ErrResourceGroupConfigUnavailable = dbterror.ClassTiKV.NewStd(errno.ErrResourceGroupConfigUnavailable) + // ErrResourceGroupThrottled is the error when resource group is exceeded quota limitation + ErrResourceGroupThrottled = dbterror.ClassTiKV.NewStd(errno.ErrResourceGroupThrottled) // ErrUnknown is the unknow error. ErrUnknown = dbterror.ClassTiKV.NewStd(errno.ErrUnknown) ) @@ -163,5 +170,18 @@ func ToTiDBErr(err error) error { return terror.ErrResultUndetermined } + var errGetResourceGroup *pderr.ErrClientGetResourceGroup + if stderrs.As(err, &errGetResourceGroup) { + return ErrResourceGroupNotExists.FastGenByArgs(errGetResourceGroup.ResourceGroupName) + } + + if stderrs.Is(err, pderr.ErrClientResourceGroupConfigUnavailable) { + return ErrResourceGroupConfigUnavailable + } + + if stderrs.Is(err, pderr.ErrClientResourceGroupThrottled) { + return ErrResourceGroupThrottled + } + return errors.Trace(err) } diff --git a/store/driver/tikv_driver.go b/store/driver/tikv_driver.go index 474a44c165699..f6de9767131ed 100644 --- a/store/driver/tikv_driver.go +++ b/store/driver/tikv_driver.go @@ -102,7 +102,7 @@ func TrySetupGlobalResourceController(ctx context.Context, serverID uint64, s kv return errors.New("cannot setup up resource controller, should use tikv storage") } - control, err := rmclient.NewResourceGroupController(serverID, store.GetPDClient(), rmclient.DefaultRequestUnitConfig()) + control, err := rmclient.NewResourceGroupController(ctx, serverID, store.GetPDClient(), nil) if err != nil { return err } diff --git a/store/mockstore/unistore/BUILD.bazel b/store/mockstore/unistore/BUILD.bazel index c25dad1d5fe6d..77821e16b18f5 100644 --- a/store/mockstore/unistore/BUILD.bazel +++ b/store/mockstore/unistore/BUILD.bazel @@ -29,6 +29,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/errorpb", "@com_github_pingcap_kvproto//pkg/keyspacepb", "@com_github_pingcap_kvproto//pkg/kvrpcpb", + "@com_github_pingcap_kvproto//pkg/meta_storagepb", "@com_github_pingcap_kvproto//pkg/metapb", "@com_github_pingcap_kvproto//pkg/mpp", "@com_github_pingcap_kvproto//pkg/pdpb", diff --git a/store/mockstore/unistore/pd.go b/store/mockstore/unistore/pd.go index 19ad6e2bd8fd0..3ec0c5926a297 100644 --- a/store/mockstore/unistore/pd.go +++ b/store/mockstore/unistore/pd.go @@ -22,6 +22,7 @@ import ( "sync/atomic" "github.com/pingcap/kvproto/pkg/keyspacepb" + "github.com/pingcap/kvproto/pkg/meta_storagepb" "github.com/pingcap/kvproto/pkg/pdpb" rmpb "github.com/pingcap/kvproto/pkg/resource_manager" us "github.com/pingcap/tidb/store/mockstore/unistore/tikv" @@ -274,17 +275,29 @@ func (c *pdClient) GetExternalTimestamp(ctx context.Context) (uint64, error) { } func (c *pdClient) GetTSWithinKeyspace(ctx context.Context, keyspaceID uint32) (int64, int64, error) { - return c.GetTS(ctx) + return 0, 0, nil } func (c *pdClient) GetTSWithinKeyspaceAsync(ctx context.Context, keyspaceID uint32) pd.TSFuture { - return c.GetTSAsync(ctx) + return nil } func (c *pdClient) GetLocalTSWithinKeyspace(ctx context.Context, dcLocation string, keyspaceID uint32) (int64, int64, error) { - return c.GetLocalTS(ctx, dcLocation) + return 0, 0, nil } func (c *pdClient) GetLocalTSWithinKeyspaceAsync(ctx context.Context, dcLocation string, keyspaceID uint32) pd.TSFuture { - return c.GetLocalTSAsync(ctx, dcLocation) + return nil +} + +func (c *pdClient) Watch(ctx context.Context, key []byte, opts ...pd.OpOption) (chan []*meta_storagepb.Event, error) { + return nil, nil +} + +func (c *pdClient) Get(ctx context.Context, key []byte, opts ...pd.OpOption) (*meta_storagepb.GetResponse, error) { + return nil, nil +} + +func (c *pdClient) Put(ctx context.Context, key []byte, value []byte, opts ...pd.OpOption) (*meta_storagepb.PutResponse, error) { + return nil, nil } diff --git a/table/tables/index.go b/table/tables/index.go index 8bc6c23c8a9f0..590d1b5fdb2a7 100644 --- a/table/tables/index.go +++ b/table/tables/index.go @@ -239,7 +239,12 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue if !distinct || skipCheck || opt.Untouched { val := idxVal - if keyIsTempIdxKey && !opt.Untouched { // Untouched key-values never occur in the storage. + if opt.Untouched && (keyIsTempIdxKey || len(tempKey) > 0) { + // Untouched key-values never occur in the storage and the temp index is not public. + // It is unnecessary to write the untouched temp index key-values. + continue + } + if keyIsTempIdxKey { tempVal := tablecodec.TempIndexValueElem{Value: idxVal, KeyVer: keyVer, Distinct: distinct} val = tempVal.Encode(nil) } @@ -248,10 +253,8 @@ func (c *index) Create(sctx sessionctx.Context, txn kv.Transaction, indexedValue return nil, err } if len(tempKey) > 0 { - if !opt.Untouched { // Untouched key-values never occur in the storage. - tempVal := tablecodec.TempIndexValueElem{Value: idxVal, KeyVer: keyVer, Distinct: distinct} - val = tempVal.Encode(nil) - } + tempVal := tablecodec.TempIndexValueElem{Value: idxVal, KeyVer: keyVer, Distinct: distinct} + val = tempVal.Encode(nil) err = txn.GetMemBuffer().Set(tempKey, val) if err != nil { return nil, err diff --git a/util/cpu/cpu.go b/util/cpu/cpu.go index 2803b4e106c49..6caa0af26037a 100644 --- a/util/cpu/cpu.go +++ b/util/cpu/cpu.go @@ -30,9 +30,12 @@ import ( var cpuUsage atomic.Float64 +// If your kernel is lower than linux 4.7, you cannot get the cpu usage in the container. +var unsupported atomic.Bool + // GetCPUUsage returns the cpu usage of the current process. -func GetCPUUsage() float64 { - return cpuUsage.Load() +func GetCPUUsage() (float64, bool) { + return cpuUsage.Load(), unsupported.Load() } // Observer is used to observe the cpu usage of the current process. @@ -56,6 +59,12 @@ func NewCPUObserver() *Observer { // Start starts the cpu observer. func (c *Observer) Start() { + _, err := cgroup.GetCgroupCPU() + if err != nil { + unsupported.Store(true) + log.Error("GetCgroupCPU", zap.Error(err)) + return + } c.wg.Add(1) go func() { ticker := time.NewTicker(100 * time.Millisecond) diff --git a/util/cpu/cpu_test.go b/util/cpu/cpu_test.go index a191227b72f78..6a6bc9943deb1 100644 --- a/util/cpu/cpu_test.go +++ b/util/cpu/cpu_test.go @@ -42,9 +42,14 @@ func TestCPUValue(t *testing.T) { }() } observer.Start() - time.Sleep(5 * time.Second) - require.GreaterOrEqual(t, GetCPUUsage(), 0.0) - require.Less(t, GetCPUUsage(), 1.0) + for n := 0; n < 10; n++ { + time.Sleep(1 * time.Second) + value, unsupported := GetCPUUsage() + require.False(t, unsupported) + require.GreaterOrEqual(t, value, 0.0) + require.Less(t, value, 1.0) + } + observer.Stop() close(exit) wg.Wait()