Skip to content
This repository has been archived by the owner on Dec 8, 2021. It is now read-only.

Commit

Permalink
backend: fix auto random default value for primary key (#457)
Browse files Browse the repository at this point in the history
* fix auto generate auto random primary key column

* fix default for auto random primary key

* fix test

* use prev row id for auto random and add a test

* replace chunck with session opt

* fix

* fix
  • Loading branch information
glorv authored Nov 9, 2020
1 parent 66281df commit 66ac09a
Show file tree
Hide file tree
Showing 19 changed files with 176 additions and 21 deletions.
2 changes: 1 addition & 1 deletion cmd/tidb-lightning-ctl/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,9 @@ import (
"strconv"
"strings"

"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/google/uuid"

kv "github.com/pingcap/tidb-lightning/lightning/backend"
"github.com/pingcap/tidb-lightning/lightning/common"
Expand Down
2 changes: 1 addition & 1 deletion lightning/backend/backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,13 @@ import (
"fmt"
"time"

"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/google/uuid"
"go.uber.org/zap"

"github.com/pingcap/tidb-lightning/lightning/common"
Expand Down
2 changes: 1 addition & 1 deletion lightning/backend/backend_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5,10 +5,10 @@ import (
"time"

"github.com/golang/mock/gomock"
"github.com/google/uuid"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/parser/mysql"
"github.com/google/uuid"

kv "github.com/pingcap/tidb-lightning/lightning/backend"
"github.com/pingcap/tidb-lightning/mock"
Expand Down
2 changes: 1 addition & 1 deletion lightning/backend/importer.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@ import (
"time"

"github.com/coreos/go-semver/semver"
"github.com/google/uuid"
"github.com/pingcap/errors"
kv "github.com/pingcap/kvproto/pkg/import_kvpb"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/table"
"github.com/google/uuid"
"go.uber.org/zap"
"google.golang.org/grpc"

Expand Down
2 changes: 1 addition & 1 deletion lightning/backend/importer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,10 +6,10 @@ import (
"testing"

"github.com/golang/mock/gomock"
"github.com/google/uuid"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/import_kvpb"
"github.com/google/uuid"

kvpb "github.com/pingcap/kvproto/pkg/import_kvpb"

Expand Down
2 changes: 2 additions & 0 deletions lightning/backend/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,6 +176,8 @@ type SessionOptions struct {
SQLMode mysql.SQLMode
Timestamp int64
RowFormatVersion string
// a seed used for tableKvEncoder's auto random bits value
AutoRandomSeed int64
}

func newSession(options *SessionOptions) *session {
Expand Down
52 changes: 39 additions & 13 deletions lightning/backend/sql2kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@
package backend

import (
"math/rand"

"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand All @@ -37,6 +39,8 @@ type tableKVEncoder struct {
tbl table.Table
se *session
recordCache []types.Datum
// auto random bits value for this chunk
autoRandomHeaderBits int64
}

func NewTableKVEncoder(tbl table.Table, options *SessionOptions) Encoder {
Expand All @@ -45,12 +49,35 @@ func NewTableKVEncoder(tbl table.Table, options *SessionOptions) Encoder {
// Set CommonAddRecordCtx to session to reuse the slices and BufStore in AddRecord
recordCtx := tables.NewCommonAddRecordCtx(len(tbl.Cols()))
tables.SetAddRecordCtx(se, recordCtx)

var autoRandomBits int64
if tbl.Meta().PKIsHandle && tbl.Meta().ContainsAutoRandomBits() {
for _, col := range tbl.Cols() {
if mysql.HasPriKeyFlag(col.Flag) {
incrementalBits := autoRandomIncrementBits(col, int(tbl.Meta().AutoRandomBits))
autoRandomBits = rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<tbl.Meta().AutoRandomBits) << incrementalBits
break
}
}
}

return &tableKVEncoder{
tbl: tbl,
se: se,
tbl: tbl,
se: se,
autoRandomHeaderBits: autoRandomBits,
}
}

func autoRandomIncrementBits(col *table.Column, randomBits int) int {
typeBitsLength := mysql.DefaultLengthOfMysqlTypes[col.Tp] * 8
incrementalBits := typeBitsLength - randomBits
hasSignBit := !mysql.HasUnsignedFlag(col.Flag)
if hasSignBit {
incrementalBits -= 1
}
return incrementalBits
}

func (kvcodec *tableKVEncoder) Close() {
metric.KvEncoderCounter.WithLabelValues("closed").Inc()
}
Expand Down Expand Up @@ -167,11 +194,7 @@ func (kvcodec *tableKVEncoder) Encode(
record = make([]types.Datum, 0, len(cols)+1)
}

isAutoRandom := false
if kvcodec.tbl.Meta().PKIsHandle && kvcodec.tbl.Meta().ContainsAutoRandomBits() {
isAutoRandom = true
}

isAutoRandom := kvcodec.tbl.Meta().PKIsHandle && kvcodec.tbl.Meta().ContainsAutoRandomBits()
for i, col := range cols {
j := columnPermutation[i]
isAutoIncCol := mysql.HasAutoIncrementFlag(col.Flag)
Expand All @@ -184,6 +207,14 @@ func (kvcodec *tableKVEncoder) Encode(
} else if isAutoIncCol {
// we still need a conversion, e.g. to catch overflow with a TINYINT column.
value, err = table.CastValue(kvcodec.se, types.NewIntDatum(rowID), col.ToInfo(), false, false)
} else if isAutoRandom && isPk {
var val types.Datum
if mysql.HasUnsignedFlag(col.Flag) {
val = types.NewUintDatum(uint64(kvcodec.autoRandomHeaderBits | rowID))
} else {
val = types.NewIntDatum(kvcodec.autoRandomHeaderBits | rowID)
}
value, err = table.CastValue(kvcodec.se, val, col.ToInfo(), false, false)
} else {
value, err = table.GetColDefaultValue(kvcodec.se, col.ToInfo())
}
Expand All @@ -194,12 +225,7 @@ func (kvcodec *tableKVEncoder) Encode(
record = append(record, value)

if isAutoRandom && isPk {
typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[col.Tp] * 8)
incrementalBits := typeBitsLength - kvcodec.tbl.Meta().AutoRandomBits
hasSignBit := !mysql.HasUnsignedFlag(col.Flag)
if hasSignBit {
incrementalBits -= 1
}
incrementalBits := autoRandomIncrementBits(col, int(kvcodec.tbl.Meta().AutoRandomBits))
kvcodec.tbl.RebaseAutoID(kvcodec.se, value.GetInt64()&((1<<incrementalBits)-1), false, autoid.AutoRandomType)
}
if isAutoIncCol {
Expand Down
46 changes: 46 additions & 0 deletions lightning/backend/sql2kv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
Expand Down Expand Up @@ -210,6 +211,51 @@ func (s *kvSuite) TestEncodeTimestamp(c *C) {
}))
}

func mockTableInfo(c *C, createSql string) *model.TableInfo {
parser := parser.New()
node, err := parser.ParseOneStmt(createSql, "", "")
c.Assert(err, IsNil)
sctx := mock.NewContext()
info, err := ddl.MockTableInfo(sctx, node.(*ast.CreateTableStmt), 1)
c.Assert(err, IsNil)
info.State = model.StatePublic
return info
}

func (s *kvSuite) TestDefaultAutoRandoms(c *C) {
tblInfo := mockTableInfo(c, "create table t (id bigint unsigned NOT NULL auto_random primary key, a varchar(100));")
// seems parser can't parse auto_random properly.
tblInfo.AutoRandomBits = 5
tbl, err := tables.TableFromMeta(NewPanickingAllocators(0), tblInfo)
c.Assert(err, IsNil)
encoder := NewTableKVEncoder(tbl, &SessionOptions{
SQLMode: mysql.ModeStrictAllTables,
Timestamp: 1234567893,
RowFormatVersion: "2",
AutoRandomSeed: 456,
})
logger := log.Logger{Logger: zap.NewNop()}
pairs, err := encoder.Encode(logger, []types.Datum{types.NewStringDatum("")}, 70, []int{-1, 0})
c.Assert(err, IsNil)
c.Assert(pairs, DeepEquals, kvPairs([]common.KvPair{
{
Key: []uint8{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x5f, 0x72, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x46},
Val: []uint8{0x80, 0x0, 0x1, 0x0, 0x0, 0x0, 0x2, 0x0, 0x0},
},
}))
c.Assert(tbl.Allocators(encoder.(*tableKVEncoder).se).Get(autoid.AutoRandomType).Base(), Equals, int64(70))

pairs, err = encoder.Encode(logger, []types.Datum{types.NewStringDatum("")}, 71, []int{-1, 0})
c.Assert(err, IsNil)
c.Assert(pairs, DeepEquals, kvPairs([]common.KvPair{
{
Key: []uint8{0x74, 0x80, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x1, 0x5f, 0x72, 0xf0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x0, 0x47},
Val: []uint8{0x80, 0x0, 0x1, 0x0, 0x0, 0x0, 0x2, 0x0, 0x0},
},
}))
c.Assert(tbl.Allocators(encoder.(*tableKVEncoder).se).Get(autoid.AutoRandomType).Base(), Equals, int64(71))
}

func (s *kvSuite) TestSplitIntoChunks(c *C) {
pairs := []common.KvPair{
{
Expand Down
2 changes: 1 addition & 1 deletion lightning/backend/tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,13 @@ import (
"strings"
"time"

"github.com/google/uuid"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/google/uuid"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"

Expand Down
2 changes: 2 additions & 0 deletions lightning/restore/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -1904,6 +1904,8 @@ func (cr *chunkRestore) restore(
SQLMode: rc.cfg.TiDB.SQLMode,
Timestamp: cr.chunk.Timestamp,
RowFormatVersion: rc.rowFormatVer,
// use chunk.PrevRowIDMax as the auto random seed, so it can stay the same value after recover from checkpoint.
AutoRandomSeed: cr.chunk.Chunk.PrevRowIDMax,
})
kvsCh := make(chan []deliveredKVs, maxKVQueueSize)
deliverCompleteCh := make(chan deliverResult)
Expand Down
2 changes: 1 addition & 1 deletion lightning/restore/restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (

"github.com/DATA-DOG/go-sqlmock"
"github.com/golang/mock/gomock"
"github.com/google/uuid"
"github.com/pingcap/br/pkg/storage"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
Expand All @@ -34,7 +35,6 @@ import (
filter "github.com/pingcap/tidb-tools/pkg/table-filter"
"github.com/pingcap/tidb/ddl"
tmock "github.com/pingcap/tidb/util/mock"
"github.com/google/uuid"

kv "github.com/pingcap/tidb-lightning/lightning/backend"
"github.com/pingcap/tidb-lightning/lightning/checkpoints"
Expand Down
2 changes: 1 addition & 1 deletion mock/backend.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

3 changes: 3 additions & 0 deletions tests/alter_random/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@ for backend in tidb importer local; do
run_sql 'DROP DATABASE IF EXISTS alter_random;'
run_lightning --backend $backend

run_sql "SELECT count(*) from alter_random.t"
check_contains "count(*): 3"

run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM alter_random.t"
check_contains 'inc: 1'
check_contains 'inc: 2'
Expand Down
2 changes: 2 additions & 0 deletions tests/auto_random_default/config.toml
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
[mydumper]
max-region-size = 200
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
CREATE DATABASE `auto_random` /*!40100 DEFAULT CHARACTER SET utf8mb4 */;
5 changes: 5 additions & 0 deletions tests/auto_random_default/data/auto_random.t-schema.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
/*!40103 SET TIME_ZONE='+00:00' */;
CREATE TABLE `t` (
`id` bigint unsigned primary key auto_random,
`s` varchar(32)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;
5 changes: 5 additions & 0 deletions tests/auto_random_default/data/auto_random.t.0.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
/*!40103 SET TIME_ZONE='+00:00' */;
INSERT INTO `t` (`s`) VALUES
("test1"),
("test2"),
("test3");
5 changes: 5 additions & 0 deletions tests/auto_random_default/data/auto_random.t.1.sql
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
/*!40103 SET TIME_ZONE='+00:00' */;
INSERT INTO `t` (`s`) VALUES
(""),
(""),
("");
58 changes: 58 additions & 0 deletions tests/auto_random_default/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
#!/bin/sh
#
# Copyright 2020 PingCAP, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# See the License for the specific language governing permissions and
# limitations under the License.

set -eu

# FIXME: auto-random is only stable on master currently.
check_cluster_version 4 0 0 AUTO_RANDOM || exit 0

for backend in tidb importer local; do
if [ "$backend" = 'local' ]; then
check_cluster_version 4 0 0 'local backend' || continue
fi

run_sql 'DROP DATABASE IF EXISTS auto_random;'
run_lightning --backend $backend

run_sql "SELECT count(*) from auto_random.t"
check_contains "count(*): 6"

run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM auto_random.t"
check_contains 'inc: 1'
check_contains 'inc: 2'
check_contains 'inc: 3'
if [ "$backend" = 'tidb' ]; then
check_contains 'inc: 4'
check_contains 'inc: 5'
check_contains 'inc: 6'
else
check_contains 'inc: 25'
check_contains 'inc: 26'
check_contains 'inc: 27'
fi


run_sql "select count(distinct id >> 58) as count from auto_random.t"
check_contains "count: 2"

# auto random base is 4
run_sql "INSERT INTO auto_random.t VALUES ();"
run_sql "SELECT id & b'000001111111111111111111111111111111111111111111111111111111111' as inc FROM auto_random.t"
if [ "$backend" = 'tidb' ]; then
check_contains 'inc: 2000001'
else
check_contains 'inc: 28'
fi
done

0 comments on commit 66ac09a

Please sign in to comment.