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

backend: fix auto random default value for primary key #457

Merged
merged 9 commits into from
Nov 9, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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' */;
kennytm marked this conversation as resolved.
Show resolved Hide resolved
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