github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/mq.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 "net/url" 19 "strconv" 20 "strings" 21 "sync/atomic" 22 "time" 23 24 "github.com/pingcap/errors" 25 "github.com/pingcap/log" 26 "github.com/pingcap/ticdc/cdc/model" 27 "github.com/pingcap/ticdc/cdc/sink/codec" 28 "github.com/pingcap/ticdc/cdc/sink/dispatcher" 29 "github.com/pingcap/ticdc/cdc/sink/producer" 30 "github.com/pingcap/ticdc/cdc/sink/producer/kafka" 31 "github.com/pingcap/ticdc/cdc/sink/producer/pulsar" 32 "github.com/pingcap/ticdc/pkg/config" 33 cerror "github.com/pingcap/ticdc/pkg/errors" 34 "github.com/pingcap/ticdc/pkg/filter" 35 "github.com/pingcap/ticdc/pkg/notify" 36 "github.com/pingcap/ticdc/pkg/security" 37 "github.com/pingcap/ticdc/pkg/util" 38 "go.uber.org/zap" 39 "golang.org/x/sync/errgroup" 40 ) 41 42 type mqSink struct { 43 mqProducer producer.Producer 44 dispatcher dispatcher.Dispatcher 45 newEncoder func() codec.EventBatchEncoder 46 filter *filter.Filter 47 protocol codec.Protocol 48 49 partitionNum int32 50 partitionInput []chan struct { 51 row *model.RowChangedEvent 52 resolvedTs uint64 53 } 54 partitionResolvedTs []uint64 55 checkpointTs uint64 56 resolvedNotifier *notify.Notifier 57 resolvedReceiver *notify.Receiver 58 59 statistics *Statistics 60 } 61 62 func newMqSink( 63 ctx context.Context, credential *security.Credential, mqProducer producer.Producer, 64 filter *filter.Filter, config *config.ReplicaConfig, opts map[string]string, errCh chan error, 65 ) (*mqSink, error) { 66 partitionNum := mqProducer.GetPartitionNum() 67 partitionInput := make([]chan struct { 68 row *model.RowChangedEvent 69 resolvedTs uint64 70 }, partitionNum) 71 for i := 0; i < int(partitionNum); i++ { 72 partitionInput[i] = make(chan struct { 73 row *model.RowChangedEvent 74 resolvedTs uint64 75 }, 12800) 76 } 77 d, err := dispatcher.NewDispatcher(config, mqProducer.GetPartitionNum()) 78 if err != nil { 79 return nil, errors.Trace(err) 80 } 81 notifier := new(notify.Notifier) 82 var protocol codec.Protocol 83 protocol.FromString(config.Sink.Protocol) 84 85 newEncoder := codec.NewEventBatchEncoder(protocol) 86 if protocol == codec.ProtocolAvro { 87 registryURI, ok := opts["registry"] 88 if !ok { 89 return nil, cerror.ErrPrepareAvroFailed.GenWithStack(`Avro protocol requires parameter "registry"`) 90 } 91 keySchemaManager, err := codec.NewAvroSchemaManager(ctx, credential, registryURI, "-key") 92 if err != nil { 93 return nil, errors.Annotate( 94 cerror.WrapError(cerror.ErrPrepareAvroFailed, err), 95 "Could not create Avro schema manager for message keys") 96 } 97 valueSchemaManager, err := codec.NewAvroSchemaManager(ctx, credential, registryURI, "-value") 98 if err != nil { 99 return nil, errors.Annotate( 100 cerror.WrapError(cerror.ErrPrepareAvroFailed, err), 101 "Could not create Avro schema manager for message values") 102 } 103 newEncoder1 := newEncoder 104 newEncoder = func() codec.EventBatchEncoder { 105 avroEncoder := newEncoder1().(*codec.AvroEventBatchEncoder) 106 avroEncoder.SetKeySchemaManager(keySchemaManager) 107 avroEncoder.SetValueSchemaManager(valueSchemaManager) 108 avroEncoder.SetTimeZone(util.TimezoneFromCtx(ctx)) 109 return avroEncoder 110 } 111 } else if (protocol == codec.ProtocolCanal || protocol == codec.ProtocolCanalJSON) && !config.EnableOldValue { 112 log.Error("Old value is not enabled when using Canal protocol. Please update changefeed config") 113 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, errors.New("Canal requires old value to be enabled")) 114 } 115 116 // pre-flight verification of encoder parameters 117 if err := newEncoder().SetParams(opts); err != nil { 118 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 119 } 120 121 newEncoder1 := newEncoder 122 newEncoder = func() codec.EventBatchEncoder { 123 ret := newEncoder1() 124 err := ret.SetParams(opts) 125 if err != nil { 126 log.Panic("MQ Encoder could not parse parameters", zap.Error(err)) 127 } 128 return ret 129 } 130 131 resolvedReceiver, err := notifier.NewReceiver(50 * time.Millisecond) 132 if err != nil { 133 return nil, err 134 } 135 k := &mqSink{ 136 mqProducer: mqProducer, 137 dispatcher: d, 138 newEncoder: newEncoder, 139 filter: filter, 140 protocol: protocol, 141 142 partitionNum: partitionNum, 143 partitionInput: partitionInput, 144 partitionResolvedTs: make([]uint64, partitionNum), 145 resolvedNotifier: notifier, 146 resolvedReceiver: resolvedReceiver, 147 148 statistics: NewStatistics(ctx, "MQ", opts), 149 } 150 151 go func() { 152 if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { 153 select { 154 case <-ctx.Done(): 155 return 156 case errCh <- err: 157 default: 158 log.Error("error channel is full", zap.Error(err)) 159 } 160 } 161 }() 162 return k, nil 163 } 164 165 func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { 166 rowsCount := 0 167 for _, row := range rows { 168 if k.filter.ShouldIgnoreDMLEvent(row.StartTs, row.Table.Schema, row.Table.Table) { 169 log.Info("Row changed event ignored", zap.Uint64("start-ts", row.StartTs)) 170 continue 171 } 172 partition := k.dispatcher.Dispatch(row) 173 select { 174 case <-ctx.Done(): 175 return ctx.Err() 176 case k.partitionInput[partition] <- struct { 177 row *model.RowChangedEvent 178 resolvedTs uint64 179 }{row: row}: 180 } 181 rowsCount++ 182 } 183 k.statistics.AddRowsCount(rowsCount) 184 return nil 185 } 186 187 func (k *mqSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { 188 if resolvedTs <= k.checkpointTs { 189 return k.checkpointTs, nil 190 } 191 192 for i := 0; i < int(k.partitionNum); i++ { 193 select { 194 case <-ctx.Done(): 195 return 0, ctx.Err() 196 case k.partitionInput[i] <- struct { 197 row *model.RowChangedEvent 198 resolvedTs uint64 199 }{resolvedTs: resolvedTs}: 200 } 201 } 202 203 // waiting for all row events are sent to mq producer 204 flushLoop: 205 for { 206 select { 207 case <-ctx.Done(): 208 return 0, ctx.Err() 209 case <-k.resolvedReceiver.C: 210 for i := 0; i < int(k.partitionNum); i++ { 211 if resolvedTs > atomic.LoadUint64(&k.partitionResolvedTs[i]) { 212 continue flushLoop 213 } 214 } 215 break flushLoop 216 } 217 } 218 err := k.mqProducer.Flush(ctx) 219 if err != nil { 220 return 0, errors.Trace(err) 221 } 222 k.checkpointTs = resolvedTs 223 k.statistics.PrintStatus(ctx) 224 return k.checkpointTs, nil 225 } 226 227 func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { 228 encoder := k.newEncoder() 229 msg, err := encoder.EncodeCheckpointEvent(ts) 230 if err != nil { 231 return errors.Trace(err) 232 } 233 if msg == nil { 234 return nil 235 } 236 err = k.writeToProducer(ctx, msg, codec.EncoderNeedSyncWrite, -1) 237 return errors.Trace(err) 238 } 239 240 func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { 241 if k.filter.ShouldIgnoreDDLEvent(ddl.StartTs, ddl.Type, ddl.TableInfo.Schema, ddl.TableInfo.Table) { 242 log.Info( 243 "DDL event ignored", 244 zap.String("query", ddl.Query), 245 zap.Uint64("startTs", ddl.StartTs), 246 zap.Uint64("commitTs", ddl.CommitTs), 247 ) 248 return cerror.ErrDDLEventIgnored.GenWithStackByArgs() 249 } 250 encoder := k.newEncoder() 251 msg, err := encoder.EncodeDDLEvent(ddl) 252 if err != nil { 253 return errors.Trace(err) 254 } 255 256 if msg == nil { 257 return nil 258 } 259 260 k.statistics.AddDDLCount() 261 log.Debug("emit ddl event", zap.String("query", ddl.Query), zap.Uint64("commit-ts", ddl.CommitTs)) 262 err = k.writeToProducer(ctx, msg, codec.EncoderNeedSyncWrite, -1) 263 return errors.Trace(err) 264 } 265 266 // Initialize registers Avro schemas for all tables 267 func (k *mqSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { 268 // No longer need it for now 269 return nil 270 } 271 272 func (k *mqSink) Close(ctx context.Context) error { 273 err := k.mqProducer.Close() 274 return errors.Trace(err) 275 } 276 277 func (k *mqSink) Barrier(cxt context.Context) error { 278 // Barrier does nothing because FlushRowChangedEvents in mq sink has flushed 279 // all buffered events forcedlly. 280 return nil 281 } 282 283 func (k *mqSink) run(ctx context.Context) error { 284 defer k.resolvedReceiver.Stop() 285 wg, ctx := errgroup.WithContext(ctx) 286 for i := int32(0); i < k.partitionNum; i++ { 287 partition := i 288 wg.Go(func() error { 289 return k.runWorker(ctx, partition) 290 }) 291 } 292 return wg.Wait() 293 } 294 295 const batchSizeLimit = 4 * 1024 * 1024 // 4MB 296 297 func (k *mqSink) runWorker(ctx context.Context, partition int32) error { 298 input := k.partitionInput[partition] 299 encoder := k.newEncoder() 300 tick := time.NewTicker(500 * time.Millisecond) 301 defer tick.Stop() 302 303 flushToProducer := func(op codec.EncoderResult) error { 304 return k.statistics.RecordBatchExecution(func() (int, error) { 305 messages := encoder.Build() 306 thisBatchSize := len(messages) 307 if thisBatchSize == 0 { 308 return 0, nil 309 } 310 311 for _, msg := range messages { 312 err := k.writeToProducer(ctx, msg, codec.EncoderNeedAsyncWrite, partition) 313 if err != nil { 314 return 0, err 315 } 316 } 317 318 if op == codec.EncoderNeedSyncWrite { 319 err := k.mqProducer.Flush(ctx) 320 if err != nil { 321 return 0, err 322 } 323 } 324 log.Debug("MQSink flushed", zap.Int("thisBatchSize", thisBatchSize)) 325 return thisBatchSize, nil 326 }) 327 } 328 for { 329 var e struct { 330 row *model.RowChangedEvent 331 resolvedTs uint64 332 } 333 select { 334 case <-ctx.Done(): 335 return ctx.Err() 336 case <-tick.C: 337 if err := flushToProducer(codec.EncoderNeedAsyncWrite); err != nil { 338 return errors.Trace(err) 339 } 340 continue 341 case e = <-input: 342 } 343 if e.row == nil { 344 if e.resolvedTs != 0 { 345 op, err := encoder.AppendResolvedEvent(e.resolvedTs) 346 if err != nil { 347 return errors.Trace(err) 348 } 349 350 if err := flushToProducer(op); err != nil { 351 return errors.Trace(err) 352 } 353 354 atomic.StoreUint64(&k.partitionResolvedTs[partition], e.resolvedTs) 355 k.resolvedNotifier.Notify() 356 } 357 continue 358 } 359 op, err := encoder.AppendRowChangedEvent(e.row) 360 if err != nil { 361 return errors.Trace(err) 362 } 363 364 if encoder.Size() >= batchSizeLimit { 365 op = codec.EncoderNeedAsyncWrite 366 } 367 368 if encoder.Size() >= batchSizeLimit || op != codec.EncoderNoOperation { 369 if err := flushToProducer(op); err != nil { 370 return errors.Trace(err) 371 } 372 } 373 } 374 } 375 376 func (k *mqSink) writeToProducer(ctx context.Context, message *codec.MQMessage, op codec.EncoderResult, partition int32) error { 377 switch op { 378 case codec.EncoderNeedAsyncWrite: 379 if partition >= 0 { 380 return k.mqProducer.SendMessage(ctx, message, partition) 381 } 382 return cerror.ErrAsyncBroadcastNotSupport.GenWithStackByArgs() 383 case codec.EncoderNeedSyncWrite: 384 if partition >= 0 { 385 err := k.mqProducer.SendMessage(ctx, message, partition) 386 if err != nil { 387 return err 388 } 389 return k.mqProducer.Flush(ctx) 390 } 391 return k.mqProducer.SyncBroadcastMessage(ctx, message) 392 } 393 394 log.Warn("writeToProducer called with no-op", 395 zap.ByteString("key", message.Key), 396 zap.ByteString("value", message.Value), 397 zap.Int32("partition", partition)) 398 return nil 399 } 400 401 func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { 402 config := kafka.NewKafkaConfig() 403 404 scheme := strings.ToLower(sinkURI.Scheme) 405 if scheme != "kafka" && scheme != "kafka+ssl" { 406 return nil, cerror.ErrKafkaInvalidConfig.GenWithStack("can't create MQ sink with unsupported scheme: %s", scheme) 407 } 408 s := sinkURI.Query().Get("partition-num") 409 if s != "" { 410 c, err := strconv.Atoi(s) 411 if err != nil { 412 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 413 } 414 config.PartitionNum = int32(c) 415 } 416 417 s = sinkURI.Query().Get("replication-factor") 418 if s != "" { 419 c, err := strconv.Atoi(s) 420 if err != nil { 421 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 422 } 423 config.ReplicationFactor = int16(c) 424 } 425 426 s = sinkURI.Query().Get("kafka-version") 427 if s != "" { 428 config.Version = s 429 } 430 431 s = sinkURI.Query().Get("max-message-bytes") 432 if s != "" { 433 c, err := strconv.Atoi(s) 434 if err != nil { 435 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 436 } 437 config.MaxMessageBytes = c 438 opts["max-message-bytes"] = s 439 } 440 441 s = sinkURI.Query().Get("max-batch-size") 442 if s != "" { 443 opts["max-batch-size"] = s 444 } 445 446 s = sinkURI.Query().Get("compression") 447 if s != "" { 448 config.Compression = s 449 } 450 451 config.ClientID = sinkURI.Query().Get("kafka-client-id") 452 453 s = sinkURI.Query().Get("protocol") 454 if s != "" { 455 replicaConfig.Sink.Protocol = s 456 } 457 458 s = sinkURI.Query().Get("ca") 459 if s != "" { 460 config.Credential.CAPath = s 461 } 462 463 s = sinkURI.Query().Get("cert") 464 if s != "" { 465 config.Credential.CertPath = s 466 } 467 468 s = sinkURI.Query().Get("key") 469 if s != "" { 470 config.Credential.KeyPath = s 471 } 472 473 s = sinkURI.Query().Get("sasl-user") 474 if s != "" { 475 config.SaslScram.SaslUser = s 476 } 477 478 s = sinkURI.Query().Get("sasl-password") 479 if s != "" { 480 config.SaslScram.SaslPassword = s 481 } 482 483 s = sinkURI.Query().Get("sasl-mechanism") 484 if s != "" { 485 config.SaslScram.SaslMechanism = s 486 } 487 488 s = sinkURI.Query().Get("auto-create-topic") 489 if s != "" { 490 autoCreate, err := strconv.ParseBool(s) 491 if err != nil { 492 return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 493 } 494 config.TopicPreProcess = autoCreate 495 } 496 497 topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { 498 return r == '/' 499 }) 500 producer, err := kafka.NewKafkaSaramaProducer(ctx, sinkURI.Host, topic, config, errCh) 501 if err != nil { 502 return nil, errors.Trace(err) 503 } 504 sink, err := newMqSink(ctx, config.Credential, producer, filter, replicaConfig, opts, errCh) 505 if err != nil { 506 return nil, errors.Trace(err) 507 } 508 return sink, nil 509 } 510 511 func newPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, replicaConfig *config.ReplicaConfig, opts map[string]string, errCh chan error) (*mqSink, error) { 512 producer, err := pulsar.NewProducer(sinkURI, errCh) 513 if err != nil { 514 return nil, errors.Trace(err) 515 } 516 s := sinkURI.Query().Get("protocol") 517 if s != "" { 518 replicaConfig.Sink.Protocol = s 519 } 520 // These two options are not used by Pulsar producer itself, but the encoders 521 s = sinkURI.Query().Get("max-message-bytes") 522 if s != "" { 523 opts["max-message-bytes"] = s 524 } 525 526 s = sinkURI.Query().Get("max-batch-size") 527 if s != "" { 528 opts["max-batch-size"] = s 529 } 530 // For now, it's a place holder. Avro format have to make connection to Schema Registery, 531 // and it may needs credential. 532 credential := &security.Credential{} 533 sink, err := newMqSink(ctx, credential, producer, filter, replicaConfig, opts, errCh) 534 if err != nil { 535 return nil, errors.Trace(err) 536 } 537 return sink, nil 538 }