Skip to content

Commit

Permalink
config: remove allow-auto-random config option (#16596) (#18613)
Browse files Browse the repository at this point in the history
  • Loading branch information
tangenta authored Jul 16, 2020
1 parent fe54331 commit b10faca
Show file tree
Hide file tree
Showing 14 changed files with 230 additions and 108 deletions.
6 changes: 0 additions & 6 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -541,8 +541,6 @@ type IsolationRead struct {
// Experimental controls the features that are still experimental: their semantics, interfaces are subject to change.
// Using these features in the production environment is not recommended.
type Experimental struct {
// Whether enable the syntax like `auto_random(3)` on the primary key column.
AllowAutoRandom bool `toml:"allow-auto-random" json:"allow-auto-random"`
// Whether enable creating expression index.
AllowsExpressionIndex bool `toml:"allow-expression-index" json:"allow-expression-index"`
}
Expand Down Expand Up @@ -681,7 +679,6 @@ var defaultConf = Config{
Engines: []string{"tikv", "tiflash", "tidb"},
},
Experimental: Experimental{
AllowAutoRandom: false,
AllowsExpressionIndex: false,
},
EnableCollectExecutionInfo: true,
Expand Down Expand Up @@ -867,9 +864,6 @@ func (c *Config) Valid() error {
return fmt.Errorf("refresh-interval in [stmt-summary] should be greater than 0")
}

if c.AlterPrimaryKey && c.Experimental.AllowAutoRandom {
return fmt.Errorf("allow-auto-random is unavailable when alter-primary-key is enabled")
}
if c.PreparedPlanCache.Capacity < 1 {
return fmt.Errorf("capacity in [prepared-plan-cache] should be at least 1")
}
Expand Down
2 changes: 0 additions & 2 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -422,8 +422,6 @@ history-size = 24
# experimental section controls the features that are still experimental: their semantics,
# interfaces are subject to change, using these features in the production environment is not recommended.
[experimental]
# enable column attribute `auto_random` to be defined on the primary key column.
allow-auto-random = false
# enable creating expression index.
allow-expression-index = false

Expand Down
15 changes: 0 additions & 15 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,7 +205,6 @@ max-sql-length=1024
refresh-interval=100
history-size=100
[experimental]
allow-auto-random = true
allow-expression-index = true
[isolation-read]
engines = ["tiflash"]
Expand Down Expand Up @@ -245,7 +244,6 @@ engines = ["tiflash"]
c.Assert(conf.MaxServerConnections, Equals, uint32(200))
c.Assert(conf.MemQuotaQuery, Equals, int64(10000))
c.Assert(conf.Experimental.AllowsExpressionIndex, IsTrue)
c.Assert(conf.Experimental.AllowAutoRandom, IsTrue)
c.Assert(conf.IsolationRead.Engines, DeepEquals, []string{"tiflash"})
c.Assert(conf.MaxIndexLength, Equals, 3080)
c.Assert(conf.SkipRegisterToDashboard, Equals, true)
Expand Down Expand Up @@ -414,19 +412,6 @@ func (s *testConfigSuite) TestTxnTotalSizeLimitValid(c *C) {
}
}

func (s *testConfigSuite) TestAllowAutoRandomValid(c *C) {
conf := NewConfig()
checkValid := func(allowAlterPK, allowAutoRand, shouldBeValid bool) {
conf.AlterPrimaryKey = allowAlterPK
conf.Experimental.AllowAutoRandom = allowAutoRand
c.Assert(conf.Valid() == nil, Equals, shouldBeValid)
}
checkValid(true, true, false)
checkValid(true, false, true)
checkValid(false, true, true)
checkValid(false, false, true)
}

func (s *testConfigSuite) TestPreparePlanCacheValid(c *C) {
conf := NewConfig()
tests := map[PreparedPlanCache]bool{
Expand Down
46 changes: 43 additions & 3 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,12 @@ package ddl

import (
"fmt"
"math/bits"
"strings"
"sync/atomic"
"time"

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
Expand All @@ -27,6 +29,7 @@ import (
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -334,17 +337,20 @@ func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ erro
oldColName := &model.CIStr{}
pos := &ast.ColumnPosition{}
var modifyColumnTp byte
err := job.DecodeArgs(newCol, oldColName, pos, &modifyColumnTp)
var updatedAutoRandomBits uint64
err := job.DecodeArgs(newCol, oldColName, pos, &modifyColumnTp, &updatedAutoRandomBits)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp)
return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp, updatedAutoRandomBits)
}

// doModifyColumn updates the column information and reorders all columns.
func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition, modifyColumnTp byte) (ver int64, _ error) {
func (w *worker) doModifyColumn(
t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr,
pos *ast.ColumnPosition, modifyColumnTp byte, newAutoRandBits uint64) (ver int64, _ error) {
dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job)
if err != nil {
return ver, errors.Trace(err)
Expand Down Expand Up @@ -386,6 +392,12 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu
}
})

