From e01852e4996e2a7cde4be2e8a9ffeb911b500c2b Mon Sep 17 00:00:00 2001 From: CharlesCheung <61726649+CharlesCheung96@users.noreply.github.com> Date: Wed, 11 Sep 2024 16:59:33 +0800 Subject: [PATCH] Revert "sink : downstream support kafka sink (#274)" This reverts commit c27cb1a142f6922147cbcd96304187189f0fd91c. --- .../event_dispatcher_manager.go | 8 +- .../sink/helper/eventrouter/event_router.go | 42 +- .../topicmanager/kafka_topic_manager.go | 7 +- downstreamadapter/sink/kafka_sink.go | 51 +- downstreamadapter/sink/sink.go | 29 - downstreamadapter/worker/kafka_ddl_worker.go | 157 --- downstreamadapter/worker/kafka_worker.go | 14 +- downstreamadapter/writer/config.go | 12 +- maintainer/maintainer.go | 4 +- maintainer/maintainer_manager.go | 3 +- maintainer/maintainer_test.go | 6 +- pkg/common/column_selector.go | 12 +- pkg/common/event.go | 1 + pkg/config/changefeed.go | 38 +- pkg/config/replica_config.go | 363 ------ pkg/config/sink_config.go | 1010 ----------------- pkg/sink/codec/avro/arvo.go | 2 +- pkg/sink/codec/bootstraper.go | 4 +- pkg/sink/codec/canal/canal_encoder.go | 2 +- .../canal/canal_json_row_event_encoder.go | 2 +- pkg/sink/codec/common/config.go | 79 +- pkg/sink/codec/craft/craft_encoder.go | 4 +- pkg/sink/codec/debezium/encoder.go | 2 +- pkg/sink/codec/encoder/encoder.go | 4 +- pkg/sink/codec/encoder_builder.go | 4 +- pkg/sink/codec/encoder_group.go | 14 +- pkg/sink/codec/open/codec.go | 5 +- pkg/sink/codec/open/codec_test.go | 20 +- pkg/sink/codec/open/encoder.go | 5 +- pkg/sink/codec/open/encoder_test.go | 12 +- pkg/sink/codec/simple/encoder.go | 2 +- pkg/sink/kafka/admin.go | 186 --- pkg/sink/kafka/factory.go | 271 ----- pkg/sink/kafka/oauth2_token_provider.go | 90 -- pkg/sink/kafka/options.go | 552 --------- pkg/sink/kafka/sarama.go | 254 ----- pkg/sink/kafka/sarama_factory.go | 145 --- pkg/sink/kafka/v2/admin.go | 268 ----- pkg/sink/kafka/v2/client.go | 33 - pkg/sink/kafka/v2/factory.go | 404 ------- pkg/sink/kafka/v2/gssapi.go | 252 ---- pkg/sink/kafka/v2/manual_balancer.go | 31 - pkg/sink/util/helper.go | 44 - 43 files changed, 154 insertions(+), 4294 deletions(-) delete mode 100644 downstreamadapter/worker/kafka_ddl_worker.go delete mode 100644 pkg/config/replica_config.go delete mode 100644 pkg/config/sink_config.go delete mode 100644 pkg/sink/kafka/admin.go delete mode 100644 pkg/sink/kafka/factory.go delete mode 100644 pkg/sink/kafka/oauth2_token_provider.go delete mode 100644 pkg/sink/kafka/options.go delete mode 100644 pkg/sink/kafka/sarama.go delete mode 100644 pkg/sink/kafka/sarama_factory.go delete mode 100644 pkg/sink/kafka/v2/admin.go delete mode 100644 pkg/sink/kafka/v2/client.go delete mode 100644 pkg/sink/kafka/v2/factory.go delete mode 100644 pkg/sink/kafka/v2/gssapi.go delete mode 100644 pkg/sink/kafka/v2/manual_balancer.go delete mode 100644 pkg/sink/util/helper.go diff --git a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go index 6c3325960..64476f68f 100644 --- a/downstreamadapter/dispatchermanager/event_dispatcher_manager.go +++ b/downstreamadapter/dispatchermanager/event_dispatcher_manager.go @@ -27,6 +27,7 @@ import ( "github.com/flowbehappy/tigate/downstreamadapter/dispatcher" "github.com/flowbehappy/tigate/downstreamadapter/eventcollector" "github.com/flowbehappy/tigate/downstreamadapter/sink" + "github.com/flowbehappy/tigate/downstreamadapter/writer" "github.com/flowbehappy/tigate/eventpb" "github.com/flowbehappy/tigate/heartbeatpb" "github.com/flowbehappy/tigate/pkg/common" @@ -130,12 +131,13 @@ func NewEventDispatcherManager(changefeedID model.ChangeFeedID, } func (e *EventDispatcherManager) InitSink() error { - sink, err := sink.NewSink(e.config, e.changefeedID) + cfg, db, err := writer.NewMysqlConfigAndDB(e.config.SinkURI) if err != nil { - log.Error("create sink failed", zap.Error(err)) + log.Error("create mysql sink failed", zap.Error(err)) return err } - e.sink = sink + + e.sink = sink.NewMysqlSink(e.changefeedID, 16, cfg, db) return nil } diff --git a/downstreamadapter/sink/helper/eventrouter/event_router.go b/downstreamadapter/sink/helper/eventrouter/event_router.go index 71c9bfb23..b29eb5347 100644 --- a/downstreamadapter/sink/helper/eventrouter/event_router.go +++ b/downstreamadapter/sink/helper/eventrouter/event_router.go @@ -17,9 +17,9 @@ import ( "github.com/flowbehappy/tigate/downstreamadapter/sink/helper/eventrouter/partition" "github.com/flowbehappy/tigate/downstreamadapter/sink/helper/eventrouter/topic" "github.com/flowbehappy/tigate/pkg/common" - ticonfig "github.com/flowbehappy/tigate/pkg/config" "github.com/pingcap/log" tableFilter "github.com/pingcap/tidb/pkg/util/table-filter" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" ) @@ -38,11 +38,11 @@ type EventRouter struct { } // NewEventRouter creates a new EventRouter. -func NewEventRouter(sinkConfig *ticonfig.SinkConfig, protocol config.Protocol, defaultTopic, scheme string) (*EventRouter, error) { +func NewEventRouter(cfg *config.ReplicaConfig, protocol config.Protocol, defaultTopic, scheme string) (*EventRouter, error) { // If an event does not match any dispatching rules in the config file, // it will be dispatched by the default partition dispatcher and // static topic dispatcher because it matches *.* rule. - ruleConfigs := append(sinkConfig.DispatchRules, &ticonfig.DispatchRule{ + ruleConfigs := append(cfg.Sink.DispatchRules, &config.DispatchRule{ Matcher: []string{"*.*"}, PartitionRule: "default", TopicRule: "", @@ -55,7 +55,7 @@ func NewEventRouter(sinkConfig *ticonfig.SinkConfig, protocol config.Protocol, d if err != nil { return nil, cerror.WrapError(cerror.ErrFilterRuleInvalid, err, ruleConfig.Matcher) } - if !sinkConfig.CaseSensitive { + if !cfg.CaseSensitive { f = tableFilter.CaseInsensitive(f) } @@ -81,25 +81,21 @@ func (s *EventRouter) GetTopicForRowChange(tableInfo *common.TableInfo) string { } // GetTopicForDDL returns the target topic for DDL. -func (s *EventRouter) GetTopicForDDL(ddl *common.DDLEvent) string { - schema := ddl.Job.SchemaName - table := ddl.Job.TableName - - // TODO: fix this - //var schema, table string - // if ddl.PreTableInfo != nil { - // if ddl.PreTableInfo.TableName.Table == "" { - // return s.defaultTopic - // } - // schema = ddl.PreTableInfo.TableName.Schema - // table = ddl.PreTableInfo.TableName.Table - // } else { - // if ddl.TableInfo.TableName.Table == "" { - // return s.defaultTopic - // } - // schema = ddl.TableInfo.TableName.Schema - // table = ddl.TableInfo.TableName.Table - // } +func (s *EventRouter) GetTopicForDDL(ddl *model.DDLEvent) string { + var schema, table string + if ddl.PreTableInfo != nil { + if ddl.PreTableInfo.TableName.Table == "" { + return s.defaultTopic + } + schema = ddl.PreTableInfo.TableName.Schema + table = ddl.PreTableInfo.TableName.Table + } else { + if ddl.TableInfo.TableName.Table == "" { + return s.defaultTopic + } + schema = ddl.TableInfo.TableName.Schema + table = ddl.TableInfo.TableName.Table + } topicGenerator := s.matchTopicGenerator(schema, table) return topicGenerator.Substitute(schema, table) diff --git a/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager.go b/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager.go index 7f9329ff8..31598cf1d 100644 --- a/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager.go +++ b/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager.go @@ -19,7 +19,6 @@ import ( "sync" "time" - tikafka "github.com/flowbehappy/tigate/pkg/sink/kafka" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -45,7 +44,7 @@ type kafkaTopicManager struct { admin kafka.ClusterAdminClient - cfg *tikafka.AutoCreateTopicConfig + cfg *kafka.AutoCreateTopicConfig topics sync.Map @@ -60,7 +59,7 @@ func GetTopicManagerAndTryCreateTopic( ctx context.Context, changefeedID model.ChangeFeedID, topic string, - topicCfg *tikafka.AutoCreateTopicConfig, + topicCfg *kafka.AutoCreateTopicConfig, adminClient kafka.ClusterAdminClient, ) (TopicManager, error) { topicManager := newKafkaTopicManager( @@ -80,7 +79,7 @@ func newKafkaTopicManager( defaultTopic string, changefeedID model.ChangeFeedID, admin kafka.ClusterAdminClient, - cfg *tikafka.AutoCreateTopicConfig, + cfg *kafka.AutoCreateTopicConfig, ) *kafkaTopicManager { mgr := &kafkaTopicManager{ defaultTopic: defaultTopic, diff --git a/downstreamadapter/sink/kafka_sink.go b/downstreamadapter/sink/kafka_sink.go index 83cf3798f..e2d94966d 100644 --- a/downstreamadapter/sink/kafka_sink.go +++ b/downstreamadapter/sink/kafka_sink.go @@ -23,21 +23,17 @@ import ( "github.com/flowbehappy/tigate/downstreamadapter/worker" "github.com/flowbehappy/tigate/downstreamadapter/worker/dmlproducer" "github.com/flowbehappy/tigate/pkg/common" - ticonfig "github.com/flowbehappy/tigate/pkg/config" "github.com/flowbehappy/tigate/pkg/sink/codec" - "github.com/flowbehappy/tigate/pkg/sink/kafka" - v2 "github.com/flowbehappy/tigate/pkg/sink/kafka/v2" - tiutils "github.com/flowbehappy/tigate/pkg/sink/util" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink/ddlsink/mq/ddlproducer" timetrics "github.com/pingcap/tiflow/cdc/sink/metrics" "github.com/pingcap/tiflow/cdc/sink/util" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/sink" - tikafka "github.com/pingcap/tiflow/pkg/sink/kafka" + "github.com/pingcap/tiflow/pkg/sink/kafka" + v2 "github.com/pingcap/tiflow/pkg/sink/kafka/v2" utils "github.com/pingcap/tiflow/pkg/util" "go.uber.org/atomic" "go.uber.org/zap" @@ -55,35 +51,34 @@ type KafkaSink struct { // It is also responsible for creating topics. topicManager topicmanager.TopicManager - dmlWorker *worker.KafkaWorker - ddlWorker *worker.KafkaDDLWorker - adminClient tikafka.ClusterAdminClient + worker *worker.KafkaWorker + adminClient kafka.ClusterAdminClient scheme string ctx context.Context cancel context.CancelCauseFunc // todo? } -func NewKafkaSink(changefeedID model.ChangeFeedID, sinkURI *url.URL, sinkConfig *ticonfig.SinkConfig) (*KafkaSink, error) { +func NewKafkaSink(changefeedID model.ChangeFeedID, sinkURI *url.URL, replicaConfig *config.ReplicaConfig) (*KafkaSink, error) { ctx, cancel := context.WithCancelCause(context.Background()) topic, err := util.GetTopic(sinkURI) if err != nil { return nil, errors.Trace(err) } scheme := sink.GetScheme(sinkURI) - protocol, err := util.GetProtocol(utils.GetOrZero(sinkConfig.Protocol)) + protocol, err := util.GetProtocol(utils.GetOrZero(replicaConfig.Sink.Protocol)) if err != nil { return nil, errors.Trace(err) } options := kafka.NewOptions() - if err := options.Apply(changefeedID, sinkURI, sinkConfig); err != nil { + if err := options.Apply(changefeedID, sinkURI, replicaConfig); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) } // todo factoryCreator := kafka.NewSaramaFactory - if utils.GetOrZero(sinkConfig.EnableKafkaSinkV2) { + if utils.GetOrZero(replicaConfig.Sink.EnableKafkaSinkV2) { factoryCreator = v2.NewFactory } @@ -97,17 +92,17 @@ func NewKafkaSink(changefeedID model.ChangeFeedID, sinkURI *url.URL, sinkConfig return nil, cerror.WrapError(cerror.ErrKafkaNewProducer, err) } - eventRouter, err := eventrouter.NewEventRouter(sinkConfig, protocol, topic, scheme) + eventRouter, err := eventrouter.NewEventRouter(replicaConfig, protocol, topic, scheme) if err != nil { return nil, errors.Trace(err) } - columnSelector, err := common.NewColumnSelectors(sinkConfig) + columnSelector, err := common.NewColumnSelectors(replicaConfig) if err != nil { return nil, errors.Trace(err) } - encoderConfig, err := tiutils.GetEncoderConfig(changefeedID, sinkURI, protocol, sinkConfig, options.MaxMessageBytes) + encoderConfig, err := util.GetEncoderConfig(changefeedID, sinkURI, protocol, replicaConfig, options.MaxMessageBytes) if err != nil { return nil, errors.Trace(err) } @@ -120,10 +115,10 @@ func NewKafkaSink(changefeedID model.ChangeFeedID, sinkURI *url.URL, sinkConfig metricsCollector := factory.MetricsCollector(utils.RoleProcessor, adminClient) dmlProducer := dmlproducer.NewKafkaDMLProducer(ctx, changefeedID, asyncProducer, metricsCollector) - encoderGroup := codec.NewEncoderGroup(ctx, sinkConfig, encoderConfig, changefeedID) + encoderGroup := codec.NewEncoderGroup(ctx, replicaConfig.Sink, encoderConfig, changefeedID) statistics := timetrics.NewStatistics(changefeedID, sink.RowSink) - dmlWorker := worker.NewKafkaWorker(changefeedID, protocol, dmlProducer, encoderGroup, statistics) + worker := worker.NewKafkaWorker(changefeedID, protocol, dmlProducer, encoderGroup, statistics) topicManager, err := topicmanager.GetTopicManagerAndTryCreateTopic( ctx, @@ -139,27 +134,13 @@ func NewKafkaSink(changefeedID model.ChangeFeedID, sinkURI *url.URL, sinkConfig } return nil, err } - - encoder, err := codec.NewEventEncoder(ctx, encoderConfig) - if err != nil { - return nil, errors.Trace(err) - } - - syncProducer, err := factory.SyncProducer(ctx) - if err != nil { - return nil, errors.Trace(err) - } - ddlProducer := ddlproducer.NewKafkaDDLProducer(ctx, changefeedID, syncProducer) - ddlWorker := worker.NewKafkaDDLWorker(changefeedID, protocol, ddlProducer, encoder, statistics) - return &KafkaSink{ ctx: ctx, cancel: cancel, topicManager: topicManager, eventRouter: eventRouter, columnSelector: columnSelector, - dmlWorker: dmlWorker, - ddlWorker: ddlWorker, + worker: worker, }, nil } @@ -206,7 +187,7 @@ func (s *KafkaSink) AddDMLEvent(event *common.DMLEvent, tableProgress *types.Tab return } - s.dmlWorker.GetEventChan() <- &common.MQRowEvent{ + s.worker.GetEventChan() <- &common.MQRowEvent{ Key: model.TopicPartitionKey{ Topic: topic, Partition: index, @@ -230,8 +211,6 @@ func (s *KafkaSink) PassDDLAndSyncPointEvent(event *common.DDLEvent, tableProgre } func (s *KafkaSink) AddDDLAndSyncPointEvent(event *common.DDLEvent, tableProgress *types.TableProgress) { - tableProgress.Add(event) - s.ddlWorker.GetEventChan() <- event } func (s *KafkaSink) Close() { diff --git a/downstreamadapter/sink/sink.go b/downstreamadapter/sink/sink.go index 8107b5275..925faeb74 100644 --- a/downstreamadapter/sink/sink.go +++ b/downstreamadapter/sink/sink.go @@ -14,17 +14,8 @@ package sink import ( - "net/url" - "github.com/flowbehappy/tigate/downstreamadapter/sink/types" - "github.com/flowbehappy/tigate/downstreamadapter/writer" "github.com/flowbehappy/tigate/pkg/common" - "github.com/flowbehappy/tigate/pkg/config" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/sink" - "go.uber.org/zap" ) type Sink interface { @@ -38,23 +29,3 @@ type Sink interface { // GetCheckpointTs(tableSpan *common.TableSpan) (uint64, bool) Close() } - -func NewSink(config *config.ChangefeedConfig, changefeedID model.ChangeFeedID) (Sink, error) { - sinkURI, err := url.Parse(config.SinkURI) - if err != nil { - return nil, cerror.WrapError(cerror.ErrSinkURIInvalid, err) - } - scheme := sink.GetScheme(sinkURI) - switch scheme { - case sink.MySQLScheme, sink.MySQLSSLScheme, sink.TiDBScheme, sink.TiDBSSLScheme: - cfg, db, err := writer.NewMysqlConfigAndDB(sinkURI) - if err != nil { - log.Error("create mysql sink failed", zap.Error(err)) - return nil, err - } - return NewMysqlSink(changefeedID, 16, cfg, db), nil - case sink.KafkaScheme, sink.KafkaSSLScheme: - return NewKafkaSink(changefeedID, sinkURI, config.SinkConfig) - } - return nil, nil -} diff --git a/downstreamadapter/worker/kafka_ddl_worker.go b/downstreamadapter/worker/kafka_ddl_worker.go deleted file mode 100644 index bee7ee54c..000000000 --- a/downstreamadapter/worker/kafka_ddl_worker.go +++ /dev/null @@ -1,157 +0,0 @@ -package worker - -import ( - "context" - "sync" - "time" - - "github.com/flowbehappy/tigate/downstreamadapter/sink/helper/eventrouter" - "github.com/flowbehappy/tigate/downstreamadapter/sink/helper/topicmanager" - "github.com/flowbehappy/tigate/pkg/common" - "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/sink/ddlsink/mq/ddlproducer" - "github.com/pingcap/tiflow/cdc/sink/metrics" - "github.com/pingcap/tiflow/cdc/sink/metrics/mq" - "github.com/pingcap/tiflow/pkg/chann" - "github.com/pingcap/tiflow/pkg/config" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" -) - -// worker will send messages to the DML producer on a batch basis. -type KafkaDDLWorker struct { - // changeFeedID indicates this sink belongs to which processor(changefeed). - changeFeedID model.ChangeFeedID - // protocol indicates the protocol used by this sink. - protocol config.Protocol - // msgChan caches the messages to be sent. - // It is an unbounded channel. - msgChan *chann.DrainableChann[*common.DDLEvent] - // ticker used to force flush the batched messages when the interval is reached. - ticker *time.Ticker - - encoder encoder.EventEncoder - // eventRouter used to route events to the right topic and partition. - eventRouter *eventrouter.EventRouter - // topicManager used to manage topics. - // It is also responsible for creating topics. - topicManager topicmanager.TopicManager - - // producer is used to send the messages to the Kafka broker. - producer ddlproducer.DDLProducer - - // metricMQWorkerSendMessageDuration tracks the time duration cost on send messages. - metricMQWorkerSendMessageDuration prometheus.Observer - // metricMQWorkerBatchSize tracks each batch's size. - metricMQWorkerBatchSize prometheus.Observer - // metricMQWorkerBatchDuration tracks the time duration cost on batch messages. - metricMQWorkerBatchDuration prometheus.Observer - // statistics is used to record DML metrics. - statistics *metrics.Statistics - partitionRule DDLDispatchRule - ctx context.Context - cancel context.CancelFunc - wg sync.WaitGroup -} - -// DDLDispatchRule is the dispatch rule for DDL event. -type DDLDispatchRule int - -const ( - // PartitionZero means the DDL event will be dispatched to partition 0. - // NOTICE: Only for canal and canal-json protocol. - PartitionZero DDLDispatchRule = iota - // PartitionAll means the DDL event will be broadcast to all the partitions. - PartitionAll -) - -func getDDLDispatchRule(protocol config.Protocol) DDLDispatchRule { - switch protocol { - case config.ProtocolCanal, config.ProtocolCanalJSON: - return PartitionZero - default: - } - return PartitionAll -} - -// newWorker creates a new flush worker. -func NewKafkaDDLWorker( - id model.ChangeFeedID, - protocol config.Protocol, - producer ddlproducer.DDLProducer, - encoder encoder.EventEncoder, - statistics *metrics.Statistics, -) *KafkaDDLWorker { - ctx, cancel := context.WithCancel(context.Background()) - w := &KafkaDDLWorker{ - ctx: ctx, - changeFeedID: id, - protocol: protocol, - msgChan: chann.NewAutoDrainChann[*common.DDLEvent](), - ticker: time.NewTicker(batchInterval), - encoder: encoder, - producer: producer, - metricMQWorkerSendMessageDuration: mq.WorkerSendMessageDuration.WithLabelValues(id.Namespace, id.ID), - metricMQWorkerBatchSize: mq.WorkerBatchSize.WithLabelValues(id.Namespace, id.ID), - metricMQWorkerBatchDuration: mq.WorkerBatchDuration.WithLabelValues(id.Namespace, id.ID), - statistics: statistics, - cancel: cancel, - partitionRule: getDDLDispatchRule(protocol), - } - - w.wg.Add(1) - go func() { - defer w.wg.Done() - // TODO:后面做一下性能测试看看要不要拆开做 pipeline - w.encodeAndSendDDLEvents() - }() - return w -} - -func (w *KafkaDDLWorker) GetEventChan() chan<- *common.DDLEvent { - return w.msgChan.In() -} - -func (w *KafkaDDLWorker) encodeAndSendDDLEvents() error { - for { - select { - case <-w.ctx.Done(): - return errors.Trace(w.ctx.Err()) - case event, ok := <-w.msgChan.Out(): - if !ok { - log.Warn("MQ sink flush worker channel closed", - zap.String("namespace", w.changeFeedID.Namespace), - zap.String("changefeed", w.changeFeedID.ID)) - return nil - } - message, err := w.encoder.EncodeDDLEvent(event) - if err != nil { - log.Error("Failed to encode ddl event", - zap.String("namespace", w.changeFeedID.Namespace), - zap.String("changefeed", w.changeFeedID.ID), - zap.Error(err)) - continue - } - - topic := w.eventRouter.GetTopicForDDL(event) - partitionNum, err := w.topicManager.GetPartitionNum(w.ctx, topic) - if err != nil { - log.Error("failed to get partition number for topic", zap.String("topic", topic), zap.Error(err)) - continue - } - - if w.partitionRule == PartitionAll { - err = w.statistics.RecordDDLExecution(func() error { - return w.producer.SyncBroadcastMessage(w.ctx, topic, partitionNum, message) - }) - return errors.Trace(err) - } - err = w.statistics.RecordDDLExecution(func() error { - return w.producer.SyncSendMessage(w.ctx, topic, 0, message) - }) - } - } -} diff --git a/downstreamadapter/worker/kafka_worker.go b/downstreamadapter/worker/kafka_worker.go index 952c306ad..deb9d3ebb 100644 --- a/downstreamadapter/worker/kafka_worker.go +++ b/downstreamadapter/worker/kafka_worker.go @@ -238,8 +238,14 @@ func (w *KafkaWorker) group(msgs []*common.MQRowEvent) map[model.TopicPartitionK } func (w *KafkaWorker) sendMessages(ctx context.Context) error { - metricSendMessageDuration := mq.WorkerSendMessageDuration.WithLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) - defer mq.WorkerSendMessageDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) + ticker := time.NewTicker(15 * time.Second) + metric := codec.EncoderGroupOutputChanSizeGauge. + WithLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) + defer func() { + ticker.Stop() + codec.EncoderGroupOutputChanSizeGauge. + DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID) + }() var err error outCh := w.encoderGroup.Output() @@ -247,6 +253,8 @@ func (w *KafkaWorker) sendMessages(ctx context.Context) error { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) + case <-ticker.C: + metric.Set(float64(len(outCh))) case future, ok := <-outCh: if !ok { log.Warn("MQ sink encoder's output channel closed", @@ -272,7 +280,7 @@ func (w *KafkaWorker) sendMessages(ctx context.Context) error { }); err != nil { return err } - metricSendMessageDuration.Observe(time.Since(start).Seconds()) + w.metricMQWorkerSendMessageDuration.Observe(time.Since(start).Seconds()) } } } diff --git a/downstreamadapter/writer/config.go b/downstreamadapter/writer/config.go index d9837ce84..8df90c420 100644 --- a/downstreamadapter/writer/config.go +++ b/downstreamadapter/writer/config.go @@ -124,11 +124,17 @@ func (c *MysqlConfig) Apply(sinkURI *url.URL) error { return nil } -func NewMysqlConfigAndDB(sinkURI *url.URL) (*MysqlConfig, *sql.DB, error) { - log.Info("create db connection", zap.String("sinkURI", sinkURI.String())) +func NewMysqlConfigAndDB(sinkURI string) (*MysqlConfig, *sql.DB, error) { + log.Info("create db connection", zap.String("sinkURI", sinkURI)) // create db connection + sinkURIParsed, err := url.Parse(sinkURI) + if err != nil { + log.Error("parse sinkURI failed", zap.Error(err)) + return nil, nil, err + } + cfg := NewMysqlConfig() - err := cfg.Apply(sinkURI) + err = cfg.Apply(sinkURIParsed) if err != nil { log.Error("Apply sinkURI failed", zap.Error(err)) return nil, nil, err diff --git a/maintainer/maintainer.go b/maintainer/maintainer.go index 13b234522..3ba3edbfc 100644 --- a/maintainer/maintainer.go +++ b/maintainer/maintainer.go @@ -51,7 +51,7 @@ import ( // 4. handle heartbeat reported by dispatcher type Maintainer struct { id model.ChangeFeedID - config *configNew.ChangeFeedInfo + config *model.ChangeFeedInfo selfNode *common.NodeInfo stream dynstream.DynamicStream[string, *Event, *Maintainer] @@ -103,7 +103,7 @@ type Maintainer struct { // NewMaintainer create the maintainer for the changefeed func NewMaintainer(cfID model.ChangeFeedID, - cfg *configNew.ChangeFeedInfo, + cfg *model.ChangeFeedInfo, selfNode *common.NodeInfo, stream dynstream.DynamicStream[string, *Event, *Maintainer], taskScheduler threadpool.ThreadPool, diff --git a/maintainer/maintainer_manager.go b/maintainer/maintainer_manager.go index 074a333ed..6fbbdd25b 100644 --- a/maintainer/maintainer_manager.go +++ b/maintainer/maintainer_manager.go @@ -20,7 +20,6 @@ import ( "time" "github.com/flowbehappy/tigate/pkg/common" - configNew "github.com/flowbehappy/tigate/pkg/config" "github.com/flowbehappy/tigate/utils/threadpool" "github.com/pingcap/tiflow/pkg/pdutil" "github.com/tikv/client-go/v2/tikv" @@ -205,7 +204,7 @@ func (m *Manager) onDispatchMaintainerRequest( cfID := model.DefaultChangeFeedID(req.GetId()) cf, ok := m.maintainers.Load(cfID) if !ok { - cfConfig := &configNew.ChangeFeedInfo{} + cfConfig := &model.ChangeFeedInfo{} err := json.Unmarshal(req.Config, cfConfig) if err != nil { log.Panic("decode changefeed fail", zap.Error(err)) diff --git a/maintainer/maintainer_test.go b/maintainer/maintainer_test.go index 159d8b475..507f59df6 100644 --- a/maintainer/maintainer_test.go +++ b/maintainer/maintainer_test.go @@ -26,7 +26,6 @@ import ( "github.com/flowbehappy/tigate/pkg/common" appcontext "github.com/flowbehappy/tigate/pkg/common/context" "github.com/flowbehappy/tigate/pkg/config" - configNew "github.com/flowbehappy/tigate/pkg/config" "github.com/flowbehappy/tigate/pkg/messaging" "github.com/flowbehappy/tigate/scheduler" "github.com/flowbehappy/tigate/server/watcher" @@ -35,6 +34,7 @@ import ( "github.com/google/uuid" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + config2 "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -231,8 +231,8 @@ func TestMaintainerSchedule(t *testing.T) { go dispatcherManager.Run(ctx) taskScheduler := threadpool.NewThreadPoolDefault() - maintainer := NewMaintainer(cfID, &configNew.ChangeFeedInfo{ - Config: configNew.GetDefaultReplicaConfig(), + maintainer := NewMaintainer(cfID, &model.ChangeFeedInfo{ + Config: config2.GetDefaultReplicaConfig(), }, node, stream, taskScheduler, nil, nil, 10) _ = stream.AddPaths(dynstream.PathAndDest[string, *Maintainer]{ Path: cfID.ID, diff --git a/pkg/common/column_selector.go b/pkg/common/column_selector.go index 0237b9ad5..0d52f95d3 100644 --- a/pkg/common/column_selector.go +++ b/pkg/common/column_selector.go @@ -14,9 +14,9 @@ package common import ( - ticonfig "github.com/flowbehappy/tigate/pkg/config" "github.com/pingcap/tidb/pkg/parser/model" filter "github.com/pingcap/tidb/pkg/util/table-filter" + "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/errors" ) @@ -40,7 +40,7 @@ type ColumnSelector struct { } func newColumnSelector( - rule *ticonfig.ColumnSelector, caseSensitive bool, + rule *config.ColumnSelector, caseSensitive bool, ) (*ColumnSelector, error) { tableM, err := filter.Parse(rule.Matcher) if err != nil { @@ -78,10 +78,10 @@ type ColumnSelectors struct { } // New return a column selectors -func NewColumnSelectors(sinkConfig *ticonfig.SinkConfig) (*ColumnSelectors, error) { - selectors := make([]*ColumnSelector, 0, len(sinkConfig.ColumnSelectors)) - for _, r := range sinkConfig.ColumnSelectors { - selector, err := newColumnSelector(r, sinkConfig.CaseSensitive) +func NewColumnSelectors(cfg *config.ReplicaConfig) (*ColumnSelectors, error) { + selectors := make([]*ColumnSelector, 0, len(cfg.Sink.ColumnSelectors)) + for _, r := range cfg.Sink.ColumnSelectors { + selector, err := newColumnSelector(r, cfg.CaseSensitive) if err != nil { return nil, err } diff --git a/pkg/common/event.go b/pkg/common/event.go index 88528b26f..055733848 100644 --- a/pkg/common/event.go +++ b/pkg/common/event.go @@ -330,6 +330,7 @@ type DDLEvent struct { // commitTS of the rawKV CommitTS Ts `json:"commit_ts"` + // Just for test now // Just for test now BlockedTables *InfluencedTables `json:"blocked_tables"` NeedDroppedTables *InfluencedTables `json:"need_dropped_tables"` diff --git a/pkg/config/changefeed.go b/pkg/config/changefeed.go index 74a0bc0f4..0a0529eb4 100644 --- a/pkg/config/changefeed.go +++ b/pkg/config/changefeed.go @@ -1,11 +1,6 @@ package config -import ( - "time" - - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" -) +import "github.com/pingcap/tiflow/pkg/config" type ChangefeedConfig struct { Namespace string `json:"namespace"` @@ -19,34 +14,5 @@ type ChangefeedConfig struct { ForceReplicate bool `json:"force_replicate" default:"false"` Filter *config.FilterConfig `toml:"filter" json:"filter"` - SinkConfig *SinkConfig `json:"sink_config"` -} - -// ChangeFeedInfo describes the detail of a ChangeFeed -type ChangeFeedInfo struct { - UpstreamID uint64 `json:"upstream-id"` - Namespace string `json:"namespace"` - ID string `json:"changefeed-id"` - SinkURI string `json:"sink-uri"` - CreateTime time.Time `json:"create-time"` - // Start sync at this commit ts if `StartTs` is specify or using the CreateTime of changefeed. - StartTs uint64 `json:"start-ts"` - // The ChangeFeed will exits until sync to timestamp TargetTs - TargetTs uint64 `json:"target-ts"` - // used for admin job notification, trigger watch event in capture - AdminJobType model.AdminJobType `json:"admin-job-type"` - Engine model.SortEngine `json:"sort-engine"` - // SortDir is deprecated - // it cannot be set by user in changefeed level, any assignment to it should be ignored. - // but can be fetched for backward compatibility - SortDir string `json:"sort-dir"` - - Config *ReplicaConfig `json:"config"` - State model.FeedState `json:"state"` - Error *model.RunningError `json:"error"` - Warning *model.RunningError `json:"warning"` - - CreatorVersion string `json:"creator-version"` - // Epoch is the epoch of a changefeed, changes on every restart. - Epoch uint64 `json:"epoch"` + SinkConfig *config.SinkConfig `json:"sink_config"` } diff --git a/pkg/config/replica_config.go b/pkg/config/replica_config.go deleted file mode 100644 index 6895119ed..000000000 --- a/pkg/config/replica_config.go +++ /dev/null @@ -1,363 +0,0 @@ -// Copyright 2021 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package config - -import ( - "database/sql/driver" - "encoding/json" - "fmt" - "net/url" - "strings" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/config/outdated" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/integrity" - "github.com/pingcap/tiflow/pkg/redo" - "github.com/pingcap/tiflow/pkg/sink" - "github.com/pingcap/tiflow/pkg/util" - "go.uber.org/zap" -) - -const ( - // minSyncPointInterval is the minimum of SyncPointInterval can be set. - minSyncPointInterval = time.Second * 30 - // minSyncPointRetention is the minimum of SyncPointRetention can be set. - minSyncPointRetention = time.Hour * 1 - minChangeFeedErrorStuckDuration = time.Minute * 30 - // DefaultTiDBSourceID is the default source ID of TiDB cluster. - DefaultTiDBSourceID = 1 -) - -var defaultReplicaConfig = &ReplicaConfig{ - MemoryQuota: config.DefaultChangefeedMemoryQuota, - CaseSensitive: false, - CheckGCSafePoint: true, - EnableSyncPoint: util.AddressOf(false), - EnableTableMonitor: util.AddressOf(false), - SyncPointInterval: util.AddressOf(10 * time.Minute), - SyncPointRetention: util.AddressOf(24 * time.Hour), - BDRMode: util.AddressOf(false), - Filter: &config.FilterConfig{ - Rules: []string{"*.*"}, - }, - Mounter: &config.MounterConfig{ - WorkerNum: 16, - }, - Sink: &SinkConfig{ - CSVConfig: &CSVConfig{ - Quote: string(DoubleQuoteChar), - Delimiter: Comma, - NullString: NULL, - BinaryEncodingMethod: BinaryEncodingBase64, - }, - EncoderConcurrency: util.AddressOf(DefaultEncoderGroupConcurrency), - Terminator: util.AddressOf(CRLF), - DateSeparator: util.AddressOf(DateSeparatorDay.String()), - EnablePartitionSeparator: util.AddressOf(true), - EnableKafkaSinkV2: util.AddressOf(false), - OnlyOutputUpdatedColumns: util.AddressOf(false), - DeleteOnlyOutputHandleKeyColumns: util.AddressOf(false), - ContentCompatible: util.AddressOf(false), - TiDBSourceID: DefaultTiDBSourceID, - AdvanceTimeoutInSec: util.AddressOf(DefaultAdvanceTimeoutInSec), - SendBootstrapIntervalInSec: util.AddressOf(DefaultSendBootstrapIntervalInSec), - SendBootstrapInMsgCount: util.AddressOf(DefaultSendBootstrapInMsgCount), - SendBootstrapToAllPartition: util.AddressOf(DefaultSendBootstrapToAllPartition), - SendAllBootstrapAtStart: util.AddressOf(DefaultSendAllBootstrapAtStart), - DebeziumDisableSchema: util.AddressOf(false), - OpenProtocol: &OpenProtocolConfig{OutputOldValue: true}, - Debezium: &DebeziumConfig{OutputOldValue: true}, - }, - Consistent: &config.ConsistentConfig{ - Level: "none", - MaxLogSize: redo.DefaultMaxLogSize, - FlushIntervalInMs: redo.DefaultFlushIntervalInMs, - MetaFlushIntervalInMs: redo.DefaultMetaFlushIntervalInMs, - EncodingWorkerNum: redo.DefaultEncodingWorkerNum, - FlushWorkerNum: redo.DefaultFlushWorkerNum, - Storage: "", - UseFileBackend: false, - Compression: "", - MemoryUsage: &config.ConsistentMemoryUsage{ - MemoryQuotaPercentage: 50, - }, - }, - Scheduler: &config.ChangefeedSchedulerConfig{ - EnableTableAcrossNodes: false, - RegionThreshold: 100_000, - WriteKeyThreshold: 0, - }, - Integrity: &integrity.Config{ - IntegrityCheckLevel: integrity.CheckLevelNone, - CorruptionHandleLevel: integrity.CorruptionHandleLevelWarn, - }, - ChangefeedErrorStuckDuration: util.AddressOf(time.Minute * 30), - SyncedStatus: &config.SyncedStatusConfig{SyncedCheckInterval: 5 * 60, CheckpointInterval: 15}, -} - -// GetDefaultReplicaConfig returns the default replica config. -func GetDefaultReplicaConfig() *ReplicaConfig { - return defaultReplicaConfig.Clone() -} - -// Duration wrap time.Duration to override UnmarshalText func -type Duration struct { - time.Duration -} - -// UnmarshalText unmarshal byte to duration -func (d *Duration) UnmarshalText(text []byte) error { - var err error - d.Duration, err = time.ParseDuration(string(text)) - return err -} - -// ReplicaConfig represents some addition replication config for a changefeed -type ReplicaConfig replicaConfig - -type replicaConfig struct { - MemoryQuota uint64 `toml:"memory-quota" json:"memory-quota"` - CaseSensitive bool `toml:"case-sensitive" json:"case-sensitive"` - ForceReplicate bool `toml:"force-replicate" json:"force-replicate"` - CheckGCSafePoint bool `toml:"check-gc-safe-point" json:"check-gc-safe-point"` - // EnableSyncPoint is only available when the downstream is a Database. - EnableSyncPoint *bool `toml:"enable-sync-point" json:"enable-sync-point,omitempty"` - EnableTableMonitor *bool `toml:"enable-table-monitor" json:"enable-table-monitor"` - // IgnoreIneligibleTable is used to store the user's config when creating a changefeed. - // not used in the changefeed's lifecycle. - IgnoreIneligibleTable bool `toml:"ignore-ineligible-table" json:"ignore-ineligible-table"` - - // BDR(Bidirectional Replication) is a feature that allows users to - // replicate data of same tables from TiDB-1 to TiDB-2 and vice versa. - // This feature is only available for TiDB. - BDRMode *bool `toml:"bdr-mode" json:"bdr-mode,omitempty"` - // SyncPointInterval is only available when the downstream is DB. - SyncPointInterval *time.Duration `toml:"sync-point-interval" json:"sync-point-interval,omitempty"` - // SyncPointRetention is only available when the downstream is DB. - SyncPointRetention *time.Duration `toml:"sync-point-retention" json:"sync-point-retention,omitempty"` - Filter *config.FilterConfig `toml:"filter" json:"filter"` - Mounter *config.MounterConfig `toml:"mounter" json:"mounter"` - Sink *SinkConfig `toml:"sink" json:"sink"` - // Consistent is only available for DB downstream with redo feature enabled. - Consistent *config.ConsistentConfig `toml:"consistent" json:"consistent,omitempty"` - // Scheduler is the configuration for scheduler. - Scheduler *config.ChangefeedSchedulerConfig `toml:"scheduler" json:"scheduler"` - // Integrity is only available when the downstream is MQ. - Integrity *integrity.Config `toml:"integrity" json:"integrity"` - ChangefeedErrorStuckDuration *time.Duration `toml:"changefeed-error-stuck-duration" json:"changefeed-error-stuck-duration,omitempty"` - SyncedStatus *config.SyncedStatusConfig `toml:"synced-status" json:"synced-status,omitempty"` - - // Deprecated: we don't use this field since v8.0.0. - SQLMode string `toml:"sql-mode" json:"sql-mode"` -} - -// Value implements the driver.Valuer interface -func (c ReplicaConfig) Value() (driver.Value, error) { - cfg, err := c.Marshal() - if err != nil { - return nil, err - } - - // TODO: refactor the meaningless type conversion. - return []byte(cfg), nil -} - -// Scan implements the sql.Scanner interface -func (c *ReplicaConfig) Scan(value interface{}) error { - b, ok := value.([]byte) - if !ok { - return errors.New("type assertion to []byte failed") - } - - return c.UnmarshalJSON(b) -} - -// Marshal returns the json marshal format of a ReplicationConfig -func (c *ReplicaConfig) Marshal() (string, error) { - cfg, err := json.Marshal(c) - if err != nil { - return "", cerror.WrapError(cerror.ErrEncodeFailed, errors.Annotatef(err, "Unmarshal data: %v", c)) - } - return string(cfg), nil -} - -// UnmarshalJSON unmarshals into *ReplicationConfig from json marshal byte slice -func (c *ReplicaConfig) UnmarshalJSON(data []byte) error { - // The purpose of casting ReplicaConfig to replicaConfig is to avoid recursive calls UnmarshalJSON, - // resulting in stack overflow - r := (*replicaConfig)(c) - err := json.Unmarshal(data, &r) - if err != nil { - return cerror.WrapError(cerror.ErrDecodeFailed, err) - } - v1 := outdated.ReplicaConfigV1{} - err = v1.Unmarshal(data) - if err != nil { - return cerror.WrapError(cerror.ErrDecodeFailed, err) - } - r.fillFromV1(&v1) - return nil -} - -// Clone clones a replica config -func (c *ReplicaConfig) Clone() *ReplicaConfig { - str, err := c.Marshal() - if err != nil { - log.Panic("failed to marshal replica config", - zap.Error(cerror.WrapError(cerror.ErrDecodeFailed, err))) - } - clone := new(ReplicaConfig) - err = clone.UnmarshalJSON([]byte(str)) - if err != nil { - log.Panic("failed to unmarshal replica config", - zap.Error(cerror.WrapError(cerror.ErrDecodeFailed, err))) - } - return clone -} - -func (c *replicaConfig) fillFromV1(v1 *outdated.ReplicaConfigV1) { - if v1 == nil || v1.Sink == nil { - return - } - for _, dispatch := range v1.Sink.DispatchRules { - c.Sink.DispatchRules = append(c.Sink.DispatchRules, &DispatchRule{ - Matcher: []string{fmt.Sprintf("%s.%s", dispatch.Schema, dispatch.Name)}, - DispatcherRule: dispatch.Rule, - }) - } -} - -// ValidateAndAdjust verifies and adjusts the replica configuration. -func (c *ReplicaConfig) ValidateAndAdjust(sinkURI *url.URL) error { // check sink uri - if c.Sink != nil { - err := c.Sink.validateAndAdjust(sinkURI) - if err != nil { - return err - } - } - - if c.Consistent != nil { - err := c.Consistent.ValidateAndAdjust() - if err != nil { - return err - } - } - - // check sync point config - if util.GetOrZero(c.EnableSyncPoint) { - if c.SyncPointInterval != nil && - *c.SyncPointInterval < minSyncPointInterval { - return cerror.ErrInvalidReplicaConfig. - FastGenByArgs( - fmt.Sprintf("The SyncPointInterval:%s must be larger than %s", - c.SyncPointInterval.String(), - minSyncPointInterval.String())) - } - if c.SyncPointRetention != nil && - *c.SyncPointRetention < minSyncPointRetention { - return cerror.ErrInvalidReplicaConfig. - FastGenByArgs( - fmt.Sprintf("The SyncPointRetention:%s must be larger than %s", - c.SyncPointRetention.String(), - minSyncPointRetention.String())) - } - } - if c.MemoryQuota == uint64(0) { - c.FixMemoryQuota() - } - if c.Scheduler == nil { - c.FixScheduler(false) - } else { - err := c.Scheduler.Validate() - if err != nil { - return err - } - } - // TODO: Remove the hack once span replication is compatible with all sinks. - if !isSinkCompatibleWithSpanReplication(sinkURI) { - c.Scheduler.EnableTableAcrossNodes = false - } - - if c.Integrity != nil { - switch strings.ToLower(sinkURI.Scheme) { - case sink.KafkaScheme, sink.KafkaSSLScheme: - default: - if c.Integrity.Enabled() { - log.Warn("integrity checksum only support kafka sink now, disable integrity") - c.Integrity.IntegrityCheckLevel = integrity.CheckLevelNone - } - } - - if err := c.Integrity.Validate(); err != nil { - return err - } - - if c.Integrity.Enabled() && len(c.Sink.ColumnSelectors) != 0 { - log.Error("it's not allowed to enable the integrity check and column selector at the same time") - return cerror.ErrInvalidReplicaConfig.GenWithStack( - "integrity check enabled and column selector set, not allowed") - - } - } - - if c.ChangefeedErrorStuckDuration != nil && - *c.ChangefeedErrorStuckDuration < minChangeFeedErrorStuckDuration { - return cerror.ErrInvalidReplicaConfig. - FastGenByArgs( - fmt.Sprintf("The ChangefeedErrorStuckDuration:%f must be larger than %f Seconds", - c.ChangefeedErrorStuckDuration.Seconds(), - minChangeFeedErrorStuckDuration.Seconds())) - } - - return nil -} - -// FixScheduler adjusts scheduler to default value -func (c *ReplicaConfig) FixScheduler(inheritV66 bool) { - if c.Scheduler == nil { - c.Scheduler = defaultReplicaConfig.Clone().Scheduler - return - } - if inheritV66 && c.Scheduler.RegionPerSpan != 0 { - c.Scheduler.EnableTableAcrossNodes = true - c.Scheduler.RegionThreshold = c.Scheduler.RegionPerSpan - c.Scheduler.RegionPerSpan = 0 - } -} - -// FixMemoryQuota adjusts memory quota to default value -func (c *ReplicaConfig) FixMemoryQuota() { - c.MemoryQuota = config.DefaultChangefeedMemoryQuota -} - -// isSinkCompatibleWithSpanReplication returns true if the sink uri is -// compatible with span replication. -func isSinkCompatibleWithSpanReplication(u *url.URL) bool { - return u != nil && - (strings.Contains(u.Scheme, "kafka") || strings.Contains(u.Scheme, "blackhole")) -} - -// MaskSensitiveData masks sensitive data in ReplicaConfig -func (c *ReplicaConfig) MaskSensitiveData() { - if c.Sink != nil { - c.Sink.MaskSensitiveData() - } - if c.Consistent != nil { - c.Consistent.MaskSensitiveData() - } -} diff --git a/pkg/config/sink_config.go b/pkg/config/sink_config.go deleted file mode 100644 index c84afacd7..000000000 --- a/pkg/config/sink_config.go +++ /dev/null @@ -1,1010 +0,0 @@ -package config - -import ( - "fmt" - "net/url" - "strconv" - "strings" - "time" - - "github.com/apache/pulsar-client-go/pulsar" - "github.com/aws/aws-sdk-go/aws" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/integrity" - "github.com/pingcap/tiflow/pkg/sink" - "github.com/pingcap/tiflow/pkg/util" - "go.uber.org/zap" -) - -const ( - // DefaultMaxMessageBytes sets the default value for max-message-bytes. - DefaultMaxMessageBytes = 10 * 1024 * 1024 // 10M - // DefaultAdvanceTimeoutInSec sets the default value for advance-timeout-in-sec. - DefaultAdvanceTimeoutInSec = uint(150) - - // TxnAtomicityKey specifies the key of the transaction-atomicity in the SinkURI. - TxnAtomicityKey = "transaction-atomicity" - // defaultTxnAtomicity is the default atomicity level. - defaultTxnAtomicity = noneTxnAtomicity - // unknownTxnAtomicity is an invalid atomicity level and will be treated as - // defaultTxnAtomicity when initializing sink in processor. - unknownTxnAtomicity AtomicityLevel = "" - // noneTxnAtomicity means atomicity of transactions is not guaranteed - noneTxnAtomicity AtomicityLevel = "none" - // tableTxnAtomicity means atomicity of single table transactions is guaranteed. - tableTxnAtomicity AtomicityLevel = "table" - - // Comma is a constant for ',' - Comma = "," - // CR is an abbreviation for carriage return - CR = '\r' - // LF is an abbreviation for line feed - LF = '\n' - // CRLF is an abbreviation for '\r\n' - CRLF = "\r\n" - // DoubleQuoteChar is a constant for '"' - DoubleQuoteChar = '"' - // Backslash is a constant for '\' - Backslash = '\\' - // NULL is a constant for '\N' - NULL = "\\N" - - // MinFileIndexWidth is the minimum width of file index. - MinFileIndexWidth = 6 // enough for 2^19 files - // MaxFileIndexWidth is the maximum width of file index. - MaxFileIndexWidth = 20 // enough for 2^64 files - // DefaultFileIndexWidth is the default width of file index. - DefaultFileIndexWidth = MaxFileIndexWidth - - // BinaryEncodingHex encodes binary data to hex string. - BinaryEncodingHex = "hex" - // BinaryEncodingBase64 encodes binary data to base64 string. - BinaryEncodingBase64 = "base64" - - // DefaultPulsarProducerCacheSize is the default size of the cache for producers - // 10240 producers maybe cost 1.1G memory - DefaultPulsarProducerCacheSize = 10240 - - // DefaultEncoderGroupConcurrency is the default concurrency of encoder group. - DefaultEncoderGroupConcurrency = 32 - - // DefaultSendBootstrapIntervalInSec is the default interval to send bootstrap message. - DefaultSendBootstrapIntervalInSec = int64(120) - // DefaultSendBootstrapInMsgCount is the default number of messages to send bootstrap message. - DefaultSendBootstrapInMsgCount = int32(10000) - // DefaultSendBootstrapToAllPartition is the default value of - // whether to send bootstrap message to all partitions. - DefaultSendBootstrapToAllPartition = true - // DefaultSendAllBootstrapAtStart is the default value of whether - // to send all tables bootstrap message at changefeed start. - DefaultSendAllBootstrapAtStart = false - - // DefaultMaxReconnectToPulsarBroker is the default max reconnect times to pulsar broker. - // The pulsar client uses an exponential backoff with jitter to reconnect to the broker. - // Based on test, when the max reconnect times is 3, - // the total time of reconnecting to brokers is about 30 seconds. - DefaultMaxReconnectToPulsarBroker = 3 -) - -// AtomicityLevel represents the atomicity level of a changefeed. -type AtomicityLevel string - -// ShouldSplitTxn returns whether the sink should split txn. -func (l AtomicityLevel) ShouldSplitTxn() bool { - if l == unknownTxnAtomicity { - l = defaultTxnAtomicity - } - return l == noneTxnAtomicity -} - -func (l AtomicityLevel) validate(scheme string) error { - switch l { - case unknownTxnAtomicity: - case noneTxnAtomicity: - // Do nothing here to avoid modifying the persistence parameters. - case tableTxnAtomicity: - // MqSink only support `noneTxnAtomicity`. - if sink.IsMQScheme(scheme) { - errMsg := fmt.Sprintf("%s level atomicity is not supported by %s scheme", l, scheme) - return cerror.ErrSinkURIInvalid.GenWithStackByArgs(errMsg) - } - default: - errMsg := fmt.Sprintf("%s level atomicity is not supported by %s scheme", l, scheme) - return cerror.ErrSinkURIInvalid.GenWithStackByArgs(errMsg) - } - return nil -} - -// SinkConfig represents sink config for a changefeed -type SinkConfig struct { - TxnAtomicity *AtomicityLevel `toml:"transaction-atomicity" json:"transaction-atomicity,omitempty"` - // Protocol is NOT available when the downstream is DB. - Protocol *string `toml:"protocol" json:"protocol,omitempty"` - - // DispatchRules is only available when the downstream is MQ. - DispatchRules []*DispatchRule `toml:"dispatchers" json:"dispatchers,omitempty"` - - ColumnSelectors []*ColumnSelector `toml:"column-selectors" json:"column-selectors,omitempty"` - // SchemaRegistry is only available when the downstream is MQ using avro protocol. - SchemaRegistry *string `toml:"schema-registry" json:"schema-registry,omitempty"` - // EncoderConcurrency is only available when the downstream is MQ. - EncoderConcurrency *int `toml:"encoder-concurrency" json:"encoder-concurrency,omitempty"` - // Terminator is NOT available when the downstream is DB. - Terminator *string `toml:"terminator" json:"terminator,omitempty"` - // DateSeparator is only available when the downstream is Storage. - DateSeparator *string `toml:"date-separator" json:"date-separator,omitempty"` - // EnablePartitionSeparator is only available when the downstream is Storage. - EnablePartitionSeparator *bool `toml:"enable-partition-separator" json:"enable-partition-separator,omitempty"` - // FileIndexWidth is only available when the downstream is Storage - FileIndexWidth *int `toml:"file-index-digit,omitempty" json:"file-index-digit,omitempty"` - - // EnableKafkaSinkV2 enabled then the kafka-go sink will be used. - // It is only available when the downstream is MQ. - EnableKafkaSinkV2 *bool `toml:"enable-kafka-sink-v2" json:"enable-kafka-sink-v2,omitempty"` - - // OnlyOutputUpdatedColumns is only available when the downstream is MQ. - OnlyOutputUpdatedColumns *bool `toml:"only-output-updated-columns" json:"only-output-updated-columns,omitempty"` - - // DeleteOnlyOutputHandleKeyColumns is only available when the downstream is MQ. - DeleteOnlyOutputHandleKeyColumns *bool `toml:"delete-only-output-handle-key-columns" json:"delete-only-output-handle-key-columns,omitempty"` - - // ContentCompatible is only available when the downstream is MQ. - ContentCompatible *bool `toml:"content-compatible" json:"content-compatible,omitempty"` - - // TiDBSourceID is the source ID of the upstream TiDB, - // which is used to set the `tidb_cdc_write_source` session variable. - // Note: This field is only used internally and only used in the MySQL sink. - TiDBSourceID uint64 `toml:"-" json:"-"` - // SafeMode is only available when the downstream is DB. - SafeMode *bool `toml:"safe-mode" json:"safe-mode,omitempty"` - KafkaConfig *KafkaConfig `toml:"kafka-config" json:"kafka-config,omitempty"` - PulsarConfig *PulsarConfig `toml:"pulsar-config" json:"pulsar-config,omitempty"` - MySQLConfig *MySQLConfig `toml:"mysql-config" json:"mysql-config,omitempty"` - CloudStorageConfig *CloudStorageConfig `toml:"cloud-storage-config" json:"cloud-storage-config,omitempty"` - - // AdvanceTimeoutInSec is a duration in second. If a table sink progress hasn't been - // advanced for this given duration, the sink will be canceled and re-established. - // Deprecated since v8.1.1 - AdvanceTimeoutInSec *uint `toml:"advance-timeout-in-sec" json:"advance-timeout-in-sec,omitempty"` - - // Simple Protocol only config, use to control the behavior of sending bootstrap message. - // Note: When one of the following conditions is set to negative value, - // bootstrap sending function will be disabled. - // SendBootstrapIntervalInSec is the interval in seconds to send bootstrap message. - SendBootstrapIntervalInSec *int64 `toml:"send-bootstrap-interval-in-sec" json:"send-bootstrap-interval-in-sec,omitempty"` - // SendBootstrapInMsgCount means bootstrap messages are being sent every SendBootstrapInMsgCount row change messages. - SendBootstrapInMsgCount *int32 `toml:"send-bootstrap-in-msg-count" json:"send-bootstrap-in-msg-count,omitempty"` - // SendBootstrapToAllPartition determines whether to send bootstrap message to all partitions. - // If set to false, bootstrap message will only be sent to the first partition of each topic. - // Default value is true. - SendBootstrapToAllPartition *bool `toml:"send-bootstrap-to-all-partition" json:"send-bootstrap-to-all-partition,omitempty"` - // SendAllBootstrapAtStart determines whether to send all tables bootstrap message at changefeed start. - SendAllBootstrapAtStart *bool `toml:"send-all-bootstrap-at-start" json:"send-all-bootstrap-at-start,omitempty"` - // Debezium only. Whether schema should be excluded in the output. - DebeziumDisableSchema *bool `toml:"debezium-disable-schema" json:"debezium-disable-schema,omitempty"` - - // CSVConfig is only available when the downstream is Storage. - CSVConfig *CSVConfig `toml:"csv" json:"csv,omitempty"` - // OpenProtocol related configurations - OpenProtocol *OpenProtocolConfig `toml:"open" json:"open,omitempty"` - // DebeziumConfig related configurations - Debezium *DebeziumConfig `toml:"debezium" json:"debezium,omitempty"` - - CaseSensitive bool `toml:"case-sensitive" json:"case-sensitive"` - // Integrity is only available when the downstream is MQ. - Integrity *integrity.Config `toml:"integrity" json:"integrity"` - ForceReplicate bool `toml:"force-replicate" json:"force-replicate"` -} - -// MaskSensitiveData masks sensitive data in SinkConfig -func (s *SinkConfig) MaskSensitiveData() { - if s.SchemaRegistry != nil { - s.SchemaRegistry = aws.String(util.MaskSensitiveDataInURI(*s.SchemaRegistry)) - } - if s.KafkaConfig != nil { - s.KafkaConfig.MaskSensitiveData() - } - if s.PulsarConfig != nil { - s.PulsarConfig.MaskSensitiveData() - } -} - -// ShouldSendBootstrapMsg returns whether the sink should send bootstrap message. -// Only enable bootstrap sending function for simple protocol -// and when both send-bootstrap-interval-in-sec and send-bootstrap-in-msg-count are > 0 -func (s *SinkConfig) ShouldSendBootstrapMsg() bool { - if s == nil { - return false - } - protocol := util.GetOrZero(s.Protocol) - - return protocol == config.ProtocolSimple.String() && - util.GetOrZero(s.SendBootstrapIntervalInSec) > 0 && - util.GetOrZero(s.SendBootstrapInMsgCount) > 0 -} - -// ShouldSendAllBootstrapAtStart returns whether the should send all bootstrap message at changefeed start. -func (s *SinkConfig) ShouldSendAllBootstrapAtStart() bool { - if s == nil { - return false - } - should := s.ShouldSendBootstrapMsg() && util.GetOrZero(s.SendAllBootstrapAtStart) - log.Info("should send all bootstrap at start", zap.Bool("should", should)) - return should -} - -// CSVConfig defines a series of configuration items for csv codec. -type CSVConfig struct { - // delimiter between fields, it can be 1 character or at most 2 characters - // It can not be CR or LF or contains CR or LF. - // It should have exclusive characters with quote. - Delimiter string `toml:"delimiter" json:"delimiter"` - // quoting character - Quote string `toml:"quote" json:"quote"` - // representation of null values - NullString string `toml:"null" json:"null"` - // whether to include commit ts - IncludeCommitTs bool `toml:"include-commit-ts" json:"include-commit-ts"` - // encoding method of binary type - BinaryEncodingMethod string `toml:"binary-encoding-method" json:"binary-encoding-method"` - // output old value - OutputOldValue bool `toml:"output-old-value" json:"output-old-value"` - // output handle key - OutputHandleKey bool `toml:"output-handle-key" json:"output-handle-key"` -} - -func (c *CSVConfig) validateAndAdjust() error { - if c == nil { - return nil - } - - // validate quote - if len(c.Quote) > 1 { - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config quote contains more than one character")) - } - if len(c.Quote) == 1 { - quote := c.Quote[0] - if quote == CR || quote == LF { - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config quote cannot be line break character")) - } - } - - // validate delimiter - switch len(c.Delimiter) { - case 0: - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config delimiter cannot be empty")) - case 1, 2, 3: - if strings.ContainsRune(c.Delimiter, CR) || strings.ContainsRune(c.Delimiter, LF) { - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config delimiter contains line break characters")) - } - default: - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config delimiter contains more than three characters, note that escape "+ - "sequences can only be used in double quotes in toml configuration items.")) - } - - if len(c.Quote) > 0 { - for _, r := range c.Delimiter { - if strings.ContainsRune(c.Quote, r) { - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config quote and delimiter has common characters which is not allowed")) - } - } - } - - // validate binary encoding method - switch c.BinaryEncodingMethod { - case BinaryEncodingHex, BinaryEncodingBase64: - default: - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New("csv config binary-encoding-method can only be hex or base64")) - } - - return nil -} - -// DateSeparator specifies the date separator in storage destination path -type DateSeparator int - -// Enum types of DateSeparator -const ( - DateSeparatorNone DateSeparator = iota - DateSeparatorYear - DateSeparatorMonth - DateSeparatorDay -) - -// FromString converts the separator from string to DateSeperator enum type. -func (d *DateSeparator) FromString(separator string) error { - switch strings.ToLower(separator) { - case "none": - *d = DateSeparatorNone - case "year": - *d = DateSeparatorYear - case "month": - *d = DateSeparatorMonth - case "day": - *d = DateSeparatorDay - default: - return cerror.ErrStorageSinkInvalidDateSeparator.GenWithStackByArgs(separator) - } - - return nil -} - -// GetPattern returns the pattern of the date separator. -func (d DateSeparator) GetPattern() string { - switch d { - case DateSeparatorNone: - return "" - case DateSeparatorYear: - return `\d{4}` - case DateSeparatorMonth: - return `\d{4}-\d{2}` - case DateSeparatorDay: - return `\d{4}-\d{2}-\d{2}` - default: - return "" - } -} - -func (d DateSeparator) String() string { - switch d { - case DateSeparatorNone: - return "none" - case DateSeparatorYear: - return "year" - case DateSeparatorMonth: - return "month" - case DateSeparatorDay: - return "day" - default: - return "unknown" - } -} - -// DispatchRule represents partition rule for a table. -type DispatchRule struct { - Matcher []string `toml:"matcher" json:"matcher"` - // Deprecated, please use PartitionRule. - DispatcherRule string `toml:"dispatcher" json:"dispatcher"` - // PartitionRule is an alias added for DispatcherRule to mitigate confusions. - // In the future release, the DispatcherRule is expected to be removed . - PartitionRule string `toml:"partition" json:"partition"` - - // IndexName is set when using index-value dispatcher with specified index. - IndexName string `toml:"index" json:"index"` - - // Columns are set when using columns dispatcher. - Columns []string `toml:"columns" json:"columns"` - - TopicRule string `toml:"topic" json:"topic"` -} - -// ColumnSelector represents a column selector for a table. -type ColumnSelector struct { - Matcher []string `toml:"matcher" json:"matcher"` - Columns []string `toml:"columns" json:"columns"` -} - -// CodecConfig represents a MQ codec configuration -type CodecConfig struct { - EnableTiDBExtension *bool `toml:"enable-tidb-extension" json:"enable-tidb-extension,omitempty"` - MaxBatchSize *int `toml:"max-batch-size" json:"max-batch-size,omitempty"` - AvroEnableWatermark *bool `toml:"avro-enable-watermark" json:"avro-enable-watermark"` - AvroDecimalHandlingMode *string `toml:"avro-decimal-handling-mode" json:"avro-decimal-handling-mode,omitempty"` - AvroBigintUnsignedHandlingMode *string `toml:"avro-bigint-unsigned-handling-mode" json:"avro-bigint-unsigned-handling-mode,omitempty"` - EncodingFormat *string `toml:"encoding-format" json:"encoding-format,omitempty"` -} - -// KafkaConfig represents a kafka sink configuration -type KafkaConfig struct { - PartitionNum *int32 `toml:"partition-num" json:"partition-num,omitempty"` - ReplicationFactor *int16 `toml:"replication-factor" json:"replication-factor,omitempty"` - KafkaVersion *string `toml:"kafka-version" json:"kafka-version,omitempty"` - MaxMessageBytes *int `toml:"max-message-bytes" json:"max-message-bytes,omitempty"` - Compression *string `toml:"compression" json:"compression,omitempty"` - KafkaClientID *string `toml:"kafka-client-id" json:"kafka-client-id,omitempty"` - AutoCreateTopic *bool `toml:"auto-create-topic" json:"auto-create-topic,omitempty"` - DialTimeout *string `toml:"dial-timeout" json:"dial-timeout,omitempty"` - WriteTimeout *string `toml:"write-timeout" json:"write-timeout,omitempty"` - ReadTimeout *string `toml:"read-timeout" json:"read-timeout,omitempty"` - RequiredAcks *int `toml:"required-acks" json:"required-acks,omitempty"` - SASLUser *string `toml:"sasl-user" json:"sasl-user,omitempty"` - SASLPassword *string `toml:"sasl-password" json:"sasl-password,omitempty"` - SASLMechanism *string `toml:"sasl-mechanism" json:"sasl-mechanism,omitempty"` - SASLGssAPIAuthType *string `toml:"sasl-gssapi-auth-type" json:"sasl-gssapi-auth-type,omitempty"` - SASLGssAPIKeytabPath *string `toml:"sasl-gssapi-keytab-path" json:"sasl-gssapi-keytab-path,omitempty"` - SASLGssAPIKerberosConfigPath *string `toml:"sasl-gssapi-kerberos-config-path" json:"sasl-gssapi-kerberos-config-path,omitempty"` - SASLGssAPIServiceName *string `toml:"sasl-gssapi-service-name" json:"sasl-gssapi-service-name,omitempty"` - SASLGssAPIUser *string `toml:"sasl-gssapi-user" json:"sasl-gssapi-user,omitempty"` - SASLGssAPIPassword *string `toml:"sasl-gssapi-password" json:"sasl-gssapi-password,omitempty"` - SASLGssAPIRealm *string `toml:"sasl-gssapi-realm" json:"sasl-gssapi-realm,omitempty"` - SASLGssAPIDisablePafxfast *bool `toml:"sasl-gssapi-disable-pafxfast" json:"sasl-gssapi-disable-pafxfast,omitempty"` - SASLOAuthClientID *string `toml:"sasl-oauth-client-id" json:"sasl-oauth-client-id,omitempty"` - SASLOAuthClientSecret *string `toml:"sasl-oauth-client-secret" json:"sasl-oauth-client-secret,omitempty"` - SASLOAuthTokenURL *string `toml:"sasl-oauth-token-url" json:"sasl-oauth-token-url,omitempty"` - SASLOAuthScopes []string `toml:"sasl-oauth-scopes" json:"sasl-oauth-scopes,omitempty"` - SASLOAuthGrantType *string `toml:"sasl-oauth-grant-type" json:"sasl-oauth-grant-type,omitempty"` - SASLOAuthAudience *string `toml:"sasl-oauth-audience" json:"sasl-oauth-audience,omitempty"` - EnableTLS *bool `toml:"enable-tls" json:"enable-tls,omitempty"` - CA *string `toml:"ca" json:"ca,omitempty"` - Cert *string `toml:"cert" json:"cert,omitempty"` - Key *string `toml:"key" json:"key,omitempty"` - InsecureSkipVerify *bool `toml:"insecure-skip-verify" json:"insecure-skip-verify,omitempty"` - CodecConfig *CodecConfig `toml:"codec-config" json:"codec-config,omitempty"` - LargeMessageHandle *config.LargeMessageHandleConfig `toml:"large-message-handle" json:"large-message-handle,omitempty"` - GlueSchemaRegistryConfig *GlueSchemaRegistryConfig `toml:"glue-schema-registry-config" json:"glue-schema-registry-config"` - - // OutputRawChangeEvent controls whether to split the update pk/uk events. - OutputRawChangeEvent *bool `toml:"output-raw-change-event" json:"output-raw-change-event,omitempty"` -} - -// GetOutputRawChangeEvent returns the value of OutputRawChangeEvent -func (k *KafkaConfig) GetOutputRawChangeEvent() bool { - if k == nil || k.OutputRawChangeEvent == nil { - return false - } - return *k.OutputRawChangeEvent -} - -// MaskSensitiveData masks sensitive data in KafkaConfig -func (k *KafkaConfig) MaskSensitiveData() { - k.SASLPassword = aws.String("******") - k.SASLGssAPIPassword = aws.String("******") - k.SASLOAuthClientSecret = aws.String("******") - k.Key = aws.String("******") - if k.GlueSchemaRegistryConfig != nil { - k.GlueSchemaRegistryConfig.AccessKey = "******" - k.GlueSchemaRegistryConfig.Token = "******" - k.GlueSchemaRegistryConfig.SecretAccessKey = "******" - } - if k.SASLOAuthTokenURL != nil { - k.SASLOAuthTokenURL = aws.String(util.MaskSensitiveDataInURI(*k.SASLOAuthTokenURL)) - } -} - -// PulsarCompressionType is the compression type for pulsar -type PulsarCompressionType string - -// Value returns the pulsar compression type -func (p *PulsarCompressionType) Value() pulsar.CompressionType { - if p == nil { - return 0 - } - switch strings.ToLower(string(*p)) { - case "lz4": - return pulsar.LZ4 - case "zlib": - return pulsar.ZLib - case "zstd": - return pulsar.ZSTD - default: - return 0 // default is no compression - } -} - -// TimeMill is the time in milliseconds -type TimeMill int - -// Duration returns the time in seconds as a duration -func (t *TimeMill) Duration() time.Duration { - if t == nil { - return 0 - } - return time.Duration(*t) * time.Millisecond -} - -// NewTimeMill returns a new time in milliseconds -func NewTimeMill(x int) *TimeMill { - t := TimeMill(x) - return &t -} - -// TimeSec is the time in seconds -type TimeSec int - -// Duration returns the time in seconds as a duration -func (t *TimeSec) Duration() time.Duration { - if t == nil { - return 0 - } - return time.Duration(*t) * time.Second -} - -// NewTimeSec returns a new time in seconds -func NewTimeSec(x int) *TimeSec { - t := TimeSec(x) - return &t -} - -// OAuth2 is the configuration for OAuth2 -type OAuth2 struct { - // OAuth2IssuerURL the URL of the authorization server. - OAuth2IssuerURL string `toml:"oauth2-issuer-url" json:"oauth2-issuer-url,omitempty"` - // OAuth2Audience the URL of the resource server. - OAuth2Audience string `toml:"oauth2-audience" json:"oauth2-audience,omitempty"` - // OAuth2PrivateKey the private key used to sign the server. - OAuth2PrivateKey string `toml:"oauth2-private-key" json:"oauth2-private-key,omitempty"` - // OAuth2ClientID the client ID of the application. - OAuth2ClientID string `toml:"oauth2-client-id" json:"oauth2-client-id,omitempty"` - // OAuth2Scope scope - OAuth2Scope string `toml:"oauth2-scope" json:"oauth2-scope,omitempty"` -} - -func (o *OAuth2) validate() (err error) { - if o == nil { - return nil - } - if len(o.OAuth2IssuerURL) == 0 || len(o.OAuth2ClientID) == 0 || len(o.OAuth2PrivateKey) == 0 || - len(o.OAuth2Audience) == 0 { - return fmt.Errorf("issuer-url and audience and private-key and client-id not be empty") - } - return nil -} - -// PulsarConfig pulsar sink configuration -type PulsarConfig struct { - TLSKeyFilePath *string `toml:"tls-key-file-path" json:"tls-key-file-path,omitempty"` - TLSCertificateFile *string `toml:"tls-certificate-file" json:"tls-certificate-file,omitempty"` - TLSTrustCertsFilePath *string `toml:"tls-trust-certs-file-path" json:"tls-trust-certs-file-path,omitempty"` - - // PulsarProducerCacheSize is the size of the cache of pulsar producers - PulsarProducerCacheSize *int32 `toml:"pulsar-producer-cache-size" json:"pulsar-producer-cache-size,omitempty"` - - // PulsarVersion print the version of pulsar - PulsarVersion *string `toml:"pulsar-version" json:"pulsar-version,omitempty"` - - // pulsar client compression - CompressionType *PulsarCompressionType `toml:"compression-type" json:"compression-type,omitempty"` - - // AuthenticationToken the token for the Pulsar server - AuthenticationToken *string `toml:"authentication-token" json:"authentication-token,omitempty"` - - // ConnectionTimeout Timeout for the establishment of a TCP connection (default: 5 seconds) - ConnectionTimeout *TimeSec `toml:"connection-timeout" json:"connection-timeout,omitempty"` - - // Set the operation timeout (default: 30 seconds) - // Producer-create, subscribe and unsubscribe operations will be retried until this interval, after which the - // operation will be marked as failed - OperationTimeout *TimeSec `toml:"operation-timeout" json:"operation-timeout,omitempty"` - - // BatchingMaxMessages specifies the maximum number of messages permitted in a batch. (default: 1000) - BatchingMaxMessages *uint `toml:"batching-max-messages" json:"batching-max-messages,omitempty"` - - // BatchingMaxPublishDelay specifies the time period within which the messages sent will be batched (default: 10ms) - // if batch messages are enabled. If set to a non zero value, messages will be queued until this time - // interval or until - BatchingMaxPublishDelay *TimeMill `toml:"batching-max-publish-delay" json:"batching-max-publish-delay,omitempty"` - - // SendTimeout specifies the timeout for a message that has not been acknowledged by the server since sent. - // Send and SendAsync returns an error after timeout. - // default: 30s - SendTimeout *TimeSec `toml:"send-timeout" json:"send-timeout,omitempty"` - - // TokenFromFile Authentication from the file token, - // the path name of the file (the third priority authentication method) - TokenFromFile *string `toml:"token-from-file" json:"token-from-file,omitempty"` - - // BasicUserName Account name for pulsar basic authentication (the second priority authentication method) - BasicUserName *string `toml:"basic-user-name" json:"basic-user-name,omitempty"` - // BasicPassword with account - BasicPassword *string `toml:"basic-password" json:"basic-password,omitempty"` - - // AuthTLSCertificatePath create new pulsar authentication provider with specified TLS certificate and private key - AuthTLSCertificatePath *string `toml:"auth-tls-certificate-path" json:"auth-tls-certificate-path,omitempty"` - // AuthTLSPrivateKeyPath private key - AuthTLSPrivateKeyPath *string `toml:"auth-tls-private-key-path" json:"auth-tls-private-key-path,omitempty"` - - // Oauth2 include oauth2-issuer-url oauth2-audience oauth2-private-key oauth2-client-id - // and 'type' always use 'client_credentials' - OAuth2 *OAuth2 `toml:"oauth2" json:"oauth2,omitempty"` - - // OutputRawChangeEvent controls whether to split the update pk/uk events. - OutputRawChangeEvent *bool `toml:"output-raw-change-event" json:"output-raw-change-event,omitempty"` - - // BrokerURL is used to configure service brokerUrl for the Pulsar service. - // This parameter is a part of the `sink-uri`. Internal use only. - BrokerURL string `toml:"-" json:"-"` - // SinkURI is the parsed sinkURI. Internal use only. - SinkURI *url.URL `toml:"-" json:"-"` -} - -// GetOutputRawChangeEvent returns the value of OutputRawChangeEvent -func (c *PulsarConfig) GetOutputRawChangeEvent() bool { - if c == nil || c.OutputRawChangeEvent == nil { - return false - } - return *c.OutputRawChangeEvent -} - -// MaskSensitiveData masks sensitive data in PulsarConfig -func (c *PulsarConfig) MaskSensitiveData() { - if c.AuthenticationToken != nil { - c.AuthenticationToken = aws.String("******") - } - if c.BasicPassword != nil { - c.BasicPassword = aws.String("******") - } - if c.OAuth2 != nil { - c.OAuth2.OAuth2PrivateKey = "******" - } -} - -// Check get broker url -func (c *PulsarConfig) validate() (err error) { - if c.OAuth2 != nil { - if err = c.OAuth2.validate(); err != nil { - return err - } - } - return nil -} - -// GetDefaultTopicName get default topic name -func (c *PulsarConfig) GetDefaultTopicName() string { - topicName := c.SinkURI.Path - return topicName[1:] -} - -// MySQLConfig represents a MySQL sink configuration -type MySQLConfig struct { - WorkerCount *int `toml:"worker-count" json:"worker-count,omitempty"` - MaxTxnRow *int `toml:"max-txn-row" json:"max-txn-row,omitempty"` - MaxMultiUpdateRowSize *int `toml:"max-multi-update-row-size" json:"max-multi-update-row-size,omitempty"` - MaxMultiUpdateRowCount *int `toml:"max-multi-update-row" json:"max-multi-update-row,omitempty"` - TiDBTxnMode *string `toml:"tidb-txn-mode" json:"tidb-txn-mode,omitempty"` - SSLCa *string `toml:"ssl-ca" json:"ssl-ca,omitempty"` - SSLCert *string `toml:"ssl-cert" json:"ssl-cert,omitempty"` - SSLKey *string `toml:"ssl-key" json:"ssl-key,omitempty"` - TimeZone *string `toml:"time-zone" json:"time-zone,omitempty"` - WriteTimeout *string `toml:"write-timeout" json:"write-timeout,omitempty"` - ReadTimeout *string `toml:"read-timeout" json:"read-timeout,omitempty"` - Timeout *string `toml:"timeout" json:"timeout,omitempty"` - EnableBatchDML *bool `toml:"enable-batch-dml" json:"enable-batch-dml,omitempty"` - EnableMultiStatement *bool `toml:"enable-multi-statement" json:"enable-multi-statement,omitempty"` - EnableCachePreparedStatement *bool `toml:"enable-cache-prepared-statement" json:"enable-cache-prepared-statement,omitempty"` -} - -// CloudStorageConfig represents a cloud storage sink configuration -type CloudStorageConfig struct { - WorkerCount *int `toml:"worker-count" json:"worker-count,omitempty"` - FlushInterval *string `toml:"flush-interval" json:"flush-interval,omitempty"` - FileSize *int `toml:"file-size" json:"file-size,omitempty"` - - OutputColumnID *bool `toml:"output-column-id" json:"output-column-id,omitempty"` - FileExpirationDays *int `toml:"file-expiration-days" json:"file-expiration-days,omitempty"` - FileCleanupCronSpec *string `toml:"file-cleanup-cron-spec" json:"file-cleanup-cron-spec,omitempty"` - FlushConcurrency *int `toml:"flush-concurrency" json:"flush-concurrency,omitempty"` - - // OutputRawChangeEvent controls whether to split the update pk/uk events. - OutputRawChangeEvent *bool `toml:"output-raw-change-event" json:"output-raw-change-event,omitempty"` -} - -// GetOutputRawChangeEvent returns the value of OutputRawChangeEvent -func (c *CloudStorageConfig) GetOutputRawChangeEvent() bool { - if c == nil || c.OutputRawChangeEvent == nil { - return false - } - return *c.OutputRawChangeEvent -} - -func (s *SinkConfig) validateAndAdjust(sinkURI *url.URL) error { - if err := s.validateAndAdjustSinkURI(sinkURI); err != nil { - return err - } - - if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) { - return nil - } - - protocol, _ := config.ParseSinkProtocolFromString(util.GetOrZero(s.Protocol)) - - if s.KafkaConfig != nil && s.KafkaConfig.LargeMessageHandle != nil { - var ( - enableTiDBExtension bool - err error - ) - if s := sinkURI.Query().Get("enable-tidb-extension"); s != "" { - enableTiDBExtension, err = strconv.ParseBool(s) - if err != nil { - return errors.Trace(err) - } - } - err = s.KafkaConfig.LargeMessageHandle.AdjustAndValidate(protocol, enableTiDBExtension) - if err != nil { - return err - } - } - - if s.SchemaRegistry != nil && - (s.KafkaConfig != nil && s.KafkaConfig.GlueSchemaRegistryConfig != nil) { - return cerror.ErrInvalidReplicaConfig. - GenWithStackByArgs("schema-registry and glue-schema-registry-config" + - "cannot be set at the same time," + - "schema-registry is used by confluent schema registry, " + - "glue-schema-registry-config is used by aws glue schema registry") - } - - if s.KafkaConfig != nil && s.KafkaConfig.GlueSchemaRegistryConfig != nil { - err := s.KafkaConfig.GlueSchemaRegistryConfig.Validate() - if err != nil { - return err - } - } - - if sink.IsPulsarScheme(sinkURI.Scheme) && s.PulsarConfig == nil { - s.PulsarConfig = &PulsarConfig{ - SinkURI: sinkURI, - } - } - if s.PulsarConfig != nil { - if err := s.PulsarConfig.validate(); err != nil { - return err - } - } - - for _, rule := range s.DispatchRules { - if rule.DispatcherRule != "" && rule.PartitionRule != "" { - log.Error("dispatcher and partition cannot be configured both", zap.Any("rule", rule)) - return cerror.WrapError(cerror.ErrSinkInvalidConfig, - errors.New(fmt.Sprintf("dispatcher and partition cannot be "+ - "configured both for rule:%v", rule))) - } - // After `validate()` is called, we only use PartitionRule to represent a partition - // dispatching rule. So when DispatcherRule is not empty, we assign its - // value to PartitionRule and clear itself. - if rule.DispatcherRule != "" { - rule.PartitionRule = rule.DispatcherRule - rule.DispatcherRule = "" - } - } - - if util.GetOrZero(s.EncoderConcurrency) < 0 { - return cerror.ErrSinkInvalidConfig.GenWithStack( - "encoder-concurrency should greater than 0, but got %d", s.EncoderConcurrency) - } - - // validate terminator - if s.Terminator == nil { - s.Terminator = util.AddressOf(CRLF) - } - - if util.GetOrZero(s.DeleteOnlyOutputHandleKeyColumns) && protocol == config.ProtocolCsv { - return cerror.ErrSinkInvalidConfig.GenWithStack( - "CSV protocol always output all columns for the delete event, " + - "do not set `delete-only-output-handle-key-columns` to true") - } - - // validate storage sink related config - if sinkURI != nil && sink.IsStorageScheme(sinkURI.Scheme) { - // validate date separator - if len(util.GetOrZero(s.DateSeparator)) > 0 { - var separator DateSeparator - if err := separator.FromString(util.GetOrZero(s.DateSeparator)); err != nil { - return cerror.WrapError(cerror.ErrSinkInvalidConfig, err) - } - } - - // File index width should be in [minFileIndexWidth, maxFileIndexWidth]. - // In most scenarios, the user does not need to change this configuration, - // so the default value of this parameter is not set and just make silent - // adjustments here. - if util.GetOrZero(s.FileIndexWidth) < MinFileIndexWidth || - util.GetOrZero(s.FileIndexWidth) > MaxFileIndexWidth { - s.FileIndexWidth = util.AddressOf(DefaultFileIndexWidth) - } - - if err := s.CSVConfig.validateAndAdjust(); err != nil { - return err - } - } - - if util.GetOrZero(s.AdvanceTimeoutInSec) == 0 { - log.Warn(fmt.Sprintf("advance-timeout-in-sec is not set, use default value: %d seconds", DefaultAdvanceTimeoutInSec)) - s.AdvanceTimeoutInSec = util.AddressOf(DefaultAdvanceTimeoutInSec) - } - - return nil -} - -// validateAndAdjustSinkURI validate and adjust `Protocol` and `TxnAtomicity` by sinkURI. -func (s *SinkConfig) validateAndAdjustSinkURI(sinkURI *url.URL) error { - if sinkURI == nil { - return nil - } - - if err := s.applyParameterBySinkURI(sinkURI); err != nil { - if !cerror.ErrIncompatibleSinkConfig.Equal(err) { - return err - } - // Ignore `ErrIncompatibleSinkConfig` here to: - // 1. Keep compatibility with old version. - // 2. Avoid throwing error when create changefeed. - log.Warn("sink-uri is not compatible with the sink config, "+ - "the configuration in sink URI will be used", zap.Error(err)) - } - - // validate that TxnAtomicity is valid and compatible with the scheme. - if err := util.GetOrZero(s.TxnAtomicity).validate(sinkURI.Scheme); err != nil { - return err - } - - log.Info("succeed to parse parameter from sink uri", - zap.String("protocol", util.GetOrZero(s.Protocol)), - zap.String("txnAtomicity", string(util.GetOrZero(s.TxnAtomicity)))) - - // Check that protocol config is compatible with the scheme. - if sink.IsMySQLCompatibleScheme(sinkURI.Scheme) && s.Protocol != nil { - return cerror.ErrSinkURIInvalid.GenWithStackByArgs(fmt.Sprintf("protocol %s "+ - "is incompatible with %s scheme", util.GetOrZero(s.Protocol), sinkURI.Scheme)) - } - // For testing purposes, any protocol should be legal for blackhole. - if sink.IsMQScheme(sinkURI.Scheme) || sink.IsStorageScheme(sinkURI.Scheme) { - return s.ValidateProtocol(sinkURI.Scheme) - } - return nil -} - -// ValidateProtocol validates the protocol configuration. -func (s *SinkConfig) ValidateProtocol(scheme string) error { - protocol, err := config.ParseSinkProtocolFromString(util.GetOrZero(s.Protocol)) - if err != nil { - return err - } - outputOldValue := false - switch protocol { - case config.ProtocolOpen: - if s.OpenProtocol != nil { - outputOldValue = s.OpenProtocol.OutputOldValue - } - case config.ProtocolDebezium: - if s.Debezium != nil { - outputOldValue = s.Debezium.OutputOldValue - } - case config.ProtocolCsv: - if s.CSVConfig != nil { - outputOldValue = s.CSVConfig.OutputOldValue - } - case config.ProtocolAvro: - outputOldValue = false - default: - return nil - } - - outputRawChangeEvent := false - switch scheme { - case sink.KafkaScheme, sink.KafkaSSLScheme: - outputRawChangeEvent = s.KafkaConfig.GetOutputRawChangeEvent() - case sink.PulsarScheme, sink.PulsarSSLScheme, sink.PulsarHTTPScheme, sink.PulsarHTTPSScheme: - outputRawChangeEvent = s.PulsarConfig.GetOutputRawChangeEvent() - default: - outputRawChangeEvent = s.CloudStorageConfig.GetOutputRawChangeEvent() - } - - if outputRawChangeEvent && !outputOldValue { - // TODO: return error if we do not need to keep backward compatibility. - log.Warn(fmt.Sprintf("TiCDC will not split the update pk/uk events if output-raw-change-event is true(scheme: %s).", scheme) + - fmt.Sprintf("It is not recommended to set output-old-value to false(protocol: %s) in this case.", protocol) + - "Otherwise, there may be data consistency issues in update pk/uk scenarios due to lack of old value information.") - } - return nil -} - -// applyParameterBySinkURI parse sinkURI and set `Protocol` and `TxnAtomicity` to `SinkConfig`. -// Return: -// - ErrIncompatibleSinkConfig to terminate `updated` changefeed operation. -func (s *SinkConfig) applyParameterBySinkURI(sinkURI *url.URL) error { - if sinkURI == nil { - return nil - } - - params := sinkURI.Query() - var errFromURI, errFromFile strings.Builder - - txnAtomicityFromURI := AtomicityLevel(params.Get(TxnAtomicityKey)) - if txnAtomicityFromURI != unknownTxnAtomicity { - if util.GetOrZero(s.TxnAtomicity) != unknownTxnAtomicity && util.GetOrZero(s.TxnAtomicity) != txnAtomicityFromURI { - errFromURI.WriteString(fmt.Sprintf("%s=%s, ", TxnAtomicityKey, txnAtomicityFromURI)) - errFromFile.WriteString(fmt.Sprintf("%s=%s, ", TxnAtomicityKey, util.GetOrZero(s.TxnAtomicity))) - } - s.TxnAtomicity = util.AddressOf(txnAtomicityFromURI) - } - - protocolFromURI := params.Get(config.ProtocolKey) - if protocolFromURI != "" { - if s.Protocol != nil && util.GetOrZero(s.Protocol) != protocolFromURI { - errFromURI.WriteString(fmt.Sprintf("%s=%s, ", config.ProtocolKey, protocolFromURI)) - errFromFile.WriteString(fmt.Sprintf("%s=%s, ", config.ProtocolKey, util.GetOrZero(s.Protocol))) - } - s.Protocol = util.AddressOf(protocolFromURI) - } - - if errFromURI.Len() == 0 && errFromFile.Len() == 0 { - return nil - } - return cerror.ErrIncompatibleSinkConfig.GenWithStackByArgs( - errFromURI.String()[0:errFromURI.Len()-2], errFromFile.String()[0:errFromFile.Len()-2]) -} - -// CheckCompatibilityWithSinkURI check whether the sinkURI is compatible with the sink config. -func (s *SinkConfig) CheckCompatibilityWithSinkURI( - oldSinkConfig *SinkConfig, sinkURIStr string, -) error { - sinkURI, err := url.Parse(sinkURIStr) - if err != nil { - return cerror.WrapError(cerror.ErrSinkURIInvalid, err) - } - - cfgParamsChanged := s.Protocol != oldSinkConfig.Protocol || - s.TxnAtomicity != oldSinkConfig.TxnAtomicity - - isURIParamsChanged := func(oldCfg SinkConfig) bool { - err := oldCfg.applyParameterBySinkURI(sinkURI) - return cerror.ErrIncompatibleSinkConfig.Equal(err) - } - uriParamsChanged := isURIParamsChanged(*oldSinkConfig) - - if !uriParamsChanged && !cfgParamsChanged { - return nil - } - - compatibilityError := s.applyParameterBySinkURI(sinkURI) - if uriParamsChanged && cerror.ErrIncompatibleSinkConfig.Equal(compatibilityError) { - // Ignore compatibility error if the sinkURI make such changes. - return nil - } - return compatibilityError -} - -// GlueSchemaRegistryConfig represents a Glue Schema Registry configuration -type GlueSchemaRegistryConfig struct { - // Name of the schema registry - RegistryName string `toml:"registry-name" json:"registry-name"` - // Region of the schema registry - Region string `toml:"region" json:"region"` - // AccessKey of the schema registry - AccessKey string `toml:"access-key" json:"access-key,omitempty"` - // SecretAccessKey of the schema registry - SecretAccessKey string `toml:"secret-access-key" json:"secret-access-key,omitempty"` - Token string `toml:"token" json:"token,omitempty"` -} - -// Validate the GlueSchemaRegistryConfig. -func (g *GlueSchemaRegistryConfig) Validate() error { - if g.RegistryName == "" { - return cerror.ErrInvalidGlueSchemaRegistryConfig. - GenWithStack("registry-name is empty, is must be set") - } - if g.Region == "" { - return cerror.ErrInvalidGlueSchemaRegistryConfig. - GenWithStack("region is empty, is must be set") - } - if g.AccessKey != "" && g.SecretAccessKey == "" { - return cerror.ErrInvalidGlueSchemaRegistryConfig. - GenWithStack("access-key is set, but access-key-secret is empty, they must be set together") - } - return nil -} - -// NoCredentials returns true if no credentials are set. -func (g *GlueSchemaRegistryConfig) NoCredentials() bool { - return g.AccessKey == "" && g.SecretAccessKey == "" && g.Token == "" -} - -// OpenProtocolConfig represents the configurations for open protocol encoding -type OpenProtocolConfig struct { - OutputOldValue bool `toml:"output-old-value" json:"output-old-value"` -} - -// DebeziumConfig represents the configurations for debezium protocol encoding -type DebeziumConfig struct { - OutputOldValue bool `toml:"output-old-value" json:"output-old-value"` -} diff --git a/pkg/sink/codec/avro/arvo.go b/pkg/sink/codec/avro/arvo.go index 94dc7e952..4383067e3 100644 --- a/pkg/sink/codec/avro/arvo.go +++ b/pkg/sink/codec/avro/arvo.go @@ -1011,7 +1011,7 @@ const ( ) // NewAvroEncoder return a avro encoder. -func NewAvroEncoder(ctx context.Context, config *ticommon.Config) (encoder.EventEncoder, error) { +func NewAvroEncoder(ctx context.Context, config *ticommon.Config) (encoder.RowEventEncoder, error) { var schemaM SchemaManager var err error diff --git a/pkg/sink/codec/bootstraper.go b/pkg/sink/codec/bootstraper.go index f7dcf151d..7c30f49be 100644 --- a/pkg/sink/codec/bootstraper.go +++ b/pkg/sink/codec/bootstraper.go @@ -40,7 +40,7 @@ const ( type bootstrapWorker struct { changefeedID model.ChangeFeedID activeTables sync.Map - rowEventEncoder encoder.EventEncoder + rowEventEncoder encoder.RowEventEncoder sendBootstrapInterval time.Duration sendBootstrapInMsgCount int32 sendBootstrapToAllPartition bool @@ -53,7 +53,7 @@ type bootstrapWorker struct { func newBootstrapWorker( changefeedID model.ChangeFeedID, outCh chan<- *future, - rowEventEncoder encoder.EventEncoder, + rowEventEncoder encoder.RowEventEncoder, sendBootstrapInterval int64, sendBootstrapInMsgCount int32, sendBootstrapToAllPartition bool, diff --git a/pkg/sink/codec/canal/canal_encoder.go b/pkg/sink/codec/canal/canal_encoder.go index 3baaaaa2b..c7ac92266 100644 --- a/pkg/sink/codec/canal/canal_encoder.go +++ b/pkg/sink/codec/canal/canal_encoder.go @@ -162,7 +162,7 @@ func (d *BatchEncoder) resetPacket() { func (d *BatchEncoder) Clean() {} // newBatchEncoder creates a new canalBatchEncoder. -func NewBatchEncoder(config *ticommon.Config) (encoder.EventEncoder, error) { +func NewBatchEncoder(config *ticommon.Config) (encoder.RowEventEncoder, error) { encoder := &BatchEncoder{ messages: &canal.Messages{}, callbackBuf: make([]func(), 0), diff --git a/pkg/sink/codec/canal/canal_json_row_event_encoder.go b/pkg/sink/codec/canal/canal_json_row_event_encoder.go index 65d512cf8..65fc5629d 100644 --- a/pkg/sink/codec/canal/canal_json_row_event_encoder.go +++ b/pkg/sink/codec/canal/canal_json_row_event_encoder.go @@ -313,7 +313,7 @@ type JSONRowEventEncoder struct { } // newJSONRowEventEncoder creates a new JSONRowEventEncoder -func NewJSONRowEventEncoder(ctx context.Context, config *ticommon.Config) (encoder.EventEncoder, error) { +func NewJSONRowEventEncoder(ctx context.Context, config *ticommon.Config) (encoder.RowEventEncoder, error) { claimCheck, err := claimcheck.New(ctx, config.LargeMessageHandle, config.ChangefeedID) if err != nil { return nil, errors.Trace(err) diff --git a/pkg/sink/codec/common/config.go b/pkg/sink/codec/common/config.go index 25c76ff4b..be97c6a20 100644 --- a/pkg/sink/codec/common/config.go +++ b/pkg/sink/codec/common/config.go @@ -18,7 +18,6 @@ import ( "net/url" "time" - ticonfig "github.com/flowbehappy/tigate/pkg/config" "github.com/gin-gonic/gin/binding" "github.com/imdario/mergo" "github.com/pingcap/errors" @@ -55,7 +54,7 @@ type Config struct { AvroConfluentSchemaRegistry string AvroDecimalHandlingMode string AvroBigintUnsignedHandlingMode string - AvroGlueSchemaRegistry *ticonfig.GlueSchemaRegistryConfig + AvroGlueSchemaRegistry *config.GlueSchemaRegistryConfig // EnableWatermarkEvent set to true, avro encode DDL and checkpoint event // and send to the downstream kafka, they cannot be consumed by the confluent official consumer // and would cause error, so this is only used for ticdc internal testing purpose, should not be @@ -175,14 +174,14 @@ type urlConfig struct { } // Apply fill the Config -func (c *Config) Apply(sinkURI *url.URL, sinkConfig *ticonfig.SinkConfig) error { +func (c *Config) Apply(sinkURI *url.URL, replicaConfig *config.ReplicaConfig) error { req := &http.Request{URL: sinkURI} var err error urlParameter := &urlConfig{} if err := binding.Query.Bind(req, urlParameter); err != nil { return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) } - if urlParameter, err = mergeConfig(sinkConfig, urlParameter); err != nil { + if urlParameter, err = mergeConfig(replicaConfig, urlParameter); err != nil { return err } @@ -215,51 +214,51 @@ func (c *Config) Apply(sinkURI *url.URL, sinkConfig *ticonfig.SinkConfig) error if urlParameter.AvroSchemaRegistry != "" { c.AvroConfluentSchemaRegistry = urlParameter.AvroSchemaRegistry } - if sinkConfig.KafkaConfig != nil && - sinkConfig.KafkaConfig.GlueSchemaRegistryConfig != nil { - c.AvroGlueSchemaRegistry = sinkConfig.KafkaConfig.GlueSchemaRegistryConfig + if replicaConfig.Sink.KafkaConfig != nil && + replicaConfig.Sink.KafkaConfig.GlueSchemaRegistryConfig != nil { + c.AvroGlueSchemaRegistry = replicaConfig.Sink.KafkaConfig.GlueSchemaRegistryConfig } - if c.Protocol == config.ProtocolAvro && sinkConfig.ForceReplicate { + if c.Protocol == config.ProtocolAvro && replicaConfig.ForceReplicate { return cerror.ErrCodecInvalidConfig.GenWithStack( `force-replicate must be disabled, when using avro protocol`) } - if sinkConfig != nil { - c.Terminator = util.GetOrZero(sinkConfig.Terminator) - if sinkConfig.CSVConfig != nil { - c.Delimiter = sinkConfig.CSVConfig.Delimiter - c.Quote = sinkConfig.CSVConfig.Quote - c.NullString = sinkConfig.CSVConfig.NullString - c.IncludeCommitTs = sinkConfig.CSVConfig.IncludeCommitTs - c.BinaryEncodingMethod = sinkConfig.CSVConfig.BinaryEncodingMethod - c.OutputOldValue = sinkConfig.CSVConfig.OutputOldValue - c.OutputHandleKey = sinkConfig.CSVConfig.OutputHandleKey + if replicaConfig.Sink != nil { + c.Terminator = util.GetOrZero(replicaConfig.Sink.Terminator) + if replicaConfig.Sink.CSVConfig != nil { + c.Delimiter = replicaConfig.Sink.CSVConfig.Delimiter + c.Quote = replicaConfig.Sink.CSVConfig.Quote + c.NullString = replicaConfig.Sink.CSVConfig.NullString + c.IncludeCommitTs = replicaConfig.Sink.CSVConfig.IncludeCommitTs + c.BinaryEncodingMethod = replicaConfig.Sink.CSVConfig.BinaryEncodingMethod + c.OutputOldValue = replicaConfig.Sink.CSVConfig.OutputOldValue + c.OutputHandleKey = replicaConfig.Sink.CSVConfig.OutputHandleKey } - if sinkConfig.KafkaConfig != nil && sinkConfig.KafkaConfig.LargeMessageHandle != nil { - c.LargeMessageHandle = sinkConfig.KafkaConfig.LargeMessageHandle + if replicaConfig.Sink.KafkaConfig != nil && replicaConfig.Sink.KafkaConfig.LargeMessageHandle != nil { + c.LargeMessageHandle = replicaConfig.Sink.KafkaConfig.LargeMessageHandle } - if !c.LargeMessageHandle.Disabled() && sinkConfig.ForceReplicate { + if !c.LargeMessageHandle.Disabled() && replicaConfig.ForceReplicate { return cerror.ErrCodecInvalidConfig.GenWithStack( `force-replicate must be disabled, when the large message handle is enabled, large message handle: "%s"`, c.LargeMessageHandle.LargeMessageHandleOption) } - if sinkConfig.OpenProtocol != nil { - c.OpenOutputOldValue = sinkConfig.OpenProtocol.OutputOldValue + if replicaConfig.Sink.OpenProtocol != nil { + c.OpenOutputOldValue = replicaConfig.Sink.OpenProtocol.OutputOldValue } - if sinkConfig.Debezium != nil { - c.DebeziumOutputOldValue = sinkConfig.Debezium.OutputOldValue + if replicaConfig.Sink.Debezium != nil { + c.DebeziumOutputOldValue = replicaConfig.Sink.Debezium.OutputOldValue } } if urlParameter.OnlyOutputUpdatedColumns != nil { c.OnlyOutputUpdatedColumns = *urlParameter.OnlyOutputUpdatedColumns } - if sinkConfig.Integrity != nil { - c.EnableRowChecksum = sinkConfig.Integrity.Enabled() + if replicaConfig.Integrity != nil { + c.EnableRowChecksum = replicaConfig.Integrity.Enabled() } - c.DeleteOnlyHandleKeyColumns = util.GetOrZero(sinkConfig.DeleteOnlyOutputHandleKeyColumns) - if c.DeleteOnlyHandleKeyColumns && sinkConfig.ForceReplicate { + c.DeleteOnlyHandleKeyColumns = util.GetOrZero(replicaConfig.Sink.DeleteOnlyOutputHandleKeyColumns) + if c.DeleteOnlyHandleKeyColumns && replicaConfig.ForceReplicate { return cerror.ErrCodecInvalidConfig.GenWithStack( `force-replicate must be disabled when configuration "delete-only-output-handle-key-columns" is true.`) } @@ -292,21 +291,21 @@ func (c *Config) Apply(sinkURI *url.URL, sinkConfig *ticonfig.SinkConfig) error } func mergeConfig( - sinkConfig *ticonfig.SinkConfig, + replicaConfig *config.ReplicaConfig, urlParameters *urlConfig, ) (*urlConfig, error) { dest := &urlConfig{} - if sinkConfig != nil { - dest.AvroSchemaRegistry = util.GetOrZero(sinkConfig.SchemaRegistry) - dest.OnlyOutputUpdatedColumns = sinkConfig.OnlyOutputUpdatedColumns - dest.ContentCompatible = sinkConfig.ContentCompatible + if replicaConfig.Sink != nil { + dest.AvroSchemaRegistry = util.GetOrZero(replicaConfig.Sink.SchemaRegistry) + dest.OnlyOutputUpdatedColumns = replicaConfig.Sink.OnlyOutputUpdatedColumns + dest.ContentCompatible = replicaConfig.Sink.ContentCompatible if util.GetOrZero(dest.ContentCompatible) { dest.OnlyOutputUpdatedColumns = util.AddressOf(true) } - if sinkConfig.KafkaConfig != nil { - dest.MaxMessageBytes = sinkConfig.KafkaConfig.MaxMessageBytes - if sinkConfig.KafkaConfig.CodecConfig != nil { - codecConfig := sinkConfig.KafkaConfig.CodecConfig + if replicaConfig.Sink.KafkaConfig != nil { + dest.MaxMessageBytes = replicaConfig.Sink.KafkaConfig.MaxMessageBytes + if replicaConfig.Sink.KafkaConfig.CodecConfig != nil { + codecConfig := replicaConfig.Sink.KafkaConfig.CodecConfig dest.EnableTiDBExtension = codecConfig.EnableTiDBExtension dest.MaxBatchSize = codecConfig.MaxBatchSize dest.AvroEnableWatermark = codecConfig.AvroEnableWatermark @@ -315,8 +314,8 @@ func mergeConfig( dest.EncodingFormatType = codecConfig.EncodingFormat } } - if sinkConfig.DebeziumDisableSchema != nil { - dest.DebeziumDisableSchema = sinkConfig.DebeziumDisableSchema + if replicaConfig.Sink.DebeziumDisableSchema != nil { + dest.DebeziumDisableSchema = replicaConfig.Sink.DebeziumDisableSchema } } if err := mergo.Merge(dest, urlParameters, mergo.WithOverride); err != nil { diff --git a/pkg/sink/codec/craft/craft_encoder.go b/pkg/sink/codec/craft/craft_encoder.go index 365572cef..41642db13 100644 --- a/pkg/sink/codec/craft/craft_encoder.go +++ b/pkg/sink/codec/craft/craft_encoder.go @@ -98,7 +98,7 @@ func (e *BatchEncoder) flush() { } // NewBatchEncoder creates a new BatchEncoder. -func NewBatchEncoder(config *ticommon.Config) encoder.EventEncoder { +func NewBatchEncoder(config *ticommon.Config) encoder.RowEventEncoder { // 64 is a magic number that come up with these assumptions and manual benchmark. // 1. Most table will not have more than 64 columns // 2. It only worth allocating slices in batch for slices that's small enough @@ -108,7 +108,7 @@ func NewBatchEncoder(config *ticommon.Config) encoder.EventEncoder { func (e *BatchEncoder) Clean() {} // NewBatchEncoderWithAllocator creates a new BatchEncoder with given allocator. -func NewBatchEncoderWithAllocator(allocator *SliceAllocator, config *ticommon.Config) encoder.EventEncoder { +func NewBatchEncoderWithAllocator(allocator *SliceAllocator, config *ticommon.Config) encoder.RowEventEncoder { return &BatchEncoder{ allocator: allocator, messageBuf: make([]*ticommon.Message, 0, 2), diff --git a/pkg/sink/codec/debezium/encoder.go b/pkg/sink/codec/debezium/encoder.go index 975cb23d8..46daea31b 100644 --- a/pkg/sink/codec/debezium/encoder.go +++ b/pkg/sink/codec/debezium/encoder.go @@ -98,7 +98,7 @@ func (d *BatchEncoder) Build() []*ticommon.Message { func (d *BatchEncoder) Clean() {} // newBatchEncoder creates a new Debezium BatchEncoder. -func NewBatchEncoder(c *ticommon.Config, clusterID string) encoder.EventEncoder { +func NewBatchEncoder(c *ticommon.Config, clusterID string) encoder.RowEventEncoder { batch := &BatchEncoder{ messages: nil, config: c, diff --git a/pkg/sink/codec/encoder/encoder.go b/pkg/sink/codec/encoder/encoder.go index 689d1ddf6..2994648c4 100644 --- a/pkg/sink/codec/encoder/encoder.go +++ b/pkg/sink/codec/encoder/encoder.go @@ -29,8 +29,8 @@ const ( MemBufShrinkThreshold = 1024 * 1024 ) -// EventEncoder is an abstraction for events encoder -type EventEncoder interface { +// RowEventEncoder is an abstraction for events encoder +type RowEventEncoder interface { // EncodeCheckpointEvent appends a checkpoint event into the batch. // This event will be broadcast to all partitions to signal a global checkpoint. EncodeCheckpointEvent(ts uint64) (*ticommon.Message, error) diff --git a/pkg/sink/codec/encoder_builder.go b/pkg/sink/codec/encoder_builder.go index d1166ee03..3709ecfc6 100644 --- a/pkg/sink/codec/encoder_builder.go +++ b/pkg/sink/codec/encoder_builder.go @@ -3,14 +3,14 @@ package codec import ( "context" - "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" "github.com/flowbehappy/tigate/pkg/sink/codec/open" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/sink/codec/common" ) -func NewEventEncoder(ctx context.Context, cfg *common.Config) (encoder.EventEncoder, error) { +func NewRowEventEncoder(ctx context.Context, cfg *common.Config) (encoder.RowEventEncoder, error) { switch cfg.Protocol { case config.ProtocolDefault, config.ProtocolOpen: return open.NewBatchEncoder(ctx, cfg) diff --git a/pkg/sink/codec/encoder_group.go b/pkg/sink/codec/encoder_group.go index ce31f234f..890fe844f 100644 --- a/pkg/sink/codec/encoder_group.go +++ b/pkg/sink/codec/encoder_group.go @@ -20,8 +20,6 @@ import ( "time" "github.com/flowbehappy/tigate/pkg/common" - ticonfig "github.com/flowbehappy/tigate/pkg/config" - newCommon "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -58,7 +56,7 @@ type encoderGroup struct { inputCh []chan *future index uint64 - rowEventEncoders []encoder.EventEncoder + rowEventEncoders []encoder.RowEventEncoder outputCh chan *future @@ -68,8 +66,8 @@ type encoderGroup struct { // NewEncoderGroup creates a new EncoderGroup instance func NewEncoderGroup( ctx context.Context, - cfg *ticonfig.SinkConfig, - encoderConfig *newCommon.Config, + cfg *config.SinkConfig, + encoderConfig *ticommon.Config, changefeedID model.ChangeFeedID, ) *encoderGroup { concurrency := util.GetOrZero(cfg.EncoderConcurrency) @@ -77,11 +75,11 @@ func NewEncoderGroup( concurrency = config.DefaultEncoderGroupConcurrency } inputCh := make([]chan *future, concurrency) - rowEventEncoders := make([]encoder.EventEncoder, concurrency) + rowEventEncoders := make([]encoder.RowEventEncoder, concurrency) var err error for i := 0; i < concurrency; i++ { inputCh[i] = make(chan *future, defaultInputChanSize) - rowEventEncoders[i], err = NewEventEncoder(ctx, encoderConfig) + rowEventEncoders[i], err = NewRowEventEncoder(ctx, encoderConfig) if err != nil { log.Error("failed to create row event encoder", zap.Error(err)) return nil @@ -91,7 +89,7 @@ func NewEncoderGroup( var bootstrapWorker *bootstrapWorker if cfg.ShouldSendBootstrapMsg() { - rowEventEncoder, err := NewEventEncoder(ctx, encoderConfig) + rowEventEncoder, err := NewRowEventEncoder(ctx, encoderConfig) if err != nil { log.Error("failed to create row event encoder", zap.Error(err)) return nil diff --git a/pkg/sink/codec/open/codec.go b/pkg/sink/codec/open/codec.go index 4539d3fbd..aee4bef8a 100644 --- a/pkg/sink/codec/open/codec.go +++ b/pkg/sink/codec/open/codec.go @@ -6,7 +6,6 @@ import ( "strconv" "github.com/flowbehappy/tigate/pkg/common" - newcommon "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" "github.com/flowbehappy/tigate/pkg/util" timodel "github.com/pingcap/tidb/pkg/parser/model" @@ -19,7 +18,7 @@ import ( ticommon "github.com/pingcap/tiflow/pkg/sink/codec/common" ) -func encodeRowChangedEvent(e *common.RowEvent, config *newcommon.Config, largeMessageOnlyHandleKeyColumns bool, claimCheckLocationName string) ([]byte, []byte, int, error) { +func encodeRowChangedEvent(e *common.RowEvent, config *ticommon.Config, largeMessageOnlyHandleKeyColumns bool, claimCheckLocationName string) ([]byte, []byte, int, error) { keyBuf := &bytes.Buffer{} valueBuf := &bytes.Buffer{} keyWriter := util.BorrowJSONWriter(keyBuf) @@ -94,7 +93,7 @@ func encodeRowChangedEvent(e *common.RowEvent, config *newcommon.Config, largeMe return key, valueCompressed, length, nil } -func encodeDDLEvent(e *common.DDLEvent, config *newcommon.Config) ([]byte, []byte, error) { +func encodeDDLEvent(e *common.DDLEvent, config *ticommon.Config) ([]byte, []byte, error) { keyBuf := &bytes.Buffer{} valueBuf := &bytes.Buffer{} keyWriter := util.BorrowJSONWriter(keyBuf) diff --git a/pkg/sink/codec/open/codec_test.go b/pkg/sink/codec/open/codec_test.go index dfdf34f55..4bbbf535b 100644 --- a/pkg/sink/codec/open/codec_test.go +++ b/pkg/sink/codec/open/codec_test.go @@ -4,9 +4,7 @@ import ( "testing" "github.com/flowbehappy/tigate/pkg/common" - ticonfig "github.com/flowbehappy/tigate/pkg/config" "github.com/flowbehappy/tigate/pkg/mounter" - newcommon "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/pingcap/tidb/pkg/util/chunk" "github.com/pingcap/tiflow/pkg/config" ticommon "github.com/pingcap/tiflow/pkg/sink/codec/common" @@ -40,7 +38,7 @@ func TestBasicType(t *testing.T) { ColumnSelector: common.NewDefaultColumnSelector(), Callback: func() {}} - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) key, value, _, err := encodeRowChangedEvent(rowEvent, protocolConfig, false, "") require.NoError(t, err) require.Equal(t, `{"ts":1,"scm":"test","tbl":"t","t":1}`, string(key)) @@ -57,7 +55,7 @@ func TestDMLEvent(t *testing.T) { job := helper.DDL2Job(`create table test.t(a tinyint primary key, b int)`) tableInfo := helper.GetTableInfo(job) - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) // Insert dmlEvent := helper.DML2Event("test", "t", `insert into test.t values (1, 123)`) @@ -128,7 +126,7 @@ func TestOnlyOutputUpdatedEvent(t *testing.T) { helper.Tk().MustExec("use test") - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) protocolConfig.OnlyOutputUpdatedColumns = true { @@ -165,7 +163,7 @@ func TestHandleOnlyEvent(t *testing.T) { job := helper.DDL2Job(`create table test.t(a tinyint primary key, b int)`) tableInfo := helper.GetTableInfo(job) - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) // Insert dmlEvent := helper.DML2Event("test", "t", `insert into test.t values (1, 123)`) @@ -195,7 +193,7 @@ func TestDDLEvent(t *testing.T) { job := helper.DDL2Job(`create table test.t(a tinyint primary key, b int)`) - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) ddlEvent := &common.DDLEvent{ Job: job, @@ -224,21 +222,21 @@ func TestEncodeWithColumnSelector(t *testing.T) { defer helper.Close() helper.Tk().MustExec("use test") - sinkConfig := ticonfig.SinkConfig{} - sinkConfig.ColumnSelectors = []*ticonfig.ColumnSelector{ + replicaConfig := config.GetDefaultReplicaConfig() + replicaConfig.Sink.ColumnSelectors = []*config.ColumnSelector{ { Matcher: []string{"test.*"}, Columns: []string{"a*"}, }, } - selectors, err := common.NewColumnSelectors(&sinkConfig) + selectors, err := common.NewColumnSelectors(replicaConfig) require.NoError(t, err) selector := selectors.GetSelector("test", "t") job := helper.DDL2Job(`create table test.t(a tinyint primary key, b int)`) tableInfo := helper.GetTableInfo(job) - protocolConfig := newcommon.NewConfig(config.ProtocolOpen) + protocolConfig := ticommon.NewConfig(config.ProtocolOpen) dmlEvent := helper.DML2Event("test", "t", `insert into test.t values (1, 123)`) require.NotNil(t, dmlEvent) diff --git a/pkg/sink/codec/open/encoder.go b/pkg/sink/codec/open/encoder.go index 0724dc7d8..cff04a59b 100644 --- a/pkg/sink/codec/open/encoder.go +++ b/pkg/sink/codec/open/encoder.go @@ -5,7 +5,6 @@ import ( "encoding/binary" "github.com/flowbehappy/tigate/pkg/common" - newcommon "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" "github.com/pingcap/errors" "github.com/pingcap/log" @@ -26,7 +25,7 @@ type BatchEncoder struct { claimCheck *claimcheck.ClaimCheck - config *newcommon.Config + config *ticommon.Config } // AppendRowChangedEvent implements the RowEventEncoder interface @@ -185,7 +184,7 @@ func enhancedKeyValue(key, value []byte) ([]byte, []byte) { } // NewBatchEncoder creates a new BatchEncoder. -func NewBatchEncoder(ctx context.Context, config *newcommon.Config) (encoder.EventEncoder, error) { +func NewBatchEncoder(ctx context.Context, config *ticommon.Config) (encoder.RowEventEncoder, error) { claimCheck, err := claimcheck.New(ctx, config.LargeMessageHandle, config.ChangefeedID) if err != nil { return nil, errors.Trace(err) diff --git a/pkg/sink/codec/open/encoder_test.go b/pkg/sink/codec/open/encoder_test.go index c87a6c39d..01c868051 100644 --- a/pkg/sink/codec/open/encoder_test.go +++ b/pkg/sink/codec/open/encoder_test.go @@ -8,12 +8,12 @@ import ( "github.com/flowbehappy/tigate/pkg/common" "github.com/flowbehappy/tigate/pkg/mounter" - newcommon "github.com/flowbehappy/tigate/pkg/sink/codec/common" "github.com/flowbehappy/tigate/pkg/sink/codec/encoder" "github.com/pingcap/log" "github.com/pingcap/tiflow/pkg/config" ticonfig "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + ticommon "github.com/pingcap/tiflow/pkg/sink/codec/common" "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -36,7 +36,7 @@ func readByteToUint(b []byte) uint64 { func TestEncoderOneMessage(t *testing.T) { ctx := context.Background() - config := newcommon.NewConfig(config.ProtocolOpen) + config := ticommon.NewConfig(config.ProtocolOpen) batchEncoder, err := NewBatchEncoder(ctx, config) require.NoError(t, err) @@ -84,7 +84,7 @@ func TestEncoderOneMessage(t *testing.T) { func TestEncoderMultipleMessage(t *testing.T) { ctx := context.Background() - config := newcommon.NewConfig(config.ProtocolOpen) + config := ticommon.NewConfig(config.ProtocolOpen) config = config.WithMaxMessageBytes(400) batchEncoder, err := NewBatchEncoder(ctx, config) @@ -157,7 +157,7 @@ func TestEncoderMultipleMessage(t *testing.T) { func TestLargeMessage(t *testing.T) { ctx := context.Background() - config := newcommon.NewConfig(config.ProtocolOpen) + config := ticommon.NewConfig(config.ProtocolOpen) config = config.WithMaxMessageBytes(100) batchEncoder, err := NewBatchEncoder(ctx, config) require.NoError(t, err) @@ -189,7 +189,7 @@ func TestLargeMessage(t *testing.T) { func TestLargeMessageWithHandle(t *testing.T) { ctx := context.Background() - config := newcommon.NewConfig(config.ProtocolOpen) + config := ticommon.NewConfig(config.ProtocolOpen) config = config.WithMaxMessageBytes(150) config.LargeMessageHandle.LargeMessageHandleOption = ticonfig.LargeMessageHandleOptionHandleKeyOnly batchEncoder, err := NewBatchEncoder(ctx, config) @@ -233,7 +233,7 @@ func TestLargeMessageWithHandle(t *testing.T) { func TestLargeMessageWithoutHandle(t *testing.T) { ctx := context.Background() - config := newcommon.NewConfig(config.ProtocolOpen) + config := ticommon.NewConfig(config.ProtocolOpen) config = config.WithMaxMessageBytes(150) config.LargeMessageHandle.LargeMessageHandleOption = ticonfig.LargeMessageHandleOptionHandleKeyOnly batchEncoder, err := NewBatchEncoder(ctx, config) diff --git a/pkg/sink/codec/simple/encoder.go b/pkg/sink/codec/simple/encoder.go index a2e77863c..745d59eb8 100644 --- a/pkg/sink/codec/simple/encoder.go +++ b/pkg/sink/codec/simple/encoder.go @@ -35,7 +35,7 @@ type Encoder struct { marshaller marshaller } -func NewEncoder(ctx context.Context, config *ticommon.Config) (encoder.EventEncoder, error) { +func NewEncoder(ctx context.Context, config *ticommon.Config) (encoder.RowEventEncoder, error) { claimCheck, err := claimcheck.New(ctx, config.LargeMessageHandle, config.ChangefeedID) if err != nil { return nil, errors.Trace(err) diff --git a/pkg/sink/kafka/admin.go b/pkg/sink/kafka/admin.go deleted file mode 100644 index fc2abbcb8..000000000 --- a/pkg/sink/kafka/admin.go +++ /dev/null @@ -1,186 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kafka - -import ( - "context" - "strconv" - "strings" - - "github.com/IBM/sarama" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - tikafka "github.com/pingcap/tiflow/pkg/sink/kafka" - "go.uber.org/zap" -) - -type saramaAdminClient struct { - changefeed model.ChangeFeedID - - client sarama.Client - admin sarama.ClusterAdmin -} - -func (a *saramaAdminClient) GetAllBrokers(_ context.Context) ([]tikafka.Broker, error) { - brokers := a.client.Brokers() - result := make([]tikafka.Broker, 0, len(brokers)) - for _, broker := range brokers { - result = append(result, tikafka.Broker{ - ID: broker.ID(), - }) - } - - return result, nil -} - -func (a *saramaAdminClient) GetBrokerConfig( - _ context.Context, - configName string, -) (string, error) { - _, controller, err := a.admin.DescribeCluster() - if err != nil { - return "", errors.Trace(err) - } - - configEntries, err := a.admin.DescribeConfig(sarama.ConfigResource{ - Type: sarama.BrokerResource, - Name: strconv.Itoa(int(controller)), - ConfigNames: []string{configName}, - }) - if err != nil { - return "", errors.Trace(err) - } - - // For compatibility with KOP, we checked all return values. - // 1. Kafka only returns requested configs. - // 2. Kop returns all configs. - for _, entry := range configEntries { - if entry.Name == configName { - return entry.Value, nil - } - } - - log.Warn("Kafka config item not found", - zap.String("namespace", a.changefeed.Namespace), - zap.String("changefeed", a.changefeed.ID), - zap.String("configName", configName)) - return "", cerror.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the broker's configuration", configName) -} - -func (a *saramaAdminClient) GetTopicConfig( - _ context.Context, topicName string, configName string, -) (string, error) { - configEntries, err := a.admin.DescribeConfig(sarama.ConfigResource{ - Type: sarama.TopicResource, - Name: topicName, - ConfigNames: []string{configName}, - }) - if err != nil { - return "", errors.Trace(err) - } - - // For compatibility with KOP, we checked all return values. - // 1. Kafka only returns requested configs. - // 2. Kop returns all configs. - for _, entry := range configEntries { - if entry.Name == configName { - log.Info("Kafka config item found", - zap.String("namespace", a.changefeed.Namespace), - zap.String("changefeed", a.changefeed.ID), - zap.String("configName", configName), - zap.String("configValue", entry.Value)) - return entry.Value, nil - } - } - - log.Warn("Kafka config item not found", - zap.String("namespace", a.changefeed.Namespace), - zap.String("changefeed", a.changefeed.ID), - zap.String("configName", configName)) - return "", cerror.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the topic's configuration", configName) -} - -func (a *saramaAdminClient) GetTopicsMeta( - _ context.Context, topics []string, ignoreTopicError bool, -) (map[string]tikafka.TopicDetail, error) { - result := make(map[string]tikafka.TopicDetail, len(topics)) - - metaList, err := a.admin.DescribeTopics(topics) - if err != nil { - return nil, errors.Trace(err) - } - - for _, meta := range metaList { - if meta.Err != sarama.ErrNoError { - if !ignoreTopicError { - return nil, meta.Err - } - log.Warn("fetch topic meta failed", - zap.String("namespace", a.changefeed.Namespace), - zap.String("changefeed", a.changefeed.ID), - zap.String("topic", meta.Name), - zap.Error(meta.Err)) - continue - } - result[meta.Name] = tikafka.TopicDetail{ - Name: meta.Name, - NumPartitions: int32(len(meta.Partitions)), - } - } - return result, nil -} - -func (a *saramaAdminClient) GetTopicsPartitionsNum( - _ context.Context, topics []string, -) (map[string]int32, error) { - result := make(map[string]int32, len(topics)) - for _, topic := range topics { - partition, err := a.client.Partitions(topic) - if err != nil { - return nil, errors.Trace(err) - } - result[topic] = int32(len(partition)) - } - - return result, nil -} - -func (a *saramaAdminClient) CreateTopic( - _ context.Context, detail *tikafka.TopicDetail, validateOnly bool, -) error { - request := &sarama.TopicDetail{ - NumPartitions: detail.NumPartitions, - ReplicationFactor: detail.ReplicationFactor, - } - - err := a.admin.CreateTopic(detail.Name, request, validateOnly) - // Ignore the already exists error because it's not harmful. - if err != nil && !strings.Contains(err.Error(), sarama.ErrTopicAlreadyExists.Error()) { - return err - } - return nil -} - -func (a *saramaAdminClient) Close() { - if err := a.admin.Close(); err != nil { - log.Warn("close admin client meet error", - zap.String("namespace", a.changefeed.Namespace), - zap.String("changefeed", a.changefeed.ID), - zap.Error(err)) - } -} diff --git a/pkg/sink/kafka/factory.go b/pkg/sink/kafka/factory.go deleted file mode 100644 index 6e231adfd..000000000 --- a/pkg/sink/kafka/factory.go +++ /dev/null @@ -1,271 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kafka - -import ( - "context" - "time" - - "github.com/IBM/sarama" - "github.com/pingcap/errors" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/sink/codec/common" - tikafka "github.com/pingcap/tiflow/pkg/sink/kafka" - "github.com/pingcap/tiflow/pkg/util" - "go.uber.org/zap" -) - -// Factory is used to produce all kafka components. -type Factory interface { - // AdminClient return a kafka cluster admin client - AdminClient(ctx context.Context) (tikafka.ClusterAdminClient, error) - // SyncProducer creates a sync producer to writer message to kafka - SyncProducer(ctx context.Context) (SyncProducer, error) - // AsyncProducer creates an async producer to writer message to kafka - AsyncProducer(ctx context.Context, failpointCh chan error) (tikafka.AsyncProducer, error) - // MetricsCollector returns the kafka metrics collector - MetricsCollector(role util.Role, adminClient tikafka.ClusterAdminClient) tikafka.MetricsCollector -} - -// FactoryCreator defines the type of factory creator. -type FactoryCreator func(*Options, model.ChangeFeedID) (Factory, error) - -// SyncProducer is the kafka sync producer -type SyncProducer interface { - // SendMessage produces a given message, and returns only when it either has - // succeeded or failed to produce. It will return the partition and the offset - // of the produced message, or an error if the message failed to produce. - SendMessage(ctx context.Context, - topic string, partitionNum int32, - message *common.Message) error - - // SendMessages produces a given set of messages, and returns only when all - // messages in the set have either succeeded or failed. Note that messages - // can succeed and fail individually; if some succeed and some fail, - // SendMessages will return an error. - SendMessages(ctx context.Context, topic string, partitionNum int32, message *common.Message) error - - // Close shuts down the producer; you must call this function before a producer - // object passes out of scope, as it may otherwise leak memory. - // You must call this before calling Close on the underlying client. - Close() -} - -// // AsyncProducer is the kafka async producer -// type AsyncProducer interface { -// // Close shuts down the producer and waits for any buffered messages to be -// // flushed. You must call this function before a producer object passes out of -// // scope, as it may otherwise leak memory. You must call this before process -// // shutting down, or you may lose messages. You must call this before calling -// // Close on the underlying client. -// Close() - -// // AsyncSend is the input channel for the user to write messages to that they -// // wish to send. -// AsyncSend(ctx context.Context, topic string, partition int32, message *common.Message) error - -// // AsyncRunCallback process the messages that has sent to kafka, -// // and run tha attached callback. the caller should call this -// // method in a background goroutine -// AsyncRunCallback(ctx context.Context) error -// } - -type saramaSyncProducer struct { - id model.ChangeFeedID - client sarama.Client - producer sarama.SyncProducer -} - -func (p *saramaSyncProducer) SendMessage( - _ context.Context, - topic string, partitionNum int32, - message *common.Message, -) error { - _, _, err := p.producer.SendMessage(&sarama.ProducerMessage{ - Topic: topic, - Key: sarama.ByteEncoder(message.Key), - Value: sarama.ByteEncoder(message.Value), - Partition: partitionNum, - }) - return err -} - -func (p *saramaSyncProducer) SendMessages(ctx context.Context, topic string, partitionNum int32, message *common.Message) error { - msgs := make([]*sarama.ProducerMessage, partitionNum) - for i := 0; i < int(partitionNum); i++ { - msgs[i] = &sarama.ProducerMessage{ - Topic: topic, - Key: sarama.ByteEncoder(message.Key), - Value: sarama.ByteEncoder(message.Value), - Partition: int32(i), - } - } - return p.producer.SendMessages(msgs) -} - -func (p *saramaSyncProducer) Close() { - go func() { - // We need to close it asynchronously. Otherwise, we might get stuck - // with an unhealthy(i.e. Network jitter, isolation) state of Kafka. - // Factory has a background thread to fetch and update the metadata. - // If we close the client synchronously, we might get stuck. - // Safety: - // * If the kafka cluster is running well, it will be closed as soon as possible. - // * If there is a problem with the kafka cluster, - // no data will be lost because this is a synchronous client. - // * There is a risk of goroutine leakage, but it is acceptable and our main - // goal is not to get stuck with the owner tick. - start := time.Now() - if err := p.client.Close(); err != nil { - log.Warn("Close Kafka DDL client with error", - zap.String("namespace", p.id.Namespace), - zap.String("changefeed", p.id.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Kafka DDL client closed", - zap.String("namespace", p.id.Namespace), - zap.String("changefeed", p.id.ID), - zap.Duration("duration", time.Since(start))) - } - start = time.Now() - err := p.producer.Close() - if err != nil { - log.Error("Close Kafka DDL producer with error", - zap.String("namespace", p.id.Namespace), - zap.String("changefeed", p.id.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Kafka DDL producer closed", - zap.String("namespace", p.id.Namespace), - zap.String("changefeed", p.id.ID), - zap.Duration("duration", time.Since(start))) - } - }() -} - -type saramaAsyncProducer struct { - client sarama.Client - producer sarama.AsyncProducer - changefeedID model.ChangeFeedID - failpointCh chan error -} - -func (p *saramaAsyncProducer) Close() { - go func() { - // We need to close it asynchronously. Otherwise, we might get stuck - // with an unhealthy(i.e. Network jitter, isolation) state of Kafka. - // Safety: - // * If the kafka cluster is running well, it will be closed as soon as possible. - // Also, we cancel all table pipelines before closed, so it's safe. - // * If there is a problem with the kafka cluster, it will shut down the client first, - // which means no more data will be sent because the connection to the broker is dropped. - // Also, we cancel all table pipelines before closed, so it's safe. - // * For Kafka Sink, duplicate data is acceptable. - // * There is a risk of goroutine leakage, but it is acceptable and our main - // goal is not to get stuck with the processor tick. - - // `client` is mainly used by `asyncProducer` to fetch metadata and perform other related - // operations. When we close the `kafkaSaramaProducer`, - // there is no need for TiCDC to make sure that all buffered messages are flushed. - // Consider the situation where the broker is irresponsive. If the client were not - // closed, `asyncProducer.Close()` would waste a mount of time to try flush all messages. - // To prevent the scenario mentioned above, close the client first. - start := time.Now() - if err := p.client.Close(); err != nil { - log.Warn("Close kafka async producer client error", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Close kafka async producer client success", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID), - zap.Duration("duration", time.Since(start))) - } - - start = time.Now() - if err := p.producer.Close(); err != nil { - log.Warn("Close kafka async producer error", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Close kafka async producer success", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID), - zap.Duration("duration", time.Since(start))) - } - }() -} - -func (p *saramaAsyncProducer) AsyncRunCallback( - ctx context.Context, -) error { - for { - select { - case <-ctx.Done(): - log.Info("async producer exit since context is done", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID)) - return errors.Trace(ctx.Err()) - case err := <-p.failpointCh: - log.Warn("Receive from failpoint chan in kafka DML producer", - zap.String("namespace", p.changefeedID.Namespace), - zap.String("changefeed", p.changefeedID.ID), - zap.Error(err)) - return errors.Trace(err) - case ack := <-p.producer.Successes(): - if ack != nil { - callback := ack.Metadata.(func()) - if callback != nil { - callback() - } - } - case err := <-p.producer.Errors(): - // We should not wrap a nil pointer if the pointer - // is of a subtype of `error` because Go would store the type info - // and the resulted `error` variable would not be nil, - // which will cause the pkg/error library to malfunction. - // See: https://go.dev/doc/faq#nil_error - if err == nil { - return nil - } - return cerror.WrapError(cerror.ErrKafkaAsyncSendMessage, err) - } - } -} - -// AsyncSend is the input channel for the user to write messages to that they -// wish to send. -func (p *saramaAsyncProducer) AsyncSend(ctx context.Context, topic string, partition int32, message *common.Message) error { - msg := &sarama.ProducerMessage{ - Topic: topic, - Partition: partition, - Key: sarama.StringEncoder(message.Key), - Value: sarama.ByteEncoder(message.Value), - Metadata: message.Callback, - } - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case p.producer.Input() <- msg: - } - return nil -} diff --git a/pkg/sink/kafka/oauth2_token_provider.go b/pkg/sink/kafka/oauth2_token_provider.go deleted file mode 100644 index 05d8e77c3..000000000 --- a/pkg/sink/kafka/oauth2_token_provider.go +++ /dev/null @@ -1,90 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kafka - -import ( - "context" - "net/url" - - "github.com/IBM/sarama" - "github.com/pingcap/errors" - "golang.org/x/oauth2" - "golang.org/x/oauth2/clientcredentials" -) - -// tsokenProvider is a user-defined callback for generating -// access tokens for SASL/OAUTHBEARER auth. -type tokenProvider struct { - tokenSource oauth2.TokenSource -} - -var _ sarama.AccessTokenProvider = (*tokenProvider)(nil) - -// Token implements the sarama.AccessTokenProvider interface. -// Token returns an access token. The implementation should ensure token -// reuse so that multiple calls at connect time do not create multiple -// tokens. The implementation should also periodically refresh the token in -// order to guarantee that each call returns an unexpired token. This -// method should not block indefinitely--a timeout error should be returned -// after a short period of inactivity so that the broker connection logic -// can log debugging information and retry. -func (t *tokenProvider) Token() (*sarama.AccessToken, error) { - token, err := t.tokenSource.Token() - if err != nil { - // Errors will result in Sarama retrying the broker connection and logging - // the transient error, with a Broker connection error surfacing after retry - // attempts have been exhausted. - return nil, err - } - - return &sarama.AccessToken{Token: token.AccessToken}, nil -} - -func newTokenProvider(ctx context.Context, o *Options) (sarama.AccessTokenProvider, error) { - // grant_type is by default going to be set to 'client_credentials' by the - // clientcredentials library as defined by the spec, however non-compliant - // auth server implementations may want a custom type - var endpointParams url.Values - if o.SASL.OAuth2.GrantType != "" { - if endpointParams == nil { - endpointParams = url.Values{} - } - endpointParams.Set("grant_type", o.SASL.OAuth2.GrantType) - } - - // audience is an optional parameter that can be used to specify the - // intended audience of the token. - if o.SASL.OAuth2.Audience != "" { - if endpointParams == nil { - endpointParams = url.Values{} - } - endpointParams.Set("audience", o.SASL.OAuth2.Audience) - } - - tokenURL, err := url.Parse(o.SASL.OAuth2.TokenURL) - if err != nil { - return nil, errors.Trace(err) - } - - cfg := clientcredentials.Config{ - ClientID: o.SASL.OAuth2.ClientID, - ClientSecret: o.SASL.OAuth2.ClientSecret, - TokenURL: tokenURL.String(), - EndpointParams: endpointParams, - Scopes: o.SASL.OAuth2.Scopes, - } - return &tokenProvider{ - tokenSource: cfg.TokenSource(ctx), - }, nil -} diff --git a/pkg/sink/kafka/options.go b/pkg/sink/kafka/options.go deleted file mode 100644 index dbf17f4c8..000000000 --- a/pkg/sink/kafka/options.go +++ /dev/null @@ -1,552 +0,0 @@ -package kafka - -import ( - "encoding/base64" - "errors" - "fmt" - "net/http" - "net/url" - "regexp" - "strings" - "time" - - ticonfig "github.com/flowbehappy/tigate/pkg/config" - "github.com/gin-gonic/gin/binding" - "github.com/imdario/mergo" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/security" - "go.uber.org/zap" -) - -const ( - // defaultPartitionNum specifies the default number of partitions when we create the topic. - defaultPartitionNum = 3 - - // the `max-message-bytes` is set equal to topic's `max.message.bytes`, and is used to check - // whether the message is larger than the max size limit. It's found some message pass the message - // size limit check at the client side and failed at the broker side since message enlarged during - // the network transmission. so we set the `max-message-bytes` to a smaller value to avoid this problem. - // maxMessageBytesOverhead is used to reduce the `max-message-bytes`. - maxMessageBytesOverhead = 128 -) - -const ( - // BrokerMessageMaxBytesConfigName specifies the largest record batch size allowed by - // Kafka brokers. - // See: https://kafka.apache.org/documentation/#brokerconfigs_message.max.bytes - BrokerMessageMaxBytesConfigName = "message.max.bytes" - // TopicMaxMessageBytesConfigName specifies the largest record batch size allowed by - // Kafka topics. - // See: https://kafka.apache.org/documentation/#topicconfigs_max.message.bytes - TopicMaxMessageBytesConfigName = "max.message.bytes" - // MinInsyncReplicasConfigName the minimum number of replicas that must acknowledge a write - // for the write to be considered successful. - // Only works if the producer's acks is "all" (or "-1"). - // See: https://kafka.apache.org/documentation/#brokerconfigs_min.insync.replicas and - // https://kafka.apache.org/documentation/#topicconfigs_min.insync.replicas - MinInsyncReplicasConfigName = "min.insync.replicas" -) - -const ( - // SASLTypePlaintext represents the plain mechanism - SASLTypePlaintext = "PLAIN" - // SASLTypeSCRAMSHA256 represents the SCRAM-SHA-256 mechanism. - SASLTypeSCRAMSHA256 = "SCRAM-SHA-256" - // SASLTypeSCRAMSHA512 represents the SCRAM-SHA-512 mechanism. - SASLTypeSCRAMSHA512 = "SCRAM-SHA-512" - // SASLTypeGSSAPI represents the gssapi mechanism. - SASLTypeGSSAPI = "GSSAPI" - // SASLTypeOAuth represents the SASL/OAUTHBEARER mechanism (Kafka 2.0.0+) - SASLTypeOAuth = "OAUTHBEARER" -) - -// RequiredAcks is used in Produce Requests to tell the broker how many replica acknowledgements -// it must see before responding. Any of the constants defined here are valid. On broker versions -// prior to 0.8.2.0 any other positive int16 is also valid (the broker will wait for that many -// acknowledgements) but in 0.8.2.0 and later this will raise an exception (it has been replaced -// by setting the `min.isr` value in the brokers configuration). -type RequiredAcks int16 - -const ( - // NoResponse doesn't send any response, the TCP ACK is all you get. - NoResponse RequiredAcks = 0 - // WaitForLocal waits for only the local commit to succeed before responding. - WaitForLocal RequiredAcks = 1 - // WaitForAll waits for all in-sync replicas to commit before responding. - // The minimum number of in-sync replicas is configured on the broker via - // the `min.insync.replicas` configuration key. - WaitForAll RequiredAcks = -1 - // Unknown should never have been use in real config. - Unknown RequiredAcks = 2 -) - -func requireAcksFromString(acks int) (RequiredAcks, error) { - switch acks { - case int(WaitForAll): - return WaitForAll, nil - case int(WaitForLocal): - return WaitForLocal, nil - case int(NoResponse): - return NoResponse, nil - default: - return Unknown, cerror.ErrKafkaInvalidRequiredAcks.GenWithStackByArgs(acks) - } -} - -type urlConfig struct { - PartitionNum *int32 `form:"partition-num"` - ReplicationFactor *int16 `form:"replication-factor"` - KafkaVersion *string `form:"kafka-version"` - MaxMessageBytes *int `form:"max-message-bytes"` - Compression *string `form:"compression"` - KafkaClientID *string `form:"kafka-client-id"` - AutoCreateTopic *bool `form:"auto-create-topic"` - DialTimeout *string `form:"dial-timeout"` - WriteTimeout *string `form:"write-timeout"` - ReadTimeout *string `form:"read-timeout"` - RequiredAcks *int `form:"required-acks"` - SASLUser *string `form:"sasl-user"` - SASLPassword *string `form:"sasl-password"` - SASLMechanism *string `form:"sasl-mechanism"` - SASLGssAPIAuthType *string `form:"sasl-gssapi-auth-type"` - SASLGssAPIKeytabPath *string `form:"sasl-gssapi-keytab-path"` - SASLGssAPIKerberosConfigPath *string `form:"sasl-gssapi-kerberos-config-path"` - SASLGssAPIServiceName *string `form:"sasl-gssapi-service-name"` - SASLGssAPIUser *string `form:"sasl-gssapi-user"` - SASLGssAPIPassword *string `form:"sasl-gssapi-password"` - SASLGssAPIRealm *string `form:"sasl-gssapi-realm"` - SASLGssAPIDisablePafxfast *bool `form:"sasl-gssapi-disable-pafxfast"` - EnableTLS *bool `form:"enable-tls"` - CA *string `form:"ca"` - Cert *string `form:"cert"` - Key *string `form:"key"` - InsecureSkipVerify *bool `form:"insecure-skip-verify"` -} - -// Options stores user specified configurations -type Options struct { - BrokerEndpoints []string - - // control whether to create topic - AutoCreate bool - PartitionNum int32 - // User should make sure that `replication-factor` not greater than the number of kafka brokers. - ReplicationFactor int16 - Version string - IsAssignedVersion bool - RequestVersion int16 - MaxMessageBytes int - Compression string - ClientID string - RequiredAcks RequiredAcks - // Only for test. User can not set this value. - // The current prod default value is 0. - MaxMessages int - - // Credential is used to connect to kafka cluster. - EnableTLS bool - Credential *security.Credential - InsecureSkipVerify bool - SASL *security.SASL - - // Timeout for network configurations, default to `10s` - DialTimeout time.Duration - WriteTimeout time.Duration - ReadTimeout time.Duration -} - -// NewOptions returns a default Kafka configuration -func NewOptions() *Options { - return &Options{ - Version: "2.4.0", - // MaxMessageBytes will be used to initialize producer - MaxMessageBytes: config.DefaultMaxMessageBytes, - ReplicationFactor: 1, - Compression: "none", - RequiredAcks: WaitForAll, - Credential: &security.Credential{}, - InsecureSkipVerify: false, - SASL: &security.SASL{}, - AutoCreate: true, - DialTimeout: 10 * time.Second, - WriteTimeout: 10 * time.Second, - ReadTimeout: 10 * time.Second, - } -} - -// SetPartitionNum set the partition-num by the topic's partition count. -func (o *Options) SetPartitionNum(realPartitionCount int32) error { - // user does not specify the `partition-num` in the sink-uri - if o.PartitionNum == 0 { - o.PartitionNum = realPartitionCount - log.Info("partitionNum is not set, set by topic's partition-num", - zap.Int32("partitionNum", realPartitionCount)) - return nil - } - - if o.PartitionNum < realPartitionCount { - log.Warn("number of partition specified in sink-uri is less than that of the actual topic. "+ - "Some partitions will not have messages dispatched to", - zap.Int32("sinkUriPartitions", o.PartitionNum), - zap.Int32("topicPartitions", realPartitionCount)) - return nil - } - - // Make sure that the user-specified `partition-num` is not greater than - // the real partition count, since messages would be dispatched to different - // partitions, this could prevent potential correctness problems. - if o.PartitionNum > realPartitionCount { - return cerror.ErrKafkaInvalidPartitionNum.GenWithStack( - "the number of partition (%d) specified in sink-uri is more than that of actual topic (%d)", - o.PartitionNum, realPartitionCount) - } - return nil -} - -// Apply the sinkURI to update Options -func (o *Options) Apply(changefeedID model.ChangeFeedID, - sinkURI *url.URL, sinkConfig *ticonfig.SinkConfig, -) error { - o.BrokerEndpoints = strings.Split(sinkURI.Host, ",") - - var err error - req := &http.Request{URL: sinkURI} - urlParameter := &urlConfig{} - if err := binding.Query.Bind(req, urlParameter); err != nil { - return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) - } - if urlParameter, err = mergeConfig(sinkConfig, urlParameter); err != nil { - return err - } - if urlParameter.PartitionNum != nil { - o.PartitionNum = *urlParameter.PartitionNum - if o.PartitionNum <= 0 { - return cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(o.PartitionNum) - } - } - - if urlParameter.ReplicationFactor != nil { - o.ReplicationFactor = *urlParameter.ReplicationFactor - } - - if urlParameter.KafkaVersion != nil { - o.Version = *urlParameter.KafkaVersion - o.IsAssignedVersion = true - } - - if urlParameter.MaxMessageBytes != nil { - o.MaxMessageBytes = *urlParameter.MaxMessageBytes - } - - if urlParameter.Compression != nil { - o.Compression = *urlParameter.Compression - } - - var kafkaClientID string - if urlParameter.KafkaClientID != nil { - kafkaClientID = *urlParameter.KafkaClientID - } - clientID, err := NewKafkaClientID( - config.GetGlobalServerConfig().AdvertiseAddr, - changefeedID, - kafkaClientID) - if err != nil { - return err - } - o.ClientID = clientID - - if urlParameter.AutoCreateTopic != nil { - o.AutoCreate = *urlParameter.AutoCreateTopic - } - - if urlParameter.DialTimeout != nil && *urlParameter.DialTimeout != "" { - a, err := time.ParseDuration(*urlParameter.DialTimeout) - if err != nil { - return err - } - o.DialTimeout = a - } - - if urlParameter.WriteTimeout != nil && *urlParameter.WriteTimeout != "" { - a, err := time.ParseDuration(*urlParameter.WriteTimeout) - if err != nil { - return err - } - o.WriteTimeout = a - } - - if urlParameter.ReadTimeout != nil && *urlParameter.ReadTimeout != "" { - a, err := time.ParseDuration(*urlParameter.ReadTimeout) - if err != nil { - return err - } - o.ReadTimeout = a - } - - if urlParameter.RequiredAcks != nil { - r, err := requireAcksFromString(*urlParameter.RequiredAcks) - if err != nil { - return err - } - o.RequiredAcks = r - } - - err = o.applySASL(urlParameter, sinkConfig) - if err != nil { - return err - } - - err = o.applyTLS(urlParameter) - if err != nil { - return err - } - - return nil -} - -func mergeConfig( - sinkConfig *ticonfig.SinkConfig, - urlParameters *urlConfig, -) (*urlConfig, error) { - dest := &urlConfig{} - if sinkConfig != nil && sinkConfig.KafkaConfig != nil { - fileConifg := sinkConfig.KafkaConfig - dest.PartitionNum = fileConifg.PartitionNum - dest.ReplicationFactor = fileConifg.ReplicationFactor - dest.KafkaVersion = fileConifg.KafkaVersion - dest.MaxMessageBytes = fileConifg.MaxMessageBytes - dest.Compression = fileConifg.Compression - dest.KafkaClientID = fileConifg.KafkaClientID - dest.AutoCreateTopic = fileConifg.AutoCreateTopic - dest.DialTimeout = fileConifg.DialTimeout - dest.WriteTimeout = fileConifg.WriteTimeout - dest.ReadTimeout = fileConifg.ReadTimeout - dest.RequiredAcks = fileConifg.RequiredAcks - dest.SASLUser = fileConifg.SASLUser - dest.SASLPassword = fileConifg.SASLPassword - dest.SASLMechanism = fileConifg.SASLMechanism - dest.SASLGssAPIDisablePafxfast = fileConifg.SASLGssAPIDisablePafxfast - dest.SASLGssAPIAuthType = fileConifg.SASLGssAPIAuthType - dest.SASLGssAPIKeytabPath = fileConifg.SASLGssAPIKeytabPath - dest.SASLGssAPIServiceName = fileConifg.SASLGssAPIServiceName - dest.SASLGssAPIKerberosConfigPath = fileConifg.SASLGssAPIKerberosConfigPath - dest.SASLGssAPIRealm = fileConifg.SASLGssAPIRealm - dest.SASLGssAPIUser = fileConifg.SASLGssAPIUser - dest.SASLGssAPIPassword = fileConifg.SASLGssAPIPassword - dest.EnableTLS = fileConifg.EnableTLS - dest.CA = fileConifg.CA - dest.Cert = fileConifg.Cert - dest.Key = fileConifg.Key - dest.InsecureSkipVerify = fileConifg.InsecureSkipVerify - } - if err := mergo.Merge(dest, urlParameters, mergo.WithOverride); err != nil { - return nil, err - } - return dest, nil -} - -func (o *Options) applyTLS(params *urlConfig) error { - if params.CA != nil && *params.CA != "" { - o.Credential.CAPath = *params.CA - } - - if params.Cert != nil && *params.Cert != "" { - o.Credential.CertPath = *params.Cert - } - - if params.Key != nil && *params.Key != "" { - o.Credential.KeyPath = *params.Key - } - - if o.Credential != nil && !o.Credential.IsEmpty() && - !o.Credential.IsTLSEnabled() { - return cerror.WrapError(cerror.ErrKafkaInvalidConfig, - errors.New("ca, cert and key files should all be supplied")) - } - - // if enable-tls is not set, but credential files are set, - // then tls should be enabled, and the self-signed CA certificate is used. - // if enable-tls is set to true, and credential files are not set, - // then tls should be enabled, and the trusted CA certificate on OS is used. - // if enable-tls is set to false, and credential files are set, - // then an error is returned. - if params.EnableTLS != nil { - enableTLS := *params.EnableTLS - - if o.Credential != nil && o.Credential.IsTLSEnabled() && !enableTLS { - return cerror.WrapError(cerror.ErrKafkaInvalidConfig, - errors.New("credential files are supplied, but 'enable-tls' is set to false")) - } - o.EnableTLS = enableTLS - } else { - if o.Credential != nil && o.Credential.IsTLSEnabled() { - o.EnableTLS = true - } - } - - // Only set InsecureSkipVerify when enable the TLS. - if o.EnableTLS && params.InsecureSkipVerify != nil { - o.InsecureSkipVerify = *params.InsecureSkipVerify - } - - return nil -} - -func (o *Options) applySASL(urlParameter *urlConfig, sinkConfig *ticonfig.SinkConfig) error { - if urlParameter.SASLUser != nil && *urlParameter.SASLUser != "" { - o.SASL.SASLUser = *urlParameter.SASLUser - } - - if urlParameter.SASLPassword != nil && *urlParameter.SASLPassword != "" { - o.SASL.SASLPassword = *urlParameter.SASLPassword - } - - if urlParameter.SASLMechanism != nil && *urlParameter.SASLMechanism != "" { - mechanism, err := security.SASLMechanismFromString(*urlParameter.SASLMechanism) - if err != nil { - return cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } - o.SASL.SASLMechanism = mechanism - } - - if urlParameter.SASLGssAPIAuthType != nil && *urlParameter.SASLGssAPIAuthType != "" { - authType, err := security.AuthTypeFromString(*urlParameter.SASLGssAPIAuthType) - if err != nil { - return cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } - o.SASL.GSSAPI.AuthType = authType - } - - if urlParameter.SASLGssAPIKeytabPath != nil && *urlParameter.SASLGssAPIKeytabPath != "" { - o.SASL.GSSAPI.KeyTabPath = *urlParameter.SASLGssAPIKeytabPath - } - - if urlParameter.SASLGssAPIKerberosConfigPath != nil && - *urlParameter.SASLGssAPIKerberosConfigPath != "" { - o.SASL.GSSAPI.KerberosConfigPath = *urlParameter.SASLGssAPIKerberosConfigPath - } - - if urlParameter.SASLGssAPIServiceName != nil && *urlParameter.SASLGssAPIServiceName != "" { - o.SASL.GSSAPI.ServiceName = *urlParameter.SASLGssAPIServiceName - } - - if urlParameter.SASLGssAPIUser != nil && *urlParameter.SASLGssAPIUser != "" { - o.SASL.GSSAPI.Username = *urlParameter.SASLGssAPIUser - } - - if urlParameter.SASLGssAPIPassword != nil && *urlParameter.SASLGssAPIPassword != "" { - o.SASL.GSSAPI.Password = *urlParameter.SASLGssAPIPassword - } - - if urlParameter.SASLGssAPIRealm != nil && *urlParameter.SASLGssAPIRealm != "" { - o.SASL.GSSAPI.Realm = *urlParameter.SASLGssAPIRealm - } - - if urlParameter.SASLGssAPIDisablePafxfast != nil { - o.SASL.GSSAPI.DisablePAFXFAST = *urlParameter.SASLGssAPIDisablePafxfast - } - - if sinkConfig != nil && sinkConfig.KafkaConfig != nil { - if sinkConfig.KafkaConfig.SASLOAuthClientID != nil { - clientID := *sinkConfig.KafkaConfig.SASLOAuthClientID - if clientID == "" { - return cerror.ErrKafkaInvalidConfig.GenWithStack("OAuth2 client ID cannot be empty") - } - o.SASL.OAuth2.ClientID = clientID - } - - if sinkConfig.KafkaConfig.SASLOAuthClientSecret != nil { - clientSecret := *sinkConfig.KafkaConfig.SASLOAuthClientSecret - if clientSecret == "" { - return cerror.ErrKafkaInvalidConfig.GenWithStack( - "OAuth2 client secret cannot be empty") - } - - // BASE64 decode the client secret - decodedClientSecret, err := base64.StdEncoding.DecodeString(clientSecret) - if err != nil { - log.Error("OAuth2 client secret is not base64 encoded", zap.Error(err)) - return cerror.ErrKafkaInvalidConfig.GenWithStack( - "OAuth2 client secret is not base64 encoded") - } - o.SASL.OAuth2.ClientSecret = string(decodedClientSecret) - } - - if sinkConfig.KafkaConfig.SASLOAuthTokenURL != nil { - tokenURL := *sinkConfig.KafkaConfig.SASLOAuthTokenURL - if tokenURL == "" { - return cerror.ErrKafkaInvalidConfig.GenWithStack( - "OAuth2 token URL cannot be empty") - } - o.SASL.OAuth2.TokenURL = tokenURL - } - - if o.SASL.OAuth2.IsEnable() { - if o.SASL.SASLMechanism != security.OAuthMechanism { - return cerror.ErrKafkaInvalidConfig.GenWithStack( - "OAuth2 is only supported with SASL mechanism type OAUTHBEARER, but got %s", - o.SASL.SASLMechanism) - } - - if err := o.SASL.OAuth2.Validate(); err != nil { - return cerror.ErrKafkaInvalidConfig.Wrap(err) - } - o.SASL.OAuth2.SetDefault() - } - - if sinkConfig.KafkaConfig.SASLOAuthScopes != nil { - o.SASL.OAuth2.Scopes = sinkConfig.KafkaConfig.SASLOAuthScopes - } - - if sinkConfig.KafkaConfig.SASLOAuthGrantType != nil { - o.SASL.OAuth2.GrantType = *sinkConfig.KafkaConfig.SASLOAuthGrantType - } - - if sinkConfig.KafkaConfig.SASLOAuthAudience != nil { - o.SASL.OAuth2.Audience = *sinkConfig.KafkaConfig.SASLOAuthAudience - } - } - - return nil -} - -// AutoCreateTopicConfig is used to create topic configuration. -type AutoCreateTopicConfig struct { - AutoCreate bool - PartitionNum int32 - ReplicationFactor int16 -} - -func (o *Options) DeriveTopicConfig() *AutoCreateTopicConfig { - return &AutoCreateTopicConfig{ - AutoCreate: o.AutoCreate, - PartitionNum: o.PartitionNum, - ReplicationFactor: o.ReplicationFactor, - } -} - -var ( - validClientID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`) - commonInvalidChar = regexp.MustCompile(`[\?:,"]`) -) - -// NewKafkaClientID generates kafka client id -func NewKafkaClientID(captureAddr string, - changefeedID model.ChangeFeedID, - configuredClientID string, -) (clientID string, err error) { - if configuredClientID != "" { - clientID = configuredClientID - } else { - clientID = fmt.Sprintf("TiCDC_producer_%s_%s_%s", - captureAddr, changefeedID.Namespace, changefeedID.ID) - clientID = commonInvalidChar.ReplaceAllString(clientID, "_") - } - if !validClientID.MatchString(clientID) { - return "", cerror.ErrKafkaInvalidClientID.GenWithStackByArgs(clientID) - } - return -} diff --git a/pkg/sink/kafka/sarama.go b/pkg/sink/kafka/sarama.go deleted file mode 100644 index df8877412..000000000 --- a/pkg/sink/kafka/sarama.go +++ /dev/null @@ -1,254 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kafka - -import ( - "context" - "crypto/tls" - "math/rand" - "strings" - "time" - - "github.com/IBM/sarama" - "github.com/pingcap/errors" - "github.com/pingcap/log" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/security" - "go.uber.org/zap" -) - -var ( - defaultKafkaVersion = sarama.V2_0_0_0 - maxKafkaVersion = sarama.V2_8_0_0 -) - -// NewSaramaConfig return the default config and set the according version and metrics -func NewSaramaConfig(ctx context.Context, o *Options) (*sarama.Config, error) { - config := sarama.NewConfig() - config.ClientID = o.ClientID - var err error - // Admin client would refresh metadata periodically, - // if metadata cannot be refreshed easily, this would indicate the network condition between the - // capture server and kafka broker is not good. - // Set the timeout to 2 minutes to ensure that the underlying client does not retry for too long. - // If retrying to obtain the metadata fails, simply return the error and let sinkManager rebuild the sink. - config.Metadata.Retry.Max = 10 - config.Metadata.Retry.Backoff = 200 * time.Millisecond - config.Metadata.Timeout = 2 * time.Minute - - config.Admin.Retry.Max = 10 - config.Admin.Retry.Backoff = 200 * time.Millisecond - // This timeout control the request timeout for each admin request. - // set it as the read timeout. - config.Admin.Timeout = 10 * time.Second - - // Producer.Retry take effect when the producer try to send message to kafka - // brokers. If kafka cluster is healthy, just the default value should be enough. - // For kafka cluster with a bad network condition, producer should not try to - // waster too much time on sending a message, get response no matter success - // or fail as soon as possible is preferred. - config.Producer.Retry.Max = 3 - config.Producer.Retry.Backoff = 100 * time.Millisecond - - // make sure sarama producer flush messages as soon as possible. - config.Producer.Flush.Bytes = 0 - config.Producer.Flush.Messages = 0 - config.Producer.Flush.Frequency = time.Duration(0) - config.Producer.Flush.MaxMessages = o.MaxMessages - - config.Net.DialTimeout = o.DialTimeout - config.Net.WriteTimeout = o.WriteTimeout - config.Net.ReadTimeout = o.ReadTimeout - - config.Producer.Partitioner = sarama.NewManualPartitioner - config.Producer.MaxMessageBytes = o.MaxMessageBytes - config.Producer.Return.Successes = true - config.Producer.Return.Errors = true - config.Producer.RequiredAcks = sarama.RequiredAcks(o.RequiredAcks) - compression := strings.ToLower(strings.TrimSpace(o.Compression)) - switch compression { - case "none": - config.Producer.Compression = sarama.CompressionNone - case "gzip": - config.Producer.Compression = sarama.CompressionGZIP - case "snappy": - config.Producer.Compression = sarama.CompressionSnappy - case "lz4": - config.Producer.Compression = sarama.CompressionLZ4 - case "zstd": - config.Producer.Compression = sarama.CompressionZSTD - default: - log.Warn("Unsupported compression algorithm", zap.String("compression", o.Compression)) - config.Producer.Compression = sarama.CompressionNone - } - if config.Producer.Compression != sarama.CompressionNone { - log.Info("Kafka producer uses " + compression + " compression algorithm") - } - - if o.EnableTLS { - // for SSL encryption with a trust CA certificate, we must populate the - // following two params of config.Net.TLS - config.Net.TLS.Enable = true - config.Net.TLS.Config = &tls.Config{ - MinVersion: tls.VersionTLS12, - NextProtos: []string{"h2", "http/1.1"}, - } - - // for SSL encryption with self-signed CA certificate, we reassign the - // config.Net.TLS.Config using the relevant credential files. - if o.Credential != nil && o.Credential.IsTLSEnabled() { - config.Net.TLS.Config, err = o.Credential.ToTLSConfig() - if err != nil { - return nil, errors.Trace(err) - } - } - - config.Net.TLS.Config.InsecureSkipVerify = o.InsecureSkipVerify - } - - err = completeSaramaSASLConfig(ctx, config, o) - if err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) - } - - kafkaVersion, err := getKafkaVersion(config, o) - if err != nil { - log.Warn("Can't get Kafka version by broker. ticdc will use default version", - zap.String("defaultVersion", kafkaVersion.String())) - } - config.Version = kafkaVersion - - if o.IsAssignedVersion { - version, err := sarama.ParseKafkaVersion(o.Version) - if err != nil { - return nil, cerror.WrapError(cerror.ErrKafkaInvalidVersion, err) - } - config.Version = version - if !version.IsAtLeast(maxKafkaVersion) && version.String() != kafkaVersion.String() { - log.Warn("The Kafka version you assigned may not be correct. "+ - "Please assign a version equal to or less than the specified version", - zap.String("assignedVersion", version.String()), - zap.String("desiredVersion", kafkaVersion.String())) - } - } - return config, nil -} - -func completeSaramaSASLConfig(ctx context.Context, config *sarama.Config, o *Options) error { - if o.SASL != nil && o.SASL.SASLMechanism != "" { - config.Net.SASL.Enable = true - config.Net.SASL.Mechanism = sarama.SASLMechanism(o.SASL.SASLMechanism) - switch o.SASL.SASLMechanism { - case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypePlaintext: - config.Net.SASL.User = o.SASL.SASLUser - config.Net.SASL.Password = o.SASL.SASLPassword - if strings.EqualFold(string(o.SASL.SASLMechanism), SASLTypeSCRAMSHA256) { - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { - return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA256} - } - } else if strings.EqualFold(string(o.SASL.SASLMechanism), SASLTypeSCRAMSHA512) { - config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient { - return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA512} - } - } - case SASLTypeGSSAPI: - config.Net.SASL.GSSAPI.AuthType = int(o.SASL.GSSAPI.AuthType) - config.Net.SASL.GSSAPI.Username = o.SASL.GSSAPI.Username - config.Net.SASL.GSSAPI.ServiceName = o.SASL.GSSAPI.ServiceName - config.Net.SASL.GSSAPI.KerberosConfigPath = o.SASL.GSSAPI.KerberosConfigPath - config.Net.SASL.GSSAPI.Realm = o.SASL.GSSAPI.Realm - config.Net.SASL.GSSAPI.DisablePAFXFAST = o.SASL.GSSAPI.DisablePAFXFAST - switch o.SASL.GSSAPI.AuthType { - case security.UserAuth: - config.Net.SASL.GSSAPI.Password = o.SASL.GSSAPI.Password - case security.KeyTabAuth: - config.Net.SASL.GSSAPI.KeyTabPath = o.SASL.GSSAPI.KeyTabPath - } - - case SASLTypeOAuth: - p, err := newTokenProvider(ctx, o) - if err != nil { - return errors.Trace(err) - } - config.Net.SASL.TokenProvider = p - } - } - - return nil -} - -func getKafkaVersion(config *sarama.Config, o *Options) (sarama.KafkaVersion, error) { - var err error - version := defaultKafkaVersion - addrs := o.BrokerEndpoints - if len(addrs) > 1 { - // Shuffle the list of addresses to randomize the order in which - // connections are attempted. This prevents routing all connections - // to the first broker (which will usually succeed). - rand.Shuffle(len(addrs), func(i, j int) { - addrs[i], addrs[j] = addrs[j], addrs[i] - }) - } - for i := range addrs { - version, err := getKafkaVersionFromBroker(config, o.RequestVersion, addrs[i]) - if err == nil { - return version, err - } - } - return version, err -} - -func getKafkaVersionFromBroker(config *sarama.Config, requestVersion int16, addr string) (sarama.KafkaVersion, error) { - KafkaVersion := defaultKafkaVersion - broker := sarama.NewBroker(addr) - err := broker.Open(config) - defer func() { - broker.Close() - }() - if err != nil { - log.Warn("Kafka fail to open broker", zap.String("addr", addr), zap.Error(err)) - return KafkaVersion, err - } - apiResponse, err := broker.ApiVersions(&sarama.ApiVersionsRequest{Version: requestVersion}) - if err != nil { - log.Warn("Kafka fail to get ApiVersions", zap.String("addr", addr), zap.Error(err)) - return KafkaVersion, err - } - // ApiKey method - // 0 Produce - // 3 Metadata (default) - version := apiResponse.ApiKeys[3].MaxVersion - if version >= 10 { - KafkaVersion = sarama.V2_8_0_0 - } else if version >= 9 { - KafkaVersion = sarama.V2_4_0_0 - } else if version >= 8 { - KafkaVersion = sarama.V2_3_0_0 - } else if version >= 7 { - KafkaVersion = sarama.V2_1_0_0 - } else if version >= 6 { - KafkaVersion = sarama.V2_0_0_0 - } else if version >= 5 { - KafkaVersion = sarama.V1_0_0_0 - } else if version >= 3 { - KafkaVersion = sarama.V0_11_0_0 - } else if version >= 2 { - KafkaVersion = sarama.V0_10_1_0 - } else if version >= 1 { - KafkaVersion = sarama.V0_10_0_0 - } else if version >= 0 { - KafkaVersion = sarama.V0_8_2_0 - } - return KafkaVersion, nil -} diff --git a/pkg/sink/kafka/sarama_factory.go b/pkg/sink/kafka/sarama_factory.go deleted file mode 100644 index b892a338c..000000000 --- a/pkg/sink/kafka/sarama_factory.go +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package kafka - -import ( - "context" - "time" - - "github.com/IBM/sarama" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/errors" - tikafka "github.com/pingcap/tiflow/pkg/sink/kafka" - "github.com/pingcap/tiflow/pkg/util" - "github.com/rcrowley/go-metrics" - "go.uber.org/zap" -) - -type saramaFactory struct { - changefeedID model.ChangeFeedID - option *Options - - registry metrics.Registry -} - -// NewSaramaFactory constructs a Factory with sarama implementation. -func NewSaramaFactory( - o *Options, - changefeedID model.ChangeFeedID, -) (Factory, error) { - return &saramaFactory{ - changefeedID: changefeedID, - option: o, - registry: metrics.NewRegistry(), - }, nil -} - -func (f *saramaFactory) AdminClient(ctx context.Context) (tikafka.ClusterAdminClient, error) { - start := time.Now() - config, err := NewSaramaConfig(ctx, f.option) - duration := time.Since(start).Seconds() - if duration > 2 { - log.Warn("new sarama config cost too much time", zap.Any("duration", duration), zap.Stringer("changefeedID", f.changefeedID)) - } - if err != nil { - return nil, err - } - - start = time.Now() - client, err := sarama.NewClient(f.option.BrokerEndpoints, config) - duration = time.Since(start).Seconds() - if duration > 2 { - log.Warn("new sarama client cost too much time", zap.Any("duration", duration), zap.Stringer("changefeedID", f.changefeedID)) - } - if err != nil { - return nil, errors.Trace(err) - } - - start = time.Now() - admin, err := sarama.NewClusterAdminFromClient(client) - duration = time.Since(start).Seconds() - if duration > 2 { - log.Warn("new sarama cluster admin cost too much time", zap.Any("duration", duration), zap.Stringer("changefeedID", f.changefeedID)) - } - if err != nil { - return nil, errors.Trace(err) - } - return &saramaAdminClient{ - client: client, - admin: admin, - changefeed: f.changefeedID, - }, nil -} - -// SyncProducer returns a Sync Producer, -// it should be the caller's responsibility to close the producer -func (f *saramaFactory) SyncProducer(ctx context.Context) (SyncProducer, error) { - config, err := NewSaramaConfig(ctx, f.option) - if err != nil { - return nil, err - } - config.MetricRegistry = f.registry - - client, err := sarama.NewClient(f.option.BrokerEndpoints, config) - if err != nil { - return nil, errors.Trace(err) - } - - p, err := sarama.NewSyncProducerFromClient(client) - if err != nil { - return nil, errors.Trace(err) - } - return &saramaSyncProducer{ - id: f.changefeedID, - client: client, - producer: p, - }, nil -} - -// AsyncProducer return an Async Producer, -// it should be the caller's responsibility to close the producer -func (f *saramaFactory) AsyncProducer( - ctx context.Context, - failpointCh chan error, -) (tikafka.AsyncProducer, error) { - config, err := NewSaramaConfig(ctx, f.option) - if err != nil { - return nil, err - } - config.MetricRegistry = f.registry - - client, err := sarama.NewClient(f.option.BrokerEndpoints, config) - if err != nil { - return nil, errors.Trace(err) - } - p, err := sarama.NewAsyncProducerFromClient(client) - if err != nil { - return nil, errors.Trace(err) - } - return &saramaAsyncProducer{ - client: client, - producer: p, - changefeedID: f.changefeedID, - failpointCh: failpointCh, - }, nil -} - -func (f *saramaFactory) MetricsCollector( - role util.Role, - adminClient tikafka.ClusterAdminClient, -) tikafka.MetricsCollector { - return tikafka.NewSaramaMetricsCollector( - f.changefeedID, role, adminClient, f.registry) -} diff --git a/pkg/sink/kafka/v2/admin.go b/pkg/sink/kafka/v2/admin.go deleted file mode 100644 index eca433f97..000000000 --- a/pkg/sink/kafka/v2/admin.go +++ /dev/null @@ -1,268 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package v2 - -import ( - "context" - "strconv" - - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/errors" - pkafka "github.com/pingcap/tiflow/pkg/sink/kafka" - "github.com/segmentio/kafka-go" - "go.uber.org/zap" -) - -type admin struct { - client Client - changefeedID model.ChangeFeedID -} - -func newClusterAdminClient( - endpoints []string, - transport *kafka.Transport, - changefeedID model.ChangeFeedID, -) pkafka.ClusterAdminClient { - client := newClient(endpoints, transport) - return &admin{ - client: client, - changefeedID: changefeedID, - } -} - -func (a *admin) clusterMetadata(ctx context.Context) (*kafka.MetadataResponse, error) { - // request is not set, so it will return all metadata - result, err := a.client.Metadata(ctx, &kafka.MetadataRequest{}) - if err != nil { - return nil, errors.Trace(err) - } - return result, nil -} - -func (a *admin) GetAllBrokers(ctx context.Context) ([]pkafka.Broker, error) { - response, err := a.clusterMetadata(ctx) - if err != nil { - return nil, errors.Trace(err) - } - - result := make([]pkafka.Broker, 0, len(response.Brokers)) - for _, broker := range response.Brokers { - result = append(result, pkafka.Broker{ - ID: int32(broker.ID), - }) - } - return result, nil -} - -func (a *admin) GetBrokerConfig(ctx context.Context, configName string) (string, error) { - response, err := a.clusterMetadata(ctx) - if err != nil { - return "", errors.Trace(err) - } - - controllerID := response.Controller.ID - request := &kafka.DescribeConfigsRequest{ - Resources: []kafka.DescribeConfigRequestResource{ - { - ResourceType: kafka.ResourceTypeBroker, - ResourceName: strconv.Itoa(controllerID), - ConfigNames: []string{configName}, - }, - }, - } - - resp, err := a.client.DescribeConfigs(ctx, request) - if err != nil { - return "", errors.Trace(err) - } - - if len(resp.Resources) == 0 || len(resp.Resources[0].ConfigEntries) == 0 { - log.Warn("Kafka config item not found", - zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the broker's configuration", configName) - } - - // For compatibility with KOP, we checked all return values. - // 1. Kafka only returns requested configs. - // 2. Kop returns all configs. - for _, entry := range resp.Resources[0].ConfigEntries { - if entry.ConfigName == configName { - return entry.ConfigValue, nil - } - } - - log.Warn("Kafka config item not found", - zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the broker's configuration", configName) -} - -func (a *admin) GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) { - request := &kafka.DescribeConfigsRequest{ - Resources: []kafka.DescribeConfigRequestResource{ - { - ResourceType: kafka.ResourceTypeTopic, - ResourceName: topicName, - ConfigNames: []string{configName}, - }, - }, - } - - resp, err := a.client.DescribeConfigs(ctx, request) - if err != nil { - return "", errors.Trace(err) - } - - if len(resp.Resources) == 0 || len(resp.Resources[0].ConfigEntries) == 0 { - log.Warn("Kafka config item not found", - zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the topic's configuration", configName) - } - - // For compatibility with KOP, we checked all return values. - // 1. Kafka only returns requested configs. - // 2. Kop returns all configs. - for _, entry := range resp.Resources[0].ConfigEntries { - if entry.ConfigName == configName { - log.Info("Kafka config item found", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID), - zap.String("configName", configName), - zap.String("configValue", entry.ConfigValue)) - return entry.ConfigValue, nil - } - } - - log.Warn("Kafka config item not found", - zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( - "cannot find the `%s` from the topic's configuration", configName) -} - -func (a *admin) GetTopicsMeta( - ctx context.Context, - topics []string, - ignoreTopicError bool, -) (map[string]pkafka.TopicDetail, error) { - resp, err := a.client.Metadata(ctx, &kafka.MetadataRequest{ - Topics: topics, - }) - if err != nil { - return nil, errors.Trace(err) - } - - result := make(map[string]pkafka.TopicDetail, len(resp.Topics)) - for _, topic := range resp.Topics { - if topic.Error != nil { - if !ignoreTopicError { - return nil, errors.Trace(topic.Error) - } - log.Warn("fetch topic meta failed", - zap.String("topic", topic.Name), zap.Error(topic.Error)) - continue - } - result[topic.Name] = pkafka.TopicDetail{ - Name: topic.Name, - NumPartitions: int32(len(topic.Partitions)), - } - } - return result, nil -} - -func (a *admin) GetTopicsPartitionsNum( - ctx context.Context, topics []string, -) (map[string]int32, error) { - resp, err := a.client.Metadata(ctx, &kafka.MetadataRequest{ - Topics: topics, - }) - if err != nil { - return nil, errors.Trace(err) - } - - result := make(map[string]int32, len(topics)) - for _, topic := range resp.Topics { - result[topic.Name] = int32(len(topic.Partitions)) - } - return result, nil -} - -func (a *admin) CreateTopic( - ctx context.Context, - detail *pkafka.TopicDetail, - validateOnly bool, -) error { - request := &kafka.CreateTopicsRequest{ - Topics: []kafka.TopicConfig{ - { - Topic: detail.Name, - NumPartitions: int(detail.NumPartitions), - ReplicationFactor: int(detail.ReplicationFactor), - }, - }, - ValidateOnly: validateOnly, - } - - response, err := a.client.CreateTopics(ctx, request) - if err != nil { - return errors.Trace(err) - } - - for _, err := range response.Errors { - if err != nil && !errors.Is(err, kafka.TopicAlreadyExists) { - return errors.Trace(err) - } - } - - return nil -} - -func (a *admin) Close() { - log.Info("admin client start closing", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID)) - client, ok := a.client.(*kafka.Client) - if !ok { - return - } - - if client.Transport == nil { - return - } - - transport, ok := client.Transport.(*kafka.Transport) - if !ok { - return - } - - transport.CloseIdleConnections() - log.Info("admin client close idle connections", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID)) - - if transport.SASL != nil { - m, ok := transport.SASL.(mechanism) - if ok && m.client != nil { - m.client.Destroy() - log.Info("destroy sasl sessions", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID)) - } - } - log.Info("kafka admin client is fully closed", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID)) -} diff --git a/pkg/sink/kafka/v2/client.go b/pkg/sink/kafka/v2/client.go deleted file mode 100644 index f9a1ed576..000000000 --- a/pkg/sink/kafka/v2/client.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package v2 - -import ( - "context" - - "github.com/segmentio/kafka-go" -) - -// Client is the interface of the kafka-go client, it contains a subset of all methods which is used -// by the kafka sink. -// This interface is mainly used to support mock kafka-go client in unit test. -type Client interface { - Metadata(ctx context.Context, req *kafka.MetadataRequest) (*kafka.MetadataResponse, error) - DescribeConfigs( - ctx context.Context, req *kafka.DescribeConfigsRequest, - ) (*kafka.DescribeConfigsResponse, error) - CreateTopics( - ctx context.Context, req *kafka.CreateTopicsRequest, - ) (*kafka.CreateTopicsResponse, error) -} diff --git a/pkg/sink/kafka/v2/factory.go b/pkg/sink/kafka/v2/factory.go deleted file mode 100644 index 342d7b24b..000000000 --- a/pkg/sink/kafka/v2/factory.go +++ /dev/null @@ -1,404 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package v2 - -import ( - "context" - "crypto/tls" - "strings" - "time" - - pkafka "github.com/flowbehappy/tigate/pkg/sink/kafka" - "github.com/jcmturner/gokrb5/v8/client" - "github.com/jcmturner/gokrb5/v8/config" - "github.com/jcmturner/gokrb5/v8/keytab" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/sink/codec/common" - tikafka "github.com/pingcap/tiflow/pkg/sink/kafka" - tiv2 "github.com/pingcap/tiflow/pkg/sink/kafka/v2" - "github.com/pingcap/tiflow/pkg/util" - "github.com/segmentio/kafka-go" - "github.com/segmentio/kafka-go/sasl" - "github.com/segmentio/kafka-go/sasl/plain" - "github.com/segmentio/kafka-go/sasl/scram" - "go.uber.org/zap" -) - -type factory struct { - // transport is used to contact kafka cluster and also maintain the `metadata cache` - // it's shared by the admin client and producers to keep the cache the same to make - // sure that the newly created topics can be found by the both. - transport *kafka.Transport - changefeedID model.ChangeFeedID - options *pkafka.Options - - writer *kafka.Writer -} - -// NewFactory returns a factory implemented based on kafka-go -func NewFactory( - options *pkafka.Options, - changefeedID model.ChangeFeedID, -) (pkafka.Factory, error) { - transport, err := newTransport(options) - if err != nil { - return nil, errors.Trace(err) - } - return &factory{ - transport: transport, - changefeedID: changefeedID, - options: options, - writer: &kafka.Writer{}, - }, nil -} - -func newClient(brokerEndpoints []string, transport *kafka.Transport) *kafka.Client { - return &kafka.Client{ - Addr: kafka.TCP(brokerEndpoints...), - // todo: make this configurable - Timeout: 10 * time.Second, - Transport: transport, - } -} - -func newTransport(o *pkafka.Options) (*kafka.Transport, error) { - mechanism, err := completeSASLConfig(o) - if err != nil { - return nil, err - } - tlsConfig, err := completeSSLConfig(o) - if err != nil { - return nil, err - } - return &kafka.Transport{ - SASL: mechanism, - TLS: tlsConfig, - DialTimeout: o.DialTimeout, - ClientID: o.ClientID, - }, nil -} - -func completeSSLConfig(options *pkafka.Options) (*tls.Config, error) { - if options.EnableTLS { - tlsConfig := &tls.Config{ - MinVersion: tls.VersionTLS12, - NextProtos: []string{"h2", "http/1.1"}, - } - - // for SSL encryption with self-signed CA certificate, we reassign the - // config.Net.TLS.Config using the relevant credential files. - if options.Credential != nil && options.Credential.IsTLSEnabled() { - tlsConfig, err := options.Credential.ToTLSConfig() - return tlsConfig, errors.Trace(err) - } - - tlsConfig.InsecureSkipVerify = options.InsecureSkipVerify - return tlsConfig, nil - } - return nil, nil -} - -func completeSASLConfig(o *pkafka.Options) (sasl.Mechanism, error) { - if o.SASL != nil && o.SASL.SASLMechanism != "" { - switch o.SASL.SASLMechanism { - case pkafka.SASLTypeSCRAMSHA256, pkafka.SASLTypeSCRAMSHA512, pkafka.SASLTypePlaintext: - if strings.EqualFold(string(o.SASL.SASLMechanism), pkafka.SASLTypeSCRAMSHA256) { - mechanism, err := scram.Mechanism(scram.SHA256, - o.SASL.SASLUser, o.SASL.SASLPassword) - return mechanism, errors.Trace(err) - } else if strings.EqualFold(string(o.SASL.SASLMechanism), pkafka.SASLTypeSCRAMSHA512) { - mechanism, err := scram.Mechanism(scram.SHA512, - o.SASL.SASLUser, o.SASL.SASLPassword) - return mechanism, errors.Trace(err) - } else { - return plain.Mechanism{ - Username: o.SASL.SASLUser, - Password: o.SASL.SASLPassword, - }, nil - } - case pkafka.SASLTypeGSSAPI: - cfg, err := config.Load(o.SASL.GSSAPI.KerberosConfigPath) - if err != nil { - return nil, errors.Trace(err) - } - var clnt *client.Client - switch o.SASL.GSSAPI.AuthType { - case security.UserAuth: - clnt = client.NewWithPassword(o.SASL.GSSAPI.Username, o.SASL.GSSAPI.Realm, - o.SASL.GSSAPI.Password, cfg, - client.DisablePAFXFAST(o.SASL.GSSAPI.DisablePAFXFAST)) - case security.KeyTabAuth: - ktab, err := keytab.Load(o.SASL.GSSAPI.KeyTabPath) - if err != nil { - return nil, errors.Trace(err) - } - clnt = client.NewWithKeytab(o.SASL.GSSAPI.Username, o.SASL.GSSAPI.Realm, ktab, cfg, - client.DisablePAFXFAST(o.SASL.GSSAPI.DisablePAFXFAST)) - } - err = clnt.Login() - if err != nil { - return nil, errors.Trace(err) - } - return Gokrb5v8(&gokrb5v8ClientImpl{clnt}, - o.SASL.GSSAPI.ServiceName), nil - - case pkafka.SASLTypeOAuth: - return nil, errors.ErrKafkaInvalidConfig.GenWithStack( - "OAuth is not yet supported in Kafka sink v2") - } - } - return nil, nil -} - -func (f *factory) newWriter(async bool) *kafka.Writer { - w := &kafka.Writer{ - Addr: kafka.TCP(f.options.BrokerEndpoints...), - Balancer: newManualPartitioner(), - Transport: f.transport, - ReadTimeout: f.options.ReadTimeout, - WriteTimeout: f.options.WriteTimeout, - // For kafka cluster with a bad network condition, - // do not waste too much time to prevent long time blocking. - MaxAttempts: 2, - WriteBackoffMin: 10 * time.Millisecond, - RequiredAcks: kafka.RequiredAcks(f.options.RequiredAcks), - BatchBytes: int64(f.options.MaxMessageBytes), - Async: async, - } - f.writer = w - compression := strings.ToLower(strings.TrimSpace(f.options.Compression)) - switch compression { - case "none": - case "gzip": - w.Compression = kafka.Gzip - case "snappy": - w.Compression = kafka.Snappy - case "lz4": - w.Compression = kafka.Lz4 - case "zstd": - w.Compression = kafka.Zstd - default: - log.Warn("Unsupported compression algorithm", - zap.String("namespace", f.changefeedID.Namespace), - zap.String("changefeed", f.changefeedID.ID), - zap.String("compression", f.options.Compression)) - } - log.Info("Kafka producer uses "+f.options.Compression+" compression algorithm", - zap.String("namespace", f.changefeedID.Namespace), - zap.String("changefeed", f.changefeedID.ID)) - return w -} - -func (f *factory) AdminClient(_ context.Context) (tikafka.ClusterAdminClient, error) { - return newClusterAdminClient(f.options.BrokerEndpoints, f.transport, f.changefeedID), nil -} - -// SyncProducer creates a sync producer to writer message to kafka -func (f *factory) SyncProducer(_ context.Context) (pkafka.SyncProducer, error) { - w := f.newWriter(false) - // set batch size to 1 to make sure the message is sent immediately - w.BatchTimeout = time.Millisecond - w.BatchSize = 1 - return &syncWriter{ - w: w, - changefeedID: f.changefeedID, - }, nil -} - -// AsyncProducer creates an async producer to writer message to kafka -func (f *factory) AsyncProducer( - ctx context.Context, - failpointCh chan error, -) (tikafka.AsyncProducer, error) { - w := f.newWriter(true) - // assume each message is 1KB, - // and set batch timeout to 5ms to avoid waste too much time on waiting for messages. - w.BatchTimeout = 5 * time.Millisecond - w.BatchSize = int(w.BatchBytes / 1024) - aw := &asyncWriter{ - w: w, - changefeedID: f.changefeedID, - failpointCh: failpointCh, - errorsChan: make(chan error, 1), - } - - w.Completion = func(messages []kafka.Message, err error) { - if err != nil { - select { - case <-ctx.Done(): - return - case aw.errorsChan <- err: - default: - log.Warn("async writer report error failed, since the err channel is full", - zap.String("namespace", aw.changefeedID.Namespace), - zap.String("changefeed", aw.changefeedID.ID), - zap.Error(err)) - } - return - } - - for _, msg := range messages { - callback := msg.WriterData.(func()) - if callback != nil { - callback() - } - } - } - - return aw, nil -} - -// MetricsCollector returns the kafka metrics collector -func (f *factory) MetricsCollector( - role util.Role, - adminClient tikafka.ClusterAdminClient, -) tikafka.MetricsCollector { - return tiv2.NewMetricsCollector(f.changefeedID, role, f.writer) -} - -type syncWriter struct { - changefeedID model.ChangeFeedID - w tiv2.Writer -} - -func (s *syncWriter) SendMessage( - ctx context.Context, - topic string, partitionNum int32, - message *common.Message, -) error { - return s.w.WriteMessages(ctx, kafka.Message{ - Topic: topic, - Partition: int(partitionNum), - Key: message.Key, - Value: message.Value, - }) -} - -// SendMessages produces a given set of messages, and returns only when all -// messages in the set have either succeeded or failed. Note that messages -// can succeed and fail individually; if some succeed and some fail, -// SendMessages will return an error. -func (s *syncWriter) SendMessages(ctx context.Context, topic string, partitionNum int32, message *common.Message) error { - msgs := make([]kafka.Message, int(partitionNum)) - for i := 0; i < int(partitionNum); i++ { - msgs[i] = kafka.Message{ - Topic: topic, - Key: message.Key, - Value: message.Value, - Partition: i, - } - } - return s.w.WriteMessages(ctx, msgs...) -} - -// Close shuts down the producer; you must call this function before a producer -// object passes out of scope, as it may otherwise leak memory. -// You must call this before calling Close on the underlying client. -func (s *syncWriter) Close() { - log.Info("kafka sync producer start closing", - zap.String("namespace", s.changefeedID.Namespace), - zap.String("changefeed", s.changefeedID.ID)) - start := time.Now() - if err := s.w.Close(); err != nil { - log.Warn("Close kafka sync producer failed", - zap.String("namespace", s.changefeedID.Namespace), - zap.String("changefeed", s.changefeedID.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Close kafka sync producer success", - zap.String("namespace", s.changefeedID.Namespace), - zap.String("changefeed", s.changefeedID.ID), - zap.Duration("duration", time.Since(start))) - } -} - -type asyncWriter struct { - w tiv2.Writer - changefeedID model.ChangeFeedID - failpointCh chan error - errorsChan chan error -} - -// Close shuts down the producer and waits for any buffered messages to be -// flushed. You must call this function before a producer object passes out of -// scope, as it may otherwise leak memory. You must call this before process -// shutting down, or you may lose messages. You must call this before calling -// Close on the underlying client. -func (a *asyncWriter) Close() { - log.Info("kafka async producer start closing", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID)) - go func() { - start := time.Now() - if err := a.w.Close(); err != nil { - log.Warn("Close kafka async producer failed", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID), - zap.Duration("duration", time.Since(start)), - zap.Error(err)) - } else { - log.Info("Close kafka async producer success", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID), - zap.Duration("duration", time.Since(start))) - } - }() -} - -// AsyncSend is the input channel for the user to write messages to that they -// wish to send. -func (a *asyncWriter) AsyncSend(ctx context.Context, topic string, partition int32, message *common.Message) error { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - default: - } - return a.w.WriteMessages(ctx, kafka.Message{ - Topic: topic, - Partition: int(partition), - Key: message.Key, - Value: message.Value, - WriterData: message.Callback, - }) -} - -// AsyncRunCallback process the messages that has sent to kafka, -// and run tha attached callback. the caller should call this -// method in a background goroutine -func (a *asyncWriter) AsyncRunCallback(ctx context.Context) error { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case err := <-a.failpointCh: - log.Warn("Receive from failpoint chan in kafka producer", - zap.String("namespace", a.changefeedID.Namespace), - zap.String("changefeed", a.changefeedID.ID), - zap.Error(err)) - return errors.Trace(err) - case err := <-a.errorsChan: - // We should not wrap a nil pointer if the pointer - // is of a subtype of `error` because Go would store the type info - // and the resulted `error` variable would not be nil, - // which will cause the pkg/error library to malfunction. - // See: https://go.dev/doc/faq#nil_error - if err == nil { - return nil - } - return errors.WrapError(errors.ErrKafkaAsyncSendMessage, err) - } -} diff --git a/pkg/sink/kafka/v2/gssapi.go b/pkg/sink/kafka/v2/gssapi.go deleted file mode 100644 index d55063381..000000000 --- a/pkg/sink/kafka/v2/gssapi.go +++ /dev/null @@ -1,252 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package v2 - -import ( - "context" - "encoding/asn1" - "encoding/binary" - - "github.com/jcmturner/gokrb5/v8/client" - "github.com/jcmturner/gokrb5/v8/credentials" - "github.com/jcmturner/gokrb5/v8/crypto" - "github.com/jcmturner/gokrb5/v8/gssapi" - "github.com/jcmturner/gokrb5/v8/iana/chksumtype" - "github.com/jcmturner/gokrb5/v8/iana/keyusage" - "github.com/jcmturner/gokrb5/v8/messages" - "github.com/jcmturner/gokrb5/v8/types" - "github.com/pingcap/errors" - "github.com/segmentio/kafka-go/sasl" -) - -const ( - // TokIDKrbApReq https://tools.ietf.org/html/rfc4121#section-4.1 - TokIDKrbApReq = "\x01\x00" -) - -// Gokrb5v8Client is the client for gokrbv8 -type Gokrb5v8Client interface { - // GetServiceTicket get a ticker form server - GetServiceTicket(spn string) (messages.Ticket, types.EncryptionKey, error) - // Destroy stops the auto-renewal of all sessions and removes - // the sessions and cache entries from the client. - Destroy() - // Credentials returns the client credentials - Credentials() *credentials.Credentials -} - -type gokrb5v8ClientImpl struct { - client *client.Client -} - -func (c *gokrb5v8ClientImpl) GetServiceTicket(spn string) ( - messages.Ticket, types.EncryptionKey, error, -) { - return c.client.GetServiceTicket(spn) -} - -func (c *gokrb5v8ClientImpl) Credentials() *credentials.Credentials { - return c.client.Credentials -} - -func (c *gokrb5v8ClientImpl) Destroy() { - c.client.Destroy() -} - -type mechanism struct { - client Gokrb5v8Client - serviceName string - host string -} - -func (m mechanism) Name() string { - return "GSSAPI" -} - -// Gokrb5v8 uses gokrb5/v8 to implement the GSSAPI mechanism. -// -// client is a github.com/gokrb5/v8/client *Client instance. -// kafkaServiceName is the name of the Kafka service in your Kerberos. -func Gokrb5v8(client Gokrb5v8Client, kafkaServiceName string) sasl.Mechanism { - return mechanism{client, kafkaServiceName, ""} -} - -// StartWithoutHostError is the error type for when Start is called on -// the GSSAPI mechanism without the host having been set by WithHost. -// -// Unless you are calling the GSSAPI SASL mechanim's Start method -// yourself for some reason, this error will never be returned. -type StartWithoutHostError struct{} - -func (e StartWithoutHostError) Error() string { - return "GSSAPI SASL handshake needs a host" -} - -func (m mechanism) Start(ctx context.Context) (sasl.StateMachine, []byte, error) { - metaData := sasl.MetadataFromContext(ctx) - m.host = metaData.Host - if m.host == "" { - return nil, nil, StartWithoutHostError{} - } - - servicePrincipalName := m.serviceName + "/" + m.host - ticket, key, err := m.client.GetServiceTicket( - servicePrincipalName, - ) - if err != nil { - return nil, nil, errors.Trace(err) - } - - authenticator, err := types.NewAuthenticator( - m.client.Credentials().Realm(), - m.client.Credentials().CName(), - ) - if err != nil { - return nil, nil, errors.Trace(err) - } - - encryptionType, err := crypto.GetEtype(key.KeyType) - if err != nil { - return nil, nil, errors.Trace(err) - } - - keySize := encryptionType.GetKeyByteSize() - err = authenticator.GenerateSeqNumberAndSubKey(key.KeyType, keySize) - if err != nil { - return nil, nil, errors.Trace(err) - } - - authenticator.Cksum = types.Checksum{ - CksumType: chksumtype.GSSAPI, - Checksum: authenticatorPseudoChecksum(), - } - apReq, err := messages.NewAPReq(ticket, key, authenticator) - if err != nil { - return nil, nil, errors.Trace(err) - } - - bytes, err := apReq.Marshal() - if err != nil { - return nil, nil, errors.Trace(err) - } - gssapiToken, err := getGssAPIToken(bytes) - if err != nil { - return nil, nil, errors.Trace(err) - } - return &gokrb5v8Session{authenticator.SubKey, false}, gssapiToken, nil -} - -func getGssAPIToken(bytes []byte) ([]byte, error) { - bytesWithPrefix := make([]byte, 0, len(TokIDKrbApReq)+len(bytes)) - bytesWithPrefix = append(bytesWithPrefix, TokIDKrbApReq...) - bytesWithPrefix = append(bytesWithPrefix, bytes...) - - return prependGSSAPITokenTag(bytesWithPrefix) -} - -func authenticatorPseudoChecksum() []byte { - // Not actually a checksum, but it goes in the checksum field. - // https://tools.ietf.org/html/rfc4121#section-4.1.1 - checksum := make([]byte, 24) - - flags := gssapi.ContextFlagInteg - // Reasons for each flag being on or off: - // Delegation: Off. We are not using delegated credentials. - // Mutual: Off. Mutual authentication is already provided - // as a result of how Kerberos works. - // Replay: Off. We don’t need replay protection because each - // packet is secured by a per-session key and is unique - // within its session. - // Sequence: Off. Out-of-order messages cannot happen in our - // case, and if it somehow happened anyway it would - // necessarily trigger other appropriate errors. - // Confidentiality: Off. Our authentication itself does not - // seem to be requesting or using any “security layers” - // in the GSSAPI sense, and this is just one of the - // security layer features. Also, if we were requesting - // a GSSAPI security layer, we would be required to - // set the mutual flag to on. - // https://tools.ietf.org/html/rfc4752#section-3.1 - // Integrity: On. Must be on when calling the standard API, - // so it probably must be set in the raw packet itself. - // https://tools.ietf.org/html/rfc4752#section-3.1 - // https://tools.ietf.org/html/rfc4752#section-7 - // Anonymous: Off. We are not using an anonymous ticket. - // https://tools.ietf.org/html/rfc6112#section-3 - - binary.LittleEndian.PutUint32(checksum[0:4], 16) - // checksum[4:20] is unused/blank channel binding settings. - binary.LittleEndian.PutUint32(checksum[20:24], uint32(flags)) - return checksum -} - -type gssapiToken struct { - OID asn1.ObjectIdentifier - Object asn1.RawValue -} - -func prependGSSAPITokenTag(payload []byte) ([]byte, error) { - // The GSSAPI "token" is almost an ASN.1 encoded object, except - // that the "token object" is raw bytes, not necessarily ASN.1. - // https://tools.ietf.org/html/rfc2743#page-81 (section 3.1) - token := gssapiToken{ - OID: asn1.ObjectIdentifier(gssapi.OIDKRB5.OID()), - Object: asn1.RawValue{FullBytes: payload}, - } - return asn1.MarshalWithParams(token, "application") -} - -type gokrb5v8Session struct { - key types.EncryptionKey - done bool -} - -func (s *gokrb5v8Session) Next(ctx context.Context, challenge []byte) (bool, []byte, error) { - if s.done { - return true, nil, nil - } - const tokenIsFromGSSAcceptor = true - challengeToken := gssapi.WrapToken{} - err := challengeToken.Unmarshal(challenge, tokenIsFromGSSAcceptor) - if err != nil { - return false, nil, errors.Trace(err) - } - - valid, err := challengeToken.Verify( - s.key, - keyusage.GSSAPI_ACCEPTOR_SEAL, - ) - if !valid { - return false, nil, errors.Trace(err) - } - - responseToken, err := gssapi.NewInitiatorWrapToken( - challengeToken.Payload, - s.key, - ) - if err != nil { - return false, nil, errors.Trace(err) - } - - response, err := responseToken.Marshal() - if err != nil { - return false, nil, errors.Trace(err) - } - - // We are done, but we can't return `true` yet because - // the SASL loop calling this needs the first return to be - // `false` any time there are response bytes to send. - s.done = true - return false, response, nil -} diff --git a/pkg/sink/kafka/v2/manual_balancer.go b/pkg/sink/kafka/v2/manual_balancer.go deleted file mode 100644 index ff852f1e7..000000000 --- a/pkg/sink/kafka/v2/manual_balancer.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package v2 - -import "github.com/segmentio/kafka-go" - -// todo: after implement the producer, remove the `unused lint`. -// -//nolint:unused -type manualPartitioner struct{} - -//nolint:unused -func newManualPartitioner() kafka.Balancer { - return &manualPartitioner{} -} - -//nolint:unused -func (m *manualPartitioner) Balance(msg kafka.Message, partitions ...int) (partition int) { - return msg.Partition -} diff --git a/pkg/sink/util/helper.go b/pkg/sink/util/helper.go deleted file mode 100644 index bee87d735..000000000 --- a/pkg/sink/util/helper.go +++ /dev/null @@ -1,44 +0,0 @@ -package util - -import ( - "net/url" - - ticonfig "github.com/flowbehappy/tigate/pkg/config" - "github.com/flowbehappy/tigate/pkg/sink/codec/common" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/util" -) - -// GetEncoderConfig returns the encoder config and validates the config. -func GetEncoderConfig( - changefeedID model.ChangeFeedID, - sinkURI *url.URL, - protocol config.Protocol, - sinkConfig *ticonfig.SinkConfig, - maxMsgBytes int, -) (*common.Config, error) { - encoderConfig := common.NewConfig(protocol) - if err := encoderConfig.Apply(sinkURI, sinkConfig); err != nil { - return nil, cerror.WrapError(cerror.ErrSinkInvalidConfig, err) - } - // Always set encoder's `MaxMessageBytes` equal to producer's `MaxMessageBytes` - // to prevent that the encoder generate batched message too large - // then cause producer meet `message too large`. - encoderConfig = encoderConfig. - WithMaxMessageBytes(maxMsgBytes). - WithChangefeedID(changefeedID) - - tz, err := util.GetTimezone(config.GetGlobalServerConfig().TZ) - if err != nil { - return nil, cerror.WrapError(cerror.ErrSinkInvalidConfig, err) - } - encoderConfig.TimeZone = tz - - if err := encoderConfig.Validate(); err != nil { - return nil, cerror.WrapError(cerror.ErrSinkInvalidConfig, err) - } - - return encoderConfig, nil -}