Skip to content

Commit

Permalink
Merge branch 'master' into fix-delete-bug
Browse files Browse the repository at this point in the history
  • Loading branch information
DQinYuan authored Jul 16, 2019
2 parents 8fc1c9f + 368119b commit ffce1cc
Show file tree
Hide file tree
Showing 186 changed files with 5,543 additions and 2,151 deletions.
281 changes: 281 additions & 0 deletions CHANGELOG.md

Large diffs are not rendered by default.

4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,14 @@ path_to_add := $(addsuffix /bin,$(subst :,/bin:,$(GOPATH))):$(PWD)/tools/bin
export PATH := $(path_to_add):$(PATH)

GO := GO111MODULE=on go
GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG)
GOBUILD := CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) -tags codes
GOTEST := CGO_ENABLED=1 $(GO) test -p 4
OVERALLS := CGO_ENABLED=1 GO111MODULE=on overalls

ARCH := "`uname -s`"
LINUX := "Linux"
MAC := "Darwin"
PACKAGE_LIST := go list ./...| grep -vE "cmd"
PACKAGE_LIST := go list ./...| grep -vE "cmd" | grep -vE "test"
PACKAGES := $$($(PACKAGE_LIST))
PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||'
FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go")
Expand Down
24 changes: 22 additions & 2 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,10 @@ const (
MaxLogFileSize = 4096 // MB
MinPessimisticTTL = time.Second * 15
MaxPessimisticTTL = time.Second * 60
// DefTxnEntryCountLimit is the default value of TxnEntryCountLimit.
DefTxnEntryCountLimit = 300 * 1000
// DefTxnTotalSizeLimit is the default value of TxnTxnTotalSizeLimit.
DefTxnTotalSizeLimit = 100 * 1024 * 1024
)

// Valid config maps
Expand Down Expand Up @@ -90,7 +94,8 @@ type Config struct {
TreatOldVersionUTF8AsUTF8MB4 bool `toml:"treat-old-version-utf8-as-utf8mb4" json:"treat-old-version-utf8-as-utf8mb4"`
// EnableTableLock indicate whether enable table lock.
// TODO: remove this after table lock features stable.
EnableTableLock bool `toml:"enable-table-lock" json:"enable-table-lock"`
EnableTableLock bool `toml:"enable-table-lock" json:"enable-table-lock"`
DelayCleanTableLock uint64 `toml:"delay-clean-table-lock" json:"delay-clean-table-lock"`
}

// Log is the log section of config.
Expand Down Expand Up @@ -192,6 +197,8 @@ type Performance struct {
PseudoEstimateRatio float64 `toml:"pseudo-estimate-ratio" json:"pseudo-estimate-ratio"`
ForcePriority string `toml:"force-priority" json:"force-priority"`
BindInfoLease string `toml:"bind-info-lease" json:"bind-info-lease"`
TxnEntryCountLimit uint64 `toml:"txn-entry-count-limit" json:"txn-entry-count-limit"`
TxnTotalSizeLimit uint64 `toml:"txn-total-size-limit" json:"txn-total-size-limit"`
}

