Skip to content

Commit

Permalink
Merge branch 'release-6.1' into cherry-pick-41370-to-release-6.1
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored Mar 29, 2023
2 parents f5b0c29 + aac228a commit 4320bd7
Show file tree
Hide file tree
Showing 86 changed files with 2,068 additions and 735 deletions.
4 changes: 0 additions & 4 deletions bindinfo/session_handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,10 +414,6 @@ func (msm *mockSessionManager) GetInternalSessionStartTSList() []uint64 {
return nil
}

func (msm *mockSessionManager) GetMinStartTS(lowerBound uint64) uint64 {
return 0
}

func TestIssue19836(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down
53 changes: 24 additions & 29 deletions br/pkg/lightning/backend/kv/sql2kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,10 +57,11 @@ type genCol struct {
type autoIDConverter func(int64) int64

type tableKVEncoder struct {
tbl table.Table
se *session
recordCache []types.Datum
genCols []genCol
tbl table.Table
autoRandomColID int64
se *session
recordCache []types.Datum
genCols []genCol
// convert auto id for shard rowid or auto random id base on row id generated by lightning
autoIDFn autoIDConverter
}
Expand All @@ -74,17 +75,15 @@ func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error
recordCtx := tables.NewCommonAddRecordCtx(len(cols))
tables.SetAddRecordCtx(se, recordCtx)

var autoRandomColID int64
autoIDFn := func(id int64) int64 { return id }
if meta.PKIsHandle && meta.ContainsAutoRandomBits() {
for _, col := range cols {
if mysql.HasPriKeyFlag(col.GetFlag()) {
incrementalBits := autoRandomIncrementBits(col, int(meta.AutoRandomBits))
autoRandomBits := rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<meta.AutoRandomBits) << incrementalBits // nolint:gosec
autoIDFn = func(id int64) int64 {
return autoRandomBits | id
}
break
}
if meta.ContainsAutoRandomBits() {
col := common.GetAutoRandomColumn(meta)
autoRandomColID = col.ID
incrementalBits := autoRandomIncrementBits(col, int(meta.AutoRandomBits))
autoRandomBits := rand.New(rand.NewSource(options.AutoRandomSeed)).Int63n(1<<meta.AutoRandomBits) << incrementalBits // nolint:gosec
autoIDFn = func(id int64) int64 {
return autoRandomBits | id
}
} else if meta.ShardRowIDBits > 0 {
rd := rand.New(rand.NewSource(options.AutoRandomSeed)) // nolint:gosec
Expand All @@ -104,14 +103,15 @@ func NewTableKVEncoder(tbl table.Table, options *SessionOptions) (Encoder, error
}

return &tableKVEncoder{
tbl: tbl,
se: se,
genCols: genCols,
autoIDFn: autoIDFn,
tbl: tbl,
autoRandomColID: autoRandomColID,
se: se,
genCols: genCols,
autoIDFn: autoIDFn,
}, nil
}

func autoRandomIncrementBits(col *table.Column, randomBits int) int {
func autoRandomIncrementBits(col *model.ColumnInfo, randomBits int) int {
typeBitsLength := mysql.DefaultLengthOfMysqlTypes[col.GetType()] * 8
incrementalBits := typeBitsLength - randomBits
hasSignBit := !mysql.HasUnsignedFlag(col.GetFlag())
Expand Down Expand Up @@ -369,8 +369,8 @@ func (kvcodec *tableKVEncoder) Encode(

record = append(record, value)

if isTableAutoRandom(meta) && isPKCol(col.ToInfo()) {
incrementalBits := autoRandomIncrementBits(col, int(meta.AutoRandomBits))
if kvcodec.isAutoRandomCol(col.ToInfo()) {
incrementalBits := autoRandomIncrementBits(col.ToInfo(), int(meta.AutoRandomBits))
alloc := kvcodec.tbl.Allocators(kvcodec.se).Get(autoid.AutoRandomType)
if err := alloc.Rebase(context.Background(), value.GetInt64()&((1<<incrementalBits)-1), false); err != nil {
return nil, errors.Trace(err)
Expand Down Expand Up @@ -427,25 +427,20 @@ func (kvcodec *tableKVEncoder) Encode(
return kvPairs, nil
}

func isTableAutoRandom(tblMeta *model.TableInfo) bool {
return tblMeta.PKIsHandle && tblMeta.ContainsAutoRandomBits()
func (kvcodec *tableKVEncoder) isAutoRandomCol(col *model.ColumnInfo) bool {
return kvcodec.tbl.Meta().ContainsAutoRandomBits() && col.ID == kvcodec.autoRandomColID
}

func isAutoIncCol(colInfo *model.ColumnInfo) bool {
return mysql.HasAutoIncrementFlag(colInfo.GetFlag())
}

func isPKCol(colInfo *model.ColumnInfo) bool {
return mysql.HasPriKeyFlag(colInfo.GetFlag())
}

func (kvcodec *tableKVEncoder) getActualDatum(rowID int64, colIndex int, inputDatum *types.Datum) (types.Datum, error) {
var (
value types.Datum
err error
)

tblMeta := kvcodec.tbl.Meta()
cols := kvcodec.tbl.Cols()

// Since this method is only called when iterating the columns in the `Encode()` method,
Expand All @@ -468,7 +463,7 @@ func (kvcodec *tableKVEncoder) getActualDatum(rowID int64, colIndex int, inputDa
case isAutoIncCol(col.ToInfo()):
// 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)
case isTableAutoRandom(tblMeta) && isPKCol(col.ToInfo()):
case kvcodec.isAutoRandomCol(col.ToInfo()):
var val types.Datum
realRowID := kvcodec.autoIDFn(rowID)
if mysql.HasUnsignedFlag(col.GetFlag()) {
Expand Down
20 changes: 20 additions & 0 deletions br/pkg/lightning/common/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/br/pkg/utils"
tmysql "github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/table/tables"
"go.uber.org/zap"
)

Expand Down Expand Up @@ -388,3 +389,22 @@ func StringSliceEqual(a, b []string) bool {
}
return true
}

// GetAutoRandomColumn return the column with auto_random, return nil if the table doesn't have it.
// todo: better put in ddl package, but this will cause import cycle since ddl package import lightning
func GetAutoRandomColumn(tblInfo *model.TableInfo) *model.ColumnInfo {
if !tblInfo.ContainsAutoRandomBits() {
return nil
}
if tblInfo.PKIsHandle {
return tblInfo.GetPkColInfo()
} else if tblInfo.IsCommonHandle {
pk := tables.FindPrimaryIndex(tblInfo)
if pk == nil {
return nil
}
offset := pk.Columns[0].Offset
return tblInfo.Columns[offset]
}
return nil
}
27 changes: 27 additions & 0 deletions br/pkg/lightning/common/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/br/pkg/lightning/log"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/util/dbutil"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -179,3 +181,28 @@ func TestInterpolateMySQLString(t *testing.T) {
assert.Equal(t, "'1''23'", common.InterpolateMySQLString("1'23"))
assert.Equal(t, "'1''2''''3'", common.InterpolateMySQLString("1'2''3"))
}

func TestGetAutoRandomColumn(t *testing.T) {
tests := []struct {
ddl string
colName string
}{
{"create table t(c int)", ""},
{"create table t(c int auto_increment)", ""},
{"create table t(c bigint auto_random primary key)", "c"},
{"create table t(a int, c bigint auto_random primary key)", "c"},
{"create table t(c bigint auto_random, a int, primary key(c,a) clustered)", "c"},
{"create table t(a int, c bigint auto_random, primary key(c,a) clustered)", "c"},
}
p := parser.New()
for _, tt := range tests {
tableInfo, err := dbutil.GetTableInfoBySQL(tt.ddl, p)
require.NoError(t, err)
col := common.GetAutoRandomColumn(tableInfo)
if tt.colName == "" {
require.Nil(t, col, tt.ddl)
} else {
require.Equal(t, tt.colName, col.Name.L, tt.ddl)
}
}
}
3 changes: 2 additions & 1 deletion br/pkg/lightning/restore/check_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -662,8 +662,9 @@ func (rc *Controller) SchemaIsValid(ctx context.Context, tableInfo *mydump.MDTab
colCountFromTiDB := len(info.Core.Columns)
core := info.Core
defaultCols := make(map[string]struct{})
autoRandomCol := common.GetAutoRandomColumn(core)
for _, col := range core.Columns {
if hasDefault(col) || (info.Core.ContainsAutoRandomBits() && mysql.HasPriKeyFlag(col.GetFlag())) {
if hasDefault(col) || (autoRandomCol != nil && autoRandomCol.ID == col.ID) {
// this column has default value or it's auto random id, so we can ignore it
defaultCols[col.Name.L] = struct{}{}
}
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/lightning/restore/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -1623,7 +1623,7 @@ func (tr *TableRestore) restoreTable(
web.BroadcastTableCheckpoint(tr.tableName, cp)

// rebase the allocator so it exceeds the number of rows.
if tr.tableInfo.Core.PKIsHandle && tr.tableInfo.Core.ContainsAutoRandomBits() {
if tr.tableInfo.Core.ContainsAutoRandomBits() {
cp.AllocBase = mathutil.Max(cp.AllocBase, tr.tableInfo.Core.AutoRandID)
if err := tr.alloc.Get(autoid.AutoRandomType).Rebase(context.Background(), cp.AllocBase, false); err != nil {
return false, err
Expand Down Expand Up @@ -2303,7 +2303,7 @@ func saveCheckpoint(rc *Controller, t *TableRestore, engineID int32, chunk *chec
// or integer primary key), which can only be obtained by reading all data.

var base int64
if t.tableInfo.Core.PKIsHandle && t.tableInfo.Core.ContainsAutoRandomBits() {
if t.tableInfo.Core.ContainsAutoRandomBits() {
base = t.alloc.Get(autoid.AutoRandomType).Base() + 1
} else {
base = t.alloc.Get(autoid.RowIDAllocType).Base() + 1
Expand Down
5 changes: 3 additions & 2 deletions br/pkg/lightning/restore/table_restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -688,11 +688,12 @@ func (tr *TableRestore) postProcess(
rc.alterTableLock.Lock()
tblInfo := tr.tableInfo.Core
var err error
if tblInfo.PKIsHandle && tblInfo.ContainsAutoRandomBits() {
if tblInfo.ContainsAutoRandomBits() {
ft := &common.GetAutoRandomColumn(tblInfo).FieldType
var maxAutoRandom, autoRandomTotalBits uint64
autoRandomTotalBits = 64
autoRandomBits := tblInfo.AutoRandomBits // range from (0, 15]
if !tblInfo.IsAutoRandomBitColUnsigned() {
if !mysql.HasUnsignedFlag(ft.GetFlag()) {
// if auto_random is signed, leave one extra bit
autoRandomTotalBits = 63
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
"a","b"
1,11
2,22
27 changes: 27 additions & 0 deletions br/tests/lightning_auto_random_default/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -60,3 +60,30 @@ for backend in tidb local; do
run_sql "SELECT max(id & b'000001111111111111111111111111111111111111111111111111111111111') >= $NEXT_AUTO_RAND_VAL as ge FROM auto_random.t"
check_contains 'ge: 1'
done

function run_for_auro_random_data2() {
create_table=$1
run_sql 'DROP DATABASE IF EXISTS auto_random;'
run_sql 'CREATE DATABASE IF NOT EXISTS auto_random;'
run_sql "$create_table"
run_lightning --backend $backend -d "tests/$TEST_NAME/data2"
run_sql 'select count(*) as count from auto_random.t where c > 0'
check_contains "count: 2"
run_sql 'select count(*) as count from auto_random.t where a=1 and b=11'
check_contains "count: 1"
run_sql 'select count(*) as count from auto_random.t where a=2 and b=22'
check_contains "count: 1"
}

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

run_for_auro_random_data2 'create table auto_random.t(c bigint auto_random primary key, a int, b int)'
run_for_auro_random_data2 'create table auto_random.t(a int, b int, c bigint auto_random primary key)'
# composite key and auto_random is the first column
run_for_auro_random_data2 'create table auto_random.t(c bigint auto_random, a int, b int, primary key(c, a) clustered)'
# composite key and auto_random is not the first column
run_for_auro_random_data2 'create table auto_random.t(a int, b int, c bigint auto_random, primary key(c, a) clustered)'
done
2 changes: 1 addition & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -892,7 +892,7 @@ var defaultConf = Config{
},
Experimental: Experimental{},
EnableCollectExecutionInfo: true,
EnableTelemetry: true,
EnableTelemetry: false,
Labels: make(map[string]string),
EnableGlobalIndex: false,
Security: Security{
Expand Down
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ skip-register-to-dashboard = false
# When enabled, usage data (for example, instance versions) will be reported to PingCAP periodically for user experience analytics.
# If this config is set to `false` on all TiDB servers, telemetry will be always disabled regardless of the value of the global variable `tidb_enable_telemetry`.
# See PingCAP privacy policy for details: https://pingcap.com/en/privacy-policy/
enable-telemetry = true
enable-telemetry = false

# deprecate-integer-display-length is used to be compatible with MySQL 8.0 in which the integer declared with display length will be returned with
# a warning like `Integer display width is deprecated and will be removed in a future release`.
Expand Down
8 changes: 4 additions & 4 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -341,23 +341,23 @@ history-size=100`)
require.NoError(t, err)
require.NoError(t, f.Sync())
require.NoError(t, conf.Load(configFile))
require.True(t, conf.EnableTelemetry)
require.False(t, conf.EnableTelemetry)

_, err = f.WriteString(`
enable-table-lock = true
`)
require.NoError(t, err)
require.NoError(t, f.Sync())
require.NoError(t, conf.Load(configFile))
require.True(t, conf.EnableTelemetry)
require.False(t, conf.EnableTelemetry)

_, err = f.WriteString(`
enable-telemetry = false
enable-telemetry = true
`)
require.NoError(t, err)
require.NoError(t, f.Sync())
require.NoError(t, conf.Load(configFile))
require.False(t, conf.EnableTelemetry)
require.True(t, conf.EnableTelemetry)

_, err = f.WriteString(`
[security]
Expand Down
3 changes: 2 additions & 1 deletion ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -658,7 +658,8 @@ func needChangeColumnData(oldCol, newCol *model.ColumnInfo) bool {
toUnsigned := mysql.HasUnsignedFlag(newCol.GetFlag())
originUnsigned := mysql.HasUnsignedFlag(oldCol.GetFlag())
needTruncationOrToggleSign := func() bool {
return (newCol.GetFlen() > 0 && newCol.GetFlen() < oldCol.GetFlen()) || (toUnsigned != originUnsigned)
return (newCol.GetFlen() > 0 && (newCol.GetFlen() < oldCol.GetFlen() || newCol.GetDecimal() < oldCol.GetDecimal())) ||
(toUnsigned != originUnsigned)
}
// Ignore the potential max display length represented by integer's flen, use default flen instead.
defaultOldColFlen, _ := mysql.GetDefaultFieldLengthAndDecimal(oldCol.GetType())
Expand Down
6 changes: 5 additions & 1 deletion ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2366,11 +2366,15 @@ func TestColumnTypeChangeBetweenFloatAndDouble(t *testing.T) {
prepare := func(createTableStmt string) {
tk.MustExec("drop table if exists t;")
tk.MustExec(createTableStmt)
tk.MustExec("insert into t values (36.4), (24.1);")
tk.MustExec("insert into t values (36.43), (24.1);")
}

prepare("create table t (a float(6,2));")
tk.MustExec("alter table t modify a double(6,2)")
tk.MustQuery("select a from t;").Check(testkit.Rows("36.43", "24.1"))

prepare("create table t (a float(6,2));")
tk.MustExec("alter table t modify a float(6,1)")
tk.MustQuery("select a from t;").Check(testkit.Rows("36.4", "24.1"))

prepare("create table t (a double(6,2));")
Expand Down
21 changes: 6 additions & 15 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1415,10 +1415,8 @@ func TestLogAndShowSlowLog(t *testing.T) {
}

func TestReportingMinStartTimestamp(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease)
_, dom, clean := testkit.CreateMockStoreAndDomainWithSchemaLease(t, dbTestLease)
defer clean()
tk := testkit.NewTestKit(t, store)
se := tk.Session()

infoSyncer := dom.InfoSyncer()
sm := &testkit.MockSessionManager{
Expand All @@ -1434,19 +1432,12 @@ func TestReportingMinStartTimestamp(t *testing.T) {
validTS := oracle.GoTimeToLowerLimitStartTS(now.Add(time.Minute), tikv.MaxTxnTimeUse)
lowerLimit := oracle.GoTimeToLowerLimitStartTS(now, tikv.MaxTxnTimeUse)
sm.PS = []*util.ProcessInfo{
{CurTxnStartTS: 0, ProtectedTSList: &se.GetSessionVars().ProtectedTSList},
{CurTxnStartTS: math.MaxUint64, ProtectedTSList: &se.GetSessionVars().ProtectedTSList},
{CurTxnStartTS: lowerLimit, ProtectedTSList: &se.GetSessionVars().ProtectedTSList},
{CurTxnStartTS: validTS, ProtectedTSList: &se.GetSessionVars().ProtectedTSList},
{CurTxnStartTS: 0},
{CurTxnStartTS: math.MaxUint64},
{CurTxnStartTS: lowerLimit},
{CurTxnStartTS: validTS},
}
infoSyncer.ReportMinStartTS(dom.Store())
require.Equal(t, validTS, infoSyncer.GetMinStartTS())

unhold := se.GetSessionVars().ProtectedTSList.HoldTS(validTS - 1)
infoSyncer.ReportMinStartTS(dom.Store())
require.Equal(t, validTS-1, infoSyncer.GetMinStartTS())

unhold()
infoSyncer.SetSessionManager(sm)
infoSyncer.ReportMinStartTS(dom.Store())
require.Equal(t, validTS, infoSyncer.GetMinStartTS())
}
Loading

0 comments on commit 4320bd7

Please sign in to comment.