github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/task.go (about) 1 // Copyright 2020 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 cdc 15 16 import ( 17 "context" 18 19 "github.com/pingcap/failpoint" 20 "github.com/pingcap/log" 21 "github.com/pingcap/ticdc/cdc/model" 22 cerror "github.com/pingcap/ticdc/pkg/errors" 23 "go.etcd.io/etcd/clientv3" 24 "go.etcd.io/etcd/mvcc" 25 "go.uber.org/zap" 26 ) 27 28 // TaskEventOp is the operation of a task 29 type TaskEventOp string 30 31 // Task Event Operatrions 32 const ( 33 TaskOpCreate TaskEventOp = "create" 34 TaskOpDelete TaskEventOp = "delete" 35 ) 36 37 // Task is dispatched by the owner 38 type Task struct { 39 ChangeFeedID string 40 CheckpointTS uint64 41 } 42 43 // TaskEvent represents a task is created or deleted 44 type TaskEvent struct { 45 Op TaskEventOp 46 Task *Task 47 Err error 48 } 49 50 // TaskWatcher watches on new tasks 51 type TaskWatcher struct { 52 capture *Capture 53 cfg *TaskWatcherConfig 54 55 events map[string]*TaskEvent 56 } 57 58 // TaskWatcherConfig configures a watcher 59 type TaskWatcherConfig struct { 60 Prefix string 61 ChannelSize int64 62 } 63 64 // NewTaskWatcher returns a TaskWatcher 65 func NewTaskWatcher(c *Capture, cfg *TaskWatcherConfig) *TaskWatcher { 66 return &TaskWatcher{capture: c, cfg: cfg} 67 } 68 69 // Watch on the new tasks, a channel is returned 70 func (w *TaskWatcher) Watch(ctx context.Context) <-chan *TaskEvent { 71 c := make(chan *TaskEvent, w.cfg.ChannelSize) 72 go w.watch(ctx, c) 73 return c 74 } 75 76 func (w *TaskWatcher) watch(ctx context.Context, c chan *TaskEvent) { 77 etcd := w.capture.etcdClient.Client 78 79 // Leader is required in this context to prevent read outdated data 80 // from a stale leader 81 ctx = clientv3.WithRequireLeader(ctx) 82 83 // Send a task event to the channel, checks ctx.Done() to avoid blocking 84 send := func(ctx context.Context, ev *TaskEvent) error { 85 select { 86 case <-ctx.Done(): 87 close(c) 88 return ctx.Err() 89 case c <- ev: 90 } 91 return nil 92 } 93 restart: 94 // Load all the existed tasks 95 events := make(map[string]*TaskEvent) 96 resp, err := etcd.Get(ctx, w.cfg.Prefix, clientv3.WithPrefix()) 97 if err != nil { 98 _ = send(ctx, &TaskEvent{Err: err}) 99 return 100 } 101 for _, kv := range resp.Kvs { 102 ev, err := w.parseTaskEvent(ctx, kv.Key, kv.Value) 103 if err != nil { 104 log.Warn("parse task event failed", 105 zap.String("capture-id", w.capture.info.ID), 106 zap.Error(err)) 107 continue 108 } 109 events[ev.Task.ChangeFeedID] = ev 110 } 111 112 // Rebuild the missed events 113 // When an error is occured during watch, the watch routine is restarted, 114 // in that case, some events maybe missed. Rebuild the events by comparing 115 // the new task list with the last successfully recorded tasks. 116 events = w.rebuildTaskEvents(events) 117 for _, ev := range events { 118 if err := send(ctx, ev); err != nil { 119 return 120 } 121 } 122 123 wch := etcd.Watch(ctx, w.cfg.Prefix, 124 clientv3.WithPrefix(), 125 clientv3.WithPrevKV(), 126 clientv3.WithRev(resp.Header.Revision+1)) 127 for wresp := range wch { 128 err := wresp.Err() 129 failpoint.Inject("restart-task-watch", func() { 130 err = mvcc.ErrCompacted 131 }) 132 if err != nil { 133 goto restart 134 } 135 for _, ev := range wresp.Events { 136 if ev.Type == clientv3.EventTypePut { 137 ev, err := w.parseTaskEvent(ctx, ev.Kv.Key, ev.Kv.Value) 138 if err != nil { 139 log.Warn("parse task event failed", 140 zap.String("capture-id", w.capture.info.ID), 141 zap.Error(err)) 142 continue 143 } 144 w.events[ev.Task.ChangeFeedID] = ev 145 if err := send(ctx, ev); err != nil { 146 return 147 } 148 } else if ev.Type == clientv3.EventTypeDelete { 149 task, err := w.parseTask(ctx, ev.PrevKv.Key) 150 if err != nil { 151 log.Warn("parse task failed", 152 zap.String("capture-id", w.capture.info.ID), 153 zap.Error(err)) 154 continue 155 } 156 delete(w.events, task.ChangeFeedID) 157 if err := send(ctx, &TaskEvent{Op: TaskOpDelete, Task: task}); err != nil { 158 return 159 } 160 } 161 } 162 } 163 close(c) 164 } 165 166 func (w *TaskWatcher) parseTask(ctx context.Context, 167 key []byte) (*Task, error) { 168 if len(key) <= len(w.cfg.Prefix) { 169 return nil, cerror.ErrInvalidTaskKey.GenWithStackByArgs(string(key)) 170 } 171 changeFeedID := string(key[len(w.cfg.Prefix)+1:]) 172 cf, err := w.capture.etcdClient.GetChangeFeedInfo(ctx, changeFeedID) 173 if err != nil { 174 return nil, err 175 } 176 status, _, err := w.capture.etcdClient.GetChangeFeedStatus(ctx, changeFeedID) 177 if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { 178 return nil, err 179 } 180 checkpointTs := cf.GetCheckpointTs(status) 181 return &Task{ChangeFeedID: changeFeedID, CheckpointTS: checkpointTs}, nil 182 } 183 184 func (w *TaskWatcher) parseTaskEvent(ctx context.Context, key, val []byte) (*TaskEvent, error) { 185 task, err := w.parseTask(ctx, key) 186 if err != nil { 187 log.Warn("parse task failed", 188 zap.String("capture-id", w.capture.info.ID), 189 zap.Error(err)) 190 return nil, err 191 } 192 193 taskStatus := &model.TaskStatus{} 194 if err := taskStatus.Unmarshal(val); err != nil { 195 log.Warn("unmarshal task status failed", 196 zap.String("capture-id", w.capture.info.ID), 197 zap.Error(err)) 198 return nil, err 199 } 200 var op TaskEventOp 201 switch taskStatus.AdminJobType { 202 case model.AdminNone, model.AdminResume: 203 op = TaskOpCreate 204 case model.AdminStop, model.AdminRemove, model.AdminFinish: 205 op = TaskOpDelete 206 } 207 return &TaskEvent{Op: op, Task: task}, nil 208 } 209 210 func (w *TaskWatcher) rebuildTaskEvents(latest map[string]*TaskEvent) map[string]*TaskEvent { 211 events := make(map[string]*TaskEvent) 212 outdated := w.events 213 for id, ev := range outdated { 214 // Check if the task still exists 215 if nev, ok := latest[id]; ok { 216 if ev.Op != nev.Op { 217 events[id] = nev 218 } 219 } else if ev.Op != TaskOpDelete { 220 events[id] = &TaskEvent{Op: TaskOpDelete, Task: ev.Task} 221 } 222 } 223 224 for id, ev := range latest { 225 if _, ok := outdated[id]; !ok { 226 events[id] = ev 227 } 228 } 229 230 // Update to the latest tasks 231 w.events = events 232 233 return events 234 }