@ -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",
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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()
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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.
|
||||
|
||||
@ -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)
|
||||
|
||||
Reference in New Issue
Block a user