github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/puller/sorter/unified_sorter.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 sorter 15 16 import ( 17 "context" 18 "os" 19 "sync" 20 21 "github.com/pingcap/errors" 22 "github.com/pingcap/failpoint" 23 "github.com/pingcap/log" 24 "github.com/pingcap/ticdc/cdc/model" 25 "github.com/pingcap/ticdc/pkg/config" 26 cerror "github.com/pingcap/ticdc/pkg/errors" 27 "github.com/pingcap/ticdc/pkg/util" 28 "golang.org/x/sync/errgroup" 29 ) 30 31 const ( 32 inputChSize = 128 33 outputChSize = 128 34 heapCollectChSize = 128 // this should be not be too small, to guarantee IO concurrency 35 ) 36 37 // UnifiedSorter provides both sorting in memory and in file. Memory pressure is used to determine which one to use. 38 type UnifiedSorter struct { 39 inputCh chan *model.PolymorphicEvent 40 outputCh chan *model.PolymorphicEvent 41 dir string 42 pool *backEndPool 43 metricsInfo *metricsInfo 44 45 closeCh chan struct{} 46 } 47 48 type metricsInfo struct { 49 changeFeedID model.ChangeFeedID 50 tableName string 51 tableID model.TableID 52 captureAddr string 53 } 54 55 type ctxKey struct { 56 } 57 58 // UnifiedSorterCheckDir checks whether the directory needed exists and is writable. 59 // If it does not exist, we try to create one. 60 // parameter: cfSortDir - the directory designated in changefeed's setting, 61 // which will be overridden by a non-empty local setting of `sort-dir`. 62 // TODO better way to organize this function after we obsolete chanegfeed setting's `sort-dir` 63 func UnifiedSorterCheckDir(cfSortDir string) error { 64 dir := cfSortDir 65 sorterConfig := config.GetGlobalServerConfig().Sorter 66 if sorterConfig.SortDir != "" { 67 // Let the local setting override the changefeed setting 68 dir = sorterConfig.SortDir 69 } 70 71 err := util.IsDirAndWritable(dir) 72 if err != nil { 73 if os.IsNotExist(errors.Cause(err)) { 74 err = os.MkdirAll(dir, 0o755) 75 if err != nil { 76 return errors.Annotate(cerror.WrapError(cerror.ErrProcessorSortDir, err), "create dir") 77 } 78 } else { 79 return errors.Annotate(cerror.WrapError(cerror.ErrProcessorSortDir, err), "sort dir check") 80 } 81 } 82 83 return nil 84 } 85 86 // NewUnifiedSorter creates a new UnifiedSorter 87 func NewUnifiedSorter( 88 dir string, 89 changeFeedID model.ChangeFeedID, 90 tableName string, 91 tableID model.TableID, 92 captureAddr string) (*UnifiedSorter, error) { 93 poolMu.Lock() 94 defer poolMu.Unlock() 95 96 if pool == nil { 97 sorterConfig := config.GetGlobalServerConfig().Sorter 98 if sorterConfig.SortDir != "" { 99 // Let the local setting override the changefeed setting 100 dir = sorterConfig.SortDir 101 } 102 var err error 103 pool, err = newBackEndPool(dir, captureAddr) 104 if err != nil { 105 return nil, errors.Trace(err) 106 } 107 } 108 109 lazyInitWorkerPool() 110 return &UnifiedSorter{ 111 inputCh: make(chan *model.PolymorphicEvent, inputChSize), 112 outputCh: make(chan *model.PolymorphicEvent, outputChSize), 113 dir: dir, 114 pool: pool, 115 metricsInfo: &metricsInfo{ 116 changeFeedID: changeFeedID, 117 tableName: tableName, 118 tableID: tableID, 119 captureAddr: captureAddr, 120 }, 121 closeCh: make(chan struct{}, 1), 122 }, nil 123 } 124 125 // UnifiedSorterCleanUp cleans up the files that might have been used. 126 func UnifiedSorterCleanUp() { 127 poolMu.Lock() 128 defer poolMu.Unlock() 129 130 if pool != nil { 131 log.Info("Unified Sorter: starting cleaning up files") 132 pool.terminate() 133 pool = nil 134 } 135 } 136 137 // ResetGlobalPoolWithoutCleanup reset the pool without cleaning up files. 138 // Note that it is used in tests only. 139 func ResetGlobalPoolWithoutCleanup() { 140 poolMu.Lock() 141 defer poolMu.Unlock() 142 143 pool = nil 144 } 145 146 // Run implements the EventSorter interface 147 func (s *UnifiedSorter) Run(ctx context.Context) error { 148 failpoint.Inject("sorterDebug", func() { 149 log.Info("sorterDebug: Running Unified Sorter in debug mode") 150 }) 151 152 defer close(s.closeCh) 153 154 finish := util.MonitorCancelLatency(ctx, "Unified Sorter") 155 defer finish() 156 157 ctx = context.WithValue(ctx, ctxKey{}, s) 158 ctx = util.PutCaptureAddrInCtx(ctx, s.metricsInfo.captureAddr) 159 ctx = util.PutChangefeedIDInCtx(ctx, s.metricsInfo.changeFeedID) 160 ctx = util.PutTableInfoInCtx(ctx, s.metricsInfo.tableID, s.metricsInfo.tableName) 161 162 sorterConfig := config.GetGlobalServerConfig().Sorter 163 numConcurrentHeaps := sorterConfig.NumConcurrentWorker 164 165 errg, subctx := errgroup.WithContext(ctx) 166 heapSorterCollectCh := make(chan *flushTask, heapCollectChSize) 167 // mergerCleanUp will consumer the remaining elements in heapSorterCollectCh to prevent any FD leak. 168 defer mergerCleanUp(heapSorterCollectCh) 169 170 heapSorterErrCh := make(chan error, 1) 171 defer close(heapSorterErrCh) 172 heapSorterErrOnce := &sync.Once{} 173 heapSorters := make([]*heapSorter, sorterConfig.NumConcurrentWorker) 174 for i := range heapSorters { 175 heapSorters[i] = newHeapSorter(i, heapSorterCollectCh) 176 heapSorters[i].init(subctx, func(err error) { 177 heapSorterErrOnce.Do(func() { 178 heapSorterErrCh <- err 179 }) 180 }) 181 } 182 183 ioCancelFunc := func() { 184 for _, heapSorter := range heapSorters { 185 // cancels async IO operations 186 heapSorter.canceller.Cancel() 187 } 188 } 189 190 errg.Go(func() error { 191 defer func() { 192 // cancelling the heapSorters from the outside 193 for _, hs := range heapSorters { 194 hs.poolHandle.Unregister() 195 } 196 // must wait for all writers to exit to close the channel. 197 close(heapSorterCollectCh) 198 failpoint.Inject("InjectHeapSorterExitDelay", func() {}) 199 }() 200 201 select { 202 case <-subctx.Done(): 203 return errors.Trace(subctx.Err()) 204 case err := <-heapSorterErrCh: 205 return errors.Trace(err) 206 } 207 }) 208 209 errg.Go(func() error { 210 return printError(runMerger(subctx, numConcurrentHeaps, heapSorterCollectCh, s.outputCh, ioCancelFunc)) 211 }) 212 213 errg.Go(func() error { 214 captureAddr := util.CaptureAddrFromCtx(ctx) 215 changefeedID := util.ChangefeedIDFromCtx(ctx) 216 217 metricSorterConsumeCount := sorterConsumeCount.MustCurryWith(map[string]string{ 218 "capture": captureAddr, 219 "changefeed": changefeedID, 220 }) 221 222 nextSorterID := 0 223 for { 224 select { 225 case <-subctx.Done(): 226 return subctx.Err() 227 case event := <-s.inputCh: 228 if event.RawKV != nil && event.RawKV.OpType == model.OpTypeResolved { 229 // broadcast resolved events 230 for _, sorter := range heapSorters { 231 select { 232 case <-subctx.Done(): 233 return subctx.Err() 234 default: 235 } 236 err := sorter.poolHandle.AddEvent(subctx, event) 237 if cerror.ErrWorkerPoolHandleCancelled.Equal(err) { 238 // no need to report ErrWorkerPoolHandleCancelled, 239 // as it may confuse the user 240 return nil 241 } 242 if err != nil { 243 return errors.Trace(err) 244 } 245 metricSorterConsumeCount.WithLabelValues("resolved").Inc() 246 } 247 continue 248 } 249 250 // dispatch a row changed event 251 targetID := nextSorterID % numConcurrentHeaps 252 nextSorterID++ 253 select { 254 case <-subctx.Done(): 255 return subctx.Err() 256 default: 257 err := heapSorters[targetID].poolHandle.AddEvent(subctx, event) 258 if err != nil { 259 if cerror.ErrWorkerPoolHandleCancelled.Equal(err) { 260 // no need to report ErrWorkerPoolHandleCancelled, 261 // as it may confuse the user 262 return nil 263 } 264 return errors.Trace(err) 265 } 266 metricSorterConsumeCount.WithLabelValues("kv").Inc() 267 } 268 } 269 } 270 }) 271 272 return printError(errg.Wait()) 273 } 274 275 // AddEntry implements the EventSorter interface 276 func (s *UnifiedSorter) AddEntry(ctx context.Context, entry *model.PolymorphicEvent) { 277 select { 278 case <-ctx.Done(): 279 return 280 case <-s.closeCh: 281 case s.inputCh <- entry: 282 } 283 } 284 285 // Output implements the EventSorter interface 286 func (s *UnifiedSorter) Output() <-chan *model.PolymorphicEvent { 287 return s.outputCh 288 } 289 290 // RunWorkerPool runs the worker pool used by the heapSorters 291 // It **must** be running for Unified Sorter to work. 292 func RunWorkerPool(ctx context.Context) error { 293 lazyInitWorkerPool() 294 errg, ctx := errgroup.WithContext(ctx) 295 errg.Go(func() error { 296 return errors.Trace(heapSorterPool.Run(ctx)) 297 }) 298 299 errg.Go(func() error { 300 return errors.Trace(heapSorterIOPool.Run(ctx)) 301 }) 302 303 return errors.Trace(errg.Wait()) 304 }