From eeafa7da9dc60fcc2cc5447d19e24c767e9fa4b5 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Wed, 7 May 2025 11:07:30 +0200 Subject: [PATCH] meta: gopls modernize (#60917) close pingcap/tidb#60916 --- build/nogo_config.json | 1 + pkg/meta/autoid/autoid_test.go | 4 ++-- pkg/meta/autoid/seq_autoid_test.go | 4 ++-- pkg/meta/meta.go | 28 ++++++++++++++-------------- pkg/meta/meta_test.go | 2 +- pkg/meta/model/job.go | 7 ++----- pkg/meta/model/job_args_test.go | 2 +- pkg/meta/model/table.go | 8 ++------ pkg/meta/model/table_test.go | 2 +- 9 files changed, 26 insertions(+), 32 deletions(-) diff --git a/build/nogo_config.json b/build/nogo_config.json index 67fba9543d..11c755d514 100644 --- a/build/nogo_config.json +++ b/build/nogo_config.json @@ -363,6 +363,7 @@ "pkg/distsql/": "distsql code", "pkg/expression/": "expression code", "pkg/infoschema/": "infoschema code", + "pkg/meta/": "meta code", "pkg/owner/": "owner code", "pkg/planner/cardinality": "pkg/planner/cardinality code", "pkg/planner/core/casetest/tpch": "pkg/planner/core/casetest/tpch code", diff --git a/pkg/meta/autoid/autoid_test.go b/pkg/meta/autoid/autoid_test.go index e28ab9c62e..2ffc9d7197 100644 --- a/pkg/meta/autoid/autoid_test.go +++ b/pkg/meta/autoid/autoid_test.go @@ -456,7 +456,7 @@ func TestConcurrentAlloc(t *testing.T) { allocIDs := func() { ctx := context.Background() alloc := autoid.NewAllocator(mockRequirement{store}, dbID, tblID, false, autoid.RowIDAllocType) - for j := 0; j < int(autoid.GetStep())+5; j++ { + for range int(autoid.GetStep()) + 5 { _, id, err1 := alloc.Alloc(ctx, 1, 1, 1) if err1 != nil { errCh <- err1 @@ -497,7 +497,7 @@ func TestConcurrentAlloc(t *testing.T) { mu.Unlock() } } - for i := 0; i < count; i++ { + for range count { num := 1 wg.Run(func() { time.Sleep(time.Duration(num%10) * time.Microsecond) diff --git a/pkg/meta/autoid/seq_autoid_test.go b/pkg/meta/autoid/seq_autoid_test.go index 4e2143fa4f..2166ff52ac 100644 --- a/pkg/meta/autoid/seq_autoid_test.go +++ b/pkg/meta/autoid/seq_autoid_test.go @@ -204,7 +204,7 @@ func TestConcurrentAllocSequence(t *testing.T) { allocSequence := func() { alloc := autoid.NewSequenceAllocator(store, 2, 2, seq) - for j := 0; j < 3; j++ { + for range 3 { base, end, _, err1 := alloc.AllocSeqCache() if err1 != nil { errCh <- err1 @@ -229,7 +229,7 @@ func TestConcurrentAllocSequence(t *testing.T) { mu.Unlock() } } - for i := 0; i < count; i++ { + for i := range count { num := i wg.Run(func() { time.Sleep(time.Duration(num%10) * time.Microsecond) diff --git a/pkg/meta/meta.go b/pkg/meta/meta.go index 92f16ecfca..e296e1b7fc 100644 --- a/pkg/meta/meta.go +++ b/pkg/meta/meta.go @@ -279,11 +279,11 @@ func (m *Mutator) GetPolicyID() (int64, error) { } func (*Mutator) policyKey(policyID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mPolicyPrefix, policyID)) + return fmt.Appendf(nil, "%s:%d", mPolicyPrefix, policyID) } func (*Mutator) resourceGroupKey(groupID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mResourceGroupPrefix, groupID)) + return fmt.Appendf(nil, "%s:%d", mResourceGroupPrefix, groupID) } func (*Mutator) dbKey(dbID int64) []byte { @@ -292,7 +292,7 @@ func (*Mutator) dbKey(dbID int64) []byte { // DBkey encodes the dbID into dbKey. func DBkey(dbID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mDBPrefix, dbID)) + return fmt.Appendf(nil, "%s:%d", mDBPrefix, dbID) } // ParseDBKey decodes the dbkey to get dbID. @@ -317,7 +317,7 @@ func (*Mutator) autoTableIDKey(tableID int64) []byte { // AutoTableIDKey decodes the auto tableID key. func AutoTableIDKey(tableID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mTableIDPrefix, tableID)) + return fmt.Appendf(nil, "%s:%d", mTableIDPrefix, tableID) } // IsAutoTableIDKey checks whether the key is auto tableID key. @@ -342,7 +342,7 @@ func (*Mutator) autoIncrementIDKey(tableID int64) []byte { // AutoIncrementIDKey decodes the auto inc table key. func AutoIncrementIDKey(tableID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mIncIDPrefix, tableID)) + return fmt.Appendf(nil, "%s:%d", mIncIDPrefix, tableID) } // IsAutoIncrementIDKey checks whether the key is auto increment key. @@ -367,7 +367,7 @@ func (*Mutator) autoRandomTableIDKey(tableID int64) []byte { // AutoRandomTableIDKey encodes the auto random tableID key. func AutoRandomTableIDKey(tableID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mRandomIDPrefix, tableID)) + return fmt.Appendf(nil, "%s:%d", mRandomIDPrefix, tableID) } // IsAutoRandomTableIDKey checks whether the key is auto random tableID key. @@ -392,7 +392,7 @@ func (*Mutator) tableKey(tableID int64) []byte { // TableKey encodes the tableID into tableKey. func TableKey(tableID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mTablePrefix, tableID)) + return fmt.Appendf(nil, "%s:%d", mTablePrefix, tableID) } // IsTableKey checks whether the tableKey comes from TableKey(). @@ -417,7 +417,7 @@ func (*Mutator) sequenceKey(sequenceID int64) []byte { // SequenceKey encodes the sequence key. func SequenceKey(sequenceID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mSequencePrefix, sequenceID)) + return fmt.Appendf(nil, "%s:%d", mSequencePrefix, sequenceID) } // IsSequenceKey checks whether the key is sequence key. @@ -437,7 +437,7 @@ func ParseSequenceKey(key []byte) (int64, error) { } func (*Mutator) sequenceCycleKey(sequenceID int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mSeqCyclePrefix, sequenceID)) + return fmt.Appendf(nil, "%s:%d", mSeqCyclePrefix, sequenceID) } // DDLJobHistoryKey is only used for testing. @@ -1004,7 +1004,7 @@ func splitRangeInt64Max(n int64) [][]string { // 9999999999999999999 is the max number than maxInt64 in string format. batch := 9999999999999999999 / uint64(n) - for k := int64(0); k < n; k++ { + for k := range n { start := batch * uint64(k) end := batch * uint64(k+1) @@ -1034,15 +1034,15 @@ func IterAllTables(ctx context.Context, store kv.Storage, startTs uint64, concur kvRanges := splitRangeInt64Max(int64(concurrency)) mu := sync.Mutex{} - for i := 0; i < concurrency; i++ { + for i := range concurrency { snapshot := store.GetSnapshot(kv.NewVersion(startTs)) snapshot.SetOption(kv.RequestSourceInternal, true) snapshot.SetOption(kv.RequestSourceType, kv.InternalTxnMeta) t := structure.NewStructure(snapshot, nil, mMetaPrefix) workGroup.Go(func() error { - startKey := []byte(fmt.Sprintf("%s:", mDBPrefix)) + startKey := fmt.Appendf(nil, "%s:", mDBPrefix) startKey = codec.EncodeBytes(startKey, []byte(kvRanges[i][0])) - endKey := []byte(fmt.Sprintf("%s:", mDBPrefix)) + endKey := fmt.Appendf(nil, "%s:", mDBPrefix) endKey = codec.EncodeBytes(endKey, []byte(kvRanges[i][1])) return t.IterateHashWithBoundedKey(startKey, endKey, func(key []byte, field []byte, value []byte) error { @@ -1798,7 +1798,7 @@ func DecodeElement(b []byte) (*Element, error) { } func (*Mutator) schemaDiffKey(schemaVersion int64) []byte { - return []byte(fmt.Sprintf("%s:%d", mSchemaDiffPrefix, schemaVersion)) + return fmt.Appendf(nil, "%s:%d", mSchemaDiffPrefix, schemaVersion) } // GetSchemaDiff gets the modification information on a given schema version. diff --git a/pkg/meta/meta_test.go b/pkg/meta/meta_test.go index 84111aa076..348c113dc7 100644 --- a/pkg/meta/meta_test.go +++ b/pkg/meta/meta_test.go @@ -550,7 +550,7 @@ func BenchmarkGenGlobalIDOneByOne(b *testing.B) { b.ResetTimer() var id int64 for i := 0; i < b.N; i++ { - for j := 0; j < 10; j++ { + for range 10 { id, _ = m.GenGlobalID() } } diff --git a/pkg/meta/model/job.go b/pkg/meta/model/job.go index fe4dc1b715..22b3503e39 100644 --- a/pkg/meta/model/job.go +++ b/pkg/meta/model/job.go @@ -561,12 +561,9 @@ func (job *Job) decodeArgs(args ...any) error { return errors.Trace(err) } - sz := len(rawArgs) - if sz > len(args) { - sz = len(args) - } + sz := min(len(rawArgs), len(args)) - for i := 0; i < sz; i++ { + for i := range sz { if err := json.Unmarshal(rawArgs[i], args[i]); err != nil { return errors.Trace(err) } diff --git a/pkg/meta/model/job_args_test.go b/pkg/meta/model/job_args_test.go index 30a72cb1d1..933872f1a3 100644 --- a/pkg/meta/model/job_args_test.go +++ b/pkg/meta/model/job_args_test.go @@ -191,7 +191,7 @@ func TestBatchCreateTableArgs(t *testing.T) { require.NoError(t, j2.Decode(getJobBytes(t, inArgs, JobVersion1, ActionCreateTables))) args, err := GetBatchCreateTableArgs(j2) require.NoError(t, err) - for i := 0; i < len(inArgs.Tables); i++ { + for i := range inArgs.Tables { require.EqualValues(t, inArgs.Tables[i].TableInfo, args.Tables[i].TableInfo) require.EqualValues(t, true, args.Tables[i].FKCheck) } diff --git a/pkg/meta/model/table.go b/pkg/meta/model/table.go index 8043796069..53b1290917 100644 --- a/pkg/meta/model/table.go +++ b/pkg/meta/model/table.go @@ -17,6 +17,7 @@ package model import ( "bytes" "fmt" + "slices" "strconv" "strings" "time" @@ -712,12 +713,7 @@ type TiFlashReplicaInfo struct { // IsPartitionAvailable checks whether the partition table replica was available. func (tr *TiFlashReplicaInfo) IsPartitionAvailable(pid int64) bool { - for _, id := range tr.AvailablePartitionIDs { - if id == pid { - return true - } - } - return false + return slices.Contains(tr.AvailablePartitionIDs, pid) } // ViewInfo provides meta data describing a DB view. diff --git a/pkg/meta/model/table_test.go b/pkg/meta/model/table_test.go index b22647a62b..6f6c7c565e 100644 --- a/pkg/meta/model/table_test.go +++ b/pkg/meta/model/table_test.go @@ -29,7 +29,7 @@ import ( func checkOffsets(t *testing.T, tbl *TableInfo, ids ...int) { require.Equal(t, len(ids), len(tbl.Columns)) - for i := 0; i < len(ids); i++ { + for i := range ids { expected := fmt.Sprintf("c_%d", ids[i]) require.Equal(t, expected, tbl.Columns[i].Name.L) require.Equal(t, i, tbl.Columns[i].Offset)