// PlanCache is the PlanCache section of the config.
Expand Down Expand Up @@ -326,6 +333,7 @@ var defaultConf = Config{
CheckMb4ValueInUTF8: true,
TreatOldVersionUTF8AsUTF8MB4: true,
EnableTableLock: false,
DelayCleanTableLock: 0,
TxnLocalLatches: TxnLocalLatches{
Enabled: true,
Capacity: 2048000,
Expand Down Expand Up @@ -359,6 +367,8 @@ var defaultConf = Config{
PseudoEstimateRatio: 0.8,
ForcePriority: "NO_PRIORITY",
BindInfoLease: "3s",
TxnEntryCountLimit: DefTxnEntryCountLimit,
TxnTotalSizeLimit: DefTxnTotalSizeLimit,
},
ProxyProtocol: ProxyProtocol{
Networks: "",
Expand Down Expand Up @@ -435,6 +445,11 @@ func GetGlobalConfig() *Config {
return globalConf.Load().(*Config)
}

// StoreGlobalConfig stores a new config to the globalConf. It mostly uses in the test to avoid some data races.
func StoreGlobalConfig(config *Config) {
globalConf.Store(config)
}

// ReloadGlobalConfig reloads global configuration for this server.
func ReloadGlobalConfig() error {
confReloadLock.Lock()
Expand Down Expand Up @@ -582,10 +597,15 @@ func hasRootPrivilege() bool {
}

// TableLockEnabled uses to check whether enabled the table lock feature.
var TableLockEnabled = func() bool {
func TableLockEnabled() bool {
return GetGlobalConfig().EnableTableLock
}

// TableLockDelayClean uses to get the time of delay clean table lock.
var TableLockDelayClean = func() uint64 {
return GetGlobalConfig().DelayCleanTableLock
}

// ToLogConfig converts *Log to *logutil.LogConfig.
func (l *Log) ToLogConfig() *logutil.LogConfig {
return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.DisableTimestamp)
Expand Down
15 changes: 15 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,9 @@ treat-old-version-utf8-as-utf8mb4 = true
# enable-table-lock is used to control table lock feature. Default is false, indicate the table lock feature is disabled.
enable-table-lock = false

# delay-clean-table-lock is used to control whether delayed-release the table lock in the abnormal situation. (Milliseconds)
delay-clean-table-lock = 0

[log]
# Log level: debug, info, warn, error, fatal.
level = "info"
Expand Down Expand Up @@ -174,6 +177,18 @@ force-priority = "NO_PRIORITY"
# Bind info lease duration, which influences the duration of loading bind info and handling invalid bind.
bind-info-lease = "3s"

# The limitation of the number for the entries in one transaction.
# If using TiKV as the storage, the entry represents a key/value pair.
# WARNING: Do not set the value too large, otherwise it will make a very large impact on the TiKV cluster.
# Please adjust this configuration carefully.
txn-entry-count-limit = 300000

# The limitation of the size in byte for the entries in one transaction.
# If using TiKV as the storage, the entry represents a key/value pair.
# WARNING: Do not set the value too large, otherwise it will make a very large impact on the TiKV cluster.
# Please adjust this configuration carefully.
txn-total-size-limit = 104857600

[proxy-protocol]
# PROXY protocol acceptable client networks.
# Empty string means disable PROXY protocol, * means all networks.
Expand Down
10 changes: 10 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ func (s *testConfigSuite) TestConfig(c *C) {
conf.Binlog.Enable = true
conf.Binlog.IgnoreError = true
conf.Binlog.Strategy = "hash"
conf.Performance.TxnEntryCountLimit = 1000
conf.Performance.TxnTotalSizeLimit = 1000
conf.TiKVClient.CommitTimeout = "10s"
configFile := "config.toml"
_, localFile, _, _ := runtime.Caller(0)
Expand All @@ -61,7 +63,10 @@ unrecognized-option-test = true
_, err = f.WriteString(`
token-limit = 0
enable-table-lock = true
delay-clean-table-lock = 5
[performance]
txn-entry-count-limit=2000
txn-total-size-limit=2000
[tikv-client]
commit-timeout="41s"
max-batch-size=128
Expand All @@ -76,10 +81,15 @@ max-batch-size=128
c.Assert(conf.Binlog.Enable, Equals, true)
c.Assert(conf.Binlog.Strategy, Equals, "hash")

// Test that the value will be overwritten by the config file.
c.Assert(conf.Performance.TxnEntryCountLimit, Equals, uint64(2000))
c.Assert(conf.Performance.TxnTotalSizeLimit, Equals, uint64(2000))

c.Assert(conf.TiKVClient.CommitTimeout, Equals, "41s")
c.Assert(conf.TiKVClient.MaxBatchSize, Equals, uint(128))
c.Assert(conf.TokenLimit, Equals, uint(1000))
c.Assert(conf.EnableTableLock, IsTrue)
c.Assert(conf.DelayCleanTableLock, Equals, uint64(5))
c.Assert(f.Close(), IsNil)
c.Assert(os.Remove(configFile), IsNil)

Expand Down
23 changes: 15 additions & 8 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,10 +384,18 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu
}
})

if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) && !mysql.HasPreventNullInsertFlag(oldCol.Flag) {
// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.Flag)
// Introduce the `mysql.HasPreventNullInsertFlag` flag to prevent users from inserting or updating null values.
ver, err = modifyColumnFromNull2NotNull(w, t, dbInfo, tblInfo, job, oldCol, newCol)
return ver, errors.Trace(err)
err = modifyColumnFromNull2NotNull(w, t, dbInfo, tblInfo, job, oldCol, newCol)
if err != nil {
return ver, err
}
// The column should get into prevent null status first.
if noPreventNullFlag {
return updateVersionAndTableInfoWithCheck(t, job, tblInfo, true)
}
}

// We need the latest column's offset and state. This information can be obtained from the store.
Expand Down Expand Up @@ -543,26 +551,25 @@ func rollbackModifyColumnJob(t *meta.Meta, tblInfo *model.TableInfo, job *model.
}

// modifyColumnFromNull2NotNull modifies the type definitions of 'null' to 'not null'.
func modifyColumnFromNull2NotNull(w *worker, t *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo, job *model.Job, oldCol, newCol *model.ColumnInfo) (ver int64, _ error) {
func modifyColumnFromNull2NotNull(w *worker, t *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo, job *model.Job, oldCol, newCol *model.ColumnInfo) error {
// Get sessionctx from context resource pool.
var ctx sessionctx.Context
ctx, err := w.sessPool.get()
if err != nil {
return ver, errors.Trace(err)
return errors.Trace(err)
}
defer w.sessPool.put(ctx)

// If there is a null value inserted, it cannot be modified and needs to be rollback.
err = checkForNullValue(ctx, oldCol.Tp == newCol.Tp, dbInfo.Name, tblInfo.Name, oldCol.Name, newCol.Name)
if err != nil {
job.State = model.JobStateRollingback
return ver, errors.Trace(err)
return errors.Trace(err)
}

// Prevent this field from inserting null values.
tblInfo.Columns[oldCol.Offset].Flag |= mysql.PreventNullInsertFlag
ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, true)
return ver, errors.Trace(err)
return nil
}

func generateOriginDefaultValue(col *model.ColumnInfo) (interface{}, error) {
Expand Down
87 changes: 73 additions & 14 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -268,13 +268,38 @@ func (s *testIntegrationSuite2) TestIssue6101(c *C) {
tk.MustExec("drop table t1")
}

func (s *testIntegrationSuite1) TestIndexLength(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table idx_len(a int(0), b timestamp(0), c datetime(0), d time(0), f float(0), g decimal(0))")
tk.MustExec("create index idx on idx_len(a)")
tk.MustExec("alter table idx_len add index idxa(a)")
tk.MustExec("create index idx1 on idx_len(b)")
tk.MustExec("alter table idx_len add index idxb(b)")
tk.MustExec("create index idx2 on idx_len(c)")
tk.MustExec("alter table idx_len add index idxc(c)")
tk.MustExec("create index idx3 on idx_len(d)")
tk.MustExec("alter table idx_len add index idxd(d)")
tk.MustExec("create index idx4 on idx_len(f)")
tk.MustExec("alter table idx_len add index idxf(f)")
tk.MustExec("create index idx5 on idx_len(g)")
tk.MustExec("alter table idx_len add index idxg(g)")
tk.MustExec("create table idx_len1(a int(0), b timestamp(0), c datetime(0), d time(0), f float(0), g decimal(0), index(a), index(b), index(c), index(d), index(f), index(g))")
}

func (s *testIntegrationSuite3) TestIssue3833(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("create table issue3833 (b char(0))")
tk.MustExec("create table issue3833 (b char(0), c binary(0), d varchar(0))")
assertErrorCode(c, tk, "create index idx on issue3833 (b)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "alter table issue3833 add index idx (b)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create table issue3833_2 (b char(0), index (b))", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(b))", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create index idx on issue3833 (c)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "alter table issue3833 add index idx (c)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(c))", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create index idx on issue3833 (d)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "alter table issue3833 add index idx (d)", tmysql.ErrWrongKeyColumn)
assertErrorCode(c, tk, "create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(d))", tmysql.ErrWrongKeyColumn)
}

func (s *testIntegrationSuite1) TestIssue2858And2717(c *C) {
Expand Down Expand Up @@ -656,17 +681,17 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) {
tk.MustExec("drop table t;")
tk.MustExec("create table t(a varchar(10)) charset utf8")
tk.MustExec("alter table t convert to charset utf8mb4;")
checkCharset := func() {
checkCharset := func(chs, coll string) {
tbl := testGetTableByName(c, s.ctx, "test", "t")
c.Assert(tbl, NotNil)
c.Assert(tbl.Meta().Charset, Equals, charset.CharsetUTF8MB4)
c.Assert(tbl.Meta().Collate, Equals, charset.CollationUTF8MB4)
c.Assert(tbl.Meta().Charset, Equals, chs)
c.Assert(tbl.Meta().Collate, Equals, coll)
for _, col := range tbl.Meta().Columns {
c.Assert(col.Charset, Equals, charset.CharsetUTF8MB4)
c.Assert(col.Collate, Equals, charset.CollationUTF8MB4)
c.Assert(col.Charset, Equals, chs)
c.Assert(col.Collate, Equals, coll)
}
}
checkCharset()
checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4)

// Test when column charset can not convert to the target charset.
tk.MustExec("drop table t;")
Expand All @@ -675,11 +700,16 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) {
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify charset from ascii to utf8mb4")

tk.MustExec("drop table t;")
tk.MustExec("create table t(a varchar(10) character set utf8) charset utf8")
tk.MustExec("alter table t convert to charset utf8 collate utf8_general_ci;")
checkCharset(charset.CharsetUTF8, "utf8_general_ci")

// Test when table charset is equal to target charset but column charset is not equal.
tk.MustExec("drop table t;")
tk.MustExec("create table t(a varchar(10) character set utf8) charset utf8mb4")
tk.MustExec("alter table t convert to charset utf8mb4;")
checkCharset()
tk.MustExec("alter table t convert to charset utf8mb4 collate utf8mb4_general_ci;")
checkCharset(charset.CharsetUTF8MB4, "utf8mb4_general_ci")

// Mock table info with charset is "". Old TiDB maybe create table with charset is "".
db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test"))
Expand Down Expand Up @@ -712,7 +742,7 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) {
c.Assert(tbl.Meta().Collate, Equals, "")
// Test when table charset is "", this for compatibility.
tk.MustExec("alter table t convert to charset utf8mb4;")
checkCharset()
checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4)

// Test when column charset is "".
tbl = testGetTableByName(c, s.ctx, "test", "t")
Expand All @@ -727,7 +757,7 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) {
c.Assert(tbl.Meta().Columns[0].Charset, Equals, "")
c.Assert(tbl.Meta().Columns[0].Collate, Equals, "")
tk.MustExec("alter table t convert to charset utf8mb4;")
checkCharset()
checkCharset(charset.CharsetUTF8MB4, charset.CollationUTF8MB4)

tk.MustExec("drop table t")
tk.MustExec("create table t (a blob) character set utf8;")
Expand All @@ -740,6 +770,33 @@ func (s *testIntegrationSuite4) TestChangingTableCharset(c *C) {

}

func (s *testIntegrationSuite5) TestModifyingColumnOption(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test")
tk.MustExec("use test")

errMsg := "[ddl:203]" // unsupported modify column with references
assertErrCode := func(sql string, errCodeStr string) {
_, err := tk.Exec(sql)
c.Assert(err, NotNil)
c.Assert(err.Error()[:len(errCodeStr)], Equals, errCodeStr)
}

tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (b char(1) default null) engine=InnoDB default charset=utf8mb4 collate=utf8mb4_general_ci")
tk.MustExec("alter table t1 modify column b char(1) character set utf8mb4 collate utf8mb4_general_ci")

tk.MustExec("drop table t1")
tk.MustExec("create table t1 (b char(1) collate utf8mb4_general_ci)")
tk.MustExec("alter table t1 modify b char(1) character set utf8mb4 collate utf8mb4_general_ci")

tk.MustExec("drop table t1")
tk.MustExec("drop table if exists t2")
tk.MustExec("create table t1 (a int)")
tk.MustExec("create table t2 (b int, c int)")
assertErrCode("alter table t2 modify column c int references t1(a)", errMsg)
}

func (s *testIntegrationSuite2) TestCaseInsensitiveCharsetAndCollate(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down Expand Up @@ -1737,10 +1794,12 @@ func (s *testIntegrationSuite5) TestChangingDBCharset(c *C) {
for _, fc := range failedCases {
c.Assert(tk.ExecToErr(fc.stmt).Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt))
}
tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8' COLLATE = 'utf8_unicode_ci'")
verifyDBCharsetAndCollate("alterdb2", "utf8", "utf8_unicode_ci")

tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8mb4'")
verifyDBCharsetAndCollate("alterdb2", "utf8mb4", "utf8mb4_bin")

err := tk.ExecToErr("ALTER SCHEMA CHARACTER SET = 'utf8mb4' COLLATE = 'utf8mb4_general_ci'")
c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify collate from utf8mb4_bin to utf8mb4_general_ci")
tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8mb4' COLLATE = 'utf8mb4_general_ci'")
verifyDBCharsetAndCollate("alterdb2", "utf8mb4", "utf8mb4_general_ci")
}
Loading

0 comments on commit ffce1cc

Please sign in to comment.