Skip to content

Commit

Permalink
Merge branch 'master' into issue35296
Browse files Browse the repository at this point in the history
  • Loading branch information
time-and-fate authored Jun 10, 2022
2 parents d9201e1 + 5d4745e commit 97f3394
Show file tree
Hide file tree
Showing 18 changed files with 9,024 additions and 8,876 deletions.
34 changes: 32 additions & 2 deletions ddl/cancel_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,19 +25,49 @@ import (
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/testkit"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
atomicutil "go.uber.org/atomic"
)

type testCancelJob struct {
sql string
ok bool
cancelState model.SchemaState
cancelState interface{} // model.SchemaState | []model.SchemaState
onJobBefore bool
onJobUpdate bool
prepareSQL []string
}

type subStates = []model.SchemaState

func testMatchCancelState(t *testing.T, job *model.Job, cancelState interface{}, sql string) bool {
switch v := cancelState.(type) {
case model.SchemaState:
if job.Type == model.ActionMultiSchemaChange {
msg := fmt.Sprintf("unexpected multi-schema change(sql: %s, cancel state: %s)", sql, v)
assert.Failf(t, msg, "use []model.SchemaState as cancel states instead")
return false
}
return job.SchemaState == v
case subStates: // For multi-schema change sub-jobs.
if job.MultiSchemaInfo == nil {
msg := fmt.Sprintf("not multi-schema change(sql: %s, cancel state: %v)", sql, v)
assert.Failf(t, msg, "use model.SchemaState as the cancel state instead")
return false
}
assert.Equal(t, len(job.MultiSchemaInfo.SubJobs), len(v), sql)
for i, subJobSchemaState := range v {
if job.MultiSchemaInfo.SubJobs[i].SchemaState != subJobSchemaState {
return false
}
}
return true
default:
return false
}
}

