diff --git a/ast/ast.go b/ast/ast.go index dab6c3ac5d4fc..f61e9519b473c 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -137,8 +137,8 @@ type RecordSet interface { // Fields gets result fields. Fields() []*ResultField - // NextChunk reads records into chunk. - NextChunk(ctx context.Context, chk *chunk.Chunk) error + // Next reads records into chunk. + Next(ctx context.Context, chk *chunk.Chunk) error // NewChunk creates a new chunk with initial capacity. NewChunk() *chunk.Chunk diff --git a/cmd/benchdb/main.go b/cmd/benchdb/main.go index 67305fdeccb51..e6a91936b9869 100644 --- a/cmd/benchdb/main.go +++ b/cmd/benchdb/main.go @@ -119,7 +119,7 @@ func (ut *benchDB) mustExec(sql string) { rs := rss[0] chk := rs.NewChunk() for { - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { log.Fatal(err) } diff --git a/ddl/util/util.go b/ddl/util/util.go index eff6b8769431e..3761356bceb95 100644 --- a/ddl/util/util.go +++ b/ddl/util/util.go @@ -58,7 +58,7 @@ func LoadDeleteRanges(ctx sessionctx.Context, safePoint uint64) (ranges []DelRan chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) if err != nil { return nil, errors.Trace(err) } diff --git a/distsql/distsql.go b/distsql/distsql.go index 4fca9ba2bd854..a33b76532cb47 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -42,12 +42,10 @@ var ( // SelectResult is an iterator of coprocessor partial results. type SelectResult interface { - // Next gets the next partial result. - Next(context.Context) (PartialResult, error) // NextRaw gets the next raw result. NextRaw(context.Context) ([]byte, error) - // NextChunk reads the data into chunk. - NextChunk(context.Context, *chunk.Chunk) error + // Next reads the data into chunk. + Next(context.Context, *chunk.Chunk) error // Close closes the iterator. Close() error // Fetch fetches partial results from client. @@ -121,23 +119,6 @@ func (r *selectResult) fetch(ctx context.Context) { } } -// Next returns the next row. -func (r *selectResult) Next(ctx context.Context) (PartialResult, error) { - re := <-r.results - if re.err != nil { - return nil, errors.Trace(re.err) - } - if re.result == nil { - return nil, nil - } - pr := &partialResult{} - pr.rowLen = r.rowLen - err := pr.unmarshal(re.result.GetData()) - r.feedback.Update(re.result.GetStartKey(), pr.resp.OutputCounts) - r.partialCount++ - return pr, errors.Trace(err) -} - // NextRaw returns the next raw partial result. func (r *selectResult) NextRaw(ctx context.Context) ([]byte, error) { re := <-r.results @@ -149,8 +130,8 @@ func (r *selectResult) NextRaw(ctx context.Context) ([]byte, error) { return re.result.GetData(), nil } -// NextChunk reads data to the chunk. -func (r *selectResult) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next reads data to the chunk. +func (r *selectResult) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for chk.NumRows() < r.ctx.GetSessionVars().MaxChunkSize { if r.selectResp == nil || r.respChkIdx == len(r.selectResp.Chunks) { diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index 3bdd8ed723d6a..27c1fbbb411ac 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -54,7 +54,7 @@ func (s *testSuite) TestSelectNormal(c *C) { colTypes = append(colTypes, colTypes[0]) colTypes = append(colTypes, colTypes[0]) - // Test NextChunk. + // Test Next. response, err := Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) c.Assert(err, IsNil) result, ok := response.(*selectResult) @@ -64,11 +64,11 @@ func (s *testSuite) TestSelectNormal(c *C) { response.Fetch(context.TODO()) - // Test NextChunk. + // Test Next. chk := chunk.NewChunk(colTypes) numAllRows := 0 for { - err = response.NextChunk(context.TODO(), chk) + err = response.Next(context.TODO(), chk) c.Assert(err, IsNil) numAllRows += chk.NumRows() if chk.NumRows() == 0 { @@ -108,7 +108,7 @@ func (s *testSuite) TestSelectStreaming(c *C) { s.sctx.GetSessionVars().EnableStreaming = true - // Test NextChunk. + // Test Next. response, err := Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) c.Assert(err, IsNil) result, ok := response.(*streamResult) @@ -117,11 +117,11 @@ func (s *testSuite) TestSelectStreaming(c *C) { response.Fetch(context.TODO()) - // Test NextChunk. + // Test Next. chk := chunk.NewChunk(colTypes) numAllRows := 0 for { - err = response.NextChunk(context.TODO(), chk) + err = response.Next(context.TODO(), chk) c.Assert(err, IsNil) numAllRows += chk.NumRows() if chk.NumRows() == 0 { diff --git a/distsql/stream.go b/distsql/stream.go index bdbf241efb130..e14edb4c4a29c 100644 --- a/distsql/stream.go +++ b/distsql/stream.go @@ -41,17 +41,7 @@ type streamResult struct { func (r *streamResult) Fetch(context.Context) {} -func (r *streamResult) Next(ctx context.Context) (PartialResult, error) { - var ret streamPartialResult - ret.rowLen = r.rowLen - finished, err := r.readDataFromResponse(ctx, r.resp, &ret.Chunk) - if err != nil || finished { - return nil, errors.Trace(err) - } - return &ret, nil -} - -func (r *streamResult) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (r *streamResult) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() maxChunkSize := r.ctx.GetSessionVars().MaxChunkSize for chk.NumRows() < maxChunkSize { diff --git a/executor/adapter.go b/executor/adapter.go index 8186c4db59326..63388b0660c70 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -79,13 +79,13 @@ func schema2ResultFields(schema *expression.Schema, defaultDB string) (rfs []*as return rfs } -// NextChunk use uses recordSet's executor to get next available chunk for later usage. +// Next use uses recordSet's executor to get next available chunk for later usage. // If chunk does not contain any rows, then we update last query found rows in session variable as current found rows. // The reason we need update is that chunk with 0 rows indicating we already finished current query, we need prepare for // next query. // If stmt is not nil and chunk with some rows inside, we simply update last query found rows by the number of row in chunk. -func (a *recordSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - err := a.executor.NextChunk(ctx, chk) +func (a *recordSet) Next(ctx context.Context, chk *chunk.Chunk) error { + err := a.executor.Next(ctx, chk) if err != nil { a.lastErr = err return errors.Trace(err) @@ -262,7 +262,7 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co a.logSlowQuery(txnTS, err == nil) }() - err = e.NextChunk(ctx, e.newChunk()) + err = e.Next(ctx, e.newChunk()) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/admin.go b/executor/admin.go index 7d6e0953289b8..5e3243f7182ec 100644 --- a/executor/admin.go +++ b/executor/admin.go @@ -58,12 +58,12 @@ type CheckIndexRangeExec struct { cols []*model.ColumnInfo } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckIndexRangeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckIndexRangeExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() handleIdx := e.schema.Len() - 1 for { - err := e.result.NextChunk(ctx, e.srcChunk) + err := e.result.Next(ctx, e.srcChunk) if err != nil { return errors.Trace(err) } @@ -318,7 +318,7 @@ func (e *RecoverIndexExec) fetchRecoverRows(ctx context.Context, srcResult dists result.scanRowCount = 0 for { - err := srcResult.NextChunk(ctx, e.srcChunk) + err := srcResult.Next(ctx, e.srcChunk) if err != nil { return nil, errors.Trace(err) } @@ -428,8 +428,8 @@ func (e *RecoverIndexExec) backfillIndexInTxn(ctx context.Context, txn kv.Transa return result, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *RecoverIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *RecoverIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil @@ -537,7 +537,7 @@ func (e *CleanupIndexExec) fetchIndex(ctx context.Context, txn kv.Transaction) e sc := e.ctx.GetSessionVars().StmtCtx for { - err := result.NextChunk(ctx, e.idxChunk) + err := result.Next(ctx, e.idxChunk) if err != nil { return errors.Trace(err) } @@ -563,8 +563,8 @@ func (e *CleanupIndexExec) fetchIndex(ctx context.Context, txn kv.Transaction) e } } -// NextChunk implements the Executor NextChunk interface. -func (e *CleanupIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CleanupIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil diff --git a/executor/aggregate.go b/executor/aggregate.go index 6cb7a10e7b086..ab94b2f138d98 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -74,8 +74,8 @@ func (e *HashAggExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *HashAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *HashAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { // In this stage we consider all data from src as a single group. if !e.executed { err := e.execute(ctx) @@ -110,11 +110,11 @@ func (e *HashAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } } -// innerNextChunk fetches Chunks from src and update each aggregate function for each row in Chunk. +// execute fetches Chunks from src and update each aggregate function for each row in Chunk. func (e *HashAggExec) execute(ctx context.Context) (err error) { inputIter := chunk.NewIterator4Chunk(e.childrenResults[0]) for { - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -217,8 +217,8 @@ func (e *StreamAggExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *StreamAggExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *StreamAggExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for !e.executed && chk.NumRows() < e.maxChunkSize { @@ -264,7 +264,7 @@ func (e *StreamAggExec) fetchChildIfNecessary(ctx context.Context, chk *chunk.Ch return nil } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } diff --git a/executor/analyze.go b/executor/analyze.go index acfaba16b5e0a..c7d50fa5b8f7d 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -52,8 +52,8 @@ const ( defaultCMSketchWidth = 2048 ) -// NextChunk implements the Executor NextChunk interface. -func (e *AnalyzeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *AnalyzeExec) Next(ctx context.Context, chk *chunk.Chunk) error { concurrency, err := getBuildStatsConcurrency(e.ctx) if err != nil { return errors.Trace(err) diff --git a/executor/checksum.go b/executor/checksum.go index e3b21953c6c0b..0beeaa89ef7ad 100644 --- a/executor/checksum.go +++ b/executor/checksum.go @@ -82,8 +82,8 @@ func (e *ChecksumTableExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ChecksumTableExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ChecksumTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil diff --git a/executor/ddl.go b/executor/ddl.go index bcf2f77a30cb4..29830223316d6 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -38,8 +38,8 @@ type DDLExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *DDLExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor Next interface. +func (e *DDLExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if e.done { return nil } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 265034802b25e..11597c2a36c8c 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -68,7 +68,7 @@ func (s *testSuite) TestCreateTable(c *C) { chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err1 := rs.NextChunk(ctx, chk) + err1 := rs.Next(ctx, chk) c.Assert(err1, IsNil) if chk.NumRows() == 0 { break @@ -82,7 +82,7 @@ func (s *testSuite) TestCreateTable(c *C) { chk = rs.NewChunk() it = chunk.NewIterator4Chunk(chk) for { - err1 := rs.NextChunk(ctx, chk) + err1 := rs.Next(ctx, chk) c.Assert(err1, IsNil) if chk.NumRows() == 0 { break @@ -156,7 +156,7 @@ func (s *testSuite) TestAlterTableAddColumn(c *C) { r, err := tk.Exec("select c2 from alter_test") c.Assert(err, IsNil) chk := r.NewChunk() - err = r.NextChunk(context.Background(), chk) + err = r.Next(context.Background(), chk) c.Assert(err, IsNil) row := chk.GetRow(0) c.Assert(row.Len(), Equals, 1) diff --git a/executor/distsql.go b/executor/distsql.go index 81e688295b04a..465e6746226a3 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -78,7 +78,7 @@ type lookupTableTask struct { // 4. task.memTracker.Consume(-task.memUsage) // // Step 1~3 are completed in "tableWorker.executeTask". - // Step 4 is completed in "IndexLookUpExecutor.NextChunk". + // Step 4 is completed in "IndexLookUpExecutor.Next". memUsage int64 memTracker *memory.Tracker } @@ -219,9 +219,9 @@ func (e *TableReaderExecutor) Close() error { return errors.Trace(err) } -// NextChunk fills data into the chunk passed by its caller. +// Next fills data into the chunk passed by its caller. // The task was actually done by tableReaderHandler. -func (e *TableReaderExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *TableReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { err := e.resultHandler.nextChunk(ctx, chk) if err != nil { e.feedback.Invalidate() @@ -357,9 +357,9 @@ func (e *IndexReaderExecutor) Close() error { return errors.Trace(err) } -// NextChunk implements the Executor NextChunk interface. -func (e *IndexReaderExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - err := e.result.NextChunk(ctx, chk) +// Next implements the Executor Next interface. +func (e *IndexReaderExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { + err := e.result.Next(ctx, chk) if err != nil { e.feedback.Invalidate() } @@ -588,8 +588,8 @@ func (e *IndexLookUpExecutor) Close() error { return nil } -// NextChunk implements Exec NextChunk interface. -func (e *IndexLookUpExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements Exec Next interface. +func (e *IndexLookUpExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() for { resultTask, err := e.getResultTask() @@ -691,7 +691,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes func (w *indexWorker) extractTaskHandles(ctx context.Context, chk *chunk.Chunk, idxResult distsql.SelectResult) (handles []int64, err error) { handles = make([]int64, 0, w.batchSize) for len(handles) < w.batchSize { - err = errors.Trace(idxResult.NextChunk(ctx, chk)) + err = errors.Trace(idxResult.Next(ctx, chk)) if err != nil { return handles, err } @@ -789,7 +789,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er task.rows = make([]chunk.Row, 0, handleCnt) for { chk := tableReader.newChunk() - err = tableReader.NextChunk(ctx, chk) + err = tableReader.Next(ctx, chk) if err != nil { log.Error(err) return errors.Trace(err) @@ -880,7 +880,7 @@ func (tr *tableResultHandler) open(optionalResult, result distsql.SelectResult) func (tr *tableResultHandler) nextChunk(ctx context.Context, chk *chunk.Chunk) error { if !tr.optionalFinished { - err := tr.optionalResult.NextChunk(ctx, chk) + err := tr.optionalResult.Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -889,7 +889,7 @@ func (tr *tableResultHandler) nextChunk(ctx context.Context, chk *chunk.Chunk) e } tr.optionalFinished = true } - return tr.result.NextChunk(ctx, chk) + return tr.result.Next(ctx, chk) } func (tr *tableResultHandler) nextRaw(ctx context.Context) (data []byte, err error) { diff --git a/executor/distsql_test.go b/executor/distsql_test.go index e0dcd23d6ddc3..988c627741d75 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -55,7 +55,7 @@ func (s *testSuite) TestIndexDoubleReadClose(c *C) { rs, err := tk.Exec("select * from dist where c_idx between 0 and 100") c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(err, IsNil) keyword := "pickAndExecTask" @@ -106,7 +106,7 @@ func (s *testSuite) TestCopClientSend(c *C) { c.Assert(err, IsNil) defer rs.Close() chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetMyDecimal(0).String(), Equals, "499500") @@ -120,7 +120,7 @@ func (s *testSuite) TestCopClientSend(c *C) { rs, err = tk.Exec("select sum(id) from copclient") c.Assert(err, IsNil) chk = rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetMyDecimal(0).String(), Equals, "499500") rs.Close() @@ -129,7 +129,7 @@ func (s *testSuite) TestCopClientSend(c *C) { rs, err = tk.Exec("select * from copclient order by id") c.Assert(err, IsNil) chk = rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) rs.Close() keyword := "(*copIterator).work" diff --git a/executor/executor.go b/executor/executor.go index 65f7bc068b6bc..cbc56398b0fc2 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -157,8 +157,8 @@ func (e *baseExecutor) retTypes() []*types.FieldType { return e.retFieldTypes } -// NextChunk fills mutiple rows into a chunk. -func (e *baseExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next fills mutiple rows into a chunk. +func (e *baseExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } @@ -187,9 +187,9 @@ type Executor interface { Schema() *expression.Schema retTypes() []*types.FieldType newChunk() *chunk.Chunk - // NextChunk fills a chunk with multiple rows + // Next fills a chunk with multiple rows // NOTE: chunk has to call Reset() method before any use. - NextChunk(ctx context.Context, chk *chunk.Chunk) error + Next(ctx context.Context, chk *chunk.Chunk) error } // CancelDDLJobsExec represents a cancel DDL jobs executor. @@ -201,8 +201,8 @@ type CancelDDLJobsExec struct { errs []error } -// NextChunk implements the Executor NextChunk interface. -func (e *CancelDDLJobsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CancelDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobIDs) { return nil @@ -230,8 +230,8 @@ type ShowDDLExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil @@ -289,8 +289,8 @@ func (e *ShowDDLJobQueriesExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLJobQueriesExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLJobQueriesExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobs) { return nil @@ -330,8 +330,8 @@ func (e *ShowDDLJobsExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowDDLJobsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowDDLJobsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.jobs) { return nil @@ -365,8 +365,8 @@ func (e *CheckTableExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckTableExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckTableExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } @@ -419,8 +419,8 @@ func (e *CheckIndexExec) Close() error { return errors.Trace(e.src.Close()) } -// NextChunk implements the Executor NextChunk interface. -func (e *CheckIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *CheckIndexExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } @@ -432,7 +432,7 @@ func (e *CheckIndexExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error } chk = e.src.newChunk() for { - err := e.src.NextChunk(ctx, chk) + err := e.src.Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -470,10 +470,10 @@ func (e *SelectLockExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *SelectLockExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SelectLockExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -512,14 +512,14 @@ type LimitExec struct { meetFirstBatch bool } -// NextChunk implements the Executor NextChunk interface. -func (e *LimitExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LimitExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= e.end { return nil } for !e.meetFirstBatch { - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -540,7 +540,7 @@ func (e *LimitExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } e.cursor += batchSize } - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -589,7 +589,7 @@ func init() { } for { chk := exec.newChunk() - err = exec.NextChunk(ctx, chk) + err = exec.Next(ctx, chk) if err != nil { return rows, errors.Trace(err) } @@ -628,10 +628,10 @@ func (e *ProjectionExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ProjectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ProjectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() - if err := e.children[0].NextChunk(ctx, e.childrenResults[0]); err != nil { + if err := e.children[0].Next(ctx, e.childrenResults[0]); err != nil { return errors.Trace(err) } return errors.Trace(e.evaluatorSuit.Run(e.ctx, e.childrenResults[0], chk)) @@ -652,8 +652,8 @@ func (e *TableDualExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *TableDualExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TableDualExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.numReturned >= e.numDualRows { return nil @@ -703,12 +703,12 @@ func (e *SelectionExec) Close() error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SelectionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.batched { - return errors.Trace(e.unBatchedNextChunk(ctx, chk)) + return errors.Trace(e.unBatchedNext(ctx, chk)) } for { @@ -721,7 +721,7 @@ func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } chk.AppendRow(e.inputRow) } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -737,10 +737,10 @@ func (e *SelectionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { } } -// unBatchedNextChunk filters input rows one by one and returns once an input row is selected. +// unBatchedNext filters input rows one by one and returns once an input row is selected. // For sql with "SETVAR" in filter and "GETVAR" in projection, for example: "SELECT @a FROM t WHERE (@a := 2) > 0", // we have to set batch size to 1 to do the evaluation of filter and projection. -func (e *SelectionExec) unBatchedNextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *SelectionExec) unBatchedNext(ctx context.Context, chk *chunk.Chunk) error { for { for ; e.inputRow != e.inputIter.End(); e.inputRow = e.inputIter.Next() { selected, err := expression.EvalBool(e.ctx, e.filters, e.inputRow) @@ -753,7 +753,7 @@ func (e *SelectionExec) unBatchedNextChunk(ctx context.Context, chk *chunk.Chunk return nil } } - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -780,8 +780,8 @@ type TableScanExec struct { virtualTableChunkIdx int } -// NextChunk implements the Executor NextChunk interface. -func (e *TableScanExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TableScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.isVirtualTable { return errors.Trace(e.nextChunk4InfoSchema(ctx, chk)) @@ -921,12 +921,12 @@ func (e *ExistsExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ExistsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExistsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.evaluated { e.evaluated = true - err := e.children[0].NextChunk(ctx, e.childrenResults[0]) + err := e.children[0].Next(ctx, e.childrenResults[0]) if err != nil { return errors.Trace(err) } @@ -956,14 +956,14 @@ func (e *MaxOneRowExec) Open(ctx context.Context) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *MaxOneRowExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *MaxOneRowExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.evaluated { return nil } e.evaluated = true - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -1075,7 +1075,7 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { return case result.chk = <-e.resourcePools[childID]: } - result.err = errors.Trace(e.children[childID].NextChunk(ctx, result.chk)) + result.err = errors.Trace(e.children[childID].Next(ctx, result.chk)) if result.err == nil && result.chk.NumRows() == 0 { return } @@ -1087,8 +1087,8 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) { } } -// NextChunk implements the Executor NextChunk interface. -func (e *UnionExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *UnionExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.initialized { e.initialize(ctx) diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index b8fb231e45627..95f547ae68039 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -89,13 +89,13 @@ func (s *testExecSuite) TestShowProcessList(c *C) { it := chunk.NewIterator4Chunk(chk) // Run test and check results. for _, p := range ps { - err = e.NextChunk(context.Background(), chk) + err = e.Next(context.Background(), chk) c.Assert(err, IsNil) for row := it.Begin(); row != it.End(); row = it.Next() { c.Assert(row.GetUint64(0), Equals, p.ID) } } - err = e.NextChunk(context.Background(), chk) + err = e.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 0) err = e.Close() diff --git a/executor/executor_test.go b/executor/executor_test.go index b3af7cb54bba8..c801d2267e6d4 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -138,7 +138,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err := tk.Exec("admin cancel ddl jobs 1") c.Assert(err, IsNil, Commentf("err %v", err)) chk := r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row := chk.GetRow(0) c.Assert(row.Len(), Equals, 2) @@ -148,7 +148,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err = tk.Exec("admin show ddl") c.Assert(err, IsNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row = chk.GetRow(0) c.Assert(row.Len(), Equals, 4) @@ -163,7 +163,7 @@ func (s *testSuite) TestAdmin(c *C) { // c.Assert(rowOwnerInfos[0], Equals, ownerInfos[0]) c.Assert(row.GetString(2), Equals, "") chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsTrue) err = txn.Rollback() @@ -173,7 +173,7 @@ func (s *testSuite) TestAdmin(c *C) { r, err = tk.Exec("admin show ddl jobs") c.Assert(err, IsNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) row = chk.GetRow(0) c.Assert(row.Len(), Equals, 2) @@ -756,7 +756,7 @@ func (s *testSuite) TestIssue2612(c *C) { rs, err := tk.Exec(`select timediff(finish_at, create_at) from t;`) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetDuration(0).String(), Equals, "-46:09:02") } @@ -2310,7 +2310,7 @@ func (s *testSuite) TestBit(c *C) { r, err := tk.Exec("select * from t where c1 = 2") c.Assert(err, IsNil) chk := r.NewChunk() - err = r.NextChunk(context.Background(), chk) + err = r.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(types.BinaryLiteral(chk.GetRow(0).GetBytes(0)), DeepEquals, types.NewBinaryLiteralFromUint(2, -1)) @@ -2475,7 +2475,7 @@ func (s *testSuite) TestEarlyClose(c *C) { c.Assert(err1, IsNil) rs := rss[0] chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) rs.Close() } @@ -2487,7 +2487,7 @@ func (s *testSuite) TestEarlyClose(c *C) { c.Assert(err, IsNil) rs := rss[0] chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, NotNil) rs.Close() } @@ -2653,7 +2653,7 @@ func (s *testSuite) TestCoprocessorStreamingFlag(c *C) { rs, err := tk.Se.Execute(ctx1, test.sql) c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) rs[0].Close() } diff --git a/executor/explain.go b/executor/explain.go index f7af00687fe49..781f026c736af 100644 --- a/executor/explain.go +++ b/executor/explain.go @@ -33,8 +33,8 @@ func (e *ExplainExec) Close() error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ExplainExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExplainExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.cursor >= len(e.rows) { return nil diff --git a/executor/grant.go b/executor/grant.go index d9522f6566250..5d82141cc8cea 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -52,8 +52,8 @@ type GrantExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *GrantExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *GrantExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 7fc096e18f836..409ac572fd0cf 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -186,8 +186,8 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork return iw } -// NextChunk implements the Executor interface. -func (e *IndexLookUpJoin) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor interface. +func (e *IndexLookUpJoin) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() e.joinResult.Reset() for { @@ -328,7 +328,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { task.memTracker.Consume(task.outerResult.MemoryUsage()) for task.outerResult.NumRows() < ow.batchSize { - err := ow.executor.NextChunk(ctx, ow.executorChk) + err := ow.executor.Next(ctx, ow.executorChk) if err != nil { return task, errors.Trace(err) } @@ -514,7 +514,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa innerResult.GetMemTracker().SetLabel("inner result") innerResult.GetMemTracker().AttachTo(task.memTracker) for { - err := innerExec.NextChunk(ctx, iw.executorChk) + err := innerExec.Next(ctx, iw.executorChk) if err != nil { return errors.Trace(err) } diff --git a/executor/join.go b/executor/join.go index 8d91194f7ec67..3f2979350221d 100644 --- a/executor/join.go +++ b/executor/join.go @@ -211,7 +211,7 @@ func (e *HashJoinExec) fetchOuterChunks(ctx context.Context) { } } outerResult := outerResource.chk - err := e.outerExec.NextChunk(ctx, outerResult) + err := e.outerExec.Next(ctx, outerResult) if err != nil { e.joinResultCh <- &hashjoinWorkerResult{ err: errors.Trace(err), @@ -233,7 +233,7 @@ func (e *HashJoinExec) fetchInnerRows(ctx context.Context) (err error) { e.innerResult.GetMemTracker().SetLabel("innerResult") for { chk := e.children[e.innerIdx].newChunk() - err = e.innerExec.NextChunk(ctx, chk) + err = e.innerExec.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return errors.Trace(err) } @@ -441,11 +441,11 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu return true, joinResult } -// NextChunk implements the Executor NextChunk interface. +// Next implements the Executor Next interface. // hash join constructs the result following these steps: // step 1. fetch data from inner child and build a hash table; // step 2. fetch data from outer child in a background goroutine and probe the hash table in multiple join workers. -func (e *HashJoinExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +func (e *HashJoinExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if !e.prepared { if err = e.fetchInnerRows(ctx); err != nil { return errors.Trace(err) @@ -571,7 +571,7 @@ func (e *NestedLoopApplyExec) fetchSelectedOuterRow(ctx context.Context, chk *ch outerIter := chunk.NewIterator4Chunk(e.outerChunk) for { if e.outerChunkCursor >= e.outerChunk.NumRows() { - err := e.outerExec.NextChunk(ctx, e.outerChunk) + err := e.outerExec.Next(ctx, e.outerChunk) if err != nil { return nil, errors.Trace(err) } @@ -608,7 +608,7 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error { e.innerList.Reset() innerIter := chunk.NewIterator4Chunk(e.innerChunk) for { - err := e.innerExec.NextChunk(ctx, e.innerChunk) + err := e.innerExec.Next(ctx, e.innerChunk) if err != nil { return errors.Trace(err) } @@ -628,8 +628,8 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error { } } -// NextChunk implements the Executor interface. -func (e *NestedLoopApplyExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor interface. +func (e *NestedLoopApplyExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { chk.Reset() for { if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() { diff --git a/executor/join_test.go b/executor/join_test.go index 4f98122176b2c..e036b81bbbd46 100644 --- a/executor/join_test.go +++ b/executor/join_test.go @@ -742,7 +742,7 @@ func (s *testSuite) TestJoinLeak(c *C) { result, err := tk.Exec("select * from t t1 left join (select 1) t2 on 1") c.Assert(err, IsNil) chk := result.NewChunk() - err = result.NextChunk(context.Background(), chk) + err = result.Next(context.Background(), chk) c.Assert(err, IsNil) time.Sleep(100 * time.Millisecond) result.Close() diff --git a/executor/load_stats.go b/executor/load_stats.go index 6f524081ddba8..2e20a074206ed 100644 --- a/executor/load_stats.go +++ b/executor/load_stats.go @@ -50,8 +50,8 @@ func (k loadStatsVarKeyType) String() string { // LoadStatsVarKey is a variable key for load statistic. const LoadStatsVarKey loadStatsVarKeyType = 0 -// NextChunk implements the Executor NextChunk interface. -func (e *LoadStatsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LoadStatsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if len(e.info.Path) == 0 { return errors.New("Load Stats: file path is empty") diff --git a/executor/merge_join.go b/executor/merge_join.go index c8b528f468a2a..b869ce2449b1f 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -132,7 +132,7 @@ func (t *mergeJoinInnerTable) nextRow() (chunk.Row, error) { if t.curRow == t.curIter.End() { t.reallocReaderResult() oldMemUsage := t.curResult.MemoryUsage() - err := t.reader.NextChunk(t.ctx, t.curResult) + err := t.reader.Next(t.ctx, t.curResult) // error happens or no more data. if err != nil || t.curResult.NumRows() == 0 { t.curRow = t.curIter.End() @@ -234,8 +234,8 @@ func (e *MergeJoinExec) prepare(ctx context.Context, chk *chunk.Chunk) error { return nil } -// NextChunk implements the Executor NextChunk interface. -func (e *MergeJoinExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *MergeJoinExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.prepared { if err := e.prepare(ctx, chk); err != nil { @@ -319,7 +319,7 @@ func (e *MergeJoinExec) fetchNextInnerRows() (err error) { // may not all belong to the same join key, but are guaranteed to be sorted // according to the join key. func (e *MergeJoinExec) fetchNextOuterRows(ctx context.Context) (err error) { - err = e.outerTable.reader.NextChunk(ctx, e.outerTable.chk) + err = e.outerTable.reader.Next(ctx, e.outerTable.chk) if err != nil { return errors.Trace(err) } diff --git a/executor/pkg_test.go b/executor/pkg_test.go index e718b081c7aa4..3a4f9a9b1c6cd 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -26,7 +26,7 @@ type MockExec struct { curRowIdx int } -func (m *MockExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (m *MockExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() colTypes := m.retTypes() for ; m.curRowIdx < len(m.Rows) && chk.NumRows() < m.maxChunkSize; m.curRowIdx++ { @@ -97,7 +97,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { joinChk := join.newChunk() it := chunk.NewIterator4Chunk(joinChk) for rowIdx := 1; ; { - err := join.NextChunk(ctx, joinChk) + err := join.Next(ctx, joinChk) c.Check(err, IsNil) if joinChk.NumRows() == 0 { break diff --git a/executor/prepared.go b/executor/prepared.go index aff2d676a22da..bd84bead3fa91 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -93,8 +93,8 @@ func NewPrepareExec(ctx sessionctx.Context, is infoschema.InfoSchema, sqlTxt str } } -// NextChunk implements the Executor NextChunk interface. -func (e *PrepareExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error { vars := e.ctx.GetSessionVars() if e.ID != 0 { // Must be the case when we retry a prepare. @@ -184,8 +184,8 @@ type ExecuteExec struct { plan plan.Plan } -// NextChunk implements the Executor NextChunk interface. -func (e *ExecuteExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ExecuteExec) Next(ctx context.Context, chk *chunk.Chunk) error { return nil } @@ -220,8 +220,8 @@ type DeallocateExec struct { Name string } -// NextChunk implements the Executor NextChunk interface. -func (e *DeallocateExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error { vars := e.ctx.GetSessionVars() id, ok := vars.PreparedStmtNameToID[e.Name] if !ok { diff --git a/executor/prepared_test.go b/executor/prepared_test.go index ec35ff2e6be7d..3c9f51016cc93 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -105,7 +105,7 @@ func (s *testSuite) TestPrepared(c *C) { rs, err = stmt.Exec(ctx) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(ctx, chk) + err = rs.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(rs.Close(), IsNil) @@ -176,7 +176,7 @@ func (s *testSuite) TestPrepared(c *C) { // Coverage. exec := &executor.ExecuteExec{} - exec.NextChunk(ctx, nil) + exec.Next(ctx, nil) exec.Close() } cfg.PreparedPlanCache.Enabled = orgEnable diff --git a/executor/revoke.go b/executor/revoke.go index 6e98bbcc50652..38abbf07a8f22 100644 --- a/executor/revoke.go +++ b/executor/revoke.go @@ -50,8 +50,8 @@ type RevokeExec struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *RevokeExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *RevokeExec) Next(ctx context.Context, chk *chunk.Chunk) error { if e.done { return nil } diff --git a/executor/set.go b/executor/set.go index ee4f9fcff943e..43cfa8c74df53 100644 --- a/executor/set.go +++ b/executor/set.go @@ -42,8 +42,8 @@ type SetExecutor struct { done bool } -// NextChunk implements the Executor NextChunk interface. -func (e *SetExecutor) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SetExecutor) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.done { return nil diff --git a/executor/show.go b/executor/show.go index 8ca2463ac9eec..6797d18f2eacc 100644 --- a/executor/show.go +++ b/executor/show.go @@ -61,8 +61,8 @@ type ShowExec struct { cursor int } -// NextChunk implements the Executor NextChunk interface. -func (e *ShowExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ShowExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.result == nil { e.result = e.newChunk() diff --git a/executor/simple.go b/executor/simple.go index 59883b4025e45..a4d4387bccd81 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -47,8 +47,8 @@ type SimpleExec struct { is infoschema.InfoSchema } -// NextChunk implements the Executor NextChunk interface. -func (e *SimpleExec) NextChunk(ctx context.Context, chk *chunk.Chunk) (err error) { +// Next implements the Executor Next interface. +func (e *SimpleExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { if e.done { return nil } diff --git a/executor/sort.go b/executor/sort.go index d93f4b024db00..e77455dd87eed 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -71,8 +71,8 @@ func (e *SortExec) Open(ctx context.Context) error { return errors.Trace(e.children[0].Open(ctx)) } -// NextChunk implements the Executor NextChunk interface. -func (e *SortExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *SortExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { err := e.fetchRowChunks(ctx) @@ -112,7 +112,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { e.rowChunks.GetMemTracker().SetLabel("rowChunks") for { chk := chunk.NewChunk(fields) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -294,8 +294,8 @@ func (e *TopNExec) Open(ctx context.Context) error { return errors.Trace(e.SortExec.Open(ctx)) } -// NextChunk implements the Executor NextChunk interface. -func (e *TopNExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *TopNExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { e.totalLimit = int(e.limit.Offset + e.limit.Count) @@ -328,7 +328,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.rowChunks.GetMemTracker().SetLabel("rowChunks") for e.rowChunks.Len() < e.totalLimit { srcChk := e.children[0].newChunk() - err := e.children[0].NextChunk(ctx, srcChk) + err := e.children[0].Next(ctx, srcChk) if err != nil { return errors.Trace(err) } @@ -364,7 +364,7 @@ func (e *TopNExec) executeTopN(ctx context.Context) error { } childRowChk := e.children[0].newChunk() for { - err := e.children[0].NextChunk(ctx, childRowChk) + err := e.children[0].Next(ctx, childRowChk) if err != nil { return errors.Trace(err) } diff --git a/executor/union_scan.go b/executor/union_scan.go index fec8cc32da422..d85d5229ce43d 100644 --- a/executor/union_scan.go +++ b/executor/union_scan.go @@ -113,8 +113,8 @@ type UnionScanExec struct { cursor4SnapshotRows int } -// NextChunk implements the Executor NextChunk interface. -func (us *UnionScanExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (us *UnionScanExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() mutableRow := chunk.MutRowFromTypes(us.retTypes()) for i, batchSize := 0, us.ctx.GetSessionVars().MaxChunkSize; i < batchSize; i++ { @@ -183,7 +183,7 @@ func (us *UnionScanExec) getSnapshotRow(ctx context.Context) (Row, error) { us.snapshotRows = us.snapshotRows[:0] for len(us.snapshotRows) == 0 { chk := chunk.NewChunkWithCapacity(us.retTypes(), us.maxChunkSize) - err = us.children[0].NextChunk(ctx, chk) + err = us.children[0].Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return nil, errors.Trace(err) } diff --git a/executor/write.go b/executor/write.go index 1143e931b9f65..1f8cb2f8da01f 100644 --- a/executor/write.go +++ b/executor/write.go @@ -182,8 +182,8 @@ type DeleteExec struct { finished bool } -// NextChunk implements the Executor NextChunk interface. -func (e *DeleteExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *DeleteExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -259,7 +259,7 @@ func (e *DeleteExec) deleteSingleTableByChunk(ctx context.Context) error { chk := e.children[0].newChunk() iter := chunk.NewIterator4Chunk(chk) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -339,7 +339,7 @@ func (e *DeleteExec) deleteMultiTablesByChunk(ctx context.Context) error { chk := e.children[0].newChunk() iter := chunk.NewIterator4Chunk(chk) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } @@ -711,8 +711,8 @@ func (k loadDataVarKeyType) String() string { // LoadDataVarKey is a variable key for load data. const LoadDataVarKey loadDataVarKeyType = 0 -// NextChunk implements the Executor NextChunk interface. -func (e *LoadData) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *LoadData) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() // TODO: support load data without local field. if !e.IsLocal { @@ -995,8 +995,8 @@ func batchMarkDupRows(ctx sessionctx.Context, t table.Table, rows [][]types.Datu return rows, nil } -// NextChunk implements Exec NextChunk interface. -func (e *InsertExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements Exec Next interface. +func (e *InsertExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -1268,7 +1268,7 @@ func (e *InsertValues) getRowsSelectChunk(ctx context.Context, cols []*table.Col chk := selectExec.newChunk() iter := chunk.NewIterator4Chunk(chk) - err := selectExec.NextChunk(ctx, chk) + err := selectExec.Next(ctx, chk) if err != nil { return nil, errors.Trace(err) } @@ -1580,8 +1580,8 @@ func (e *ReplaceExec) exec(ctx context.Context, rows [][]types.Datum) (Row, erro return nil, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *ReplaceExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *ReplaceExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if e.finished { return nil @@ -1673,8 +1673,8 @@ func (e *UpdateExec) exec(ctx context.Context, schema *expression.Schema) (Row, return Row{}, nil } -// NextChunk implements the Executor NextChunk interface. -func (e *UpdateExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +// Next implements the Executor Next interface. +func (e *UpdateExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() if !e.fetched { err := e.fetchChunkRows(ctx) @@ -1714,7 +1714,7 @@ func (e *UpdateExec) fetchChunkRows(ctx context.Context) error { globalRowIdx := 0 for { chk := chunk.NewChunk(fields) - err := e.children[0].NextChunk(ctx, chk) + err := e.children[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 4f7d067841b61..6a127362a634d 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -186,7 +186,7 @@ func (p *MySQLPrivilege) loadTable(sctx sessionctx.Context, sql string, chk := rs.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } diff --git a/server/conn.go b/server/conn.go index 74db5aba8c5f6..df2cb26a7b25a 100644 --- a/server/conn.go +++ b/server/conn.go @@ -955,13 +955,13 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool chk := rs.NewChunk() gotColumnInfo := false for { - // Here server.tidbResultSet implements NextChunk method. - err := rs.NextChunk(ctx, chk) + // Here server.tidbResultSet implements Next method. + err := rs.Next(ctx, chk) if err != nil { return errors.Trace(err) } if !gotColumnInfo { - // We need to call NextChunk before we get columns. + // We need to call Next before we get columns. // Otherwise, we will get incorrect columns info. columns := rs.Columns() err = cc.writeColumnInfo(columns) diff --git a/server/driver.go b/server/driver.go index 42968c6c2fd23..9b2b1eaceb598 100644 --- a/server/driver.go +++ b/server/driver.go @@ -119,6 +119,6 @@ type PreparedStatement interface { type ResultSet interface { Columns() []*ColumnInfo NewChunk() *chunk.Chunk - NextChunk(context.Context, *chunk.Chunk) error + Next(context.Context, *chunk.Chunk) error Close() error } diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 85735baccddc1..016561b2b18cd 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -293,8 +293,8 @@ func (trs *tidbResultSet) NewChunk() *chunk.Chunk { return trs.recordSet.NewChunk() } -func (trs *tidbResultSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { - return trs.recordSet.NextChunk(ctx, chk) +func (trs *tidbResultSet) Next(ctx context.Context, chk *chunk.Chunk) error { + return trs.recordSet.Next(ctx, chk) } func (trs *tidbResultSet) Close() error { diff --git a/server/tidb_test.go b/server/tidb_test.go index 3d030a8107dd6..ffdb3fdad39d0 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -400,7 +400,7 @@ func (ts *TidbTestSuite) TestCreateTableFlen(c *C) { c.Assert(err, IsNil) rs, err := qctx.Execute(ctx, "show create table t1") chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) cols := rs[0].Columns() c.Assert(err, IsNil) @@ -429,7 +429,7 @@ func (ts *TidbTestSuite) TestShowTablesFlen(c *C) { c.Assert(err, IsNil) rs, err := qctx.Execute(ctx, "show tables") chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) cols := rs[0].Columns() c.Assert(err, IsNil) diff --git a/session/bench_test.go b/session/bench_test.go index accdb4e117935..777876e570c02 100644 --- a/session/bench_test.go +++ b/session/bench_test.go @@ -84,7 +84,7 @@ func prepareJoinBenchData(se Session, colType string, valueFormat string, valueC func readResult(ctx context.Context, rs ast.RecordSet, count int) { chk := rs.NewChunk() for count > 0 { - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { log.Fatal(err) } diff --git a/session/bootstrap.go b/session/bootstrap.go index a00c5159232bc..06b05084c5514 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -275,7 +275,7 @@ func getTiDBVar(s Session, name string) (sVal string, isNull bool, e error) { r := rs[0] defer terror.Call(r.Close) chk := r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return "", true, errors.Trace(err) } @@ -489,7 +489,7 @@ func upgradeToVer12(s Session) { defer terror.Call(r.Close) chk := r.NewChunk() it := chunk.NewIterator4Chunk(chk) - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) for err == nil && chk.NumRows() != 0 { for row := it.Begin(); row != it.End(); row = it.Next() { user := row.GetString(0) @@ -501,7 +501,7 @@ func upgradeToVer12(s Session) { updateSQL := fmt.Sprintf(`UPDATE mysql.user set password = "%s" where user="%s" and host="%s"`, newPass, user, host) sqls = append(sqls, updateSQL) } - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) } terror.MustNil(err) diff --git a/session/bootstrap_test.go b/session/bootstrap_test.go index 66cb14bbad793..2d6a7cbcb7f42 100644 --- a/session/bootstrap_test.go +++ b/session/bootstrap_test.go @@ -51,7 +51,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(r, NotNil) ctx := context.Background() chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) datums := ast.RowToDatums(chk.GetRow(0), r.Fields()) @@ -67,7 +67,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { r = mustExecSQL(c, se, "SELECT COUNT(*) from mysql.global_variables;") c.Assert(r, NotNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).GetInt64(0), Equals, globalVarsCount()) @@ -88,7 +88,7 @@ func (s *testBootstrapSuite) TestBootstrap(c *C) { c.Assert(r, NotNil) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) datums = ast.RowToDatums(chk.GetRow(0), r.Fields()) match(c, datums, 3) @@ -148,7 +148,7 @@ func (s *testBootstrapSuite) testBootstrapWithError(c *C) { mustExecSQL(c, se, "USE mysql;") r := mustExecSQL(c, se, `select * from user;`) chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row := chk.GetRow(0) @@ -162,14 +162,14 @@ func (s *testBootstrapSuite) testBootstrapWithError(c *C) { // Check global variables. r = mustExecSQL(c, se, "SELECT COUNT(*) from mysql.global_variables;") chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) v := chk.GetRow(0) c.Assert(v.GetInt64(0), Equals, globalVarsCount()) r = mustExecSQL(c, se, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="bootstrapped";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row = chk.GetRow(0) @@ -192,7 +192,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { // bootstrap with currentBootstrapVersion r := mustExecSQL(c, se, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk := r.NewChunk() - err := r.NextChunk(ctx, chk) + err := r.Next(ctx, chk) row := chk.GetRow(0) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) @@ -221,7 +221,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { // Make sure the version is downgraded. r = mustExecSQL(c, se1, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsTrue) @@ -236,7 +236,7 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { se2 := newSession(c, store, s.dbName) r = mustExecSQL(c, se2, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) chk = r.NewChunk() - err = r.NextChunk(ctx, chk) + err = r.Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row = chk.GetRow(0) diff --git a/session/session.go b/session/session.go index 3609a9f605f4d..e209bae05baf1 100644 --- a/session/session.go +++ b/session/session.go @@ -610,7 +610,7 @@ func drainRecordSet(ctx context.Context, rs ast.RecordSet) ([]types.Row, error) var rows []types.Row for { chk := rs.NewChunk() - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil || chk.NumRows() == 0 { return rows, errors.Trace(err) } @@ -860,7 +860,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields // So we have to call PrepareTxnCtx here. s.PrepareTxnCtx(ctx) prepareExec := executor.NewPrepareExec(s, executor.GetInfoSchema(s), sql) - err = prepareExec.NextChunk(ctx, nil) + err = prepareExec.Next(ctx, nil) if err != nil { err = errors.Trace(err) return diff --git a/session/session_test.go b/session/session_test.go index 95b1da81a7148..8cbff9ce28aef 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -934,7 +934,7 @@ func (s *testSessionSuite) TestResultType(c *C) { rs, err := tk.Exec(`select cast(null as char(30))`) c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.Background(), chk) + err = rs.Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.GetRow(0).IsNull(0), IsTrue) c.Assert(rs.Fields()[0].Column.FieldType.Tp, Equals, mysql.TypeVarString) @@ -1655,7 +1655,7 @@ func (s *testSchemaSuite) TestTableReaderChunk(c *C) { var count int var numChunks int for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { @@ -1688,7 +1688,7 @@ func (s *testSchemaSuite) TestInsertExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break @@ -1722,7 +1722,7 @@ func (s *testSchemaSuite) TestUpdateExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break @@ -1757,7 +1757,7 @@ func (s *testSchemaSuite) TestDeleteExecChunk(c *C) { c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 1) @@ -1789,7 +1789,7 @@ func (s *testSchemaSuite) TestDeleteMultiTableExecChunk(c *C) { var idx int for { chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { @@ -1809,7 +1809,7 @@ func (s *testSchemaSuite) TestDeleteMultiTableExecChunk(c *C) { c.Assert(err, IsNil) chk := rs.NewChunk() - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows(), Equals, 0) rs.Close() @@ -1834,7 +1834,7 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { chk := rs.NewChunk() var count int for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { @@ -1854,7 +1854,7 @@ func (s *testSchemaSuite) TestIndexLookUpReaderChunk(c *C) { chk = rs.NewChunk() count = 0 for { - err = rs.NextChunk(context.TODO(), chk) + err = rs.Next(context.TODO(), chk) c.Assert(err, IsNil) numRows := chk.NumRows() if numRows == 0 { diff --git a/session/tidb.go b/session/tidb.go index 55eda92682c26..edfd96c2c4ec3 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -222,7 +222,7 @@ func GetRows4Test(ctx context.Context, sctx sessionctx.Context, rs ast.RecordSet chk := rs.NewChunk() iter := chunk.NewIterator4Chunk(chk) - err := rs.NextChunk(ctx, chk) + err := rs.Next(ctx, chk) if err != nil { return nil, errors.Trace(err) } diff --git a/statistics/boostrap.go b/statistics/boostrap.go index 8df521565221e..7b5f3d5fe21f2 100644 --- a/statistics/boostrap.go +++ b/statistics/boostrap.go @@ -61,7 +61,7 @@ func (h *Handle) initStatsMeta(is infoschema.InfoSchema) (statsCache, error) { chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return nil, errors.Trace(err) } @@ -128,7 +128,7 @@ func (h *Handle) initStatsHistograms(is infoschema.InfoSchema, tables statsCache chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } @@ -198,7 +198,7 @@ func (h *Handle) initStatsBuckets(tables statsCache) error { chk := rc[0].NewChunk() iter := chunk.NewIterator4Chunk(chk) for { - err := rc[0].NextChunk(context.TODO(), chk) + err := rc[0].Next(context.TODO(), chk) if err != nil { return errors.Trace(err) } diff --git a/statistics/ddl.go b/statistics/ddl.go index 145cb0b282906..8f4905e7677c8 100644 --- a/statistics/ddl.go +++ b/statistics/ddl.go @@ -101,7 +101,7 @@ func (h *Handle) insertColStats2KV(tableID int64, colInfo *model.ColumnInfo) err return errors.Trace(err) } chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) if err != nil { return errors.Trace(err) } diff --git a/statistics/sample.go b/statistics/sample.go index 33065fa35f986..279fb883b14f7 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -158,7 +158,7 @@ func (s SampleBuilder) CollectColumnStats() ([]*SampleCollector, *SortedBuilder, chk := s.RecordSet.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err := s.RecordSet.NextChunk(ctx, chk) + err := s.RecordSet.Next(ctx, chk) if err != nil { return nil, nil, errors.Trace(err) } diff --git a/statistics/statistics_test.go b/statistics/statistics_test.go index 41429322e3969..98e709816a5d6 100644 --- a/statistics/statistics_test.go +++ b/statistics/statistics_test.go @@ -82,7 +82,7 @@ func (r *recordSet) getNext() []types.Datum { return row } -func (r *recordSet) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (r *recordSet) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() row := r.getNext() if row != nil { @@ -174,7 +174,7 @@ func buildPK(sctx sessionctx.Context, numBuckets, id int64, records ast.RecordSe ctx := context.Background() for { chk := records.NewChunk() - err := records.NextChunk(ctx, chk) + err := records.Next(ctx, chk) if err != nil { return 0, nil, errors.Trace(err) } @@ -200,7 +200,7 @@ func buildIndex(sctx sessionctx.Context, numBuckets, id int64, records ast.Recor chk := records.NewChunk() it := chunk.NewIterator4Chunk(chk) for { - err := records.NextChunk(ctx, chk) + err := records.Next(ctx, chk) if err != nil { return 0, nil, nil, errors.Trace(err) } diff --git a/store/mockstore/mocktikv/analyze.go b/store/mockstore/mocktikv/analyze.go index 7f8e0737613e2..b04647b19e997 100644 --- a/store/mockstore/mocktikv/analyze.go +++ b/store/mockstore/mocktikv/analyze.go @@ -210,16 +210,7 @@ func (e *analyzeColumnsExec) getNext(ctx context.Context) ([]types.Datum, error) return datumRow, nil } -// Next implements the ast.RecordSet Next interface. -func (e *analyzeColumnsExec) Next(ctx context.Context) (types.Row, error) { - row, err := e.getNext(ctx) - if row == nil || err != nil { - return nil, errors.Trace(err) - } - return types.DatumRow(row), nil -} - -func (e *analyzeColumnsExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error { +func (e *analyzeColumnsExec) Next(ctx context.Context, chk *chunk.Chunk) error { chk.Reset() row, err := e.getNext(ctx) if row == nil || err != nil { diff --git a/store/tikv/gcworker/gc_worker.go b/store/tikv/gcworker/gc_worker.go index 955225ab549b6..5aa1622c1d415 100644 --- a/store/tikv/gcworker/gc_worker.go +++ b/store/tikv/gcworker/gc_worker.go @@ -889,7 +889,7 @@ func (w *GCWorker) loadValueFromSysTable(key string, s session.Session) (string, return "", errors.Trace(err) } chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) if err != nil { return "", errors.Trace(err) } diff --git a/store/tikv/sql_fail_test.go b/store/tikv/sql_fail_test.go index 643194cf2b72a..eec2437652c7f 100644 --- a/store/tikv/sql_fail_test.go +++ b/store/tikv/sql_fail_test.go @@ -73,7 +73,7 @@ func (s *testSQLSuite) TestFailBusyServerCop(c *C) { } c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(context.Background(), chk) + err = rs[0].Next(context.Background(), chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) c.Assert(chk.GetRow(0).GetString(0), Equals, "True") @@ -109,7 +109,7 @@ func (s *testSQLSuite) TestCoprocessorStreamRecvTimeout(c *C) { chk := res[0].NewChunk() for { - err := res[0].NextChunk(ctx, chk) + err := res[0].Next(ctx, chk) c.Assert(err, IsNil) if chk.NumRows() == 0 { break diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 4602503c9a6db..a056d7eec50ed 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -159,7 +159,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err := ts.se.Execute(ctx, "select * from test.t where c1 = 1") c.Assert(err, IsNil) chk := rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) _, err = ts.se.Execute(ctx, "drop table test.t") @@ -172,7 +172,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err = ts.se.Execute(ctx, "select * from test.t where c1 = 1") c.Assert(err, IsNil) chk = rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) row := chk.GetRow(0) @@ -187,7 +187,7 @@ func (ts *testSuite) TestTypes(c *C) { rs, err = ts.se.Execute(ctx, "select c1 + 1 from test.t where c1 = 1") c.Assert(err, IsNil) chk = rs[0].NewChunk() - err = rs[0].NextChunk(ctx, chk) + err = rs[0].Next(ctx, chk) c.Assert(err, IsNil) c.Assert(chk.NumRows() == 0, IsFalse) c.Assert(chk.GetRow(0).GetFloat64(0), DeepEquals, float64(2))