if newAutoRandBits > 0 {
if err := checkAndApplyNewAutoRandomBits(job, t, tblInfo, newCol, oldName, newAutoRandBits); err != nil {
return ver, errors.Trace(err)
}
}

// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.Flag)
Expand Down Expand Up @@ -476,6 +488,34 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu
return ver, nil
}

func checkAndApplyNewAutoRandomBits(job *model.Job, t *meta.Meta, tblInfo *model.TableInfo,
newCol *model.ColumnInfo, oldName *model.CIStr, newAutoRandBits uint64) error {
schemaID := job.SchemaID
newLayout := autoid.NewAutoRandomIDLayout(&newCol.FieldType, newAutoRandBits)

// GenAutoRandomID first to prevent concurrent update.
_, err := t.GenAutoRandomID(schemaID, tblInfo.ID, 1)
if err != nil {
return err
}
currentIncBitsVal, err := t.GetAutoRandomID(schemaID, tblInfo.ID)
if err != nil {
return err
}
// Find the max number of available shard bits by
// counting leading zeros in current inc part of auto_random ID.
availableBits := bits.LeadingZeros64(uint64(currentIncBitsVal))
isOccupyingIncBits := newLayout.TypeBitsLength-newLayout.IncrementalBits > uint64(availableBits)
if isOccupyingIncBits {
availableBits := mathutil.Min(autoid.MaxAutoRandomBits, availableBits)
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, availableBits, newAutoRandBits, oldName.O)
job.State = model.JobStateCancelled
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
tblInfo.AutoRandomBits = newAutoRandBits
return nil
}

// checkForNullValue ensure there are no null values of the column of this table.
// `isDataTruncated` indicates whether the new field and the old field type are the same, in order to be compatible with mysql.
func checkForNullValue(ctx sessionctx.Context, isDataTruncated bool, schema, table, newCol model.CIStr, oldCols ...*model.ColumnInfo) error {
Expand Down
72 changes: 72 additions & 0 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,14 @@ import (
"context"
"fmt"
"sync"
"sync/atomic"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
Expand Down Expand Up @@ -151,6 +153,76 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) {
s.testAddColumnNoDefault(c, ctx, d, tblInfo)
}

func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) {
d := testNewDDLAndStart(
context.Background(),
c,
WithStore(s.store),
WithLease(testLease),
)
defer d.Stop()

ids, err := d.genGlobalIDs(1)
tableID := ids[0]
c.Assert(err, IsNil)
colInfo := &model.ColumnInfo{
Name: model.NewCIStr("a"),
Offset: 0,
State: model.StatePublic,
FieldType: *types.NewFieldType(mysql.TypeLonglong),
}
tblInfo := &model.TableInfo{
ID: tableID,
Name: model.NewCIStr("auto_random_table_name"),
Columns: []*model.ColumnInfo{colInfo},
AutoRandomBits: 5,
}
colInfo.ID = allocateColumnID(tblInfo)
ctx := testNewContext(d)
testCreateTable(c, ctx, d, s.dbInfo, tblInfo)

tc := &TestDDLCallback{}
var errCount int32 = 3
var genAutoRandErr error
tc.onJobRunBefore = func(job *model.Job) {
if atomic.LoadInt32(&errCount) > 0 && job.Type == model.ActionModifyColumn {
atomic.AddInt32(&errCount, -1)
genAutoRandErr = kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error {
t := meta.NewMeta(txn)
_, err1 := t.GenAutoRandomID(s.dbInfo.ID, tableID, 1)
return err1
})
}
}
d.SetHook(tc)
const newAutoRandomBits uint64 = 10
job := &model.Job{
SchemaID: s.dbInfo.ID,
TableID: tblInfo.ID,
SchemaName: s.dbInfo.Name.L,
Type: model.ActionModifyColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{colInfo, colInfo.Name, ast.ColumnPosition{}, 0, newAutoRandomBits},
}
err = d.doDDLJob(ctx, job)
c.Assert(err, IsNil)
c.Assert(errCount == 0, IsTrue)
c.Assert(genAutoRandErr, IsNil)
testCheckJobDone(c, d, job, true)
var newTbInfo *model.TableInfo
err = kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error {
t := meta.NewMeta(txn)
var err error
newTbInfo, err = t.GetTable(s.dbInfo.ID, tableID)
if err != nil {
return errors.Trace(err)
}
return nil
})
c.Assert(err, IsNil)
c.Assert(newTbInfo.AutoRandomBits, Equals, newAutoRandomBits)
}