var allTestCase = []testCancelJob{
// Add index.
{"create unique index c3_index on t_partition (c1)", true, model.StateWriteReorganization, true, true, nil},
Expand Down Expand Up @@ -246,7 +276,7 @@ func TestCancel(t *testing.T) {
cancelWhenReorgNotStart := false

hookFunc := func(job *model.Job) {
if job.SchemaState == allTestCase[i.Load()].cancelState && !cancel {
if testMatchCancelState(t, job, allTestCase[i.Load()].cancelState, allTestCase[i.Load()].sql) && !cancel {
if !cancelWhenReorgNotStart && job.SchemaState == model.StateWriteReorganization && job.MayNeedReorg() && job.RowCount == 0 {
return
}
Expand Down
11 changes: 7 additions & 4 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/testkit/external"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/mock"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -63,7 +64,6 @@ func TestColumnAdd(t *testing.T) {
var jobID int64
tc.OnJobUpdatedExported = func(job *model.Job) {
jobID = job.ID
require.NoError(t, dom.Reload())
tbl, exist := dom.InfoSchema().TableByID(job.TableID)
require.True(t, exist)
switch job.SchemaState {
Expand Down Expand Up @@ -96,11 +96,14 @@ func TestColumnAdd(t *testing.T) {
}
}
tc.OnJobUpdatedExported = func(job *model.Job) {
if job.NotStarted() {
return
}
jobID = job.ID
tbl := external.GetTableByName(t, internal, "test", "t")
if job.SchemaState != model.StatePublic {
for _, col := range tbl.Cols() {
require.NotEqualf(t, col.ID, dropCol.ID, "column is not dropped")
assert.NotEqualf(t, col.ID, dropCol.ID, "column is not dropped")
}
}
}
Expand Down Expand Up @@ -224,7 +227,7 @@ func checkAddWriteOnly(ctx sessionctx.Context, deleteOnlyTable, writeOnlyTable t
return errors.Trace(err)
}
err = checkResult(ctx, writeOnlyTable, writeOnlyTable.WritableCols(), [][]string{
{"1", "2", "<nil>"},
{"1", "2", "3"},
{"2", "3", "3"},
})
if err != nil {
Expand All @@ -236,7 +239,7 @@ func checkAddWriteOnly(ctx sessionctx.Context, deleteOnlyTable, writeOnlyTable t
return errors.Trace(err)
}
got := fmt.Sprintf("%v", row)
expect := fmt.Sprintf("%v", []types.Datum{types.NewDatum(1), types.NewDatum(2), types.NewDatum(nil)})
expect := fmt.Sprintf("%v", []types.Datum{types.NewDatum(1), types.NewDatum(2), types.NewDatum(3)})
if got != expect {
return errors.Errorf("expect %v, got %v", expect, got)
}
Expand Down
1 change: 1 addition & 0 deletions ddl/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ func TestMain(m *testing.M) {

autoid.SetStep(5000)
ddl.ReorgWaitTimeout = 30 * time.Millisecond
ddl.RunInGoTest = true
ddl.SetBatchInsertDeleteRangeSize(2)

config.UpdateGlobal(func(conf *config.Config) {
Expand Down
9 changes: 9 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6070,3 +6070,12 @@ func TestIsFastPlan(t *testing.T) {
require.Equal(t, ca.isFastPlan, ok)
}
}

func TestShowSessionStates(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustQuery("show session_states").Check(testkit.Rows())
tk.MustExec("set session_states 'x'")
tk.MustGetErrCode("set session_states 1", errno.ErrParse)
}
6 changes: 6 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,6 +235,8 @@ func (e *ShowExec) fetchAll(ctx context.Context) error {
return e.fetchShowPlacementForTable(ctx)
case ast.ShowPlacementForPartition:
return e.fetchShowPlacementForPartition(ctx)
case ast.ShowSessionStates:
return e.fetchShowSessionStates(ctx)
}
return nil
}
Expand Down Expand Up @@ -1927,6 +1929,10 @@ func (e *ShowExec) fetchShowBuiltins() error {
return nil
}

func (e *ShowExec) fetchShowSessionStates(ctx context.Context) error {
return nil
}

// tryFillViewColumnType fill the columns type info of a view.
// Because view's underlying table's column could change or recreate, so view's column type may change over time.
// To avoid this situation we need to generate a logical plan and extract current column types from Schema.
Expand Down
6 changes: 6 additions & 0 deletions executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,6 +151,8 @@ func (e *SimpleExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
err = e.executeRenameUser(x)
case *ast.SetPwdStmt:
err = e.executeSetPwd(ctx, x)
case *ast.SetSessionStatesStmt:
err = e.executeSetSessionStates(ctx, x)
case *ast.KillStmt:
err = e.executeKillStmt(ctx, x)
case *ast.BinlogStmt:
Expand Down Expand Up @@ -1684,6 +1686,10 @@ func asyncDelayShutdown(p *os.Process, delay time.Duration) {
}
}

func (e *SimpleExec) executeSetSessionStates(ctx context.Context, s *ast.SetSessionStatesStmt) error {
return nil
}

func (e *SimpleExec) executeAdmin(s *ast.AdminStmt) error {
switch s.Tp {
case ast.AdminReloadStatistics:
Expand Down
3 changes: 3 additions & 0 deletions parser/ast/dml.go
Original file line number Diff line number Diff line change
Expand Up @@ -2674,6 +2674,7 @@ const (
ShowPlacementForTable
ShowPlacementForPartition
ShowPlacementLabels
ShowSessionStates
)

const (
Expand Down Expand Up @@ -3025,6 +3026,8 @@ func (n *ShowStmt) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord("PLACEMENT")
case ShowPlacementLabels:
ctx.WriteKeyWord("PLACEMENT LABELS")
case ShowSessionStates:
ctx.WriteKeyWord("SESSION_STATES")
default:
return errors.New("Unknown ShowStmt type")
}
Expand Down
23 changes: 23 additions & 0 deletions parser/ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ var (
_ StmtNode = &SetRoleStmt{}
_ StmtNode = &SetDefaultRoleStmt{}
_ StmtNode = &SetStmt{}
_ StmtNode = &SetSessionStatesStmt{}
_ StmtNode = &UseStmt{}
_ StmtNode = &FlushStmt{}
_ StmtNode = &KillStmt{}
Expand Down Expand Up @@ -1055,6 +1056,28 @@ func (n *SetConfigStmt) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

// SetSessionStatesStmt is a statement to restore session states.
type SetSessionStatesStmt struct {
stmtNode

SessionStates string
}

func (n *SetSessionStatesStmt) Restore(ctx *format.RestoreCtx) error {
ctx.WriteKeyWord("SET SESSION_STATES ")
ctx.WriteString(n.SessionStates)
return nil
}

func (n *SetSessionStatesStmt) Accept(v Visitor) (Node, bool) {
newNode, skipChildren := v.Enter(n)
if skipChildren {
return v.Leave(newNode)
}
n = newNode.(*SetSessionStatesStmt)
return v.Leave(n)
}

/*
// SetCharsetStmt is a statement to assign values to character and collation variables.
// See https://dev.mysql.com/doc/refman/5.7/en/set-statement.html
Expand Down
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -632,6 +632,7 @@ var tokenMap = map[string]int{
"SERIAL": serial,
"SERIALIZABLE": serializable,
"SESSION": session,
"SESSION_STATES": sessionStates,
"SET": set,
"SETVAL": setval,
"SHARD_ROW_ID_BITS": shardRowIDBits,
Expand Down
18 changes: 18 additions & 0 deletions parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ const (
ActionAlterTableStatsOptions ActionType = 58
ActionAlterNoCacheTable ActionType = 59
ActionCreateTables ActionType = 60
ActionMultiSchemaChange ActionType = 61
)

var actionMap = map[ActionType]string{
Expand Down Expand Up @@ -157,6 +158,7 @@ var actionMap = map[ActionType]string{
ActionAlterCacheTable: "alter table cache",
ActionAlterNoCacheTable: "alter table nocache",
ActionAlterTableStatsOptions: "alter table statistics options",
ActionMultiSchemaChange: "alter table multi-schema change",

// `ActionAlterTableAlterPartition` is removed and will never be used.
// Just left a tombstone here for compatibility.
Expand Down Expand Up @@ -256,6 +258,22 @@ func NewDDLReorgMeta() *DDLReorgMeta {
// MultiSchemaInfo keeps some information for multi schema change.
type MultiSchemaInfo struct {
Warnings []*errors.Error

SubJobs []*SubJob `json:"sub_jobs"`
Revertible bool `json:"revertible"`
}

type SubJob struct {
Type ActionType `json:"type"`
Args []interface{} `json:"-"`
RawArgs json.RawMessage `json:"raw_args"`
SchemaState SchemaState `json:"schema_state"`
SnapshotVer uint64 `json:"snapshot_ver"`
Revertible bool `json:"revertible"`
State JobState `json:"state"`
RowCount int64 `json:"row_count"`
Warning *terror.Error `json:"warning"`
CtxVars []interface{} `json:"-"`
}

// Job is for a DDL operation.
Expand Down
Loading

0 comments on commit 97f3394

Please sign in to comment.