Skip to content

Commit

Permalink
Merge branch 'master' into improve_str_to_date
Browse files Browse the repository at this point in the history
  • Loading branch information
JaySon-Huang authored Jun 22, 2021
2 parents 431f902 + b858ce4 commit 7955cde
Show file tree
Hide file tree
Showing 282 changed files with 3,105 additions and 39,707 deletions.
18 changes: 1 addition & 17 deletions .build.ps1
Original file line number Diff line number Diff line change
Expand Up @@ -287,14 +287,6 @@ task ExplainTest -If (-not ((Get-Content cmd\explaintest\r\explain.result -Raw)
}
}

# Synopsis: Check dependency.
task CheckDep {
$list = go list -json github.com/pingcap/tidb/store/tikv | ConvertFrom-Json
if ($list.Imports | Where-Object { Select-String -Pattern '^github.com/pingcap/parser$' -InputObject $_ }) {
throw 'incorrect import of github.com/pingcap/parser'
}
}

# Synopsis: Run unit tests.
task GoTest BuildFailPoint, {
Enable-FailPoint
Expand Down Expand Up @@ -343,14 +335,6 @@ task GoLeakTest BuildFailPoint, {
$env:TZ = $Task.Data.tz
}

# Synopsis: Run some tests with real TiKV.
task TiKVIntegrationTest BuildFailPoint, {
Enable-FailPoint
{ & $GO test -p $P github.com/pingcap/tidb/store/tikv -with-tikv=true }
} -Done {
Disable-FailPoint
}

# Synopsis: Ensure generated code is up to date.
task GoGenerate {
exec { & $GO generate ./... }
Expand All @@ -361,7 +345,7 @@ task GoGenerate {
}

# Synopsis: Run common tests.
task Test ExplainTest, CheckDep, GoTest, GoGenerate
task Test ExplainTest, GoTest, GoGenerate

# Synopsis: Check and Test.
task Dev Check, Test
Expand Down
9 changes: 1 addition & 8 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ test: test_part_1 test_part_2

test_part_1: checklist explaintest

test_part_2: checkdep gotest gogenerate
test_part_2: gotest gogenerate

explaintest: server_check
@cd cmd/explaintest && ./run-tests.sh -s ../../bin/tidb-server
Expand Down Expand Up @@ -153,10 +153,6 @@ leak: failpoint-enable
$(GOTEST) -tags leak $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

tikv_integration_test: failpoint-enable
$(GOTEST) ./store/tikv/. -with-tikv=true || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

server:
ifeq ($(TARGET), "")
CGO_ENABLED=1 $(GOBUILD) $(RACE_FLAG) -ldflags '$(LDFLAGS) $(CHECK_FLAG)' -o bin/tidb-server tidb-server/main.go
Expand Down Expand Up @@ -208,9 +204,6 @@ failpoint-disable: tools/bin/failpoint-ctl
# Restoring gofail failpoints...
@$(FAILPOINT_DISABLE)

checkdep:
$(GO) list -f '{{ join .Imports "\n" }}' github.com/pingcap/tidb/store/tikv | grep ^github.com/pingcap/parser$$ || exit 0; exit 1

tools/bin/megacheck: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/megacheck honnef.co/go/tools/cmd/megacheck
Expand Down
16 changes: 16 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/bindinfo"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -2098,3 +2099,18 @@ func (s *testSuite) TestBindingWithoutCharset(c *C) {
c.Assert(rows[0][0], Equals, "select * from `test` . `t` where `a` = ?")
c.Assert(rows[0][1], Equals, "SELECT * FROM `test`.`t` WHERE `a` = 'aa'")
}

func (s *testSuite) TestTemporaryTable(c *C) {
tk := testkit.NewTestKit(c, s.store)
s.cleanBindingEnv(tk)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("set tidb_enable_global_temporary_table = true")
tk.MustExec("create global temporary table t(a int, b int, key(a), key(b)) on commit delete rows")
tk.MustExec("create table t2(a int, b int, key(a), key(b))")
tk.MustGetErrCode("create session binding for select * from t where b = 123 using select * from t ignore index(b) where b = 123;", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for insert into t select * from t2 where t2.b = 1 and t2.c > 1 using insert into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for replace into t select * from t2 where t2.b = 1 and t2.c > 1 using replace into t select /*+ use_index(t2,c) */ * from t2 where t2.b = 1 and t2.c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for update t set a = 1 where b = 1 and c > 1 using update /*+ use_index(t, c) */ t set a = 1 where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
tk.MustGetErrCode("create binding for delete from t where b = 1 and c > 1 using delete /*+ use_index(t, c) */ from t where b = 1 and c > 1", errno.ErrOptOnTemporaryTable)
}
2 changes: 1 addition & 1 deletion cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ var (
func main() {
flag.Parse()
flag.PrintDefaults()
err := logutil.InitZapLogger(logutil.NewLogConfig(*logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(*logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
terror.MustNil(err)
err = store.Register("tikv", driver.TiKVDriver{})
terror.MustNil(err)
Expand Down
2 changes: 1 addition & 1 deletion cmd/benchfilesort/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,7 @@ func driveRunCmd() {
}

func init() {
err := logutil.InitZapLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
terror.MustNil(err)
cwd, err1 := os.Getwd()
terror.MustNil(err1)
Expand Down
2 changes: 1 addition & 1 deletion cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ func openDBWithRetry(driverName, dataSourceName string) (mdb *sql.DB, err error)
func main() {
flag.Parse()

err := logutil.InitZapLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
err := logutil.InitLogger(logutil.NewLogConfig(logLevel, logutil.DefaultLogFormat, "", logutil.EmptyFileLogConfig, false))
if err != nil {
panic("init logger fail, " + err.Error())
}
Expand Down
3 changes: 2 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,7 +494,8 @@ type Binlog struct {
// PessimisticTxn is the config for pessimistic transaction.
type PessimisticTxn struct {
// The max count of retry for a single statement in a pessimistic transaction.
MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"`
MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"`
// The max count of deadlock events that will be recorded in the information_schema.deadlocks table.
DeadlockHistoryCapacity uint `toml:"deadlock-history-capacity" json:"deadlock-history-capacity"`
}

Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -443,6 +443,9 @@ strategy = "range"
# max retry count for a statement in a pessimistic transaction.
max-retry-count = 256

# The max count of deadlock events that will be recorded in the information_schema.deadlocks table.
deadlock-history-capacity = 10

[stmt-summary]
# enable statement summary.
enable = true
Expand Down
3 changes: 3 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,8 @@ group= "abc"
zone= "dc-1"
[security]
spilled-file-encryption-method = "plaintext"
[pessimistic-txn]
deadlock-history-capacity = 123
`)

c.Assert(err, IsNil)
Expand Down Expand Up @@ -284,6 +286,7 @@ spilled-file-encryption-method = "plaintext"
c.Assert(conf.EnableEnumLengthLimit, Equals, false)
c.Assert(conf.EnableForwarding, Equals, true)
c.Assert(conf.StoresRefreshInterval, Equals, uint64(30))
c.Assert(conf.PessimisticTxn.DeadlockHistoryCapacity, Equals, uint(123))

_, err = f.WriteString(`
[log.file]
Expand Down
4 changes: 2 additions & 2 deletions config/config_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func flatten(flatMap map[string]interface{}, nested interface{}, prefix string)
}
}

// GetTxnScopeFromConfig extracts @@txn_scope value from config
func GetTxnScopeFromConfig() (bool, string) {
// GetTxnScopeFromConfig extracts @@txn_scope value from the config.
func GetTxnScopeFromConfig() string {
return tikvcfg.GetTxnScopeFromConfig()
}
53 changes: 38 additions & 15 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -798,6 +798,39 @@ func getModifyColumnInfo(t *meta.Meta, job *model.Job) (*model.DBInfo, *model.Ta
return dbInfo, tblInfo, oldCol, jobParam, errors.Trace(err)
}

// getOriginDefaultValueForModifyColumn gets the original default value for modifying column.
// Since column type change is implemented as adding a new column then substituting the old one.
// Case exists when update-where statement fetch a NULL for not-null column without any default data,
// it will errors.
// So we set original default value here to prevent this error. If the oldCol has the original default value, we use it.
// Otherwise we set the zero value as original default value.
// Besides, in insert & update records, we have already implement using the casted value of relative column to insert
// rather than the original default value.
func getOriginDefaultValueForModifyColumn(d *ddlCtx, changingCol, oldCol *model.ColumnInfo) (interface{}, error) {
var err error
originDefVal := oldCol.GetOriginDefaultValue()
if originDefVal != nil {
sessCtx := newContext(d.store)
odv, err := table.CastValue(sessCtx, types.NewDatum(originDefVal), changingCol, false, false)
if err != nil {
logutil.BgLogger().Info("[ddl] cast origin default value failed", zap.Error(err))
}
if !odv.IsNull() {
if originDefVal, err = odv.ToString(); err != nil {
originDefVal = nil
logutil.BgLogger().Info("[ddl] convert default value to string failed", zap.Error(err))
}
}
}
if originDefVal == nil {
originDefVal, err = generateOriginDefaultValue(changingCol)
if err != nil {
return nil, errors.Trace(err)
}
}
return originDefVal, nil
}

func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
dbInfo, tblInfo, oldCol, jobParam, err := getModifyColumnInfo(t, job)
if err != nil {
Expand Down Expand Up @@ -857,19 +890,9 @@ func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in
jobParam.changingCol = jobParam.newCol.Clone()
jobParam.changingCol.Name = newColName
jobParam.changingCol.ChangeStateInfo = &model.ChangeStateInfo{DependencyColumnOffset: oldCol.Offset}

// Since column type change is implemented as adding a new column then substituting the old one.
// Case exists when update-where statement fetch a NULL for not-null column without any default data,
// it will errors.
// So we set zero original default value here to prevent this error. Besides, in insert & update records,
// we have already implement using the casted value of relative column to insert rather than the origin
// default value.
originDefVal := oldCol.GetOriginDefaultValue()
if originDefVal == nil {
originDefVal, err = generateOriginDefaultValue(jobParam.newCol)
if err != nil {
return ver, errors.Trace(err)
}
originDefVal, err := getOriginDefaultValueForModifyColumn(d, jobParam.changingCol, oldCol)
if err != nil {
return ver, errors.Trace(err)
}
if err = jobParam.changingCol.SetOriginDefaultValue(originDefVal); err != nil {
return ver, errors.Trace(err)
Expand Down Expand Up @@ -1760,7 +1783,7 @@ func rollbackModifyColumnJob(t *meta.Meta, tblInfo *model.TableInfo, job *model.
// modifyColsFromNull2NotNull modifies the type definitions of 'null' to 'not null'.
// Introduce the `mysql.PreventNullInsertFlag` flag to prevent users from inserting or updating null values.
func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.TableInfo, cols []*model.ColumnInfo,
newColName model.CIStr, isModifiedType bool) error {
newColName model.CIStr, isDataTruncated bool) error {
// Get sessionctx from context resource pool.
var ctx sessionctx.Context
ctx, err := w.sessPool.get()
Expand All @@ -1777,7 +1800,7 @@ func modifyColsFromNull2NotNull(w *worker, dbInfo *model.DBInfo, tblInfo *model.
})
if !skipCheck {
// If there is a null value inserted, it cannot be modified and needs to be rollback.
err = checkForNullValue(ctx, isModifiedType, dbInfo.Name, tblInfo.Name, newColName, cols...)
err = checkForNullValue(ctx, isDataTruncated, dbInfo.Name, tblInfo.Name, newColName, cols...)
if err != nil {
return errors.Trace(err)
}
Expand Down
95 changes: 89 additions & 6 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,9 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenInteger(c *C) {
tk.MustExec("alter table t modify column b int not null")

tk.MustExec("insert into t(a, b) values (null, 1)")
// Modify column from null to not null in same type will cause ErrInvalidUseOfNull
tk.MustGetErrCode("alter table t modify column a int not null", mysql.ErrInvalidUseOfNull)
// Modify column from null to not null in same type will cause ErrWarnDataTruncated
_, err := tk.Exec("alter table t modify column a int not null")
c.Assert(err.Error(), Equals, "[ddl:1265]Data truncated for column 'a' at row 1")

// Modify column from null to not null in different type will cause WarnDataTruncated.
tk.MustGetErrCode("alter table t modify column a tinyint not null", mysql.WarnDataTruncated)
Expand Down Expand Up @@ -131,7 +132,7 @@ func (s *testColumnTypeChangeSuite) TestColumnTypeChangeBetweenInteger(c *C) {
tk.MustGetErrCode("alter table t modify column a mediumint", mysql.ErrDataOutOfRange)
tk.MustGetErrCode("alter table t modify column a smallint", mysql.ErrDataOutOfRange)
tk.MustGetErrCode("alter table t modify column a tinyint", mysql.ErrDataOutOfRange)
_, err := tk.Exec("admin check table t")
_, err = tk.Exec("admin check table t")
c.Assert(err, IsNil)
}

Expand Down Expand Up @@ -1671,7 +1672,7 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValue(c *C) {
tk.MustExec("drop table if exists t")
}

func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol(c *C) {
func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddColAndCastSucc(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

Expand Down Expand Up @@ -1708,8 +1709,11 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol
checkErr = errors.New("assert the writable column number error")
return
}
if tbl.WritableCols()[3].OriginDefaultValue.(string) != "1971-06-09 00:00:00" {
checkErr = errors.New("assert the write only column origin default value error")
originalDV := fmt.Sprintf("%v", tbl.WritableCols()[3].OriginDefaultValue)
expectVal := "1971-06-09"
if originalDV != expectVal {
errMsg := fmt.Sprintf("expect: %v, got: %v", expectVal, originalDV)
checkErr = errors.New("assert the write only column origin default value error" + errMsg)
return
}
}
Expand Down Expand Up @@ -1752,6 +1756,62 @@ func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddCol
tk.MustExec("drop table if exists t")
}

// TestChangingColOriginDefaultValueAfterAddColAndCastFail tests #25383.
func (s *testColumnTypeChangeSuite) TestChangingColOriginDefaultValueAfterAddColAndCastFail(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")

tk1 := testkit.NewTestKit(c, s.store)
tk1.MustExec("use test")

tk.MustExec(fmt.Sprintf("set time_zone = 'UTC'"))
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a VARCHAR(31) NULL DEFAULT 'wwrzfwzb01j6ddj', b DECIMAL(12,0) NULL DEFAULT '-729850476163')")
tk.MustExec("ALTER TABLE t ADD COLUMN x CHAR(218) NULL DEFAULT 'lkittuae'")

tbl := testGetTableByName(c, tk.Se, "test", "t")
originalHook := s.dom.DDL().GetHook()
hook := &ddl.TestDDLCallback{Do: s.dom}
var checkErr error
hook.OnJobRunBeforeExported = func(job *model.Job) {
if checkErr != nil {
return
}
if tbl.Meta().ID != job.TableID {
return
}

if job.SchemaState == model.StateWriteOnly || job.SchemaState == model.StateWriteReorganization {
tbl := testGetTableByName(c, tk1.Se, "test", "t")
if len(tbl.WritableCols()) != 4 {
errMsg := fmt.Sprintf("cols len:%v", len(tbl.WritableCols()))
checkErr = errors.New("assert the writable column number error" + errMsg)
return
}
originalDV := fmt.Sprintf("%v", tbl.WritableCols()[3].OriginDefaultValue)
expectVal := "0000-00-00 00:00:00"
if originalDV != expectVal {
errMsg := fmt.Sprintf("expect: %v, got: %v", expectVal, originalDV)
checkErr = errors.New("assert the write only column origin default value error" + errMsg)
return
}
// The casted value will be inserted into changing column too.
_, err := tk1.Exec("UPDATE t SET a = '18apf' WHERE x = '' AND a = 'mul'")
if err != nil {
checkErr = err
return
}
}
}

s.dom.DDL().(ddl.DDLForTest).SetHook(hook)
tk.MustExec("alter table t modify column x DATETIME NULL DEFAULT '3771-02-28 13:00:11' AFTER b;")
s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook)
c.Assert(checkErr, IsNil)
tk.MustQuery("select * from t order by a").Check(testkit.Rows())
tk.MustExec("drop table if exists t")
}

// Close issue #22820
func (s *testColumnTypeChangeSuite) TestChangingAttributeOfColumnWithFK(c *C) {
tk := testkit.NewTestKit(c, s.store)
Expand Down Expand Up @@ -2046,3 +2106,26 @@ func (s *testColumnTypeChangeSuite) TestChangePrefixedIndexColumnToNonPrefixOne(
tk.MustExec("create table t(a varchar(700), key(a(700)));")
tk.MustGetErrCode("alter table t change column a a tinytext;", mysql.ErrBlobKeyWithoutLength)
}

// Fix issue https://github.com/pingcap/tidb/issues/25469
func (s *testColumnTypeChangeSuite) TestCastToTimeStampDecodeError(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")

tk.MustExec("CREATE TABLE `t` (" +
" `a` datetime DEFAULT '1764-06-11 02:46:14'" +
") ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin COMMENT='7b84832e-f857-4116-8872-82fc9dcc4ab3'")
tk.MustExec("insert into `t` values();")
tk.MustGetErrCode("alter table `t` change column `a` `b` TIMESTAMP NULL DEFAULT '2015-11-14 07:12:24';", mysql.ErrTruncatedWrongValue)

tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE `t` (" +
" `a` date DEFAULT '1764-06-11 02:46:14'" +
") ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin COMMENT='7b84832e-f857-4116-8872-82fc9dcc4ab3'")
tk.MustExec("insert into `t` values();")
tk.MustGetErrCode("alter table `t` change column `a` `b` TIMESTAMP NULL DEFAULT '2015-11-14 07:12:24';", mysql.ErrTruncatedWrongValue)
tk.MustExec("drop table if exists t")

// Normal cast datetime to timestamp can succeed.
tk.MustQuery("select timestamp(cast('1000-11-11 12-3-1' as date));").Check(testkit.Rows("1000-11-11 00:00:00"))
}
Loading

0 comments on commit 7955cde

Please sign in to comment.