github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/simple_mysql_tester.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 sink 15 16 import ( 17 "context" 18 "database/sql" 19 "fmt" 20 "net/url" 21 "strings" 22 "sync" 23 24 dmysql "github.com/go-sql-driver/mysql" 25 "github.com/pingcap/errors" 26 "github.com/pingcap/failpoint" 27 "github.com/pingcap/log" 28 "github.com/pingcap/ticdc/cdc/model" 29 "github.com/pingcap/ticdc/pkg/config" 30 cerror "github.com/pingcap/ticdc/pkg/errors" 31 "github.com/pingcap/ticdc/pkg/filter" 32 "github.com/pingcap/ticdc/pkg/quotes" 33 "go.uber.org/zap" 34 ) 35 36 func init() { 37 failpoint.Inject("SimpleMySQLSinkTester", func() { 38 sinkIniterMap["simple-mysql"] = func(ctx context.Context, changefeedID model.ChangeFeedID, sinkURI *url.URL, 39 filter *filter.Filter, config *config.ReplicaConfig, opts map[string]string, errCh chan error) (Sink, error) { 40 return newSimpleMySQLSink(ctx, sinkURI, config) 41 } 42 }) 43 } 44 45 type simpleMySQLSink struct { 46 enableOldValue bool 47 enableCheckOldValue bool 48 db *sql.DB 49 rowsBuffer []*model.RowChangedEvent 50 rowsBufferLock sync.Mutex 51 } 52 53 func newSimpleMySQLSink(ctx context.Context, sinkURI *url.URL, config *config.ReplicaConfig) (*simpleMySQLSink, error) { 54 var db *sql.DB 55 56 // dsn format of the driver: 57 // [username[:password]@][protocol[(address)]]/dbname[?param1=value1&...¶mN=valueN] 58 username := sinkURI.User.Username() 59 password, _ := sinkURI.User.Password() 60 port := sinkURI.Port() 61 if username == "" { 62 username = "root" 63 } 64 if port == "" { 65 port = "4000" 66 } 67 68 dsnStr := fmt.Sprintf("%s:%s@tcp(%s:%s)/?multiStatements=true", username, password, sinkURI.Hostname(), port) 69 dsn, err := dmysql.ParseDSN(dsnStr) 70 if err != nil { 71 return nil, cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) 72 } 73 74 // create test db used for parameter detection 75 if dsn.Params == nil { 76 dsn.Params = make(map[string]string, 1) 77 } 78 testDB, err := sql.Open("mysql", dsn.FormatDSN()) 79 if err != nil { 80 return nil, errors.Annotate( 81 cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection when configuring sink") 82 } 83 defer testDB.Close() 84 85 db, err = sql.Open("mysql", dsnStr) 86 if err != nil { 87 return nil, errors.Annotate( 88 cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed") 89 } 90 err = db.PingContext(ctx) 91 if err != nil { 92 return nil, errors.Annotate( 93 cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") 94 } 95 96 sink := &simpleMySQLSink{ 97 db: db, 98 enableOldValue: config.EnableOldValue, 99 } 100 if strings.ToLower(sinkURI.Query().Get("check-old-value")) == "true" { 101 sink.enableCheckOldValue = true 102 log.Info("the old value checker is enabled") 103 } 104 return sink, nil 105 } 106 107 func (s *simpleMySQLSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { 108 // do nothing 109 return nil 110 } 111 112 // EmitRowChangedEvents sends Row Changed Event to Sink 113 // EmitRowChangedEvents may write rows to downstream directly; 114 func (s *simpleMySQLSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { 115 s.rowsBufferLock.Lock() 116 defer s.rowsBufferLock.Unlock() 117 s.rowsBuffer = append(s.rowsBuffer, rows...) 118 return nil 119 } 120 121 func (s *simpleMySQLSink) executeRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { 122 var sql string 123 var args []interface{} 124 if s.enableOldValue { 125 for _, row := range rows { 126 if len(row.PreColumns) != 0 && len(row.Columns) != 0 { 127 // update 128 if s.enableCheckOldValue { 129 err := s.checkOldValue(ctx, row) 130 if err != nil { 131 return errors.Trace(err) 132 } 133 } 134 sql, args = prepareReplace(row.Table.QuoteString(), row.Columns, true, false /* translateToInsert */) 135 } else if len(row.PreColumns) == 0 { 136 // insert 137 sql, args = prepareReplace(row.Table.QuoteString(), row.Columns, true, false /* translateToInsert */) 138 } else if len(row.Columns) == 0 { 139 // delete 140 if s.enableCheckOldValue { 141 err := s.checkOldValue(ctx, row) 142 if err != nil { 143 return errors.Trace(err) 144 } 145 } 146 sql, args = prepareDelete(row.Table.QuoteString(), row.PreColumns, true) 147 } 148 _, err := s.db.ExecContext(ctx, sql, args...) 149 if err != nil { 150 return errors.Trace(err) 151 } 152 } 153 } else { 154 for _, row := range rows { 155 if row.IsDelete() { 156 sql, args = prepareDelete(row.Table.QuoteString(), row.PreColumns, true) 157 } else { 158 sql, args = prepareReplace(row.Table.QuoteString(), row.Columns, true, false) 159 } 160 _, err := s.db.ExecContext(ctx, sql, args...) 161 if err != nil { 162 return errors.Trace(err) 163 } 164 } 165 } 166 return nil 167 } 168 169 // EmitDDLEvent sends DDL Event to Sink 170 // EmitDDLEvent should execute DDL to downstream synchronously 171 func (s *simpleMySQLSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { 172 var sql string 173 if len(ddl.TableInfo.Table) == 0 { 174 sql = ddl.Query 175 } else { 176 sql = fmt.Sprintf("use %s;%s", ddl.TableInfo.Schema, ddl.Query) 177 } 178 _, err := s.db.ExecContext(ctx, sql) 179 if err != nil && isIgnorableDDLError(err) { 180 log.Info("execute DDL failed, but error can be ignored", zap.String("query", ddl.Query), zap.Error(err)) 181 return nil 182 } 183 return err 184 } 185 186 // FlushRowChangedEvents flushes each row which of commitTs less than or equal to `resolvedTs` into downstream. 187 // TiCDC guarantees that all of Event which of commitTs less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` 188 func (s *simpleMySQLSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { 189 s.rowsBufferLock.Lock() 190 defer s.rowsBufferLock.Unlock() 191 newBuffer := make([]*model.RowChangedEvent, 0, len(s.rowsBuffer)) 192 for _, row := range s.rowsBuffer { 193 if row.CommitTs <= resolvedTs { 194 err := s.executeRowChangedEvents(ctx, row) 195 if err != nil { 196 return 0, err 197 } 198 } else { 199 newBuffer = append(newBuffer, row) 200 } 201 } 202 s.rowsBuffer = newBuffer 203 return resolvedTs, nil 204 } 205 206 // EmitCheckpointTs sends CheckpointTs to Sink 207 // TiCDC guarantees that all Events **in the cluster** which of commitTs less than or equal `checkpointTs` are sent to downstream successfully. 208 func (s *simpleMySQLSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { 209 // do nothing 210 return nil 211 } 212 213 // Close closes the Sink 214 func (s *simpleMySQLSink) Close(ctx context.Context) error { 215 return s.db.Close() 216 } 217 218 func (s *simpleMySQLSink) Barrier(ctx context.Context) error { 219 return nil 220 } 221 222 func prepareCheckSQL(quoteTable string, cols []*model.Column) (string, []interface{}) { 223 var builder strings.Builder 224 builder.WriteString("SELECT count(1) FROM " + quoteTable + " WHERE ") 225 226 colNames, wargs := whereSlice(cols, true) 227 if len(wargs) == 0 { 228 return "", nil 229 } 230 args := make([]interface{}, 0, len(wargs)) 231 for i := 0; i < len(colNames); i++ { 232 if i > 0 { 233 builder.WriteString(" AND ") 234 } 235 if wargs[i] == nil { 236 builder.WriteString(quotes.QuoteName(colNames[i]) + " IS NULL") 237 } else { 238 builder.WriteString(quotes.QuoteName(colNames[i]) + " = ?") 239 args = append(args, wargs[i]) 240 } 241 } 242 builder.WriteString(" LIMIT 1;") 243 sql := builder.String() 244 return sql, args 245 } 246 247 func (s *simpleMySQLSink) checkOldValue(ctx context.Context, row *model.RowChangedEvent) error { 248 sql, args := prepareCheckSQL(row.Table.QuoteString(), row.PreColumns) 249 result, err := s.db.QueryContext(ctx, sql, args...) 250 if err != nil { 251 return errors.Trace(err) 252 } 253 var count int 254 if result.Next() { 255 err := result.Scan(&count) 256 if err != nil { 257 return errors.Trace(err) 258 } 259 } 260 if count == 0 { 261 log.Error("can't pass the check, the old value of this row is not exist", zap.Any("row", row)) 262 return errors.New("check failed") 263 } 264 log.Debug("pass the old value check", zap.String("sql", sql), zap.Any("args", args), zap.Int("count", count)) 265 return nil 266 }