Skip to content

Commit

Permalink
ddl, meta: add GenGlobalIDs and tiny update
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Mar 11, 2020
1 parent 93c2486 commit 40a73d5
Show file tree
Hide file tree
Showing 8 changed files with 80 additions and 12 deletions.
6 changes: 1 addition & 5 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,11 +110,7 @@ func testDropColumn(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo,
}

func (s *testColumnSuite) TestColumn(c *C) {
d := newDDL(
context.Background(),
WithStore(s.store),
WithLease(testLease),
)
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
defer d.Stop()

tblInfo := testTableInfo(c, d, "t1", 3)
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -443,7 +443,7 @@ func (d *ddl) start(ctx context.Context, ctxPool *pools.ResourcePool) {
func() { d.limitDDLJobs() },
func(r interface{}) {
if r != nil {
logutil.BgLogger().Error("[ddl] limit DDL jobs meet panic",
logutil.Logger(ddlLogCtx).Error("[ddl] limit DDL jobs meet panic",
zap.String("ID", d.uuid), zap.Reflect("r", r), zap.Stack("stack trace"))
metrics.PanicCounter.WithLabelValues(metrics.LabelDDL).Inc()
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func (d *ddl) restartWorkers(ctx context.Context) {
util.WithRecovery(
func() { d.limitDDLJobs() },
func(r interface{}) {
logutil.BgLogger().Error("[ddl] DDL add batch DDL jobs meet panic",
logutil.Logger(ddlLogCtx).Error("[ddl] DDL add batch DDL jobs meet panic",
zap.String("ID", d.uuid), zap.Reflect("r", r), zap.Stack("stack trace"))
metrics.PanicCounter.WithLabelValues(metrics.LabelDDL).Inc()
})
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,7 +240,7 @@ func (d *ddl) addBatchDDLJobs(tasks []*limitJobTask) {
metrics.DDLWorkerHistogram.WithLabelValues(metrics.WorkerAddDDLJob, task.job.Type.String(),
metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds())
}
logutil.BgLogger().Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs))
logutil.Logger(ddlLogCtx).Info("[ddl] add DDL jobs", zap.Int("batch count", len(tasks)), zap.String("jobs", jobs))
}

// getHistoryDDLJob gets a DDL job with job's ID from history queue.
Expand Down
4 changes: 0 additions & 4 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/log"
"github.com/pingcap/parser"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/terror"
Expand All @@ -54,10 +53,7 @@ import (
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tidb/util/testutil"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
)

func TestT(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d/go.mo
github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI=
github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI=
github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU=
github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI=
github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w=
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
Expand Down
17 changes: 17 additions & 0 deletions meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,23 @@ func (m *Meta) GenGlobalID() (int64, error) {
return m.txn.Inc(mNextGlobalIDKey, 1)
}

// GenGlobalIDs generates the next n global IDs.
func (m *Meta) GenGlobalIDs(n int) ([]int64, error) {
globalIDMutex.Lock()
defer globalIDMutex.Unlock()

newID, err := m.txn.Inc(mNextGlobalIDKey, int64(n))
if err != nil {
return nil, err
}
origID := newID - int64(n)
ids := make([]int64, 0, n)
for i := origID + 1; i <= newID; i++ {
ids = append(ids, i)
}
return ids, nil
}

// GetGlobalID gets current global id.
func (m *Meta) GetGlobalID() (int64, error) {
return m.txn.GetInt64(mNextGlobalIDKey)
Expand Down
58 changes: 58 additions & 0 deletions meta/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"context"
"math"
"strconv"
"sync"
"testing"
"time"

Expand Down Expand Up @@ -57,6 +58,24 @@ func (s *testSuite) TestMeta(c *C) {
c.Assert(err, IsNil)
c.Assert(n, Equals, int64(1))

var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
ids, err := t.GenGlobalIDs(3)
c.Assert(err, IsNil)
anyMatch(c, ids, []int64{2, 3, 4}, []int64{6, 7, 8})
}()

wg.Add(1)
go func() {
defer wg.Done()
ids, err := t.GenGlobalIDs(4)
c.Assert(err, IsNil)
anyMatch(c, ids, []int64{5, 6, 7, 8}, []int64{2, 3, 4, 5})
}()
wg.Wait()

n, err = t.GetSchemaVersion()
c.Assert(err, IsNil)
c.Assert(n, Equals, int64(0))
Expand Down Expand Up @@ -416,3 +435,42 @@ func (s *testSuite) TestDDL(c *C) {
err = txn1.Commit(context.Background())
c.Assert(err, IsNil)
}

func (s *testSuite) BenchmarkGenGlobalIDs(c *C) {
defer testleak.AfterTest(c)()
store, err := mockstore.NewMockTikvStore()
c.Assert(err, IsNil)
defer store.Close()

txn, err := store.Begin()
c.Assert(err, IsNil)
defer txn.Rollback()

t := meta.NewMeta(txn)

c.ResetTimer()
var ids []int64
for i := 0; i < c.N; i++ {
ids, _ = t.GenGlobalIDs(10)
}
c.Assert(ids, HasLen, 10)
c.Assert(ids[9], Equals, int64(c.N)*10)
}

func anyMatch(c *C, ids []int64, candidates ...[]int64) {
var match bool
OUTER:
for _, cand := range candidates {
if len(ids) != len(cand) {
continue
}
for i, v := range cand {
if ids[i] != v {
continue OUTER
}
}
match = true
break
}
c.Assert(match, IsTrue)
}

0 comments on commit 40a73d5

Please sign in to comment.