Skip to content

Commit

Permalink
*: tiny clean up the RecordSet NewChunk() method (pingcap#29232)
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Nov 2, 2021
1 parent a96deab commit 55a38a1
Show file tree
Hide file tree
Showing 49 changed files with 133 additions and 158 deletions.
2 changes: 1 addition & 1 deletion bindinfo/bind_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -234,7 +234,7 @@ func TestErrorBind(t *testing.T) {

rs, err := tk.Exec("show global bindings")
require.NoError(t, err)
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
require.NoError(t, err)
require.Equal(t, 0, chk.NumRows())
Expand Down
4 changes: 2 additions & 2 deletions bindinfo/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -815,7 +815,7 @@ func getHintsForSQL(sctx sessionctx.Context, sql string) (string, error) {
if err != nil {
return "", err
}
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
if err != nil {
return "", err
Expand Down Expand Up @@ -1046,7 +1046,7 @@ func runSQL(ctx context.Context, sctx sessionctx.Context, sql string, resultChan
resultChan <- err
return
}
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
for {
err = rs.Next(ctx, chk)
if err != nil || chk.NumRows() == 0 {
Expand Down
4 changes: 2 additions & 2 deletions bindinfo/handle_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ func TestGlobalBinding(t *testing.T) {

rs, err := tk.Exec("show global bindings")
require.Nil(t, err)
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
require.Nil(t, err)
require.Equal(t, 1, chk.NumRows())
Expand Down Expand Up @@ -447,7 +447,7 @@ func TestGlobalBinding(t *testing.T) {

rs, err = tk.Exec("show global bindings")
require.Nil(t, err)
chk = rs.NewChunk()
chk = rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
require.Nil(t, err)
require.Equal(t, 0, chk.NumRows())
Expand Down
4 changes: 2 additions & 2 deletions bindinfo/session_handle_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,14 +135,14 @@ func TestSessionBinding(t *testing.T) {

rs, err := tk.Exec("show global bindings")
require.NoError(t, err)
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
require.NoError(t, err)
require.Equal(t, 0, chk.NumRows())

rs, err = tk.Exec("show session bindings")
require.NoError(t, err)
chk = rs.NewChunk()
chk = rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
require.NoError(t, err)
require.Equal(t, 1, chk.NumRows())
Expand Down
12 changes: 6 additions & 6 deletions br/pkg/lightning/checkpoints/glue_checkpoint.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ func (g GlueCheckpointsDB) TaskCheckpoint(ctx context.Context) (*TaskCheckpoint,
}
r := rs[0]
defer r.Close()
req := r.NewChunk()
req := r.NewChunk(nil)
err = r.Next(ctx, req)
if err != nil {
return err
Expand Down Expand Up @@ -242,7 +242,7 @@ func (g GlueCheckpointsDB) Get(ctx context.Context, tableName string) (*TableChe
return errors.Trace(err)
}
r := rs[0]
req := r.NewChunk()
req := r.NewChunk(nil)
it := chunk.NewIterator4Chunk(req)
for {
err = r.Next(ctx, req)
Expand Down Expand Up @@ -272,7 +272,7 @@ func (g GlueCheckpointsDB) Get(ctx context.Context, tableName string) (*TableChe
return errors.Trace(err)
}
r = rs[0]
req = r.NewChunk()
req = r.NewChunk(nil)
it = chunk.NewIterator4Chunk(req)
for {
err = r.Next(ctx, req)
Expand Down Expand Up @@ -323,7 +323,7 @@ func (g GlueCheckpointsDB) Get(ctx context.Context, tableName string) (*TableChe
}
r = rs[0]
defer r.Close()
req = r.NewChunk()
req = r.NewChunk(nil)
err = r.Next(ctx, req)
if err != nil {
return err
Expand Down Expand Up @@ -708,7 +708,7 @@ func (g GlueCheckpointsDB) DestroyErrorCheckpoint(ctx context.Context, tableName
return errors.Trace(err)
}
r := rs[0]
req := r.NewChunk()
req := r.NewChunk(nil)
it := chunk.NewIterator4Chunk(req)
for {
err = r.Next(ctx, req)
Expand Down Expand Up @@ -787,7 +787,7 @@ func drainFirstRecordSet(ctx context.Context, rss []sqlexec.RecordSet) ([]chunk.
}
rs := rss[0]
var rows []chunk.Row
req := rs.NewChunk()
req := rs.NewChunk(nil)
for {
err := rs.Next(ctx, req)
if err != nil || req.NumRows() == 0 {
Expand Down
2 changes: 1 addition & 1 deletion cmd/benchdb/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func (ut *benchDB) mustExec(sql string, args ...interface{}) {
}
if rs != nil {
ctx := context.Background()
req := rs.NewChunk()
req := rs.NewChunk(nil)
for {
err := rs.Next(ctx, req)
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,7 +147,7 @@ func (s *serialTestStateChangeSuite) TestShowCreateTable(c *C) {
return
}
}
req := result.NewChunk()
req := result.NewChunk(nil)
checkErr = result.Next(context.Background(), req)
if checkErr != nil {
return
Expand Down
2 changes: 1 addition & 1 deletion ddl/rollingback_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func (s *testRollingBackSuite) TestCancelAddIndexJobError(c *C) {
jobID = job.ID
res, checkErr = tk1.Exec("admin cancel ddl jobs " + strconv.Itoa(int(job.ID)))
// drain the result set here, otherwise the cancel action won't take effect immediately.
chk := res.NewChunk()
chk := res.NewChunk(nil)
if err := res.Next(context.Background(), chk); err != nil {
checkErr = err
return
Expand Down
2 changes: 1 addition & 1 deletion ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ func loadDeleteRangesFromTable(ctx sessionctx.Context, table string, safePoint u
return nil, errors.Trace(err)
}

req := rs.NewChunk()
req := rs.NewChunk(nil)
it := chunk.NewIterator4Chunk(req)
for {
err = rs.Next(context.TODO(), req)
Expand Down
21 changes: 10 additions & 11 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,17 +162,16 @@ func (a *recordSet) Next(ctx context.Context, req *chunk.Chunk) (err error) {
return nil
}

// NewChunkFromAllocator create a chunk base on top-level executor's newFirstChunk().
func (a *recordSet) NewChunkFromAllocator(alloc chunk.Allocator) *chunk.Chunk {
// NewChunk create a chunk base on top-level executor's newFirstChunk().
func (a *recordSet) NewChunk(alloc chunk.Allocator) *chunk.Chunk {
if alloc == nil {
return newFirstChunk(a.executor)
}

base := a.executor.base()
return alloc.Alloc(base.retFieldTypes, base.initCap, base.maxChunkSize)
}

// NewChunk create a chunk base on top-level executor's newFirstChunk().
func (a *recordSet) NewChunk() *chunk.Chunk {
return newFirstChunk(a.executor)
}

func (a *recordSet) Close() error {
err := a.executor.Close()
a.stmt.CloseRecordSet(a.txnStartTS, a.lastErr)
Expand Down Expand Up @@ -511,11 +510,11 @@ func (c *chunkRowRecordSet) Next(ctx context.Context, chk *chunk.Chunk) error {
return nil
}

func (c *chunkRowRecordSet) NewChunk() *chunk.Chunk {
return newFirstChunk(c.e)
}
func (c *chunkRowRecordSet) NewChunk(alloc chunk.Allocator) *chunk.Chunk {
if alloc == nil {
return newFirstChunk(c.e)
}

func (c *chunkRowRecordSet) NewChunkFromAllocator(alloc chunk.Allocator) *chunk.Chunk {
base := c.e.base()
return alloc.Alloc(base.retFieldTypes, base.initCap, base.maxChunkSize)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze.go
Original file line number Diff line number Diff line change
Expand Up @@ -1620,7 +1620,7 @@ func (e *AnalyzeFastExec) calculateEstimateSampleStep() (err error) {
return
}
defer terror.Call(rs.Close)
chk := rs.NewChunk()
chk := rs.NewChunk(nil)
err = rs.Next(context.TODO(), chk)
if err != nil {
return
Expand Down
8 changes: 4 additions & 4 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ func (s *testSuite6) TestCreateTable(c *C) {
rs, err := tk.Exec(`desc issue312_1`)
c.Assert(err, IsNil)
ctx := context.Background()
req := rs.NewChunk()
req := rs.NewChunk(nil)
it := chunk.NewIterator4Chunk(req)
for {
err1 := rs.Next(ctx, req)
Expand All @@ -125,7 +125,7 @@ func (s *testSuite6) TestCreateTable(c *C) {
}
rs, err = tk.Exec(`desc issue312_2`)
c.Assert(err, IsNil)
req = rs.NewChunk()
req = rs.NewChunk(nil)
it = chunk.NewIterator4Chunk(req)
for {
err1 := rs.Next(ctx, req)
Expand Down Expand Up @@ -536,7 +536,7 @@ func (s *testSuite6) TestAlterTableAddColumn(c *C) {
now := time.Now().Add(-1 * time.Millisecond).Format(types.TimeFormat)
r, err := tk.Exec("select c2 from alter_test")
c.Assert(err, IsNil)
req := r.NewChunk()
req := r.NewChunk(nil)
err = r.Next(context.Background(), req)
c.Assert(err, IsNil)
row := req.GetRow(0)
Expand Down Expand Up @@ -564,7 +564,7 @@ func (s *testSuite6) TestAlterTableAddColumns(c *C) {
tk.MustExec("alter table alter_test add column (c7 timestamp default current_timestamp, c3 varchar(50) default 'CURRENT_TIMESTAMP')")
r, err := tk.Exec("select c2 from alter_test")
c.Assert(err, IsNil)
req := r.NewChunk()
req := r.NewChunk(nil)
err = r.Next(context.Background(), req)
c.Assert(err, IsNil)
row := req.GetRow(0)
Expand Down
6 changes: 3 additions & 3 deletions executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func (s *testSuite3) TestCopClientSend(c *C) {
// Send coprocessor request when the table split.
rs, err := tk.Exec("select sum(id) from copclient")
c.Assert(err, IsNil)
req := rs.NewChunk()
req := rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(req.GetRow(0).GetMyDecimal(0).String(), Equals, "499500")
Expand All @@ -96,7 +96,7 @@ func (s *testSuite3) TestCopClientSend(c *C) {
// Check again.
rs, err = tk.Exec("select sum(id) from copclient")
c.Assert(err, IsNil)
req = rs.NewChunk()
req = rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(req.GetRow(0).GetMyDecimal(0).String(), Equals, "499500")
Expand All @@ -105,7 +105,7 @@ func (s *testSuite3) TestCopClientSend(c *C) {
// Check there is no goroutine leak.
rs, err = tk.Exec("select * from copclient order by id")
c.Assert(err, IsNil)
req = rs.NewChunk()
req = rs.NewChunk(nil)
err = rs.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(rs.Close(), IsNil)
Expand Down
16 changes: 8 additions & 8 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -392,7 +392,7 @@ func (s *testSuite3) TestAdmin(c *C) {
// cancel DDL jobs test
r, err := tk.Exec("admin cancel ddl jobs 1")
c.Assert(err, IsNil, Commentf("err %v", err))
req := r.NewChunk()
req := r.NewChunk(nil)
err = r.Next(ctx, req)
c.Assert(err, IsNil)
row := req.GetRow(0)
Expand All @@ -403,7 +403,7 @@ func (s *testSuite3) TestAdmin(c *C) {
// show ddl test;
r, err = tk.Exec("admin show ddl")
c.Assert(err, IsNil)
req = r.NewChunk()
req = r.NewChunk(nil)
err = r.Next(ctx, req)
c.Assert(err, IsNil)
row = req.GetRow(0)
Expand All @@ -422,7 +422,7 @@ func (s *testSuite3) TestAdmin(c *C) {
c.Assert(row.GetString(2), Equals, serverInfo.IP+":"+
strconv.FormatUint(uint64(serverInfo.Port), 10))
c.Assert(row.GetString(3), Equals, "")
req = r.NewChunk()
req = r.NewChunk(nil)
err = r.Next(ctx, req)
c.Assert(err, IsNil)
c.Assert(req.NumRows() == 0, IsTrue)
Expand All @@ -432,7 +432,7 @@ func (s *testSuite3) TestAdmin(c *C) {
// show DDL jobs test
r, err = tk.Exec("admin show ddl jobs")
c.Assert(err, IsNil)
req = r.NewChunk()
req = r.NewChunk(nil)
err = r.Next(ctx, req)
c.Assert(err, IsNil)
row = req.GetRow(0)
Expand All @@ -448,7 +448,7 @@ func (s *testSuite3) TestAdmin(c *C) {

r, err = tk.Exec("admin show ddl jobs 20")
c.Assert(err, IsNil)
req = r.NewChunk()
req = r.NewChunk(nil)
err = r.Next(ctx, req)
c.Assert(err, IsNil)
row = req.GetRow(0)
Expand Down Expand Up @@ -1167,7 +1167,7 @@ func (s *testSuiteP1) TestIssue2612(c *C) {
tk.MustExec(`insert into t values ('2016-02-13 15:32:24', '2016-02-11 17:23:22');`)
rs, err := tk.Exec(`select timediff(finish_at, create_at) from t;`)
c.Assert(err, IsNil)
req := rs.NewChunk()
req := rs.NewChunk(nil)
err = rs.Next(context.Background(), req)
c.Assert(err, IsNil)
c.Assert(req.GetRow(0).GetDuration(0, 0).String(), Equals, "-46:09:02")
Expand Down Expand Up @@ -3650,7 +3650,7 @@ func (s *testSuite) TestBit(c *C) {
c.Assert(err, NotNil)
r, err := tk.Exec("select * from t where c1 = 2")
c.Assert(err, IsNil)
req := r.NewChunk()
req := r.NewChunk(nil)
err = r.Next(context.Background(), req)
c.Assert(err, IsNil)
c.Assert(types.BinaryLiteral(req.GetRow(0).GetBytes(0)), DeepEquals, types.NewBinaryLiteralFromUint(2, -1))
Expand Down Expand Up @@ -4551,7 +4551,7 @@ func (s *testSuite3) TestMaxOneRow(c *C) {
rs, err := tk.Exec(`select (select t1.a from t1 where t1.a > t2.a) as a from t2;`)
c.Assert(err, IsNil)

err = rs.Next(context.TODO(), rs.NewChunk())
err = rs.Next(context.TODO(), rs.NewChunk(nil))
c.Assert(err.Error(), Equals, "[executor:1242]Subquery returns more than 1 row")

c.Assert(rs.Close(), IsNil)
Expand Down
2 changes: 1 addition & 1 deletion executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -807,7 +807,7 @@ func getRowsAndFields(ctx sessionctx.Context, rs sqlexec.RecordSet) ([]chunk.Row

func getRowFromRecordSet(ctx context.Context, se sessionctx.Context, rs sqlexec.RecordSet) ([]chunk.Row, error) {
var rows []chunk.Row
req := rs.NewChunk()
req := rs.NewChunk(nil)
for {
err := rs.Next(ctx, req)
if err != nil || req.NumRows() == 0 {
Expand Down
6 changes: 3 additions & 3 deletions executor/index_lookup_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func (s *testSuite1) TestIndexLookupJoinHang(c *C) {

rs, err := tk.Exec("select /*+ INL_JOIN(i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0")
c.Assert(err, IsNil)
req := rs.NewChunk()
req := rs.NewChunk(nil)
for i := 0; i < 5; i++ {
// FIXME: cannot check err, since err exists, Panic: [tikv:1690]BIGINT UNSIGNED value is out of range in '(Column#0 - 3)'
_ = rs.Next(context.Background(), req)
Expand All @@ -46,7 +46,7 @@ func (s *testSuite1) TestIndexLookupJoinHang(c *C) {

rs, err = tk.Exec("select /*+ INL_HASH_JOIN(i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0")
c.Assert(err, IsNil)
req = rs.NewChunk()
req = rs.NewChunk(nil)
for i := 0; i < 5; i++ {
// to fix: cannot check err, since err exists, Panic: [tikv:1690]BIGINT UNSIGNED value is out of range in '(Column#0 - 3)'
_ = rs.Next(context.Background(), req)
Expand All @@ -56,7 +56,7 @@ func (s *testSuite1) TestIndexLookupJoinHang(c *C) {

rs, err = tk.Exec("select /*+ INL_MERGE_JOIN(i)*/ * from idxJoinOuter o left join idxJoinInner i on o.a = i.a where o.a in (1, 2) and (i.a - 3) > 0")
c.Assert(err, IsNil)
req = rs.NewChunk()
req = rs.NewChunk(nil)
for i := 0; i < 5; i++ {
// to fix: cannot check err, since err exists, Panic: [tikv:1690]BIGINT UNSIGNED value is out of range in '(Column#0 - 3)'
_ = rs.Next(context.Background(), req)
Expand Down
2 changes: 1 addition & 1 deletion executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1107,7 +1107,7 @@ func (s *testSuiteJoin1) TestJoinLeak(c *C) {
tk.MustExec("commit")
result, err := tk.Exec("select * from t t1 left join (select 1) t2 on 1")
c.Assert(err, IsNil)
req := result.NewChunk()
req := result.NewChunk(nil)
err = result.Next(context.Background(), req)
c.Assert(err, IsNil)
time.Sleep(time.Millisecond)
Expand Down
2 changes: 1 addition & 1 deletion executor/memory_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ func (s *testMemoryLeak) TestPBMemoryLeak(c *C) {
c.Assert(err, IsNil)
record := records[0]
rowCnt := 0
chk := record.NewChunk()
chk := record.NewChunk(nil)
for {
c.Assert(record.Next(context.Background(), chk), IsNil)
rowCnt += chk.NumRows()
Expand Down
2 changes: 1 addition & 1 deletion executor/plan_replayer.go
Original file line number Diff line number Diff line change
Expand Up @@ -462,7 +462,7 @@ func getRows(ctx context.Context, rs sqlexec.RecordSet) ([]chunk.Row, error) {
return nil, nil
}
var rows []chunk.Row
req := rs.NewChunk()
req := rs.NewChunk(nil)
// Must reuse `req` for imitating server.(*clientConn).writeChunks
for {
err := rs.Next(ctx, req)
Expand Down
Loading

0 comments on commit 55a38a1

Please sign in to comment.