func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo) {
tc := &TestDDLCallback{}
// set up hook
Expand Down
50 changes: 28 additions & 22 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1106,13 +1106,9 @@ func checkConstraintNames(constraints []*ast.Constraint) error {
}

func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error {
allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom
pkColName := tbInfo.GetPkName()
for _, col := range colDefs {
if containsColumnOption(col, ast.ColumnOptionAutoRandom) {
if !allowAutoRandom {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomExperimentalDisabledErrMsg)
}
if col.Tp.Tp != mysql.TypeLonglong {
return ErrInvalidAutoRandom.GenWithStackByArgs(
fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(col.Tp.Tp)))
Expand Down Expand Up @@ -3031,7 +3027,8 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
return nil, errors.Trace(err)
}

if err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil {
var newAutoRandBits uint64
if newAutoRandBits, err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil {
return nil, errors.Trace(err)
}

Expand All @@ -3041,7 +3038,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
SchemaName: schema.Name.L,
Type: model.ActionModifyColumn,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp},
Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp, newAutoRandBits},
}
return job, nil
}
Expand Down Expand Up @@ -3083,34 +3080,43 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol
return nil
}

func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) error {
if !config.GetGlobalConfig().Experimental.AllowAutoRandom && containsColumnOption(specNewColumn, ast.ColumnOptionAutoRandom) {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomExperimentalDisabledErrMsg)
}
// Disallow add/drop/modify actions on auto_random.
newAutoRandomBit, err := extractAutoRandomBitsFromColDef(specNewColumn)
func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) {
// Disallow add/drop actions on auto_random.
oldRandBits := tableInfo.AutoRandomBits
newRandBits, err := extractAutoRandomBitsFromColDef(specNewColumn)
if err != nil {
return errors.Trace(err)
}
if tableInfo.AutoRandomBits != newAutoRandomBit {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg)
return 0, errors.Trace(err)
}

if tableInfo.AutoRandomBits != 0 {
switch {
case oldRandBits == newRandBits:
break
case oldRandBits == 0 || newRandBits == 0:
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg)
case autoid.MaxAutoRandomBits < newRandBits:
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg,
autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
case oldRandBits < newRandBits:
break // Increasing auto_random shard bits is allowed.
case oldRandBits > newRandBits:
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg)
}

if oldRandBits != 0 {
// Disallow changing the column field type.
if originCol.Tp != specNewColumn.Tp.Tp {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg)
}
// Disallow changing auto_increment on auto_random column.
if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) != mysql.HasAutoIncrementFlag(originCol.Flag) {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg)
}
// Disallow specifying a default value on auto_random column.
if containsColumnOption(specNewColumn, ast.ColumnOptionDefaultValue) {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg)
}
}
return nil
return newRandBits, nil
}

// ChangeColumn renames an existing column and modifies the column's definition,
Expand Down
Loading

0 comments on commit b10faca

Please sign in to comment.