Skip to content

Commit a6094f7

Browse files
authored
ddl: fix job's row count for global sort (pingcap#59898)
close pingcap#59897
1 parent bbb9623 commit a6094f7

File tree

3 files changed

+66
-102
lines changed

3 files changed

+66
-102
lines changed

pkg/ddl/backfilling_operators.go

+50-94
Original file line numberDiff line numberDiff line change
@@ -181,8 +181,8 @@ func NewAddIndexIngestPipeline(
181181
srcOp := NewTableScanTaskSource(ctx, store, tbl, startKey, endKey, backendCtx)
182182
scanOp := NewTableScanOperator(ctx, sessPool, copCtx, srcChkPool, readerCnt,
183183
reorgMeta.GetBatchSize(), rm, backendCtx)
184-
ingestOp := NewIndexIngestOperator(ctx, copCtx, backendCtx, sessPool,
185-
tbl, indexes, engines, srcChkPool, writerCnt, reorgMeta, rowCntListener)
184+
ingestOp := NewIndexIngestOperator(ctx, copCtx, sessPool,
185+
tbl, indexes, engines, srcChkPool, writerCnt, reorgMeta)
186186
sinkOp := newIndexWriteResultSink(ctx, backendCtx, tbl, indexes, rowCntListener)
187187

188188
operator.Compose[TableScanTask](srcOp, scanOp)
@@ -667,19 +667,17 @@ func NewWriteExternalStoreOperator(
667667
writers = append(writers, writer)
668668
}
669669

670-
return &indexIngestExternalWorker{
671-
indexIngestBaseWorker: indexIngestBaseWorker{
672-
ctx: ctx,
673-
tbl: tbl,
674-
indexes: indexes,
675-
copCtx: copCtx,
676-
se: nil,
677-
sessPool: sessPool,
678-
writers: writers,
679-
srcChunkPool: srcChunkPool,
680-
reorgMeta: reorgMeta,
681-
totalCount: totalCount,
682-
},
670+
return &indexIngestWorker{
671+
ctx: ctx,
672+
tbl: tbl,
673+
indexes: indexes,
674+
copCtx: copCtx,
675+
se: nil,
676+
sessPool: sessPool,
677+
writers: writers,
678+
srcChunkPool: srcChunkPool,
679+
reorgMeta: reorgMeta,
680+
totalCount: totalCount,
683681
}
684682
})
685683
return &WriteExternalStoreOperator{
@@ -700,7 +698,6 @@ func (o *WriteExternalStoreOperator) Close() error {
700698
type IndexWriteResult struct {
701699
ID int
702700
Added int
703-
Total int
704701
}
705702

706703
// IndexIngestOperator writes index records to ingest engine.
@@ -712,15 +709,13 @@ type IndexIngestOperator struct {
712709
func NewIndexIngestOperator(
713710
ctx *OperatorCtx,
714711
copCtx copr.CopContext,
715-
backendCtx ingest.BackendCtx,
716712
sessPool opSessPool,
717713
tbl table.PhysicalTable,
718714
indexes []table.Index,
719715
engines []ingest.Engine,
720716
srcChunkPool *sync.Pool,
721717
concurrency int,
722718
reorgMeta *model.DDLReorgMeta,
723-
rowCntListener RowCountListener,
724719
) *IndexIngestOperator {
725720
writerCfg := getLocalWriterConfig(len(indexes), concurrency)
726721

@@ -742,77 +737,25 @@ func NewIndexIngestOperator(
742737
writers = append(writers, writer)
743738
}
744739

745-
return &indexIngestLocalWorker{
746-
indexIngestBaseWorker: indexIngestBaseWorker{
747-
ctx: ctx,
748-
tbl: tbl,
749-
indexes: indexes,
750-
copCtx: copCtx,
751-
752-
se: nil,
753-
sessPool: sessPool,
754-
writers: writers,
755-
srcChunkPool: srcChunkPool,
756-
reorgMeta: reorgMeta,
757-
},
758-
backendCtx: backendCtx,
759-
rowCntListener: rowCntListener,
740+
return &indexIngestWorker{
741+
ctx: ctx,
742+
tbl: tbl,
743+
indexes: indexes,
744+
copCtx: copCtx,
745+
746+
se: nil,
747+
sessPool: sessPool,
748+
writers: writers,
749+
srcChunkPool: srcChunkPool,
750+
reorgMeta: reorgMeta,
760751
}
761752
})
762753
return &IndexIngestOperator{
763754
AsyncOperator: operator.NewAsyncOperator[IndexRecordChunk, IndexWriteResult](ctx, pool),
764755
}
765756
}
766757

767-
type indexIngestExternalWorker struct {
768-
indexIngestBaseWorker
769-
}
770-
771-
func (w *indexIngestExternalWorker) HandleTask(ck IndexRecordChunk, send func(IndexWriteResult)) {
772-
defer func() {
773-
if ck.Chunk != nil {
774-
w.srcChunkPool.Put(ck.Chunk)
775-
}
776-
}()
777-
rs, err := w.indexIngestBaseWorker.HandleTask(ck)
778-
if err != nil {
779-
w.ctx.onError(err)
780-
return
781-
}
782-
send(rs)
783-
}
784-
785-
type indexIngestLocalWorker struct {
786-
indexIngestBaseWorker
787-
backendCtx ingest.BackendCtx
788-
rowCntListener RowCountListener
789-
}
790-
791-
func (w *indexIngestLocalWorker) HandleTask(ck IndexRecordChunk, send func(IndexWriteResult)) {
792-
defer func() {
793-
if ck.Chunk != nil {
794-
w.srcChunkPool.Put(ck.Chunk)
795-
}
796-
}()
797-
rs, err := w.indexIngestBaseWorker.HandleTask(ck)
798-
if err != nil {
799-
w.ctx.onError(err)
800-
return
801-
}
802-
if rs.Added == 0 {
803-
return
804-
}
805-
w.rowCntListener.Written(rs.Added)
806-
err = w.backendCtx.IngestIfQuotaExceeded(w.ctx, ck.ID, rs.Added)
807-
if err != nil {
808-
w.ctx.onError(err)
809-
return
810-
}
811-
rs.Total = w.backendCtx.TotalKeyCount()
812-
send(rs)
813-
}
814-
815-
type indexIngestBaseWorker struct {
758+
type indexIngestWorker struct {
816759
ctx *OperatorCtx
817760

818761
tbl table.PhysicalTable
@@ -830,23 +773,28 @@ type indexIngestBaseWorker struct {
830773
totalCount *atomic.Int64
831774
}
832775

833-
func (w *indexIngestBaseWorker) HandleTask(rs IndexRecordChunk) (IndexWriteResult, error) {
776+
func (w *indexIngestWorker) HandleTask(ck IndexRecordChunk, send func(IndexWriteResult)) {
777+
defer func() {
778+
if ck.Chunk != nil {
779+
w.srcChunkPool.Put(ck.Chunk)
780+
}
781+
}()
834782
failpoint.Inject("injectPanicForIndexIngest", func() {
835783
panic("mock panic")
836784
})
837785

838786
result := IndexWriteResult{
839-
ID: rs.ID,
787+
ID: ck.ID,
840788
}
841789
w.initSessCtx()
842-
count, _, err := w.WriteChunk(&rs)
790+
count, _, err := w.WriteChunk(&ck)
843791
if err != nil {
844792
w.ctx.onError(err)
845-
return result, err
793+
return
846794
}
847795
if count == 0 {
848-
logutil.Logger(w.ctx).Info("finish a index ingest task", zap.Int("id", rs.ID))
849-
return result, nil
796+
logutil.Logger(w.ctx).Info("finish a index ingest task", zap.Int("id", ck.ID))
797+
return
850798
}
851799
if w.totalCount != nil {
852800
w.totalCount.Add(int64(count))
@@ -855,10 +803,10 @@ func (w *indexIngestBaseWorker) HandleTask(rs IndexRecordChunk) (IndexWriteResul
855803
if ResultCounterForTest != nil {
856804
ResultCounterForTest.Add(1)
857805
}
858-
return result, nil
806+
send(result)
859807
}
860808

861-
func (w *indexIngestBaseWorker) initSessCtx() {
809+
func (w *indexIngestWorker) initSessCtx() {
862810
if w.se == nil {
863811
sessCtx, err := w.sessPool.Get()
864812
if err != nil {
@@ -874,7 +822,7 @@ func (w *indexIngestBaseWorker) initSessCtx() {
874822
}
875823
}
876824

877-
func (w *indexIngestBaseWorker) Close() {
825+
func (w *indexIngestWorker) Close() {
878826
// TODO(lance6716): unify the real write action for engineInfo and external
879827
// writer.
880828
for _, writer := range w.writers {
@@ -894,7 +842,7 @@ func (w *indexIngestBaseWorker) Close() {
894842
}
895843

896844
// WriteChunk will write index records to lightning engine.
897-
func (w *indexIngestBaseWorker) WriteChunk(rs *IndexRecordChunk) (count int, nextKey kv.Key, err error) {
845+
func (w *indexIngestWorker) WriteChunk(rs *IndexRecordChunk) (count int, nextKey kv.Key, err error) {
898846
failpoint.Inject("mockWriteLocalError", func(_ failpoint.Value) {
899847
failpoint.Return(0, nil, errors.New("mock write local error"))
900848
})
@@ -955,20 +903,28 @@ func (s *indexWriteResultSink) collectResult() error {
955903
select {
956904
case <-s.ctx.Done():
957905
return s.ctx.Err()
958-
case _, ok := <-s.source.Channel():
906+
case rs, ok := <-s.source.Channel():
959907
if !ok {
960908
err := s.flush()
961909
if err != nil {
962910
s.ctx.onError(err)
963911
}
964-
if s.backendCtx != nil {
912+
if s.backendCtx != nil { // for local sort only
965913
total := s.backendCtx.TotalKeyCount()
966914
if total > 0 {
967915
s.rowCntListener.SetTotal(total)
968916
}
969917
}
970918
return err
971919
}
920+
s.rowCntListener.Written(rs.Added)
921+
if s.backendCtx != nil { // for local sort only
922+
err := s.backendCtx.IngestIfQuotaExceeded(s.ctx, rs.ID, rs.Added)
923+
if err != nil {
924+
s.ctx.onError(err)
925+
return err
926+
}
927+
}
972928
}
973929
}
974930
}

tests/realtikvtest/addindextest2/global_sort_test.go

+12-3
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,15 @@ func checkFileCleaned(t *testing.T, jobID int64, sortStorageURI string) {
7272
require.Equal(t, 0, len(statFiles))
7373
}
7474

75+
func checkDataAndShowJobs(t *testing.T, tk *testkit.TestKit, count int) {
76+
tk.MustExec("admin check table t;")
77+
rs := tk.MustQuery("admin show ddl jobs 1;").Rows()
78+
require.Len(t, rs, 1)
79+
require.Contains(t, rs[0][12], "ingest")
80+
require.Contains(t, rs[0][12], "cloud")
81+
require.Equal(t, rs[0][7], strconv.Itoa(count))
82+
}
83+
7584
func TestGlobalSortBasic(t *testing.T) {
7685
gcsHost, gcsPort, cloudStorageURI := genStorageURI(t)
7786
opt := fakestorage.Options{
@@ -120,18 +129,18 @@ func TestGlobalSortBasic(t *testing.T) {
120129
})
121130

122131
tk.MustExec("alter table t add index idx(a);")
123-
tk.MustExec("admin check table t;")
132+
checkDataAndShowJobs(t, tk, size)
124133
<-ch
125134
checkFileCleaned(t, jobID, cloudStorageURI)
126135

127136
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/forceMergeSort", "return()")
128137
tk.MustExec("alter table t add index idx1(a);")
129-
tk.MustExec("admin check table t;")
138+
checkDataAndShowJobs(t, tk, size)
130139
<-ch
131140
checkFileCleaned(t, jobID, cloudStorageURI)
132141

133142
tk.MustExec("alter table t add unique index idx2(a);")
134-
tk.MustExec("admin check table t;")
143+
checkDataAndShowJobs(t, tk, size)
135144
<-ch
136145
checkFileCleaned(t, jobID, cloudStorageURI)
137146
}

tests/realtikvtest/addindextest3/operator_test.go

+4-5
Original file line numberDiff line numberDiff line change
@@ -171,8 +171,8 @@ func TestBackfillOperators(t *testing.T) {
171171
src := testutil.NewOperatorTestSource(chunkResults...)
172172
reorgMeta := ddl.NewDDLReorgMeta(tk.Session())
173173
ingestOp := ddl.NewIndexIngestOperator(
174-
opCtx, copCtx, bcCtx, sessPool, pTbl, []table.Index{index}, []ingest.Engine{mockEngine},
175-
srcChkPool, 3, reorgMeta, &ddl.EmptyRowCntListener{})
174+
opCtx, copCtx, sessPool, pTbl, []table.Index{index}, []ingest.Engine{mockEngine},
175+
srcChkPool, 3, reorgMeta)
176176
sink := testutil.NewOperatorTestSink[ddl.IndexWriteResult]()
177177

178178
operator.Compose[ddl.IndexRecordChunk](src, ingestOp)
@@ -443,9 +443,8 @@ func TestTuneWorkerPoolSize(t *testing.T) {
443443
require.NoError(t, err)
444444
defer bcCtx.Close()
445445
mockEngine := ingest.NewMockEngineInfo(nil)
446-
ingestOp := ddl.NewIndexIngestOperator(opCtx, copCtx, bcCtx, sessPool, pTbl, []table.Index{index},
447-
[]ingest.Engine{mockEngine}, nil, 2, nil,
448-
&ddl.EmptyRowCntListener{})
446+
ingestOp := ddl.NewIndexIngestOperator(opCtx, copCtx, sessPool, pTbl, []table.Index{index},
447+
[]ingest.Engine{mockEngine}, nil, 2, nil)
449448

450449
ingestOp.Open()
451450
require.Equal(t, ingestOp.GetWorkerPoolSize(), int32(2))

0 commit comments

Comments
 (0)