github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner/ddl_puller.go (about) 1 // Copyright 2021 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package owner 15 16 import ( 17 "context" 18 "sync" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/log" 22 timodel "github.com/pingcap/parser/model" 23 "github.com/pingcap/ticdc/cdc/entry" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/cdc/puller" 26 cdcContext "github.com/pingcap/ticdc/pkg/context" 27 "github.com/pingcap/ticdc/pkg/filter" 28 "github.com/pingcap/ticdc/pkg/regionspan" 29 "github.com/pingcap/ticdc/pkg/util" 30 "go.uber.org/zap" 31 "golang.org/x/sync/errgroup" 32 ) 33 34 // DDLPuller is a wrapper of the Puller interface for the owner 35 // DDLPuller starts a puller, listens to the DDL range, adds the received DDLs into an internal queue 36 type DDLPuller interface { 37 // Run runs the DDLPuller 38 Run(ctx cdcContext.Context) error 39 // FrontDDL returns the first DDL job in the internal queue 40 FrontDDL() (uint64, *timodel.Job) 41 // PopFrontDDL returns and pops the first DDL job in the internal queue 42 PopFrontDDL() (uint64, *timodel.Job) 43 // Close closes the DDLPuller 44 Close() 45 } 46 47 type ddlPullerImpl struct { 48 puller puller.Puller 49 filter *filter.Filter 50 51 mu sync.Mutex 52 resolvedTS uint64 53 pendingDDLJobs []*timodel.Job 54 lastDDLJobID int64 55 cancel context.CancelFunc 56 } 57 58 func newDDLPuller(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { 59 pdCli := ctx.GlobalVars().PDClient 60 f, err := filter.NewFilter(ctx.ChangefeedVars().Info.Config) 61 if err != nil { 62 return nil, errors.Trace(err) 63 } 64 var plr puller.Puller 65 kvStorage := ctx.GlobalVars().KVStorage 66 // kvStorage can be nil only in the test 67 if kvStorage != nil { 68 plr = puller.NewPuller(ctx, pdCli, ctx.GlobalVars().GrpcPool, kvStorage, startTs, 69 []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()}, false) 70 } 71 72 return &ddlPullerImpl{ 73 puller: plr, 74 resolvedTS: startTs, 75 filter: f, 76 cancel: func() {}, 77 }, nil 78 } 79 80 const ddlPullerName = "DDL_PULLER" 81 82 func (h *ddlPullerImpl) Run(ctx cdcContext.Context) error { 83 ctx, cancel := cdcContext.WithCancel(ctx) 84 h.cancel = cancel 85 log.Debug("DDL puller started", zap.String("changefeed-id", ctx.ChangefeedVars().ID)) 86 stdCtx := util.PutTableInfoInCtx(ctx, -1, ddlPullerName) 87 errg, stdCtx := errgroup.WithContext(stdCtx) 88 ctx = cdcContext.WithStd(ctx, stdCtx) 89 90 errg.Go(func() error { 91 return h.puller.Run(ctx) 92 }) 93 94 rawDDLCh := puller.SortOutput(ctx, h.puller.Output()) 95 96 receiveDDL := func(rawDDL *model.RawKVEntry) error { 97 if rawDDL == nil { 98 return nil 99 } 100 if rawDDL.OpType == model.OpTypeResolved { 101 h.mu.Lock() 102 defer h.mu.Unlock() 103 if rawDDL.CRTs > h.resolvedTS { 104 h.resolvedTS = rawDDL.CRTs 105 } 106 return nil 107 } 108 job, err := entry.UnmarshalDDL(rawDDL) 109 if err != nil { 110 return errors.Trace(err) 111 } 112 if job == nil { 113 return nil 114 } 115 if h.filter.ShouldDiscardDDL(job.Type) { 116 log.Info("discard the ddl job", zap.Int64("jobID", job.ID), zap.String("query", job.Query)) 117 return nil 118 } 119 if job.ID == h.lastDDLJobID { 120 log.Warn("ignore duplicated DDL job", zap.Any("job", job)) 121 return nil 122 } 123 h.mu.Lock() 124 defer h.mu.Unlock() 125 h.pendingDDLJobs = append(h.pendingDDLJobs, job) 126 h.lastDDLJobID = job.ID 127 return nil 128 } 129 130 errg.Go(func() error { 131 for { 132 select { 133 case <-ctx.Done(): 134 return ctx.Err() 135 case e := <-rawDDLCh: 136 if err := receiveDDL(e); err != nil { 137 return errors.Trace(err) 138 } 139 } 140 } 141 }) 142 143 return errg.Wait() 144 } 145 146 func (h *ddlPullerImpl) FrontDDL() (uint64, *timodel.Job) { 147 h.mu.Lock() 148 defer h.mu.Unlock() 149 if len(h.pendingDDLJobs) == 0 { 150 return h.resolvedTS, nil 151 } 152 job := h.pendingDDLJobs[0] 153 return job.BinlogInfo.FinishedTS, job 154 } 155 156 func (h *ddlPullerImpl) PopFrontDDL() (uint64, *timodel.Job) { 157 h.mu.Lock() 158 defer h.mu.Unlock() 159 if len(h.pendingDDLJobs) == 0 { 160 return h.resolvedTS, nil 161 } 162 job := h.pendingDDLJobs[0] 163 h.pendingDDLJobs = h.pendingDDLJobs[1:] 164 return job.BinlogInfo.FinishedTS, job 165 } 166 167 func (h *ddlPullerImpl) Close() { 168 h.cancel() 169 }