From 3c1b30b42ab803599f195e4bafe143ddd6e34c51 Mon Sep 17 00:00:00 2001 From: Arenatlx Date: Mon, 1 Jun 2020 10:47:17 +0800 Subject: [PATCH 1/2] cherry pick #17271 to release-3.0 Signed-off-by: sre-bot --- meta/autoid/autoid.go | 243 ++++++++++++++++++++++++++-- meta/autoid/autoid_test.go | 315 +++++++++++++++++++++++++++++++++++++ 2 files changed, 542 insertions(+), 16 deletions(-) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index f5ad8c78f1fc3..83cef8a7c91b5 100755 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -245,6 +245,11 @@ func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error // NextStep return new auto id step according to previous step and consuming time. func NextStep(curStep int64, consumeDur time.Duration) int64 { + failpoint.Inject("mockAutoIDCustomize", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(3) + } + }) failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) { if val.(bool) { failpoint.Return(step) @@ -387,14 +392,6 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset consumeDur := startTime.Sub(alloc.lastAllocTime) nextStep = NextStep(alloc.step, consumeDur) } - // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. - if nextStep <= n1 { - nextStep = mathutil.MinInt64(n1*2, maxStep) - } - // Store the step for non-customized-step allocator to calculate next dynamic step. - if !alloc.customStep { - alloc.step = nextStep - } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -402,6 +399,12 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset if err1 != nil { return err1 } + // CalcNeededBatchSize calculates the total batch size needed on global base. + n1 = CalcNeededBatchSize(newBase, int64(n), increment, offset, alloc.isUnsigned) + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. + if nextStep < n1 { + nextStep = n1 + } tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, nextStep) // The global rest is not enough for alloc. if tmpStep < n1 { @@ -414,6 +417,10 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset if err != nil { return 0, 0, err } + // Store the step for non-customized-step allocator to calculate next dynamic step. + if !alloc.customStep { + alloc.step = nextStep + } alloc.lastAllocTime = time.Now() if newBase == math.MaxInt64 { return 0, 0, ErrAutoincReadFailed @@ -454,14 +461,6 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse consumeDur := startTime.Sub(alloc.lastAllocTime) nextStep = NextStep(alloc.step, consumeDur) } - // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. - if nextStep <= n1 { - nextStep = mathutil.MinInt64(n1*2, maxStep) - } - // Store the step for non-customized-step allocator to calculate next dynamic step. - if !alloc.customStep { - alloc.step = nextStep - } err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error @@ -469,6 +468,12 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse if err1 != nil { return err1 } + // CalcNeededBatchSize calculates the total batch size needed on new base. + n1 = CalcNeededBatchSize(newBase, int64(n), increment, offset, alloc.isUnsigned) + // Although the step is customized by user, we still need to make sure nextStep is big enough for insert batch. + if nextStep < n1 { + nextStep = n1 + } tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(nextStep))) // The global rest is not enough for alloc. if tmpStep < n1 { @@ -481,6 +486,10 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse if err != nil { return 0, 0, err } + // Store the step for non-customized-step allocator to calculate next dynamic step. + if !alloc.customStep { + alloc.step = nextStep + } alloc.lastAllocTime = time.Now() if uint64(newBase) == math.MaxUint64 { return 0, 0, ErrAutoincReadFailed @@ -497,3 +506,205 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse alloc.base = int64(uint64(alloc.base) + uint64(n1)) return min, alloc.base, nil } +<<<<<<< HEAD +======= + +// alloc4Sequence is used to alloc value for sequence, there are several aspects different from autoid logic. +// 1: sequence allocation don't need check rebase. +// 2: sequence allocation don't need auto step. +// 3: sequence allocation may have negative growth. +// 4: sequence allocation batch length can be dissatisfied. +// 5: sequence batch allocation will be consumed immediately. +func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, round int64, err error) { + increment := alloc.sequence.Increment + offset := alloc.sequence.Start + minValue := alloc.sequence.MinValue + maxValue := alloc.sequence.MaxValue + cacheSize := alloc.sequence.CacheValue + if !alloc.sequence.Cache { + cacheSize = 1 + } + + var newBase, newEnd int64 + startTime := time.Now() + err = kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + var ( + err1 error + seqStep int64 + ) + // Get the real offset if the sequence is in cycle. + // round is used to count cycle times in sequence with cycle option. + if alloc.sequence.Cycle { + // GetSequenceCycle is used to get the flag `round`, which indicates whether the sequence is already in cycle. + round, err1 = m.GetSequenceCycle(alloc.dbID, tableID) + if err1 != nil { + return err1 + } + if round > 0 { + if increment > 0 { + offset = alloc.sequence.MinValue + } else { + offset = alloc.sequence.MaxValue + } + } + } + + // Get the global new base. + newBase, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + if err1 != nil { + return err1 + } + + // CalcNeededBatchSize calculates the total batch size needed. + seqStep, err1 = CalcSequenceBatchSize(newBase, cacheSize, increment, offset, minValue, maxValue) + + if err1 != nil && err1 == ErrAutoincReadFailed { + if !alloc.sequence.Cycle { + return err1 + } + // Reset the sequence base and offset. + if alloc.sequence.Increment > 0 { + newBase = alloc.sequence.MinValue - 1 + offset = alloc.sequence.MinValue + } else { + newBase = alloc.sequence.MaxValue + 1 + offset = alloc.sequence.MaxValue + } + err1 = m.SetSequenceValue(alloc.dbID, tableID, newBase) + if err1 != nil { + return err1 + } + + // Reset sequence round state value. + round++ + // SetSequenceCycle is used to store the flag `round` which indicates whether the sequence is already in cycle. + // round > 0 means the sequence is already in cycle, so the offset should be minvalue / maxvalue rather than sequence.start. + // TiDB is a stateless node, it should know whether the sequence is already in cycle when restart. + err1 = m.SetSequenceCycle(alloc.dbID, tableID, round) + if err1 != nil { + return err1 + } + + // Recompute the sequence next batch size. + seqStep, err1 = CalcSequenceBatchSize(newBase, cacheSize, increment, offset, minValue, maxValue) + if err1 != nil { + return err1 + } + } + var delta int64 + if alloc.sequence.Increment > 0 { + delta = seqStep + } else { + delta = -seqStep + } + newEnd, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, delta, alloc.allocType) + return err1 + }) + + // TODO: sequence metrics + metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + if err != nil { + return 0, 0, 0, err + } + logutil.Logger(context.TODO()).Debug("alloc sequence value", + zap.Uint64(" from value", uint64(newBase)), + zap.Uint64("to value", uint64(newEnd)), + zap.Int64("table ID", tableID), + zap.Int64("database ID", alloc.dbID)) + return newBase, newEnd, round, nil +} + +func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType AllocatorType) (int64, error) { + switch allocType { + // Currently, row id allocator and auto-increment value allocator shares the same key-value pair. + case RowIDAllocType, AutoIncrementType: + return m.GetAutoTableID(dbID, tableID) + case AutoRandomType: + return m.GetAutoRandomID(dbID, tableID) + case SequenceType: + return m.GetSequenceValue(dbID, tableID) + default: + return 0, ErrInvalidAllocatorType.GenWithStackByArgs() + } +} + +func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocType AllocatorType) (int64, error) { + switch allocType { + case RowIDAllocType, AutoIncrementType: + return m.GenAutoTableID(dbID, tableID, step) + case AutoRandomType: + return m.GenAutoRandomID(dbID, tableID, step) + case SequenceType: + return m.GenSequenceValue(dbID, tableID, step) + default: + return 0, ErrInvalidAllocatorType.GenWithStackByArgs() + } +} + +const signMask uint64 = 0x8000000000000000 + +// EncodeIntToCmpUint make int v to comparable uint type +func EncodeIntToCmpUint(v int64) uint64 { + return uint64(v) ^ signMask +} + +// DecodeCmpUintToInt decodes the u that encoded by EncodeIntToCmpUint +func DecodeCmpUintToInt(u uint64) int64 { + return int64(u ^ signMask) +} + +// TestModifyBaseAndEndInjection exported for testing modifying the base and end. +func TestModifyBaseAndEndInjection(alloc Allocator, base, end int64) { + alloc.(*allocator).mu.Lock() + alloc.(*allocator).base = base + alloc.(*allocator).end = end + alloc.(*allocator).mu.Unlock() +} + +// AutoRandomIDLayout is used to calculate the bits length of different section in auto_random id. +// The primary key with auto_random can only be `bigint` column, the total layout length of auto random is 64 bits. +// These are two type of layout: +// 1. Signed bigint: +// | [sign_bit] | [shard_bits] | [incremental_bits] | +// sign_bit(1 fixed) + shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed) +// 2. Unsigned bigint: +// | [shard_bits] | [incremental_bits] | +// shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed) +// Please always use NewAutoRandomIDLayout() to instantiate. +type AutoRandomIDLayout struct { + FieldType *types.FieldType + ShardBits uint64 + // Derived fields. + TypeBitsLength uint64 + IncrementalBits uint64 + HasSignBit bool +} + +// NewAutoRandomIDLayout create an instance of AutoRandomIDLayout. +func NewAutoRandomIDLayout(fieldType *types.FieldType, shardBits uint64) *AutoRandomIDLayout { + typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[mysql.TypeLonglong] * 8) + incrementalBits := typeBitsLength - shardBits + hasSignBit := !mysql.HasUnsignedFlag(fieldType.Flag) + if hasSignBit { + incrementalBits -= 1 + } + return &AutoRandomIDLayout{ + FieldType: fieldType, + ShardBits: shardBits, + TypeBitsLength: typeBitsLength, + IncrementalBits: incrementalBits, + HasSignBit: hasSignBit, + } +} + +// IncrementalBitsCapacity returns the max capacity of incremental section of the current layout. +func (l *AutoRandomIDLayout) IncrementalBitsCapacity() uint64 { + return uint64(math.Pow(2, float64(l.IncrementalBits)) - 1) +} + +// IncrementalMask returns 00..0[11..1], where [xxx] is the incremental section of the current layout. +func (l *AutoRandomIDLayout) IncrementalMask() int64 { + return (1 << l.IncrementalBits) - 1 +} +>>>>>>> 9162cfa... meta: fix the allocator batch size compute logic (#17271) diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index c1efb8ae711fb..59d402de87df6 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -565,3 +565,318 @@ func BenchmarkAllocator_Alloc(b *testing.B) { alloc.Alloc(2, 1, 1, 1) } } +<<<<<<< HEAD +======= + +func BenchmarkAllocator_SequenceAlloc(b *testing.B) { + b.StopTimer() + store, err := mockstore.NewMockStore() + if err != nil { + return + } + defer store.Close() + var seq *model.SequenceInfo + var sequenceBase int64 + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) + if err != nil { + return err + } + seq = &model.SequenceInfo{ + Start: 1, + Cycle: true, + Cache: false, + MinValue: -10, + MaxValue: math.MaxInt64, + Increment: 2, + CacheValue: 2000000, + } + seqTable := &model.TableInfo{ + ID: 1, + Name: model.NewCIStr("seq"), + Sequence: seq, + } + sequenceBase = seq.Start - 1 + err = m.CreateSequenceAndSetSeqValue(1, seqTable, sequenceBase) + return err + }) + if err != nil { + return + } + alloc := autoid.NewSequenceAllocator(store, 1, seq) + b.StartTimer() + for i := 0; i < b.N; i++ { + _, _, _, err := alloc.AllocSeqCache(1) + if err != nil { + fmt.Println("err") + } + } +} + +func BenchmarkAllocator_Seek(b *testing.B) { + base := int64(21421948021) + offset := int64(-351354365326) + increment := int64(3) + b.StartTimer() + for i := 0; i < b.N; i++ { + autoid.CalcSequenceBatchSize(base, 3, increment, offset, math.MinInt64, math.MaxInt64) + } +} + +func (*testSuite) TestSequenceAutoid(c *C) { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + defer store.Close() + + var seq *model.SequenceInfo + var sequenceBase int64 + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) + c.Assert(err, IsNil) + seq = &model.SequenceInfo{ + Start: 1, + Cycle: true, + Cache: true, + MinValue: -10, + MaxValue: 10, + Increment: 2, + CacheValue: 3, + } + seqTable := &model.TableInfo{ + ID: 1, + Name: model.NewCIStr("seq"), + Sequence: seq, + } + sequenceBase = seq.Start - 1 + err = m.CreateSequenceAndSetSeqValue(1, seqTable, sequenceBase) + c.Assert(err, IsNil) + return nil + }) + c.Assert(err, IsNil) + + alloc := autoid.NewSequenceAllocator(store, 1, seq) + c.Assert(alloc, NotNil) + + // allocate sequence cache. + base, end, round, err := alloc.AllocSeqCache(1) + c.Assert(err, IsNil) + c.Assert(base, Equals, int64(0)) + c.Assert(end, Equals, int64(5)) + c.Assert(round, Equals, int64(0)) + + // test the sequence batch size. + offset := seq.Start + size, err := autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) + c.Assert(err, IsNil) + c.Assert(size, Equals, end-base) + + // simulate the next value allocation. + nextVal, ok := autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(1)) + base = nextVal + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(3)) + base = nextVal + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(5)) + + base, end, round, err = alloc.AllocSeqCache(1) + c.Assert(err, IsNil) + c.Assert(base, Equals, int64(5)) + c.Assert(end, Equals, int64(10)) + c.Assert(round, Equals, int64(0)) + + // test the sequence batch size. + size, err = autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) + c.Assert(err, IsNil) + c.Assert(size, Equals, end-base) + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(7)) + base = nextVal + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(9)) + base = nextVal + + _, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + // the rest in cache in not enough for next value. + c.Assert(ok, Equals, false) + + base, end, round, err = alloc.AllocSeqCache(1) + c.Assert(err, IsNil) + c.Assert(base, Equals, int64(-11)) + c.Assert(end, Equals, int64(-6)) + // the round is already in cycle. + c.Assert(round, Equals, int64(1)) + + // test the sequence batch size. + size, err = autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) + c.Assert(err, IsNil) + c.Assert(size, Equals, end-base) + + offset = seq.MinValue + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(-10)) + base = nextVal + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(-8)) + base = nextVal + + nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + c.Assert(ok, Equals, true) + c.Assert(nextVal, Equals, int64(-6)) + base = nextVal + + _, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) + // the cache is already empty. + c.Assert(ok, Equals, false) +} + +func (*testSuite) TestConcurrentAllocSequence(c *C) { + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + defer store.Close() + + var seq *model.SequenceInfo + var sequenceBase int64 + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err1 := m.CreateDatabase(&model.DBInfo{ID: 2, Name: model.NewCIStr("a")}) + c.Assert(err1, IsNil) + seq = &model.SequenceInfo{ + Start: 100, + Cycle: false, + Cache: true, + MinValue: -100, + MaxValue: 100, + Increment: -2, + CacheValue: 3, + } + seqTable := &model.TableInfo{ + ID: 2, + Name: model.NewCIStr("seq"), + Sequence: seq, + } + if seq.Increment >= 0 { + sequenceBase = seq.Start - 1 + } else { + sequenceBase = seq.Start + 1 + } + err1 = m.CreateSequenceAndSetSeqValue(2, seqTable, sequenceBase) + c.Assert(err1, IsNil) + return nil + }) + c.Assert(err, IsNil) + + var mu sync.Mutex + wg := sync.WaitGroup{} + m := map[int64]struct{}{} + count := 10 + errCh := make(chan error, count) + + allocSequence := func() { + alloc := autoid.NewSequenceAllocator(store, 2, seq) + for j := 0; j < 3; j++ { + base, end, _, err1 := alloc.AllocSeqCache(2) + if err1 != nil { + errCh <- err1 + break + } + + errFlag := false + mu.Lock() + // sequence is negative-growth here. + for i := base - 1; i >= end; i-- { + if _, ok := m[i]; ok { + errCh <- fmt.Errorf("duplicate id:%v", i) + errFlag = true + mu.Unlock() + break + } + m[i] = struct{}{} + } + if errFlag { + break + } + mu.Unlock() + } + } + for i := 0; i < count; i++ { + wg.Add(1) + go func(num int) { + time.Sleep(time.Duration(num%10) * time.Microsecond) + allocSequence() + wg.Done() + }(i) + } + wg.Wait() + + close(errCh) + err = <-errCh + c.Assert(err, IsNil) +} + +// Fix a computation logic bug in allocator computation. +func (*testSuite) TestAllocComputationIssue(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDCustomize", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDCustomize"), IsNil) + }() + + store, err := mockstore.NewMockStore() + c.Assert(err, IsNil) + defer store.Close() + + err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) + c.Assert(err, IsNil) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 1, Name: model.NewCIStr("t")}) + c.Assert(err, IsNil) + err = m.CreateTableOrView(1, &model.TableInfo{ID: 2, Name: model.NewCIStr("t1")}) + c.Assert(err, IsNil) + return nil + }) + c.Assert(err, IsNil) + + // Since the test here is applicable to any type of allocators, autoid.RowIDAllocType is chosen. + unsignedAlloc := autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + c.Assert(unsignedAlloc, NotNil) + signedAlloc := autoid.NewAllocator(store, 1, false, autoid.RowIDAllocType) + c.Assert(signedAlloc, NotNil) + + // the next valid two value must be 13 & 16, batch size = 6. + err = unsignedAlloc.Rebase(1, 10, false) + c.Assert(err, IsNil) + // the next valid two value must be 10 & 13, batch size = 6. + err = signedAlloc.Rebase(2, 7, false) + c.Assert(err, IsNil) + // Simulate the rest cache is not enough for next batch, assuming 10 & 13, batch size = 4. + autoid.TestModifyBaseAndEndInjection(unsignedAlloc, 9, 9) + // Simulate the rest cache is not enough for next batch, assuming 10 & 13, batch size = 4. + autoid.TestModifyBaseAndEndInjection(signedAlloc, 4, 6) + + // Here will recompute the new allocator batch size base on new base = 10, which will get 6. + min, max, err := unsignedAlloc.Alloc(1, 2, 3, 1) + c.Assert(err, IsNil) + c.Assert(min, Equals, int64(10)) + c.Assert(max, Equals, int64(16)) + min, max, err = signedAlloc.Alloc(2, 2, 3, 1) + c.Assert(err, IsNil) + c.Assert(min, Equals, int64(7)) + c.Assert(max, Equals, int64(13)) +} +>>>>>>> 9162cfa... meta: fix the allocator batch size compute logic (#17271) From 3326ac1a1662a97647da0504c87d49094b66381e Mon Sep 17 00:00:00 2001 From: AilinKid <314806019@qq.com> Date: Mon, 1 Jun 2020 11:54:36 +0800 Subject: [PATCH 2/2] . Signed-off-by: AilinKid <314806019@qq.com> --- meta/autoid/autoid.go | 194 -------------------------- meta/autoid/autoid_test.go | 270 +------------------------------------ 2 files changed, 3 insertions(+), 461 deletions(-) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 83cef8a7c91b5..85d5b119c5c3e 100755 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -506,153 +506,6 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse alloc.base = int64(uint64(alloc.base) + uint64(n1)) return min, alloc.base, nil } -<<<<<<< HEAD -======= - -// alloc4Sequence is used to alloc value for sequence, there are several aspects different from autoid logic. -// 1: sequence allocation don't need check rebase. -// 2: sequence allocation don't need auto step. -// 3: sequence allocation may have negative growth. -// 4: sequence allocation batch length can be dissatisfied. -// 5: sequence batch allocation will be consumed immediately. -func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, round int64, err error) { - increment := alloc.sequence.Increment - offset := alloc.sequence.Start - minValue := alloc.sequence.MinValue - maxValue := alloc.sequence.MaxValue - cacheSize := alloc.sequence.CacheValue - if !alloc.sequence.Cache { - cacheSize = 1 - } - - var newBase, newEnd int64 - startTime := time.Now() - err = kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { - m := meta.NewMeta(txn) - var ( - err1 error - seqStep int64 - ) - // Get the real offset if the sequence is in cycle. - // round is used to count cycle times in sequence with cycle option. - if alloc.sequence.Cycle { - // GetSequenceCycle is used to get the flag `round`, which indicates whether the sequence is already in cycle. - round, err1 = m.GetSequenceCycle(alloc.dbID, tableID) - if err1 != nil { - return err1 - } - if round > 0 { - if increment > 0 { - offset = alloc.sequence.MinValue - } else { - offset = alloc.sequence.MaxValue - } - } - } - - // Get the global new base. - newBase, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) - if err1 != nil { - return err1 - } - - // CalcNeededBatchSize calculates the total batch size needed. - seqStep, err1 = CalcSequenceBatchSize(newBase, cacheSize, increment, offset, minValue, maxValue) - - if err1 != nil && err1 == ErrAutoincReadFailed { - if !alloc.sequence.Cycle { - return err1 - } - // Reset the sequence base and offset. - if alloc.sequence.Increment > 0 { - newBase = alloc.sequence.MinValue - 1 - offset = alloc.sequence.MinValue - } else { - newBase = alloc.sequence.MaxValue + 1 - offset = alloc.sequence.MaxValue - } - err1 = m.SetSequenceValue(alloc.dbID, tableID, newBase) - if err1 != nil { - return err1 - } - - // Reset sequence round state value. - round++ - // SetSequenceCycle is used to store the flag `round` which indicates whether the sequence is already in cycle. - // round > 0 means the sequence is already in cycle, so the offset should be minvalue / maxvalue rather than sequence.start. - // TiDB is a stateless node, it should know whether the sequence is already in cycle when restart. - err1 = m.SetSequenceCycle(alloc.dbID, tableID, round) - if err1 != nil { - return err1 - } - - // Recompute the sequence next batch size. - seqStep, err1 = CalcSequenceBatchSize(newBase, cacheSize, increment, offset, minValue, maxValue) - if err1 != nil { - return err1 - } - } - var delta int64 - if alloc.sequence.Increment > 0 { - delta = seqStep - } else { - delta = -seqStep - } - newEnd, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, delta, alloc.allocType) - return err1 - }) - - // TODO: sequence metrics - metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) - if err != nil { - return 0, 0, 0, err - } - logutil.Logger(context.TODO()).Debug("alloc sequence value", - zap.Uint64(" from value", uint64(newBase)), - zap.Uint64("to value", uint64(newEnd)), - zap.Int64("table ID", tableID), - zap.Int64("database ID", alloc.dbID)) - return newBase, newEnd, round, nil -} - -func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType AllocatorType) (int64, error) { - switch allocType { - // Currently, row id allocator and auto-increment value allocator shares the same key-value pair. - case RowIDAllocType, AutoIncrementType: - return m.GetAutoTableID(dbID, tableID) - case AutoRandomType: - return m.GetAutoRandomID(dbID, tableID) - case SequenceType: - return m.GetSequenceValue(dbID, tableID) - default: - return 0, ErrInvalidAllocatorType.GenWithStackByArgs() - } -} - -func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocType AllocatorType) (int64, error) { - switch allocType { - case RowIDAllocType, AutoIncrementType: - return m.GenAutoTableID(dbID, tableID, step) - case AutoRandomType: - return m.GenAutoRandomID(dbID, tableID, step) - case SequenceType: - return m.GenSequenceValue(dbID, tableID, step) - default: - return 0, ErrInvalidAllocatorType.GenWithStackByArgs() - } -} - -const signMask uint64 = 0x8000000000000000 - -// EncodeIntToCmpUint make int v to comparable uint type -func EncodeIntToCmpUint(v int64) uint64 { - return uint64(v) ^ signMask -} - -// DecodeCmpUintToInt decodes the u that encoded by EncodeIntToCmpUint -func DecodeCmpUintToInt(u uint64) int64 { - return int64(u ^ signMask) -} // TestModifyBaseAndEndInjection exported for testing modifying the base and end. func TestModifyBaseAndEndInjection(alloc Allocator, base, end int64) { @@ -661,50 +514,3 @@ func TestModifyBaseAndEndInjection(alloc Allocator, base, end int64) { alloc.(*allocator).end = end alloc.(*allocator).mu.Unlock() } - -// AutoRandomIDLayout is used to calculate the bits length of different section in auto_random id. -// The primary key with auto_random can only be `bigint` column, the total layout length of auto random is 64 bits. -// These are two type of layout: -// 1. Signed bigint: -// | [sign_bit] | [shard_bits] | [incremental_bits] | -// sign_bit(1 fixed) + shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed) -// 2. Unsigned bigint: -// | [shard_bits] | [incremental_bits] | -// shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed) -// Please always use NewAutoRandomIDLayout() to instantiate. -type AutoRandomIDLayout struct { - FieldType *types.FieldType - ShardBits uint64 - // Derived fields. - TypeBitsLength uint64 - IncrementalBits uint64 - HasSignBit bool -} - -// NewAutoRandomIDLayout create an instance of AutoRandomIDLayout. -func NewAutoRandomIDLayout(fieldType *types.FieldType, shardBits uint64) *AutoRandomIDLayout { - typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[mysql.TypeLonglong] * 8) - incrementalBits := typeBitsLength - shardBits - hasSignBit := !mysql.HasUnsignedFlag(fieldType.Flag) - if hasSignBit { - incrementalBits -= 1 - } - return &AutoRandomIDLayout{ - FieldType: fieldType, - ShardBits: shardBits, - TypeBitsLength: typeBitsLength, - IncrementalBits: incrementalBits, - HasSignBit: hasSignBit, - } -} - -// IncrementalBitsCapacity returns the max capacity of incremental section of the current layout. -func (l *AutoRandomIDLayout) IncrementalBitsCapacity() uint64 { - return uint64(math.Pow(2, float64(l.IncrementalBits)) - 1) -} - -// IncrementalMask returns 00..0[11..1], where [xxx] is the incremental section of the current layout. -func (l *AutoRandomIDLayout) IncrementalMask() int64 { - return (1 << l.IncrementalBits) - 1 -} ->>>>>>> 9162cfa... meta: fix the allocator batch size compute logic (#17271) diff --git a/meta/autoid/autoid_test.go b/meta/autoid/autoid_test.go index 59d402de87df6..14364d5edba7b 100644 --- a/meta/autoid/autoid_test.go +++ b/meta/autoid/autoid_test.go @@ -565,269 +565,6 @@ func BenchmarkAllocator_Alloc(b *testing.B) { alloc.Alloc(2, 1, 1, 1) } } -<<<<<<< HEAD -======= - -func BenchmarkAllocator_SequenceAlloc(b *testing.B) { - b.StopTimer() - store, err := mockstore.NewMockStore() - if err != nil { - return - } - defer store.Close() - var seq *model.SequenceInfo - var sequenceBase int64 - err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { - m := meta.NewMeta(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) - if err != nil { - return err - } - seq = &model.SequenceInfo{ - Start: 1, - Cycle: true, - Cache: false, - MinValue: -10, - MaxValue: math.MaxInt64, - Increment: 2, - CacheValue: 2000000, - } - seqTable := &model.TableInfo{ - ID: 1, - Name: model.NewCIStr("seq"), - Sequence: seq, - } - sequenceBase = seq.Start - 1 - err = m.CreateSequenceAndSetSeqValue(1, seqTable, sequenceBase) - return err - }) - if err != nil { - return - } - alloc := autoid.NewSequenceAllocator(store, 1, seq) - b.StartTimer() - for i := 0; i < b.N; i++ { - _, _, _, err := alloc.AllocSeqCache(1) - if err != nil { - fmt.Println("err") - } - } -} - -func BenchmarkAllocator_Seek(b *testing.B) { - base := int64(21421948021) - offset := int64(-351354365326) - increment := int64(3) - b.StartTimer() - for i := 0; i < b.N; i++ { - autoid.CalcSequenceBatchSize(base, 3, increment, offset, math.MinInt64, math.MaxInt64) - } -} - -func (*testSuite) TestSequenceAutoid(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer store.Close() - - var seq *model.SequenceInfo - var sequenceBase int64 - err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { - m := meta.NewMeta(txn) - err = m.CreateDatabase(&model.DBInfo{ID: 1, Name: model.NewCIStr("a")}) - c.Assert(err, IsNil) - seq = &model.SequenceInfo{ - Start: 1, - Cycle: true, - Cache: true, - MinValue: -10, - MaxValue: 10, - Increment: 2, - CacheValue: 3, - } - seqTable := &model.TableInfo{ - ID: 1, - Name: model.NewCIStr("seq"), - Sequence: seq, - } - sequenceBase = seq.Start - 1 - err = m.CreateSequenceAndSetSeqValue(1, seqTable, sequenceBase) - c.Assert(err, IsNil) - return nil - }) - c.Assert(err, IsNil) - - alloc := autoid.NewSequenceAllocator(store, 1, seq) - c.Assert(alloc, NotNil) - - // allocate sequence cache. - base, end, round, err := alloc.AllocSeqCache(1) - c.Assert(err, IsNil) - c.Assert(base, Equals, int64(0)) - c.Assert(end, Equals, int64(5)) - c.Assert(round, Equals, int64(0)) - - // test the sequence batch size. - offset := seq.Start - size, err := autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) - c.Assert(err, IsNil) - c.Assert(size, Equals, end-base) - - // simulate the next value allocation. - nextVal, ok := autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(1)) - base = nextVal - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(3)) - base = nextVal - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(5)) - - base, end, round, err = alloc.AllocSeqCache(1) - c.Assert(err, IsNil) - c.Assert(base, Equals, int64(5)) - c.Assert(end, Equals, int64(10)) - c.Assert(round, Equals, int64(0)) - - // test the sequence batch size. - size, err = autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) - c.Assert(err, IsNil) - c.Assert(size, Equals, end-base) - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(7)) - base = nextVal - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(9)) - base = nextVal - - _, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - // the rest in cache in not enough for next value. - c.Assert(ok, Equals, false) - - base, end, round, err = alloc.AllocSeqCache(1) - c.Assert(err, IsNil) - c.Assert(base, Equals, int64(-11)) - c.Assert(end, Equals, int64(-6)) - // the round is already in cycle. - c.Assert(round, Equals, int64(1)) - - // test the sequence batch size. - size, err = autoid.CalcSequenceBatchSize(sequenceBase, seq.CacheValue, seq.Increment, offset, seq.MinValue, seq.MaxValue) - c.Assert(err, IsNil) - c.Assert(size, Equals, end-base) - - offset = seq.MinValue - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(-10)) - base = nextVal - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(-8)) - base = nextVal - - nextVal, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - c.Assert(ok, Equals, true) - c.Assert(nextVal, Equals, int64(-6)) - base = nextVal - - _, ok = autoid.SeekToFirstSequenceValue(base, seq.Increment, offset, base, end) - // the cache is already empty. - c.Assert(ok, Equals, false) -} - -func (*testSuite) TestConcurrentAllocSequence(c *C) { - store, err := mockstore.NewMockStore() - c.Assert(err, IsNil) - defer store.Close() - - var seq *model.SequenceInfo - var sequenceBase int64 - err = kv.RunInNewTxn(store, false, func(txn kv.Transaction) error { - m := meta.NewMeta(txn) - err1 := m.CreateDatabase(&model.DBInfo{ID: 2, Name: model.NewCIStr("a")}) - c.Assert(err1, IsNil) - seq = &model.SequenceInfo{ - Start: 100, - Cycle: false, - Cache: true, - MinValue: -100, - MaxValue: 100, - Increment: -2, - CacheValue: 3, - } - seqTable := &model.TableInfo{ - ID: 2, - Name: model.NewCIStr("seq"), - Sequence: seq, - } - if seq.Increment >= 0 { - sequenceBase = seq.Start - 1 - } else { - sequenceBase = seq.Start + 1 - } - err1 = m.CreateSequenceAndSetSeqValue(2, seqTable, sequenceBase) - c.Assert(err1, IsNil) - return nil - }) - c.Assert(err, IsNil) - - var mu sync.Mutex - wg := sync.WaitGroup{} - m := map[int64]struct{}{} - count := 10 - errCh := make(chan error, count) - - allocSequence := func() { - alloc := autoid.NewSequenceAllocator(store, 2, seq) - for j := 0; j < 3; j++ { - base, end, _, err1 := alloc.AllocSeqCache(2) - if err1 != nil { - errCh <- err1 - break - } - - errFlag := false - mu.Lock() - // sequence is negative-growth here. - for i := base - 1; i >= end; i-- { - if _, ok := m[i]; ok { - errCh <- fmt.Errorf("duplicate id:%v", i) - errFlag = true - mu.Unlock() - break - } - m[i] = struct{}{} - } - if errFlag { - break - } - mu.Unlock() - } - } - for i := 0; i < count; i++ { - wg.Add(1) - go func(num int) { - time.Sleep(time.Duration(num%10) * time.Microsecond) - allocSequence() - wg.Done() - }(i) - } - wg.Wait() - - close(errCh) - err = <-errCh - c.Assert(err, IsNil) -} // Fix a computation logic bug in allocator computation. func (*testSuite) TestAllocComputationIssue(c *C) { @@ -836,7 +573,7 @@ func (*testSuite) TestAllocComputationIssue(c *C) { c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDCustomize"), IsNil) }() - store, err := mockstore.NewMockStore() + store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) defer store.Close() @@ -853,9 +590,9 @@ func (*testSuite) TestAllocComputationIssue(c *C) { c.Assert(err, IsNil) // Since the test here is applicable to any type of allocators, autoid.RowIDAllocType is chosen. - unsignedAlloc := autoid.NewAllocator(store, 1, true, autoid.RowIDAllocType) + unsignedAlloc := autoid.NewAllocator(store, 1, true) c.Assert(unsignedAlloc, NotNil) - signedAlloc := autoid.NewAllocator(store, 1, false, autoid.RowIDAllocType) + signedAlloc := autoid.NewAllocator(store, 1, false) c.Assert(signedAlloc, NotNil) // the next valid two value must be 13 & 16, batch size = 6. @@ -879,4 +616,3 @@ func (*testSuite) TestAllocComputationIssue(c *C) { c.Assert(min, Equals, int64(7)) c.Assert(max, Equals, int64(13)) } ->>>>>>> 9162cfa... meta: fix the allocator batch size compute logic (#17271)