github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/pipeline/cyclic_mark.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 pipeline 15 16 import ( 17 "github.com/pingcap/errors" 18 "github.com/pingcap/log" 19 "github.com/pingcap/ticdc/cdc/entry" 20 "github.com/pingcap/ticdc/cdc/model" 21 "github.com/pingcap/ticdc/pkg/cyclic/mark" 22 "github.com/pingcap/ticdc/pkg/pipeline" 23 "go.uber.org/zap" 24 ) 25 26 // cyclicMarkNode match the mark rows and normal rows, set the ReplicaID of normal rows and filter the mark rows 27 // and filter the normal rows by the FilterReplicaID config item. 28 type cyclicMarkNode struct { 29 localReplicaID uint64 30 filterReplicaID map[uint64]struct{} 31 markTableID model.TableID 32 33 // startTs -> rows 34 rowsUnknownReplicaID map[model.Ts][]*model.PolymorphicEvent 35 // startTs -> replicaID 36 currentReplicaIDs map[model.Ts]uint64 37 currentCommitTs uint64 38 } 39 40 func newCyclicMarkNode(markTableID model.TableID) pipeline.Node { 41 return &cyclicMarkNode{ 42 markTableID: markTableID, 43 rowsUnknownReplicaID: make(map[model.Ts][]*model.PolymorphicEvent), 44 currentReplicaIDs: make(map[model.Ts]uint64), 45 } 46 } 47 48 func (n *cyclicMarkNode) Init(ctx pipeline.NodeContext) error { 49 n.localReplicaID = ctx.ChangefeedVars().Info.Config.Cyclic.ReplicaID 50 filterReplicaID := ctx.ChangefeedVars().Info.Config.Cyclic.FilterReplicaID 51 n.filterReplicaID = make(map[uint64]struct{}) 52 for _, rID := range filterReplicaID { 53 n.filterReplicaID[rID] = struct{}{} 54 } 55 // do nothing 56 return nil 57 } 58 59 // Receive receives the message from the previous node 60 // In the previous nodes(puller node and sorter node), 61 // the change logs of mark table and normal table are listen by one puller, 62 // and sorted by one sorter. So, this node will receive a commitTs-ordered stream which include the mark rows and normal rows. 63 // Under the above conditions, we need to cache at most one transaction's rows to matching rows. 64 // For every row event, Receive function flushes every the last transaction's rows, and adds the mark row or normal row into the cache. 65 func (n *cyclicMarkNode) Receive(ctx pipeline.NodeContext) error { 66 msg := ctx.Message() 67 switch msg.Tp { 68 case pipeline.MessageTypePolymorphicEvent: 69 event := msg.PolymorphicEvent 70 n.flush(ctx, event.CRTs) 71 if event.RawKV.OpType == model.OpTypeResolved { 72 ctx.SendToNextNode(msg) 73 return nil 74 } 75 tableID, err := entry.DecodeTableID(event.RawKV.Key) 76 if err != nil { 77 return errors.Trace(err) 78 } 79 if tableID == n.markTableID { 80 err := n.appendMarkRow(ctx, event) 81 if err != nil { 82 return errors.Trace(err) 83 } 84 } else { 85 n.appendNormalRow(ctx, event) 86 } 87 return nil 88 } 89 ctx.SendToNextNode(msg) 90 return nil 91 } 92 93 // appendNormalRow adds the normal row into the cache 94 func (n *cyclicMarkNode) appendNormalRow(ctx pipeline.NodeContext, event *model.PolymorphicEvent) { 95 if event.CRTs != n.currentCommitTs { 96 log.Panic("the CommitTs of the received event is not equal to the currentCommitTs, please report a bug", zap.Reflect("event", event), zap.Uint64("currentCommitTs", n.currentCommitTs)) 97 } 98 if replicaID, exist := n.currentReplicaIDs[event.StartTs]; exist { 99 // we already know the replicaID of this startTs, it means that the mark row of this startTs is already in cached. 100 event.ReplicaID = replicaID 101 n.sendNormalRowToNextNode(ctx, event.ReplicaID, event) 102 return 103 } 104 // for all normal rows which we don't know the replicaID for now. we cache them in rowsUnknownReplicaID. 105 n.rowsUnknownReplicaID[event.StartTs] = append(n.rowsUnknownReplicaID[event.StartTs], event) 106 } 107 108 // appendMarkRow adds the mark row into the cache 109 func (n *cyclicMarkNode) appendMarkRow(ctx pipeline.NodeContext, event *model.PolymorphicEvent) error { 110 if event.CRTs != n.currentCommitTs { 111 log.Panic("the CommitTs of the received event is not equal to the currentCommitTs, please report a bug", zap.Reflect("event", event), zap.Uint64("currentCommitTs", n.currentCommitTs)) 112 } 113 err := event.WaitPrepare(ctx) 114 if err != nil { 115 return errors.Trace(err) 116 } 117 markRow := event.Row 118 if markRow == nil { 119 return nil 120 } 121 replicaID := extractReplicaID(markRow) 122 // Establishing the mapping from StartTs to ReplicaID 123 n.currentReplicaIDs[markRow.StartTs] = replicaID 124 if rows, exist := n.rowsUnknownReplicaID[markRow.StartTs]; exist { 125 // the replicaID of these rows we did not know before, but now we know through received mark row now. 126 delete(n.rowsUnknownReplicaID, markRow.StartTs) 127 n.sendNormalRowToNextNode(ctx, replicaID, rows...) 128 } 129 return nil 130 } 131 132 func (n *cyclicMarkNode) flush(ctx pipeline.NodeContext, commitTs uint64) { 133 if n.currentCommitTs == commitTs { 134 return 135 } 136 // all mark rows and normal rows in current transaction is received now. 137 // there are still unmatched normal rows in the cache, their replicaID should be local replicaID. 138 for _, rows := range n.rowsUnknownReplicaID { 139 for _, row := range rows { 140 row.ReplicaID = n.localReplicaID 141 } 142 n.sendNormalRowToNextNode(ctx, n.localReplicaID, rows...) 143 } 144 if len(n.rowsUnknownReplicaID) != 0 { 145 n.rowsUnknownReplicaID = make(map[model.Ts][]*model.PolymorphicEvent) 146 } 147 if len(n.currentReplicaIDs) != 0 { 148 n.currentReplicaIDs = make(map[model.Ts]uint64) 149 } 150 n.currentCommitTs = commitTs 151 } 152 153 // sendNormalRowToNextNode filter the specified normal rows by the FilterReplicaID config item, and send rows to the next node. 154 func (n *cyclicMarkNode) sendNormalRowToNextNode(ctx pipeline.NodeContext, replicaID uint64, rows ...*model.PolymorphicEvent) { 155 if _, shouldFilter := n.filterReplicaID[replicaID]; shouldFilter { 156 return 157 } 158 for _, row := range rows { 159 row.ReplicaID = replicaID 160 ctx.SendToNextNode(pipeline.PolymorphicEventMessage(row)) 161 } 162 } 163 164 func (n *cyclicMarkNode) Destroy(ctx pipeline.NodeContext) error { 165 // do nothing 166 return nil 167 } 168 169 // extractReplicaID extracts replica ID from the given mark row. 170 func extractReplicaID(markRow *model.RowChangedEvent) uint64 { 171 for _, c := range markRow.Columns { 172 if c == nil { 173 continue 174 } 175 if c.Name == mark.CyclicReplicaIDCol { 176 return c.Value.(uint64) 177 } 178 } 179 log.Panic("bad mark table, " + mark.CyclicReplicaIDCol + " not found") 180 return 0 181 }