github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/canal_flat.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 codec 15 16 import ( 17 "encoding/json" 18 19 "github.com/pingcap/errors" 20 "github.com/pingcap/log" 21 "github.com/pingcap/ticdc/cdc/model" 22 cerrors "github.com/pingcap/ticdc/pkg/errors" 23 canal "github.com/pingcap/ticdc/proto/canal" 24 "go.uber.org/zap" 25 ) 26 27 // CanalFlatEventBatchEncoder encodes Canal flat messages in JSON format 28 type CanalFlatEventBatchEncoder struct { 29 builder *canalEntryBuilder 30 unresolvedBuf []*canalFlatMessage 31 resolvedBuf []*canalFlatMessage 32 } 33 34 // NewCanalFlatEventBatchEncoder creates a new CanalFlatEventBatchEncoder 35 func NewCanalFlatEventBatchEncoder() EventBatchEncoder { 36 return &CanalFlatEventBatchEncoder{ 37 builder: NewCanalEntryBuilder(), 38 unresolvedBuf: make([]*canalFlatMessage, 0), 39 resolvedBuf: make([]*canalFlatMessage, 0), 40 } 41 } 42 43 // adapted from https://github.com/alibaba/canal/blob/master/protocol/src/main/java/com/alibaba/otter/canal/protocol/FlatMessage.java 44 type canalFlatMessage struct { 45 // ignored by consumers 46 ID int64 `json:"id"` 47 Schema string `json:"database"` 48 Table string `json:"table"` 49 PKNames []string `json:"pkNames"` 50 IsDDL bool `json:"isDdl"` 51 EventType string `json:"type"` 52 ExecutionTime int64 `json:"es"` 53 // officially the time of building the MQ message, actually ignored 54 BuildTime uint64 `json:"ts"` 55 Query string `json:"sql"` 56 SQLType map[string]int32 `json:"sqlType"` 57 MySQLType map[string]string `json:"mysqlType"` 58 // A Datum should be a string or nil 59 Data []map[string]interface{} `json:"data"` 60 Old []map[string]interface{} `json:"old"` 61 // Used internally by CanalFlatEventBatchEncoder 62 tikvTs uint64 63 } 64 65 func (c *CanalFlatEventBatchEncoder) newFlatMessageForDML(e *model.RowChangedEvent) (*canalFlatMessage, error) { 66 eventType := convertRowEventType(e) 67 header := c.builder.buildHeader(e.CommitTs, e.Table.Schema, e.Table.Table, eventType, 1) 68 rowData, err := c.builder.buildRowData(e) 69 if err != nil { 70 return nil, cerrors.WrapError(cerrors.ErrCanalEncodeFailed, err) 71 } 72 73 pkCols := e.PrimaryKeyColumns() 74 pkNames := make([]string, len(pkCols)) 75 for i := range pkNames { 76 pkNames[i] = pkCols[i].Name 77 } 78 79 var nonTrivialRow []*canal.Column 80 if e.IsDelete() { 81 nonTrivialRow = rowData.BeforeColumns 82 } else { 83 nonTrivialRow = rowData.AfterColumns 84 } 85 86 sqlType := make(map[string]int32, len(nonTrivialRow)) 87 mysqlType := make(map[string]string, len(nonTrivialRow)) 88 for i := range nonTrivialRow { 89 sqlType[nonTrivialRow[i].Name] = nonTrivialRow[i].SqlType 90 mysqlType[nonTrivialRow[i].Name] = nonTrivialRow[i].MysqlType 91 } 92 93 var ( 94 data map[string]interface{} 95 oldData map[string]interface{} 96 ) 97 98 if len(rowData.BeforeColumns) > 0 { 99 oldData = make(map[string]interface{}, len(rowData.BeforeColumns)) 100 for i := range rowData.BeforeColumns { 101 if !rowData.BeforeColumns[i].GetIsNull() { 102 oldData[rowData.BeforeColumns[i].Name] = rowData.BeforeColumns[i].Value 103 } else { 104 oldData[rowData.BeforeColumns[i].Name] = nil 105 } 106 } 107 } 108 109 if len(rowData.AfterColumns) > 0 { 110 data = make(map[string]interface{}, len(rowData.AfterColumns)) 111 for i := range rowData.AfterColumns { 112 if !rowData.AfterColumns[i].GetIsNull() { 113 data[rowData.AfterColumns[i].Name] = rowData.AfterColumns[i].Value 114 } else { 115 data[rowData.AfterColumns[i].Name] = nil 116 } 117 } 118 } else { 119 // The event type is DELETE 120 // The following line is important because Alibaba's adapter expects this, and so does Flink. 121 data = oldData 122 } 123 124 ret := &canalFlatMessage{ 125 ID: 0, // ignored by both Canal Adapter and Flink 126 Schema: header.SchemaName, 127 Table: header.TableName, 128 PKNames: pkNames, 129 IsDDL: false, 130 EventType: header.GetEventType().String(), 131 ExecutionTime: header.ExecuteTime, 132 BuildTime: 0, // ignored by both Canal Adapter and Flink 133 Query: "", 134 SQLType: sqlType, 135 MySQLType: mysqlType, 136 Data: make([]map[string]interface{}, 0), 137 Old: make([]map[string]interface{}, 0), 138 tikvTs: e.CommitTs, 139 } 140 141 // We need to ensure that both Data and Old have exactly one element, 142 // even if the element could be nil. Changing this could break Alibaba's adapter 143 ret.Data = append(ret.Data, data) 144 ret.Old = append(ret.Old, oldData) 145 146 return ret, nil 147 } 148 149 func (c *CanalFlatEventBatchEncoder) newFlatMessageForDDL(e *model.DDLEvent) *canalFlatMessage { 150 header := c.builder.buildHeader(e.CommitTs, e.TableInfo.Schema, e.TableInfo.Table, convertDdlEventType(e), 1) 151 152 ret := &canalFlatMessage{ 153 ID: 0, // ignored by both Canal Adapter and Flink 154 Schema: header.SchemaName, 155 Table: header.TableName, 156 IsDDL: true, 157 EventType: header.GetEventType().String(), 158 ExecutionTime: header.ExecuteTime, 159 BuildTime: 0, // ignored by both Canal Adapter and Flink 160 Query: e.Query, 161 tikvTs: e.CommitTs, 162 } 163 return ret 164 } 165 166 // EncodeCheckpointEvent is no-op 167 func (c *CanalFlatEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { 168 return nil, nil 169 } 170 171 // AppendRowChangedEvent implements the interface EventBatchEncoder 172 func (c *CanalFlatEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { 173 msg, err := c.newFlatMessageForDML(e) 174 if err != nil { 175 return EncoderNoOperation, errors.Trace(err) 176 } 177 c.unresolvedBuf = append(c.unresolvedBuf, msg) 178 return EncoderNoOperation, nil 179 } 180 181 // AppendResolvedEvent receives the latest resolvedTs 182 func (c *CanalFlatEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { 183 nextIdx := 0 184 for _, msg := range c.unresolvedBuf { 185 if msg.tikvTs <= ts { 186 c.resolvedBuf = append(c.resolvedBuf, msg) 187 } else { 188 break 189 } 190 nextIdx++ 191 } 192 c.unresolvedBuf = c.unresolvedBuf[nextIdx:] 193 if len(c.resolvedBuf) > 0 { 194 return EncoderNeedAsyncWrite, nil 195 } 196 return EncoderNoOperation, nil 197 } 198 199 // EncodeDDLEvent encodes DDL events 200 func (c *CanalFlatEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) { 201 msg := c.newFlatMessageForDDL(e) 202 value, err := json.Marshal(msg) 203 if err != nil { 204 return nil, cerrors.WrapError(cerrors.ErrCanalEncodeFailed, err) 205 } 206 return newDDLMQMessage(ProtocolCanalJSON, nil, value, e), nil 207 } 208 209 // Build implements the EventBatchEncoder interface 210 func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { 211 if len(c.resolvedBuf) == 0 { 212 return nil 213 } 214 ret := make([]*MQMessage, len(c.resolvedBuf)) 215 for i, msg := range c.resolvedBuf { 216 value, err := json.Marshal(msg) 217 if err != nil { 218 log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) 219 return nil 220 } 221 ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) 222 } 223 c.resolvedBuf = c.resolvedBuf[0:0] 224 return ret 225 } 226 227 // MixedBuild is not used here 228 func (c *CanalFlatEventBatchEncoder) MixedBuild(withVersion bool) []byte { 229 panic("MixedBuild not supported by CanalFlatEventBatchEncoder") 230 } 231 232 // Size implements the EventBatchEncoder interface 233 func (c *CanalFlatEventBatchEncoder) Size() int { 234 return -1 235 } 236 237 // Reset is only supported by JSONEventBatchEncoder 238 func (c *CanalFlatEventBatchEncoder) Reset() { 239 panic("not supported") 240 } 241 242 // SetParams is no-op for now 243 func (c *CanalFlatEventBatchEncoder) SetParams(params map[string]string) error { 244 // no op 245 return nil 246 }