From e3b49f6b43b640126055dda89bea4febbcd45a30 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Sat, 20 Oct 2018 11:40:14 +0800 Subject: [PATCH 1/3] *: move `Statement` and `RecordSet` from ast to sqlexec package Statement use RecordSet and RecordSet use chunk After moving them out, the ast package do not depends on chunk any more --- ast/ast.go | 40 ------------------------- ddl/db_change_test.go | 3 +- executor/adapter.go | 11 +++---- executor/prepared.go | 2 +- server/driver_tidb.go | 3 +- session/bench_test.go | 4 +-- session/session.go | 27 +++++++++-------- session/tidb.go | 11 +++---- session/tidb_test.go | 6 ++-- statistics/ddl.go | 3 +- statistics/sample.go | 3 +- statistics/sample_test.go | 4 +-- statistics/statistics_test.go | 15 +++++----- store/mockstore/mocktikv/analyze.go | 4 +-- util/mock/context.go | 3 +- util/sqlexec/restricted_sql_executor.go | 40 ++++++++++++++++++++++++- util/testkit/testkit.go | 10 +++---- 17 files changed, 96 insertions(+), 93 deletions(-) diff --git a/ast/ast.go b/ast/ast.go index eef7ac63016fc..a99d806ca9020 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -20,8 +20,6 @@ import ( "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/types" - "github.com/pingcap/tidb/util/chunk" - "golang.org/x/net/context" ) // Node is the basic element of the AST. @@ -130,22 +128,6 @@ type ResultField struct { Referenced bool } -// RecordSet is an abstract result set interface to help get data from Plan. -type RecordSet interface { - // Fields gets result fields. - Fields() []*ResultField - - // Next reads records into chunk. - Next(ctx context.Context, chk *chunk.Chunk) error - - // NewChunk creates a new chunk with initial capacity. - NewChunk() *chunk.Chunk - - // Close closes the underlying iterator, call Next after Close will - // restart the iteration. - Close() error -} - // ResultSetNode interface has a ResultFields property, represents a Node that returns result set. // Implementations include SelectStmt, SubqueryExpr, TableSource, TableName and Join. type ResultSetNode interface { @@ -159,28 +141,6 @@ type SensitiveStmtNode interface { SecureText() string } -// Statement is an interface for SQL execution. -// NOTE: all Statement implementations must be safe for -// concurrent using by multiple goroutines. -// If the Exec method requires any Execution domain local data, -// they must be held out of the implementing instance. -type Statement interface { - // OriginText gets the origin SQL text. - OriginText() string - - // Exec executes SQL and gets a Recordset. - Exec(ctx context.Context) (RecordSet, error) - - // IsPrepared returns whether this statement is prepared statement. - IsPrepared() bool - - // IsReadOnly returns if the statement is read only. For example: SelectStmt without lock. - IsReadOnly() bool - - // RebuildPlan rebuilds the plan of the statement. - RebuildPlan() (schemaVersion int64, err error) -} - // Visitor visits a Node. type Visitor interface { // Enter is called before children nodes are visited. diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 9cbc5e4818b6d..bac31f693cd78 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" @@ -229,7 +230,7 @@ func (s *testStateChangeSuite) test(c *C, tableName, alterTableSQL string, testI type stateCase struct { session session.Session rawStmt ast.StmtNode - stmt ast.Statement + stmt sqlexec.Statement expectedExecErr error expectedCompileErr error } diff --git a/executor/adapter.go b/executor/adapter.go index b5062cf80112c..3c1593769b7b5 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -45,7 +46,7 @@ type processinfoSetter interface { SetProcessInfo(string, time.Time, byte) } -// recordSet wraps an executor, implements ast.RecordSet interface +// recordSet wraps an executor, implements sqlexec.RecordSet interface type recordSet struct { fields []*ast.ResultField executor Executor @@ -122,7 +123,7 @@ func (a *recordSet) Close() error { return errors.Trace(err) } -// ExecStmt implements the ast.Statement interface, it builds a planner.Plan to an ast.Statement. +// ExecStmt implements the sqlexec.Statement interface, it builds a planner.Plan to an sqlexec.Statement. type ExecStmt struct { // InfoSchema stores a reference to the schema information. InfoSchema infoschema.InfoSchema @@ -183,8 +184,8 @@ func (a *ExecStmt) RebuildPlan() (int64, error) { // Exec builds an Executor from a plan. If the Executor doesn't return result, // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns -// result, execution is done after this function returns, in the returned ast.RecordSet Next method. -func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { +// result, execution is done after this function returns, in the returned sqlexec.RecordSet Next method. +func (a *ExecStmt) Exec(ctx context.Context) (sqlexec.RecordSet, error) { a.StartTime = time.Now() sctx := a.Ctx if _, ok := a.Plan.(*plannercore.Analyze); ok && sctx.GetSessionVars().InRestrictedSQL { @@ -247,7 +248,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (ast.RecordSet, error) { }, nil } -func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Context, e Executor) (ast.RecordSet, error) { +func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Context, e Executor) (sqlexec.RecordSet, error) { // Check if "tidb_snapshot" is set for the write executors. // In history read mode, we can not do write operations. switch e.(type) { diff --git a/executor/prepared.go b/executor/prepared.go index 5fded52276104..d857979c4b69a 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -240,7 +240,7 @@ func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error { } // CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement. -func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (ast.Statement, error) { +func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (sqlexec.Statement, error) { execStmt := &ast.ExecuteStmt{ExecID: ID} if err := ResetContextOfStmt(ctx, execStmt); err != nil { return nil, err diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 0824ac2018b18..a7df39d6f7683 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" ) @@ -344,7 +345,7 @@ func (tc *TiDBContext) GetSessionVars() *variable.SessionVars { } type tidbResultSet struct { - recordSet ast.RecordSet + recordSet sqlexec.RecordSet columns []*ColumnInfo rows []chunk.Row closed bool diff --git a/session/bench_test.go b/session/bench_test.go index f19dd711e257f..b68a1bdf76823 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -19,10 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/sqlexec" log "github.com/sirupsen/logrus" "golang.org/x/net/context" ) @@ -83,7 +83,7 @@ func prepareJoinBenchData(se Session, colType string, valueFormat string, valueC mustExecute(se, "commit") } -func readResult(ctx context.Context, rs ast.RecordSet, count int) { +func readResult(ctx context.Context, rs sqlexec.RecordSet, count int) { chk := rs.NewChunk() for count > 0 { err := rs.Next(ctx, chk) diff --git a/session/session.go b/session/session.go index 5b29952d90400..a44b941e60682 100644 --- a/session/session.go +++ b/session/session.go @@ -53,6 +53,7 @@ import ( "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-binlog" "github.com/pkg/errors" @@ -63,17 +64,17 @@ import ( // Session context type Session interface { sessionctx.Context - Status() uint16 // Flag of current status, such as autocommit. - LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. - AffectedRows() uint64 // Affected rows by latest executed stmt. - Execute(context.Context, string) ([]ast.RecordSet, error) // Execute a sql statement. - String() string // String is used to debug. + Status() uint16 // Flag of current status, such as autocommit. + LastInsertID() uint64 // LastInsertID is the last inserted auto_increment ID. + AffectedRows() uint64 // Affected rows by latest executed stmt. + Execute(context.Context, string) ([]sqlexec.RecordSet, error) // Execute a sql statement. + String() string // String is used to debug. CommitTxn(context.Context) error RollbackTxn(context.Context) error // PrepareStmt executes prepare statement in binary protocol. PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*ast.ResultField, err error) // ExecutePreparedStmt executes a prepared statement. - ExecutePreparedStmt(ctx context.Context, stmtID uint32, param ...interface{}) (ast.RecordSet, error) + ExecutePreparedStmt(ctx context.Context, stmtID uint32, param ...interface{}) (sqlexec.RecordSet, error) DropPreparedStmt(stmtID uint32) error SetClientCapability(uint32) // Set client capability flags. SetConnectionID(uint64) @@ -97,7 +98,7 @@ var ( type stmtRecord struct { stmtID uint32 - st ast.Statement + st sqlexec.Statement stmtCtx *stmtctx.StatementContext params []interface{} } @@ -108,7 +109,7 @@ type StmtHistory struct { } // Add appends a stmt to history list. -func (h *StmtHistory) Add(stmtID uint32, st ast.Statement, stmtCtx *stmtctx.StatementContext, params ...interface{}) { +func (h *StmtHistory) Add(stmtID uint32, st sqlexec.Statement, stmtCtx *stmtctx.StatementContext, params ...interface{}) { s := &stmtRecord{ stmtID: stmtID, st: st, @@ -613,7 +614,7 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R } } -func drainRecordSet(ctx context.Context, se *session, rs ast.RecordSet) ([]chunk.Row, error) { +func drainRecordSet(ctx context.Context, se *session, rs sqlexec.RecordSet) ([]chunk.Row, error) { var rows []chunk.Row chk := rs.NewChunk() for { @@ -728,7 +729,7 @@ func (s *session) SetProcessInfo(sql string, t time.Time, command byte) { s.processInfo.Store(pi) } -func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode ast.StmtNode, stmt ast.Statement, recordSets []ast.RecordSet) ([]ast.RecordSet, error) { +func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode ast.StmtNode, stmt sqlexec.Statement, recordSets []sqlexec.RecordSet) ([]sqlexec.RecordSet, error) { s.SetValue(sessionctx.QueryString, stmt.OriginText()) if _, ok := stmtNode.(ast.DDLNode); ok { s.SetValue(sessionctx.LastExecuteDDL, true) @@ -757,7 +758,7 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode return recordSets, nil } -func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.RecordSet, err error) { +func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec.RecordSet, err error) { if recordSets, err = s.execute(ctx, sql); err != nil { err = errors.Trace(err) s.sessionVars.StmtCtx.AppendError(err) @@ -765,7 +766,7 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []ast.Rec return } -func (s *session) execute(ctx context.Context, sql string) (recordSets []ast.RecordSet, err error) { +func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec.RecordSet, err error) { s.PrepareTxnCtx(ctx) connID := s.sessionVars.ConnectionID err = s.loadCommonGlobalVariablesIfNeeded() @@ -908,7 +909,7 @@ func checkArgs(args ...interface{}) error { } // ExecutePreparedStmt executes a prepared statement. -func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args ...interface{}) (ast.RecordSet, error) { +func (s *session) ExecutePreparedStmt(ctx context.Context, stmtID uint32, args ...interface{}) (sqlexec.RecordSet, error) { err := checkArgs(args...) if err != nil { return nil, errors.Trace(err) diff --git a/session/tidb.go b/session/tidb.go index 1357c3e753f01..460620da25b50 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" log "github.com/sirupsen/logrus" "golang.org/x/net/context" @@ -134,16 +135,16 @@ func Parse(ctx sessionctx.Context, src string) ([]ast.StmtNode, error) { } // Compile is safe for concurrent use by multiple goroutines. -func Compile(ctx context.Context, sctx sessionctx.Context, stmtNode ast.StmtNode) (ast.Statement, error) { +func Compile(ctx context.Context, sctx sessionctx.Context, stmtNode ast.StmtNode) (sqlexec.Statement, error) { compiler := executor.Compiler{Ctx: sctx} stmt, err := compiler.Compile(ctx, stmtNode) return stmt, errors.Trace(err) } -// runStmt executes the ast.Statement and commit or rollback the current transaction. -func runStmt(ctx context.Context, sctx sessionctx.Context, s ast.Statement) (ast.RecordSet, error) { +// runStmt executes the sqlexec.Statement and commit or rollback the current transaction. +func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (sqlexec.RecordSet, error) { var err error - var rs ast.RecordSet + var rs sqlexec.RecordSet se := sctx.(*session) rs, err = s.Exec(ctx) // All the history should be added here. @@ -194,7 +195,7 @@ func GetHistory(ctx sessionctx.Context) *StmtHistory { } // GetRows4Test gets all the rows from a RecordSet, only used for test. -func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs ast.RecordSet) ([]chunk.Row, error) { +func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs sqlexec.RecordSet) ([]chunk.Row, error) { if rs == nil { return nil, nil } diff --git a/session/tidb_test.go b/session/tidb_test.go index 1821351008d94..10c0ad259dea8 100644 --- a/session/tidb_test.go +++ b/session/tidb_test.go @@ -22,7 +22,6 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" @@ -30,6 +29,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/auth" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testleak" "github.com/pkg/errors" "golang.org/x/net/context" @@ -195,7 +195,7 @@ func removeStore(c *C, dbPath string) { os.RemoveAll(dbPath) } -func exec(se Session, sql string, args ...interface{}) (ast.RecordSet, error) { +func exec(se Session, sql string, args ...interface{}) (sqlexec.RecordSet, error) { ctx := context.Background() if len(args) == 0 { rs, err := se.Execute(ctx, sql) @@ -215,7 +215,7 @@ func exec(se Session, sql string, args ...interface{}) (ast.RecordSet, error) { return rs, nil } -func mustExecSQL(c *C, se Session, sql string, args ...interface{}) ast.RecordSet { +func mustExecSQL(c *C, se Session, sql string, args ...interface{}) sqlexec.RecordSet { rs, err := exec(se, sql, args...) c.Assert(err, IsNil) return rs diff --git a/statistics/ddl.go b/statistics/ddl.go index da1052249028c..022583ca2198e 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -16,7 +16,6 @@ package statistics import ( "fmt" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/mysql" @@ -119,7 +118,7 @@ func (h *Handle) insertColStats2KV(physicalID int64, colInfo *model.ColumnInfo) // If we didn't update anything by last SQL, it means the stats of this table does not exist. if h.mu.ctx.GetSessionVars().StmtCtx.AffectedRows() > 0 { // By this step we can get the count of this table, then we can sure the count and repeats of bucket. - var rs []ast.RecordSet + var rs []sqlexec.RecordSet rs, err = exec.Execute(ctx, fmt.Sprintf("select count from mysql.stats_meta where table_id = %d", physicalID)) if len(rs) > 0 { defer terror.Call(rs[0].Close) diff --git a/statistics/sample.go b/statistics/sample.go index cac1d4229cdaa..9e43f984873a6 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/pkg/errors" "golang.org/x/net/context" @@ -132,7 +133,7 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e // Also, if primary key is handle, it will directly build histogram for it. type SampleBuilder struct { Sc *stmtctx.StatementContext - RecordSet ast.RecordSet + RecordSet sqlexec.RecordSet ColLen int // ColLen is the number of columns need to be sampled. PkBuilder *SortedBuilder MaxBucketSize int64 diff --git a/statistics/sample_test.go b/statistics/sample_test.go index dc3cc3e222303..66d8ceec313fb 100644 --- a/statistics/sample_test.go +++ b/statistics/sample_test.go @@ -17,18 +17,18 @@ import ( "time" . "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/sqlexec" ) var _ = Suite(&testSampleSuite{}) type testSampleSuite struct { count int - rs ast.RecordSet + rs sqlexec.RecordSet } func (s *testSampleSuite) SetUpSuite(c *C) { diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 02c9da7ae279b..6b71236e76164 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pkg/errors" "golang.org/x/net/context" ) @@ -43,8 +44,8 @@ var _ = Suite(&testStatisticsSuite{}) type testStatisticsSuite struct { count int samples []types.Datum - rc ast.RecordSet - pk ast.RecordSet + rc sqlexec.RecordSet + pk sqlexec.RecordSet } type recordSet struct { @@ -169,7 +170,7 @@ func encodeKey(key types.Datum) types.Datum { return types.NewBytesDatum(buf) } -func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSet) (int64, *Histogram, error) { +func buildPK(sctx sessionctx.Context, numBuckets, id int64, records sqlexec.RecordSet) (int64, *Histogram, error) { b := NewSortedBuilder(sctx.GetSessionVars().StmtCtx, numBuckets, id, types.NewFieldType(mysql.TypeLonglong)) ctx := context.Background() for { @@ -193,7 +194,7 @@ func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSe return b.Count, b.hist, nil } -func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSet) (int64, *Histogram, *CMSketch, error) { +func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records sqlexec.RecordSet) (int64, *Histogram, *CMSketch, error) { b := NewSortedBuilder(sctx.GetSessionVars().StmtCtx, numBuckets, id, types.NewFieldType(mysql.TypeBlob)) cms := NewCMSketch(8, 2048) ctx := context.Background() @@ -282,7 +283,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { checkRepeats(c, col) c.Assert(col.Len(), Equals, 250) - tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, ast.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, bucketCount, 1, sqlexec.RecordSet(s.rc)) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -299,7 +300,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { c.Check(int(count), Equals, 0) s.pk.(*recordSet).cursor = 0 - tblCount, col, err = buildPK(ctx, bucketCount, 4, ast.RecordSet(s.pk)) + tblCount, col, err = buildPK(ctx, bucketCount, 4, sqlexec.RecordSet(s.pk)) c.Check(err, IsNil) checkRepeats(c, col) col.PreCalculateScalar() @@ -338,7 +339,7 @@ func (s *testStatisticsSuite) TestBuild(c *C) { func (s *testStatisticsSuite) TestHistogramProtoConversion(c *C) { ctx := mock.NewContext() s.rc.Close() - tblCount, col, _, err := buildIndex(ctx, 256, 1, ast.RecordSet(s.rc)) + tblCount, col, _, err := buildIndex(ctx, 256, 1, sqlexec.RecordSet(s.rc)) c.Check(err, IsNil) c.Check(int(tblCount), Equals, 100000) diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index bbda6c3020b00..9b9f799880acb 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -188,7 +188,7 @@ func (h *rpcHandler) handleAnalyzeColumnsReq(req *coprocessor.Request, analyzeRe return &coprocessor.Response{Data: data}, nil } -// Fields implements the ast.RecordSet Fields interface. +// Fields implements the sqlexec.RecordSet Fields interface. func (e *analyzeColumnsExec) Fields() []*ast.ResultField { return e.fields } @@ -232,7 +232,7 @@ func (e *analyzeColumnsExec) NewChunk() *chunk.Chunk { return chunk.NewChunkWithCapacity(fields, 1) } -// Close implements the ast.RecordSet Close interface. +// Close implements the sqlexec.RecordSet Close interface. func (e *analyzeColumnsExec) Close() error { return nil } diff --git a/util/mock/context.go b/util/mock/context.go index 2eb7631e16769..e33879460288b 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -19,7 +19,6 @@ import ( "sync" "time" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" "github.com/pingcap/tidb/sessionctx" @@ -50,7 +49,7 @@ type Context struct { } // Execute implements sqlexec.SQLExecutor Execute interface. -func (c *Context) Execute(ctx context.Context, sql string) ([]ast.RecordSet, error) { +func (c *Context) Execute(ctx context.Context, sql string) ([]sqlexec.RecordSet, error) { return nil, errors.Errorf("Not Support.") } diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go index e95574ea8c354..fd1d5adb846de 100644 --- a/util/sqlexec/restricted_sql_executor.go +++ b/util/sqlexec/restricted_sql_executor.go @@ -42,7 +42,7 @@ type RestrictedSQLExecutor interface { // For example, privilege/privileges package need execute SQL, if it use // session.Session.Execute, then privilege/privileges and tidb would become a circle. type SQLExecutor interface { - Execute(ctx context.Context, sql string) ([]ast.RecordSet, error) + Execute(ctx context.Context, sql string) ([]RecordSet, error) } // SQLParser is an interface provides parsing sql statement. @@ -52,3 +52,41 @@ type SQLExecutor interface { type SQLParser interface { ParseSQL(sql, charset, collation string) ([]ast.StmtNode, error) } + +// Statement is an interface for SQL execution. +// NOTE: all Statement implementations must be safe for +// concurrent using by multiple goroutines. +// If the Exec method requires any Execution domain local data, +// they must be held out of the implementing instance. +type Statement interface { + // OriginText gets the origin SQL text. + OriginText() string + + // Exec executes SQL and gets a Recordset. + Exec(ctx context.Context) (RecordSet, error) + + // IsPrepared returns whether this statement is prepared statement. + IsPrepared() bool + + // IsReadOnly returns if the statement is read only. For example: SelectStmt without lock. + IsReadOnly() bool + + // RebuildPlan rebuilds the plan of the statement. + RebuildPlan() (schemaVersion int64, err error) +} + +// RecordSet is an abstract result set interface to help get data from Plan. +type RecordSet interface { + // Fields gets result fields. + Fields() []*ast.ResultField + + // Next reads records into chunk. + Next(ctx context.Context, chk *chunk.Chunk) error + + // NewChunk creates a new chunk with initial capacity. + NewChunk() *chunk.Chunk + + // Close closes the underlying iterator, call Next after Close will + // restart the iteration. + Close() error +} diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 08cc9a2d87355..cea25aab3ef75 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -20,9 +20,9 @@ import ( "sync/atomic" "github.com/pingcap/check" - "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tidb/util/testutil" "github.com/pkg/errors" "golang.org/x/net/context" @@ -123,7 +123,7 @@ func NewTestKitWithInit(c *check.C, store kv.Storage) *TestKit { var connectionID uint64 // Exec executes a sql statement. -func (tk *TestKit) Exec(sql string, args ...interface{}) (ast.RecordSet, error) { +func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, error) { var err error if tk.Se == nil { tk.Se, err = session.CreateSession4Test(tk.store) @@ -133,7 +133,7 @@ func (tk *TestKit) Exec(sql string, args ...interface{}) (ast.RecordSet, error) } ctx := context.Background() if len(args) == 0 { - var rss []ast.RecordSet + var rss []sqlexec.RecordSet rss, err = tk.Se.Execute(ctx, sql) if err == nil && len(rss) > 0 { return rss[0], nil @@ -180,9 +180,9 @@ func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { return tk.ResultSetToResult(rs, comment) } -// ResultSetToResult converts ast.RecordSet to testkit.Result. +// ResultSetToResult converts sqlexec.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. -func (tk *TestKit) ResultSetToResult(rs ast.RecordSet, comment check.CommentInterface) *Result { +func (tk *TestKit) ResultSetToResult(rs sqlexec.RecordSet, comment check.CommentInterface) *Result { rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) err = rs.Close() From 04661ba2bbd6e000a97b3815feebdbe03af8593e Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Oct 2018 18:44:09 +0800 Subject: [PATCH 2/3] tiny update --- model/model.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/model/model.go b/model/model.go index 1c050ca73da46..b8a9e1c6e4199 100644 --- a/model/model.go +++ b/model/model.go @@ -103,7 +103,7 @@ func (c *ColumnInfo) SetDefaultValue(value interface{}) error { c.DefaultValueBit = []byte(v) return nil } - return types.ErrInvalidDefault + return types.ErrInvalidDefault.GenWithStackByArgs(c.Name) } return nil } From aeb53f458905576ef4d9466873b383f8a0398fbe Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 23 Oct 2018 15:42:53 +0800 Subject: [PATCH 3/3] address comment --- parser/types/etc.go | 146 ------------------------------------------ types/convert_test.go | 2 +- types/datum.go | 23 +++++++ types/etc.go | 84 ++---------------------- 4 files changed, 28 insertions(+), 227 deletions(-) diff --git a/parser/types/etc.go b/parser/types/etc.go index 9b535e89cbad6..983b81503b5d8 100644 --- a/parser/types/etc.go +++ b/parser/types/etc.go @@ -21,9 +21,7 @@ import ( "strings" "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/terror" - "github.com/pkg/errors" ) // IsTypeBlob returns a boolean indicating whether the tp is a blob type. @@ -42,57 +40,6 @@ func IsTypeChar(tp byte) bool { return tp == mysql.TypeString || tp == mysql.TypeVarchar } -// IsTypeVarchar returns a boolean indicating -// whether the tp is the varchar type like a varstring type or a varchar type. -func IsTypeVarchar(tp byte) bool { - return tp == mysql.TypeVarString || tp == mysql.TypeVarchar -} - -// IsTypeUnspecified returns a boolean indicating whether the tp is the Unspecified type. -func IsTypeUnspecified(tp byte) bool { - return tp == mysql.TypeUnspecified -} - -// IsTypePrefixable returns a boolean indicating -// whether an index on a column with the tp can be defined with a prefix. -func IsTypePrefixable(tp byte) bool { - return IsTypeBlob(tp) || IsTypeChar(tp) -} - -// IsTypeFractionable returns a boolean indicating -// whether the tp can has time fraction. -func IsTypeFractionable(tp byte) bool { - return tp == mysql.TypeDatetime || tp == mysql.TypeDuration || tp == mysql.TypeTimestamp -} - -// IsTypeTime returns a boolean indicating -// whether the tp is time type like datetime, date or timestamp. -func IsTypeTime(tp byte) bool { - return tp == mysql.TypeDatetime || tp == mysql.TypeDate || tp == mysql.TypeTimestamp -} - -// IsTypeNumeric returns a boolean indicating whether the tp is numeric type. -func IsTypeNumeric(tp byte) bool { - switch tp { - case mysql.TypeBit, mysql.TypeTiny, mysql.TypeInt24, mysql.TypeLong, mysql.TypeLonglong, mysql.TypeNewDecimal, - mysql.TypeDecimal, mysql.TypeFloat, mysql.TypeDouble, mysql.TypeShort: - return true - } - return false -} - -// IsTemporalWithDate returns a boolean indicating -// whether the tp is time type with date. -func IsTemporalWithDate(tp byte) bool { - return IsTypeTime(tp) -} - -// IsString returns a boolean indicating -// whether the field type is a string type. -func IsString(tp byte) bool { - return IsTypeChar(tp) || IsTypeBlob(tp) || IsTypeVarchar(tp) || IsTypeUnspecified(tp) -} - var type2Str = map[byte]string{ mysql.TypeBit: "bit", mysql.TypeBlob: "text", @@ -123,38 +70,11 @@ var type2Str = map[byte]string{ mysql.TypeYear: "year", } -var kind2Str = map[byte]string{ - KindNull: "null", - KindInt64: "bigint", - KindUint64: "unsigned bigint", - KindFloat32: "float", - KindFloat64: "double", - KindString: "char", - KindBytes: "bytes", - KindBinaryLiteral: "bit/hex literal", - KindMysqlDecimal: "decimal", - KindMysqlDuration: "time", - KindMysqlEnum: "enum", - KindMysqlBit: "bit", - KindMysqlSet: "set", - KindMysqlTime: "datetime", - KindInterface: "interface", - KindMinNotNull: "min_not_null", - KindMaxValue: "max_value", - KindRaw: "raw", - KindMysqlJSON: "json", -} - // TypeStr converts tp to a string. func TypeStr(tp byte) (r string) { return type2Str[tp] } -// KindStr converts kind to a string. -func KindStr(kind byte) (r string) { - return kind2Str[kind] -} - // TypeToStr converts a field to a string. // It is used for converting Text to Blob, // or converting Char to Binary. @@ -174,80 +94,14 @@ func TypeToStr(tp byte, cs string) (r string) { return ts } -// InvOp2 returns an invalid operation error. -func InvOp2(x, y interface{}, o opcode.Op) (interface{}, error) { - return nil, errors.Errorf("Invalid operation: %v %v %v (mismatched types %T and %T)", x, o, y, x, y) -} - -// IsTypeTemporal checks if a type is a temporal type. -func IsTypeTemporal(tp byte) bool { - switch tp { - case mysql.TypeDuration, mysql.TypeDatetime, mysql.TypeTimestamp, - mysql.TypeDate, mysql.TypeNewDate: - return true - } - return false -} - -// Kind constants. -const ( - KindNull byte = 0 - KindInt64 byte = 1 - KindUint64 byte = 2 - KindFloat32 byte = 3 - KindFloat64 byte = 4 - KindString byte = 5 - KindBytes byte = 6 - KindBinaryLiteral byte = 7 // Used for BIT / HEX literals. - KindMysqlDecimal byte = 8 - KindMysqlDuration byte = 9 - KindMysqlEnum byte = 10 - KindMysqlBit byte = 11 // Used for BIT table column values. - KindMysqlSet byte = 12 - KindMysqlTime byte = 13 - KindInterface byte = 14 - KindMinNotNull byte = 15 - KindMaxValue byte = 16 - KindRaw byte = 17 - KindMysqlJSON byte = 18 -) - var ( dig2bytes = [10]int{0, 1, 1, 2, 2, 3, 3, 4, 4, 4} ) -// RoundMode is the type for round mode. -type RoundMode string - // constant values. const ( - ten0 = 1 - ten1 = 10 - ten2 = 100 - ten3 = 1000 - ten4 = 10000 - ten5 = 100000 - ten6 = 1000000 - ten7 = 10000000 - ten8 = 100000000 - ten9 = 1000000000 - - maxWordBufLen = 9 // A MyDecimal holds 9 words. digitsPerWord = 9 // A word holds 9 digits. wordSize = 4 // A word is 4 bytes int32. - digMask = ten8 - wordBase = ten9 - wordMax = wordBase - 1 - notFixedDec = 31 - - DivFracIncr = 4 - - // ModeHalfEven rounds normally. - ModeHalfEven RoundMode = "ModeHalfEven" - // Truncate just truncates the decimal. - ModeTruncate RoundMode = "Truncate" - // Ceiling is not supported now. - modeCeiling RoundMode = "Ceiling" ) const ( diff --git a/types/convert_test.go b/types/convert_test.go index f74431f2ef843..35cb6d45df524 100644 --- a/types/convert_test.go +++ b/types/convert_test.go @@ -464,7 +464,7 @@ func (s *testTypeConvertSuite) TestStrToNum(c *C) { func (s *testTypeConvertSuite) TestFieldTypeToStr(c *C) { defer testleak.AfterTest(c)() v := TypeToStr(mysql.TypeUnspecified, "not binary") - c.Assert(v, Equals, type2Str[mysql.TypeUnspecified]) + c.Assert(v, Equals, TypeStr(mysql.TypeUnspecified)) v = TypeToStr(mysql.TypeBlob, charset.CharsetBin) c.Assert(v, Equals, "blob") v = TypeToStr(mysql.TypeString, charset.CharsetBin) diff --git a/types/datum.go b/types/datum.go index c683e2e70a88d..959231e633b0f 100644 --- a/types/datum.go +++ b/types/datum.go @@ -32,6 +32,29 @@ import ( log "github.com/sirupsen/logrus" ) +// Kind constants. +const ( + KindNull byte = 0 + KindInt64 byte = 1 + KindUint64 byte = 2 + KindFloat32 byte = 3 + KindFloat64 byte = 4 + KindString byte = 5 + KindBytes byte = 6 + KindBinaryLiteral byte = 7 // Used for BIT / HEX literals. + KindMysqlDecimal byte = 8 + KindMysqlDuration byte = 9 + KindMysqlEnum byte = 10 + KindMysqlBit byte = 11 // Used for BIT table column values. + KindMysqlSet byte = 12 + KindMysqlTime byte = 13 + KindInterface byte = 14 + KindMinNotNull byte = 15 + KindMaxValue byte = 16 + KindRaw byte = 17 + KindMysqlJSON byte = 18 +) + // Datum is a data box holds different kind of data. // It has better performance and is easier to use than `interface{}`. type Datum struct { diff --git a/types/etc.go b/types/etc.go index 0629520b4847e..9454b3cb3dcbc 100644 --- a/types/etc.go +++ b/types/etc.go @@ -19,7 +19,6 @@ package types import ( "io" - "strings" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/parser/opcode" @@ -29,44 +28,12 @@ import ( "github.com/pkg/errors" ) -// Kind constants. -const ( - KindNull = ast.KindNull - KindInt64 = ast.KindInt64 - KindUint64 = ast.KindUint64 - KindFloat32 = ast.KindFloat32 - KindFloat64 = ast.KindFloat64 - KindString = ast.KindString - KindBytes = ast.KindBytes - KindBinaryLiteral = ast.KindBinaryLiteral - KindMysqlDecimal = ast.KindMysqlDecimal - KindMysqlDuration = ast.KindMysqlDuration - KindMysqlEnum = ast.KindMysqlEnum - KindMysqlBit = ast.KindMysqlBit - KindMysqlSet = ast.KindMysqlSet - KindMysqlTime = ast.KindMysqlTime - KindInterface = ast.KindInterface - KindMinNotNull = ast.KindMinNotNull - KindMaxValue = ast.KindMaxValue - KindRaw = ast.KindRaw - KindMysqlJSON = ast.KindMysqlJSON -) - // IsTypeBlob returns a boolean indicating whether the tp is a blob type. -func IsTypeBlob(tp byte) bool { - switch tp { - case mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeBlob, mysql.TypeLongBlob: - return true - default: - return false - } -} +var IsTypeBlob = ast.IsTypeBlob // IsTypeChar returns a boolean indicating // whether the tp is the char type like a string type or a varchar type. -func IsTypeChar(tp byte) bool { - return tp == mysql.TypeString || tp == mysql.TypeVarchar -} +var IsTypeChar = ast.IsTypeChar // IsTypeVarchar returns a boolean indicating // whether the tp is the varchar type like a varstring type or a varchar type. @@ -137,36 +104,6 @@ func IsString(tp byte) bool { return IsTypeChar(tp) || IsTypeBlob(tp) || IsTypeVarchar(tp) || IsTypeUnspecified(tp) } -var type2Str = map[byte]string{ - mysql.TypeBit: "bit", - mysql.TypeBlob: "text", - mysql.TypeDate: "date", - mysql.TypeDatetime: "datetime", - mysql.TypeDecimal: "unspecified", - mysql.TypeNewDecimal: "decimal", - mysql.TypeDouble: "double", - mysql.TypeEnum: "enum", - mysql.TypeFloat: "float", - mysql.TypeGeometry: "geometry", - mysql.TypeInt24: "mediumint", - mysql.TypeJSON: "json", - mysql.TypeLong: "int", - mysql.TypeLonglong: "bigint", - mysql.TypeLongBlob: "longtext", - mysql.TypeMediumBlob: "mediumtext", - mysql.TypeNull: "null", - mysql.TypeSet: "set", - mysql.TypeShort: "smallint", - mysql.TypeString: "char", - mysql.TypeDuration: "time", - mysql.TypeTimestamp: "timestamp", - mysql.TypeTiny: "tinyint", - mysql.TypeTinyBlob: "tinytext", - mysql.TypeVarchar: "varchar", - mysql.TypeVarString: "var_string", - mysql.TypeYear: "year", -} - var kind2Str = map[byte]string{ KindNull: "null", KindInt64: "bigint", @@ -190,9 +127,7 @@ var kind2Str = map[byte]string{ } // TypeStr converts tp to a string. -func TypeStr(tp byte) (r string) { - return type2Str[tp] -} +var TypeStr = ast.TypeStr // KindStr converts kind to a string. func KindStr(kind byte) (r string) { @@ -205,18 +140,7 @@ func KindStr(kind byte) (r string) { // Args: // tp: type enum // cs: charset -func TypeToStr(tp byte, cs string) (r string) { - ts := type2Str[tp] - if cs != charset.CharsetBin { - return ts - } - if IsTypeBlob(tp) { - ts = strings.Replace(ts, "text", "blob", 1) - } else if IsTypeChar(tp) { - ts = strings.Replace(ts, "char", "binary", 1) - } - return ts -} +var TypeToStr = ast.TypeToStr // EOFAsNil filtrates errors, // If err is equal to io.EOF returns nil.