diff --git a/Makefile b/Makefile index 3dd8543b3..985ceb185 100644 --- a/Makefile +++ b/Makefile @@ -37,12 +37,19 @@ GITHASH := $(shell git rev-parse HEAD) GITBRANCH := $(shell git rev-parse --abbrev-ref HEAD) GOVERSION := $(shell go version) +# Since TiDB add a new dependency on github.com/cloudfoundry/gosigar, +# We need to add CGO_ENABLED=1 to make it work when build TiCDC in Darwin OS. +# These logic is to check if the OS is Darwin, if so, add CGO_ENABLED=1. +# ref: https://github.com/cloudfoundry/gosigar/issues/58#issuecomment-1150925711 +# ref: https://github.com/pingcap/tidb/pull/39526#issuecomment-1407952955 OS := "$(shell go env GOOS)" SED_IN_PLACE ?= $(shell which sed) IS_ALPINE := $(shell if [ -f /etc/os-release ]; then grep -qi Alpine /etc/os-release && echo 1; else echo 0; fi) ifeq (${OS}, "linux") + CGO := 0 SED_IN_PLACE += -i else ifeq (${OS}, "darwin") + CGO := 1 SED_IN_PLACE += -i '' endif @@ -53,6 +60,7 @@ GOEXPERIMENT= ifeq ("${ENABLE_FIPS}", "1") BUILD_FLAG = -tags boringcrypto GOEXPERIMENT = GOEXPERIMENT=boringcrypto + CGO = 1 endif RELEASE_VERSION = @@ -81,8 +89,8 @@ CONSUMER_BUILD_FLAG= ifeq ("${IS_ALPINE}", "1") CONSUMER_BUILD_FLAG = -tags musl endif - -GOBUILD := $(GOEXPERIMENT) CGO_ENABLED=1 $(GO) build $(BUILD_FLAG) -trimpath $(GOVENDORFLAG) +GOBUILD := $(GOEXPERIMENT) CGO_ENABLED=$(CGO) $(GO) build $(BUILD_FLAG) -trimpath $(GOVENDORFLAG) +CONSUMER_GOBUILD := $(GOEXPERIMENT) CGO_ENABLED=1 $(GO) build $(CONSUMER_BUILD_FLAG) -trimpath $(GOVENDORFLAG) PACKAGE_LIST := go list ./... | grep -vE 'vendor|proto|ticdc/tests|integration|testing_utils|pb|pbmock|ticdc/bin' PACKAGES := $$($(PACKAGE_LIST)) @@ -124,7 +132,7 @@ cdc: $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc ./cmd/cdc kafka_consumer: - $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_kafka_consumer ./cmd/kafka-consumer + $(CONSUMER_GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_kafka_consumer ./cmd/kafka-consumer storage_consumer: $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc_storage_consumer ./cmd/storage-consumer/main.go diff --git a/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager_test.go b/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager_test.go index 48ef6e722..bf02658b2 100644 --- a/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager_test.go +++ b/downstreamadapter/sink/helper/topicmanager/kafka_topic_manager_test.go @@ -25,21 +25,16 @@ import ( func TestCreateTopic(t *testing.T) { t.Parallel() - options := kafka.NewOptions() - changefeed := common.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) - require.NoError(t, err) - - adminClient, err := factory.AdminClient() - require.NoError(t, err) + adminClient := kafka.NewClusterAdminClientMockImpl() defer adminClient.Close() cfg := &kafka.AutoCreateTopicConfig{ AutoCreate: true, PartitionNum: 2, ReplicationFactor: 1, } - ctx := context.Background() + changefeedID := common.NewChangefeedID4Test("test", "test") + ctx := context.Background() manager := newKafkaTopicManager(ctx, kafka.DefaultMockTopicName, changefeedID, adminClient, cfg) defer manager.Close() partitionNum, err := manager.CreateTopicAndWaitUntilVisible(ctx, kafka.DefaultMockTopicName) @@ -85,13 +80,7 @@ func TestCreateTopic(t *testing.T) { func TestCreateTopicWithDelay(t *testing.T) { t.Parallel() - options := kafka.NewOptions() - changefeed := common.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) - require.NoError(t, err) - - adminClient, err := factory.AdminClient() - require.NoError(t, err) + adminClient := kafka.NewClusterAdminClientMockImpl() defer adminClient.Close() cfg := &kafka.AutoCreateTopicConfig{ AutoCreate: true, @@ -99,9 +88,9 @@ func TestCreateTopicWithDelay(t *testing.T) { ReplicationFactor: 1, } - ctx := context.Background() topic := "new_topic" changefeedID := common.NewChangefeedID4Test("test", "test") + ctx := context.Background() manager := newKafkaTopicManager(ctx, topic, changefeedID, adminClient, cfg) defer manager.Close() partitionNum, err := manager.createTopic(ctx, topic) diff --git a/downstreamadapter/sink/kafka_sink.go b/downstreamadapter/sink/kafka_sink.go index 70b3b44a7..fe5c6b2cb 100644 --- a/downstreamadapter/sink/kafka_sink.go +++ b/downstreamadapter/sink/kafka_sink.go @@ -85,7 +85,7 @@ func newKafkaSink( }() statistics := metrics.NewStatistics(changefeedID, "KafkaSink") - asyncProducer, err := kafkaComponent.Factory.AsyncProducer() + asyncProducer, err := kafkaComponent.Factory.AsyncProducer(ctx) if err != nil { return nil, errors.WrapError(errors.ErrKafkaNewProducer, err) } @@ -121,8 +121,8 @@ func newKafkaSink( adminClient: kafkaComponent.AdminClient, topicManager: kafkaComponent.TopicManager, statistics: statistics, - metricsCollector: kafkaComponent.Factory.MetricsCollector(), ctx: ctx, + metricsCollector: kafkaComponent.Factory.MetricsCollector(kafkaComponent.AdminClient), } return sink, nil } @@ -203,11 +203,10 @@ func newKafkaSinkForTest() (*KafkaSink, producer.DMLProducer, producer.DDLProduc ctx := context.Background() changefeedID := common.NewChangefeedID4Test("test", "test") openProtocol := "open-protocol" - sinkConfig := config.GetDefaultReplicaConfig().Clone().Sink - sinkConfig.Protocol = &openProtocol + sinkConfig := &config.SinkConfig{Protocol: &openProtocol} uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" + "&max-message-bytes=1048576&partition-num=1" + - "&kafka-client-id=unit-test&auto-create-topic=true&compression=gzip&protocol=open-protocol" + "&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=open-protocol" uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName) sinkURI, err := url.Parse(uri) @@ -257,7 +256,7 @@ func newKafkaSinkForTest() (*KafkaSink, producer.DMLProducer, producer.DDLProduc adminClient: kafkaComponent.AdminClient, topicManager: kafkaComponent.TopicManager, statistics: statistics, - metricsCollector: kafkaComponent.Factory.MetricsCollector(), + metricsCollector: kafkaComponent.Factory.MetricsCollector(kafkaComponent.AdminClient), } go sink.Run(ctx) return sink, dmlMockProducer, ddlMockProducer, nil diff --git a/downstreamadapter/worker/helper.go b/downstreamadapter/worker/helper.go index fad94d3af..e0642e967 100644 --- a/downstreamadapter/worker/helper.go +++ b/downstreamadapter/worker/helper.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/ticdc/pkg/sink/codec" "github.com/pingcap/ticdc/pkg/sink/codec/common" "github.com/pingcap/ticdc/pkg/sink/kafka" + v2 "github.com/pingcap/ticdc/pkg/sink/kafka/v2" "github.com/pingcap/ticdc/pkg/sink/util" "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tiflow/pkg/sink" @@ -59,7 +60,7 @@ func getKafkaSinkComponentWithFactory(ctx context.Context, return kafkaComponent, protocol, errors.WrapError(errors.ErrKafkaInvalidConfig, err) } - kafkaComponent.Factory, err = factoryCreator(options, changefeedID) + kafkaComponent.Factory, err = factoryCreator(ctx, options, changefeedID) if err != nil { return kafkaComponent, protocol, errors.WrapError(errors.ErrKafkaNewProducer, err) } @@ -93,9 +94,7 @@ func getKafkaSinkComponentWithFactory(ctx context.Context, options.DeriveTopicConfig(), kafkaComponent.AdminClient, ) - if err != nil { - return kafkaComponent, protocol, errors.Trace(err) - } + scheme := sink.GetScheme(sinkURI) kafkaComponent.EventRouter, err = eventrouter.NewEventRouter(sinkConfig, protocol, topic, scheme) if err != nil { @@ -130,7 +129,10 @@ func GetKafkaSinkComponent( sinkURI *url.URL, sinkConfig *config.SinkConfig, ) (KafkaComponent, config.Protocol, error) { - factoryCreator := kafka.NewFactory + factoryCreator := kafka.NewSaramaFactory + if utils.GetOrZero(sinkConfig.EnableKafkaSinkV2) { + factoryCreator = v2.NewFactory + } return getKafkaSinkComponentWithFactory(ctx, changefeedID, sinkURI, sinkConfig, factoryCreator) } diff --git a/downstreamadapter/worker/kafka_ddl_worker_test.go b/downstreamadapter/worker/kafka_ddl_worker_test.go index 980f541d7..2d372ea84 100644 --- a/downstreamadapter/worker/kafka_ddl_worker_test.go +++ b/downstreamadapter/worker/kafka_ddl_worker_test.go @@ -38,11 +38,10 @@ func kafkaDDLWorkerForTest(t *testing.T) *KafkaDDLWorker { ctx := context.Background() changefeedID := common.NewChangefeedID4Test("test", "test") openProtocol := "open-protocol" - sinkConfig := config.GetDefaultReplicaConfig().Clone().Sink - sinkConfig.Protocol = &openProtocol + sinkConfig := &config.SinkConfig{Protocol: &openProtocol} uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" + "&max-message-bytes=1048576&partition-num=1" + - "&kafka-client-id=unit-test&auto-create-topic=true&compression=gzip&protocol=open-protocol" + "&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=open-protocol" uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName) sinkURI, err := url.Parse(uri) diff --git a/downstreamadapter/worker/kafka_dml_worker_test.go b/downstreamadapter/worker/kafka_dml_worker_test.go index 5dcfb04f1..63b6396b4 100644 --- a/downstreamadapter/worker/kafka_dml_worker_test.go +++ b/downstreamadapter/worker/kafka_dml_worker_test.go @@ -36,11 +36,10 @@ func kafkaDMLWorkerForTest(t *testing.T) *KafkaDMLWorker { ctx := context.Background() changefeedID := common.NewChangefeedID4Test("test", "test") openProtocol := "open-protocol" - sinkConfig := config.GetDefaultReplicaConfig().Clone().Sink - sinkConfig.Protocol = &openProtocol + sinkConfig := &config.SinkConfig{Protocol: &openProtocol} uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" + "&max-message-bytes=1048576&partition-num=1" + - "&kafka-client-id=unit-test&auto-create-topic=true&compression=gzip&protocol=open-protocol" + "&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=open-protocol" uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName) sinkURI, err := url.Parse(uri) diff --git a/downstreamadapter/worker/producer/kafka_ddl_producer_test.go b/downstreamadapter/worker/producer/kafka_ddl_producer_test.go index 831dace78..7a437e975 100644 --- a/downstreamadapter/worker/producer/kafka_ddl_producer_test.go +++ b/downstreamadapter/worker/producer/kafka_ddl_producer_test.go @@ -15,11 +15,10 @@ package producer import ( "context" - "fmt" "testing" "time" - confluentKafka "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/IBM/sarama" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/errors" cerror "github.com/pingcap/ticdc/pkg/errors" @@ -34,8 +33,9 @@ func TestDDLSyncBroadcastMessage(t *testing.T) { options.MaxMessages = 1 changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t syncProducer, err := factory.SyncProducer() require.NoError(t, err) @@ -43,7 +43,7 @@ func TestDDLSyncBroadcastMessage(t *testing.T) { p := NewKafkaDDLProducer(ctx, changefeed, syncProducer) for i := 0; i < kafka.DefaultMockPartitionNum; i++ { - syncProducer.(*kafka.MockSyncProducer).Producer.ExpectSendMessageAndSucceed() + syncProducer.(*kafka.MockSaramaSyncProducer).Producer.ExpectSendMessageAndSucceed() } err = p.SyncBroadcastMessage(ctx, kafka.DefaultMockTopicName, kafka.DefaultMockPartitionNum, &common.Message{}) @@ -61,15 +61,16 @@ func TestDDLSyncSendMessage(t *testing.T) { options := getOptions() changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t syncProducer, err := factory.SyncProducer() require.NoError(t, err) p := NewKafkaDDLProducer(ctx, changefeed, syncProducer) - syncProducer.(*kafka.MockSyncProducer).Producer.ExpectSendMessageAndSucceed() + syncProducer.(*kafka.MockSaramaSyncProducer).Producer.ExpectSendMessageAndSucceed() err = p.SyncSendMessage(ctx, kafka.DefaultMockTopicName, 0, &common.Message{}) require.NoError(t, err) @@ -90,8 +91,9 @@ func TestDDLProducerSendMsgFailed(t *testing.T) { // This will make the first send failed. changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t syncProducer, err := factory.SyncProducer() require.NoError(t, err) @@ -99,9 +101,9 @@ func TestDDLProducerSendMsgFailed(t *testing.T) { p := NewKafkaDDLProducer(ctx, changefeed, syncProducer) defer p.Close() + syncProducer.(*kafka.MockSaramaSyncProducer).Producer.ExpectSendMessageAndFail(sarama.ErrMessageTooLarge) err = p.SyncSendMessage(ctx, kafka.DefaultMockTopicName, 0, &common.Message{}) - fmt.Println(err) - require.ErrorIs(t, err, confluentKafka.NewError(confluentKafka.ErrMsgSizeTooLarge, "", false)) + require.ErrorIs(t, err, sarama.ErrMessageTooLarge) } func TestDDLProducerDoubleClose(t *testing.T) { @@ -110,8 +112,9 @@ func TestDDLProducerDoubleClose(t *testing.T) { options := getOptions() changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t syncProducer, err := factory.SyncProducer() require.NoError(t, err) diff --git a/downstreamadapter/worker/producer/kafka_dml_producer_test.go b/downstreamadapter/worker/producer/kafka_dml_producer_test.go index 2cd2003bb..25927101e 100644 --- a/downstreamadapter/worker/producer/kafka_dml_producer_test.go +++ b/downstreamadapter/worker/producer/kafka_dml_producer_test.go @@ -19,7 +19,7 @@ import ( "testing" "time" - confluentKafka "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/IBM/sarama" commonType "github.com/pingcap/ticdc/pkg/common" "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" @@ -45,16 +45,16 @@ func TestProducerAck(t *testing.T) { errCh := make(chan error, 1) ctx, cancel := context.WithCancel(context.Background()) - config := kafka.NewConfig(options) - val, err := config.Get("queue.buffering.max.messages", -1) - require.NoError(t, err) - require.Equal(t, 1, val) + config, err := kafka.NewSaramaConfig(ctx, options) + require.Nil(t, err) + require.Equal(t, 1, config.Producer.Flush.MaxMessages) changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t - asyncProducer, err := factory.AsyncProducer() + asyncProducer, err := factory.AsyncProducer(ctx) require.NoError(t, err) producer := NewKafkaDMLProducer(changefeed, asyncProducer) @@ -64,7 +64,7 @@ func TestProducerAck(t *testing.T) { messageCount := 20 for i := 0; i < messageCount; i++ { - asyncProducer.(*kafka.MockAsyncProducer).Producer.ExpectInputAndSucceed() + asyncProducer.(*kafka.MockSaramaAsyncProducer).AsyncProducer.ExpectInputAndSucceed() } count := atomic.NewInt64(0) @@ -113,14 +113,17 @@ func TestProducerSendMsgFailed(t *testing.T) { errCh := make(chan error, 1) ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() + _, err := kafka.NewSaramaConfig(ctx, options) + require.NoError(t, err) options.MaxMessages = 1 options.MaxMessageBytes = 1 changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t - asyncProducer, err := factory.AsyncProducer() + asyncProducer, err := factory.AsyncProducer(ctx) require.NoError(t, err) producer := NewKafkaDMLProducer(changefeed, asyncProducer) @@ -143,6 +146,7 @@ func TestProducerSendMsgFailed(t *testing.T) { go func(t *testing.T) { defer wg.Done() + asyncProducer.(*kafka.MockSaramaAsyncProducer).AsyncProducer.ExpectInputAndFail(sarama.ErrMessageTooLarge) err = producer.AsyncSendMessage(ctx, kafka.DefaultMockTopicName, int32(0), &common.Message{ Key: []byte("test-key-1"), Value: []byte("test-value"), @@ -162,7 +166,7 @@ func TestProducerSendMsgFailed(t *testing.T) { case <-ctx.Done(): t.Errorf("TestProducerSendMessageFailed timed out") case err := <-errCh: - require.ErrorIs(t, err, confluentKafka.NewError(confluentKafka.ErrMsgSizeTooLarge, "", false)) + require.ErrorIs(t, err, sarama.ErrMessageTooLarge) } }() @@ -176,10 +180,11 @@ func TestProducerDoubleClose(t *testing.T) { defer cancel() changefeed := commonType.NewChangefeedID4Test("test", "test") - factory, err := kafka.NewMockFactory(options, changefeed) + factory, err := kafka.NewMockFactory(ctx, options, changefeed) require.NoError(t, err) + factory.(*kafka.MockFactory).ErrorReporter = t - asyncProducer, err := factory.AsyncProducer() + asyncProducer, err := factory.AsyncProducer(ctx) require.NoError(t, err) producer := NewKafkaDMLProducer(changefeed, asyncProducer) diff --git a/go.mod b/go.mod index 44e5a0536..b42038761 100644 --- a/go.mod +++ b/go.mod @@ -7,6 +7,7 @@ toolchain go1.23.2 require ( github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c github.com/DATA-DOG/go-sqlmock v1.5.0 + github.com/IBM/sarama v1.41.2 github.com/apache/pulsar-client-go v0.11.0 github.com/aws/aws-sdk-go-v2 v1.19.1 github.com/aws/aws-sdk-go-v2/config v1.18.30 @@ -30,6 +31,7 @@ require ( github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 github.com/hashicorp/golang-lru v0.5.1 github.com/imdario/mergo v0.3.16 + github.com/jcmturner/gokrb5/v8 v8.4.4 github.com/json-iterator/go v1.1.12 github.com/klauspost/compress v1.17.9 github.com/linkedin/goavro/v2 v2.11.1 @@ -43,9 +45,10 @@ require ( github.com/pingcap/tidb-dashboard v0.0.0-20240326110213-9768844ff5d7 github.com/pingcap/tidb/pkg/parser v0.0.0-20241014034929-94b2ac04a0c4 github.com/pingcap/tiflow v0.0.0-20241023094956-dd2d54ad4c19 - github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.20.4 github.com/r3labs/diff v1.1.0 + github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 + github.com/segmentio/kafka-go v0.4.41-0.20230526171612-f057b1d369cd github.com/spf13/cobra v1.8.1 github.com/spf13/pflag v1.0.5 github.com/stretchr/testify v1.9.0 @@ -63,6 +66,7 @@ require ( go.uber.org/goleak v1.3.0 go.uber.org/zap v1.27.0 golang.org/x/net v0.30.0 + golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 golang.org/x/term v0.25.0 golang.org/x/text v0.19.0 @@ -87,7 +91,6 @@ require ( github.com/Azure/go-ntlmssp v0.0.0-20221128193559-754e69321358 // indirect github.com/AzureAD/microsoft-authentication-library-for-go v1.2.2 // indirect github.com/DataDog/zstd v1.5.5 // indirect - github.com/IBM/sarama v1.41.2 // indirect github.com/KimMachineGun/automemlimit v0.2.4 // indirect github.com/KyleBanks/depth v1.2.1 // indirect github.com/Masterminds/semver v1.5.0 // indirect @@ -198,7 +201,6 @@ require ( github.com/jcmturner/aescts/v2 v2.0.0 // indirect github.com/jcmturner/dnsutils/v2 v2.0.0 // indirect github.com/jcmturner/gofork v1.7.6 // indirect - github.com/jcmturner/gokrb5/v8 v8.4.4 // indirect github.com/jcmturner/rpc/v2 v2.0.3 // indirect github.com/jedib0t/go-pretty/v6 v6.2.2 // indirect github.com/jellydator/ttlcache/v3 v3.0.1 // indirect @@ -254,6 +256,7 @@ require ( github.com/pingcap/sysutil v1.0.1-0.20240311050922-ae81ee01f3a5 // indirect github.com/pingcap/tipb v0.0.0-20241008083645-0bcddae67837 // indirect github.com/pkg/browser v0.0.0-20240102092130-5ac0b6a4141c // indirect + github.com/pkg/errors v0.9.1 // indirect github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect github.com/power-devops/perfstat v0.0.0-20221212215047-62379fc7944b // indirect github.com/prometheus/client_model v0.6.1 // indirect @@ -261,7 +264,6 @@ require ( github.com/prometheus/procfs v0.15.1 // indirect github.com/qri-io/jsonpointer v0.1.1 // indirect github.com/qri-io/jsonschema v0.2.1 // indirect - github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect github.com/remyoudompheng/bigfft v0.0.0-20230129092748-24d4a6f8daec // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/robfig/cron v1.2.0 // indirect @@ -270,7 +272,6 @@ require ( github.com/rs/cors v1.7.0 // indirect github.com/sasha-s/go-deadlock v0.3.5 // indirect github.com/segmentio/asm v1.2.0 // indirect - github.com/segmentio/kafka-go v0.4.41-0.20230526171612-f057b1d369cd // indirect github.com/shirou/gopsutil/v3 v3.24.5 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect github.com/shopspring/decimal v1.3.0 // indirect @@ -328,7 +329,6 @@ require ( golang.org/x/crypto v0.28.0 // indirect golang.org/x/exp v0.0.0-20240909161429-701f63a606c0 // indirect golang.org/x/mod v0.21.0 // indirect - golang.org/x/oauth2 v0.23.0 // indirect golang.org/x/sys v0.26.0 // indirect golang.org/x/tools v0.26.0 // indirect golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect diff --git a/pkg/leakutil/leak_helper.go b/pkg/leakutil/leak_helper.go index f6ca32a82..90e25bbf5 100644 --- a/pkg/leakutil/leak_helper.go +++ b/pkg/leakutil/leak_helper.go @@ -24,6 +24,12 @@ var defaultOpts = []goleak.Option{ goleak.IgnoreTopFunction("go.etcd.io/etcd/client/pkg/v3/logutil.(*MergeLogger).outputLoop"), goleak.IgnoreTopFunction("go.opencensus.io/stats/view.(*worker).start"), goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), + // library used by sarama, ref: https://github.com/rcrowley/go-metrics/pull/266 + goleak.IgnoreTopFunction("github.com/rcrowley/go-metrics.(*meterArbiter).tick"), + // Because we close the sarama producer asynchronously, so we have to ignore these funcs. + goleak.IgnoreTopFunction("github.com/Shopify/sarama.(*client).backgroundMetadataUpdater"), + goleak.IgnoreTopFunction("github.com/Shopify/sarama.(*Broker).responseReceiver"), + goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), } // VerifyNone verifies that no unexpected leaks occur diff --git a/pkg/logger/log.go b/pkg/logger/log.go index d5861bd0e..06eb9ee8b 100644 --- a/pkg/logger/log.go +++ b/pkg/logger/log.go @@ -21,6 +21,7 @@ import ( "strconv" "strings" + "github.com/IBM/sarama" "github.com/gin-gonic/gin" "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" @@ -88,9 +89,10 @@ func SetLogLevel(level string) error { // loggerOp is the op for logger control type loggerOp struct { - isInitGRPCLogger bool - isInitMySQLLogger bool - output zapcore.WriteSyncer + isInitGRPCLogger bool + isInitSaramaLogger bool + isInitMySQLLogger bool + output zapcore.WriteSyncer } func (op *loggerOp) applyOpts(opts []LoggerOpt) { @@ -109,6 +111,13 @@ func WithInitGRPCLogger() LoggerOpt { } } +// WithInitSaramaLogger enables sarama logger initialization when initializes global logger +func WithInitSaramaLogger() LoggerOpt { + return func(op *loggerOp) { + op.isInitSaramaLogger = true + } +} + // WithInitMySQLLogger enables mysql logger initialization when initializes global logger func WithInitMySQLLogger() LoggerOpt { return func(op *loggerOp) { @@ -129,6 +138,7 @@ func InitLogger(cfg *Config, opts ...LoggerOpt) error { var op loggerOp opts = []LoggerOpt{ WithInitGRPCLogger(), + WithInitSaramaLogger(), WithInitMySQLLogger(), } op.applyOpts(opts) @@ -166,7 +176,7 @@ func InitLogger(cfg *Config, opts ...LoggerOpt) error { // initOptionalComponent initializes some optional components func initOptionalComponent(op *loggerOp, cfg *Config) error { var level zapcore.Level - if op.isInitGRPCLogger { + if op.isInitGRPCLogger || op.isInitSaramaLogger { err := level.UnmarshalText([]byte(cfg.Level)) if err != nil { return errors.Trace(err) @@ -179,6 +189,12 @@ func initOptionalComponent(op *loggerOp, cfg *Config) error { } } + if op.isInitSaramaLogger { + if err := initSaramaLogger(level); err != nil { + return err + } + } + if op.isInitMySQLLogger { if err := initMySQLLogger(); err != nil { return err @@ -210,6 +226,16 @@ func initMySQLLogger() error { return mysql.SetLogger(logger) } +// initSaramaLogger hacks logger used in sarama lib +func initSaramaLogger(level zapcore.Level) error { + logger, err := zap.NewStdLogAt(log.L().With(zap.String("component", "sarama")), level) + if err != nil { + return errors.Trace(err) + } + sarama.Logger = logger + return nil +} + type loggerWriter struct { logFunc func(msg string, fields ...zap.Field) } diff --git a/pkg/sink/codec/common/message.go b/pkg/sink/codec/common/message.go index c38b36c6b..25c84489a 100644 --- a/pkg/sink/codec/common/message.go +++ b/pkg/sink/codec/common/message.go @@ -18,9 +18,11 @@ import ( "encoding/json" ) -// MaxRecordOverhead is used to calculate message size. +// MaxRecordOverhead is used to calculate message size by sarama kafka client. +// reference: https://github.com/IBM/sarama/blob/ +// 66521126c71c522c15a36663ae9cddc2b024c799/async_producer.go#L233 // For TiCDC, minimum supported kafka version is `0.11.0.2`, -// which will be treated as `version = 2` by producer. +// which will be treated as `version = 2` by sarama producer. const MaxRecordOverhead = 5*binary.MaxVarintLen32 + binary.MaxVarintLen64 + 1 // MessageType is the type of message, which is used by MqSink and RedoLog. diff --git a/pkg/sink/kafka/admin.go b/pkg/sink/kafka/admin.go index f6f548dd4..914a5c381 100644 --- a/pkg/sink/kafka/admin.go +++ b/pkg/sink/kafka/admin.go @@ -1,4 +1,4 @@ -// Copyright 2025 PingCAP, Inc. +// 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. @@ -16,208 +16,170 @@ package kafka import ( "context" "strconv" + "strings" - "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/IBM/sarama" + "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/ticdc/pkg/errors" "go.uber.org/zap" ) -const defaultTimeoutMs = 1000 +type saramaAdminClient struct { + changefeed common.ChangeFeedID -type admin struct { - ClusterAdminClient - client *kafka.AdminClient - changefeedID common.ChangeFeedID + client sarama.Client + admin sarama.ClusterAdmin } -func newClusterAdminClient( - client *kafka.AdminClient, - changefeedID common.ChangeFeedID, -) ClusterAdminClient { - return &admin{ - client: client, - changefeedID: changefeedID, - } -} - -func (a *admin) clusterMetadata(_ context.Context) (*kafka.Metadata, error) { - // request is not set, so it will return all metadata - return a.client.GetMetadata(nil, true, defaultTimeoutMs) -} - -func (a *admin) GetAllBrokers(ctx context.Context) ([]Broker, error) { - response, err := a.clusterMetadata(ctx) - if err != nil { - return nil, errors.Trace(err) - } - result := make([]Broker, 0, len(response.Brokers)) - for _, broker := range response.Brokers { +func (a *saramaAdminClient) GetAllBrokers(_ context.Context) ([]Broker, error) { + brokers := a.client.Brokers() + result := make([]Broker, 0, len(brokers)) + for _, broker := range brokers { result = append(result, Broker{ - ID: broker.ID, + ID: broker.ID(), }) } + return result, nil } -func (a *admin) GetBrokerConfig(ctx context.Context, configName string) (string, error) { - response, err := a.clusterMetadata(ctx) +func (a *saramaAdminClient) GetBrokerConfig( + _ context.Context, + configName string, +) (string, error) { + _, controller, err := a.admin.DescribeCluster() if err != nil { return "", errors.Trace(err) } - controllerID := response.Brokers[0].ID - resources := []kafka.ConfigResource{ - { - Type: kafka.ResourceBroker, - Name: strconv.Itoa(int(controllerID)), - Config: []kafka.ConfigEntry{{Name: configName}}, - }, - } - results, err := a.client.DescribeConfigs(ctx, resources) + + 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) } - if len(results) == 0 || len(results[0].Config) == 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 results[0].Config { + 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.Name()), zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( + return "", cerror.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) { - resources := []kafka.ConfigResource{ - { - Type: kafka.ResourceTopic, - Name: topicName, - Config: []kafka.ConfigEntry{{Name: configName}}, - }, - } - results, err := a.client.DescribeConfigs(ctx, resources) +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) } - if len(results) == 0 || len(results[0].Config) == 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 results[0].Config { + for _, entry := range configEntries { if entry.Name == configName { log.Info("Kafka config item found", - zap.String("namespace", a.changefeedID.Namespace()), - zap.String("changefeed", a.changefeedID.Name()), + zap.String("namespace", a.changefeed.Namespace()), + zap.String("changefeed", a.changefeed.Name()), 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.Name()), zap.String("configName", configName)) - return "", errors.ErrKafkaConfigNotFound.GenWithStack( + return "", cerror.ErrKafkaConfigNotFound.GenWithStack( "cannot find the `%s` from the topic's configuration", configName) } -func (a *admin) GetTopicsMeta( - ctx context.Context, - topics []string, - ignoreTopicError bool, +func (a *saramaAdminClient) GetTopicsMeta( + _ context.Context, topics []string, ignoreTopicError bool, ) (map[string]TopicDetail, error) { - resp, err := a.clusterMetadata(ctx) + result := make(map[string]TopicDetail, len(topics)) + + metaList, err := a.admin.DescribeTopics(topics) if err != nil { return nil, errors.Trace(err) } - result := make(map[string]TopicDetail, len(resp.Topics)) - for _, topic := range topics { - msg, ok := resp.Topics[topic] - if !ok { - log.Warn("fetch topic meta failed", - zap.String("topic", topic), zap.Error(msg.Error)) - continue - } - if msg.Error.Code() != kafka.ErrNoError { + + for _, meta := range metaList { + if meta.Err != sarama.ErrNoError { if !ignoreTopicError { - return nil, errors.Trace(msg.Error) + return nil, meta.Err } log.Warn("fetch topic meta failed", - zap.String("topic", topic), zap.Error(msg.Error)) + zap.String("namespace", a.changefeed.Namespace()), + zap.String("changefeed", a.changefeed.Name()), + zap.String("topic", meta.Name), + zap.Error(meta.Err)) continue } - result[topic] = TopicDetail{ - Name: topic, - NumPartitions: int32(len(msg.Partitions)), + result[meta.Name] = TopicDetail{ + Name: meta.Name, + NumPartitions: int32(len(meta.Partitions)), } } return result, nil } -func (a *admin) GetTopicsPartitionsNum( - ctx context.Context, topics []string, +func (a *saramaAdminClient) GetTopicsPartitionsNum( + _ context.Context, topics []string, ) (map[string]int32, error) { - resp, err := a.clusterMetadata(ctx) - if err != nil { - return nil, errors.Trace(err) - } result := make(map[string]int32, len(topics)) for _, topic := range topics { - msg, ok := resp.Topics[topic] - if !ok { - log.Warn("fetch topic meta failed", - zap.String("topic", topic), zap.Error(msg.Error)) - continue + partition, err := a.client.Partitions(topic) + if err != nil { + return nil, errors.Trace(err) } - result[topic] = int32(len(msg.Partitions)) + result[topic] = int32(len(partition)) } + return result, nil } -func (a *admin) CreateTopic( - ctx context.Context, - detail *TopicDetail, - validateOnly bool, +func (a *saramaAdminClient) CreateTopic( + _ context.Context, detail *TopicDetail, validateOnly bool, ) error { - topics := []kafka.TopicSpecification{ - { - Topic: detail.Name, - NumPartitions: int(detail.NumPartitions), - ReplicationFactor: int(detail.ReplicationFactor), - }, - } - results, err := a.client.CreateTopics(ctx, topics) - if err != nil { - return errors.Trace(err) + request := &sarama.TopicDetail{ + NumPartitions: detail.NumPartitions, + ReplicationFactor: detail.ReplicationFactor, } - for _, res := range results { - if res.Error.Code() == kafka.ErrTopicAlreadyExists { - return errors.Trace(err) - } + + 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 *admin) Close() { - log.Info("admin client start closing", - zap.String("namespace", a.changefeedID.Namespace()), - zap.String("changefeed", a.changefeedID.Name())) - a.client.Close() - log.Info("kafka admin client is fully closed", - zap.String("namespace", a.changefeedID.Namespace()), - zap.String("changefeed", a.changefeedID.Name())) +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.Name()), + zap.Error(err)) + } } diff --git a/pkg/sink/kafka/cluster_admin_client.go b/pkg/sink/kafka/cluster_admin_client.go new file mode 100644 index 000000000..2277540d1 --- /dev/null +++ b/pkg/sink/kafka/cluster_admin_client.go @@ -0,0 +1,57 @@ +// Copyright 2025 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" +) + +// TopicDetail represent a topic's detail information. +type TopicDetail struct { + Name string + NumPartitions int32 + ReplicationFactor int16 +} + +// Broker represents a Kafka broker. +type Broker struct { + ID int32 +} + +// ClusterAdminClient is the administrative client for Kafka, +// which supports managing and inspecting topics, brokers, configurations and ACLs. +type ClusterAdminClient interface { + // GetAllBrokers return all brokers among the cluster + GetAllBrokers(ctx context.Context) ([]Broker, error) + + // GetBrokerConfig return the broker level configuration with the `configName` + GetBrokerConfig(ctx context.Context, configName string) (string, error) + + // GetTopicConfig return the topic level configuration with the `configName` + GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) + + // GetTopicsMeta return all target topics' metadata + // if `ignoreTopicError` is true, ignore the topic error and return the metadata of valid topics + GetTopicsMeta(ctx context.Context, + topics []string, ignoreTopicError bool) (map[string]TopicDetail, error) + + // GetTopicsPartitionsNum return the number of partitions of each topic. + GetTopicsPartitionsNum(ctx context.Context, topics []string) (map[string]int32, error) + + // CreateTopic creates a new topic. + CreateTopic(ctx context.Context, detail *TopicDetail, validateOnly bool) error + + // Close shuts down the admin client. + Close() +} diff --git a/pkg/sink/kafka/config.go b/pkg/sink/kafka/config.go deleted file mode 100644 index 2a4a2d05d..000000000 --- a/pkg/sink/kafka/config.go +++ /dev/null @@ -1,142 +0,0 @@ -// Copyright 2025 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 ( - "crypto/x509" - "strings" - "time" - - "github.com/confluentinc/confluent-kafka-go/v2/kafka" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/pkg/security" - "github.com/pkg/errors" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" -) - -var ( - ProtocolDefault = "plaintext" - ProtocolSSL = "ssl" - ProtocolSASL = "sasl_plaintext" - ProtocolSASL_SSL = "sasl_ssl" -) - -func NewConfig(options *Options) *kafka.ConfigMap { - config := &kafka.ConfigMap{ - "bootstrap.servers": strings.Join(options.BrokerEndpoints, ","), - "allow.auto.create.topics": options.AutoCreate, - // retries may cause reordering unless enable.idempotence is set to true. - "message.send.max.retries": 0, - "max.in.flight": 1, - "request.required.acks": int(options.RequiredAcks), - "queue.buffering.max.ms": time.Duration(0), - "queue.buffering.max.messages": options.MaxMessages, - "message.max.bytes": options.MaxMessageBytes, - "socket.timeout.ms": int(options.DialTimeout.Milliseconds()), - "log_level": getLogLevel(), - } - if options.EnableTLS { - _ = config.SetKey("security.protocol", ProtocolSSL) - if options.Credential != nil && options.Credential.IsTLSEnabled() { - _ = config.SetKey("ssl.ca.location", options.Credential.CAPath) - _ = config.SetKey("ssl.certificate.location", options.Credential.CertPath) - _ = config.SetKey("ssl.key.location", options.Credential.KeyPath) - addVerifyPeerCertificate(config, options.Credential.CertAllowedCN) - - } - _ = config.SetKey("enable.ssl.certificate.verification", !options.InsecureSkipVerify) - } - - completeSASLConfig(config, options) - compression := strings.ToLower(strings.TrimSpace(options.Compression)) - config.SetKey("compression.codec", compression) - log.Info("kafka producer config", zap.Any("config", config)) - return config -} - -func completeSASLConfig(config *kafka.ConfigMap, o *Options) error { - if o.SASL != nil && o.SASL.SASLMechanism != "" { - if o.EnableTLS { - _ = config.SetKey("security.protocol", ProtocolSASL_SSL) - } else { - _ = config.SetKey("security.protocol", ProtocolSASL) - } - _ = config.SetKey("sasl.mechanisms", o.SASL.SASLMechanism) - switch o.SASL.SASLMechanism { - case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypePlaintext: - _ = config.SetKey("sasl.username", o.SASL.SASLUser) - _ = config.SetKey("sasl.password", o.SASL.SASLPassword) - case SASLTypeGSSAPI: - _ = config.SetKey("sasl.username", o.SASL.SASLUser) - _ = config.SetKey("sasl.kerberos.service.name", o.SASL.GSSAPI.ServiceName) - switch o.SASL.GSSAPI.AuthType { - case security.UserAuth: - _ = config.SetKey("sasl.password", o.SASL.SASLPassword) - case security.KeyTabAuth: - _ = config.SetKey("sasl.kerberos.keytab", o.SASL.GSSAPI.KeyTabPath) - } - case SASLTypeOAuth: - _ = config.SetKey("sasl.oauthbearer.token.endpoint.url", o.SASL.OAuth2.TokenURL) - } - } - - return nil -} - -func addVerifyPeerCertificate(config *kafka.ConfigMap, verifyCN []string) { - if len(verifyCN) != 0 { - checkCN := make(map[string]struct{}) - for _, cn := range verifyCN { - cn = strings.TrimSpace(cn) - checkCN[cn] = struct{}{} - } - verifyPeerCertificate := func( - rawCerts [][]byte, verifiedChains [][]*x509.Certificate, - ) error { - cns := make([]string, 0, len(verifiedChains)) - for _, chains := range verifiedChains { - for _, chain := range chains { - cns = append(cns, chain.Subject.CommonName) - if _, match := checkCN[chain.Subject.CommonName]; match { - return nil - } - } - } - return errors.Errorf("client certificate authentication failed. "+ - "The Common Name from the client certificate %v was not found "+ - "in the configuration cluster-verify-cn with value: %s", cns, verifyCN) - } - _ = config.SetKey("ssl.certificate.verify_cb", verifyPeerCertificate) - } -} - -func getLogLevel() int { - switch log.GetLevel() { - case zapcore.DebugLevel: - return 7 - case zapcore.InfoLevel: - return 6 - case zapcore.WarnLevel: - return 4 - case zapcore.ErrorLevel: - return 3 - case zapcore.DPanicLevel: - return 2 - case zapcore.FatalLevel: - return 0 - default: - return 6 - } -} diff --git a/pkg/sink/kafka/factory.go b/pkg/sink/kafka/factory.go index 4a5b641b6..68891f45c 100644 --- a/pkg/sink/kafka/factory.go +++ b/pkg/sink/kafka/factory.go @@ -17,183 +17,257 @@ import ( "context" "time" - "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/IBM/sarama" + "github.com/pingcap/errors" "github.com/pingcap/log" commonType "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" "go.uber.org/zap" ) -type factory struct { - config *kafka.ConfigMap - changefeedID commonType.ChangeFeedID +// Factory is used to produce all kafka components. +type Factory interface { + // AdminClient return a kafka cluster admin client + AdminClient() (ClusterAdminClient, error) + // SyncProducer creates a sync producer to writer message to kafka + SyncProducer() (SyncProducer, error) + // AsyncProducer creates an async producer to writer message to kafka + AsyncProducer(ctx context.Context) (AsyncProducer, error) + // MetricsCollector returns the kafka metrics collector + MetricsCollector(adminClient ClusterAdminClient) MetricsCollector } -// NewFactory returns a factory implemented based on kafka-go -func NewFactory( - options *Options, - changefeedID commonType.ChangeFeedID, -) (Factory, error) { - config := NewConfig(options) - return &factory{ - config: config, - changefeedID: changefeedID, - }, nil -} +// FactoryCreator defines the type of factory creator. +type FactoryCreator func(context.Context, *Options, commonType.ChangeFeedID) (Factory, error) -func (f *factory) AdminClient() (ClusterAdminClient, error) { - client, err := kafka.NewAdminClient(f.config) - if err != nil { - return nil, err - } - return newClusterAdminClient(client, f.changefeedID), nil -} +// 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 -// SyncProducer creates a sync producer to Producer message to kafka -func (f *factory) SyncProducer() (SyncProducer, error) { - p, err := kafka.NewProducer(f.config) - if err != nil { - return nil, err - } - return &syncProducer{changefeedID: f.changefeedID, p: p, deliveryChan: make(chan kafka.Event)}, err -} + // 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 -// AsyncProducer creates an async producer to Producer message to kafka -func (f *factory) AsyncProducer() (AsyncProducer, error) { - p, err := kafka.NewProducer(f.config) - if err != nil { - return nil, err - } - return &asyncProducer{changefeedID: f.changefeedID, p: p}, err + // 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() } -// MetricsCollector returns the kafka metrics collector -func (f *factory) MetricsCollector() MetricsCollector { - return NewMetricsCollector(f.changefeedID, f.config) +// 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 syncProducer struct { - changefeedID commonType.ChangeFeedID - p *kafka.Producer - deliveryChan chan kafka.Event +type saramaSyncProducer struct { + id commonType.ChangeFeedID + client sarama.Client + producer sarama.SyncProducer } -func (s *syncProducer) SendMessage( - ctx context.Context, - topic string, partition int32, +func (p *saramaSyncProducer) SendMessage( + _ context.Context, + topic string, partitionNum int32, message *common.Message, ) error { - msg := &kafka.Message{ - TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: partition}, - Key: message.Key, - Value: message.Value, - } - err := s.p.Produce(msg, s.deliveryChan) - if err != nil { - return err - } - select { - case <-ctx.Done(): - return ctx.Err() - case event := <-s.deliveryChan: - switch e := event.(type) { - case *kafka.Error: - return e - } - } - return nil + _, _, err := p.producer.SendMessage(&sarama.ProducerMessage{ + Topic: topic, + Key: sarama.ByteEncoder(message.Key), + Value: sarama.ByteEncoder(message.Value), + Partition: partitionNum, + }) + return err } -// 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 *syncProducer) SendMessages(ctx context.Context, topic string, partitionNum int32, message *common.Message) error { - var err error +func (p *saramaSyncProducer) SendMessages( + _ context.Context, topic string, partitionNum int32, message *common.Message, +) error { + msgs := make([]*sarama.ProducerMessage, partitionNum) for i := 0; i < int(partitionNum); i++ { - e := s.SendMessage(ctx, topic, int32(i), message) - if e != nil { - err = e + msgs[i] = &sarama.ProducerMessage{ + Topic: topic, + Key: sarama.ByteEncoder(message.Key), + Value: sarama.ByteEncoder(message.Value), + Partition: int32(i), } } - return err + return p.producer.SendMessages(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 *syncProducer) Close() { - log.Info("kafka sync producer start closing", - zap.String("namespace", s.changefeedID.Namespace()), - zap.String("changefeed", s.changefeedID.Name())) - s.p.Close() - close(s.deliveryChan) +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.Name()), + 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.Name()), + 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.Name()), + 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.Name()), + zap.Duration("duration", time.Since(start))) + } + }() } -type asyncProducer struct { - p *kafka.Producer +type saramaAsyncProducer struct { + client sarama.Client + producer sarama.AsyncProducer changefeedID commonType.ChangeFeedID + failpointCh 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 *asyncProducer) Close() { - log.Info("kafka async producer start closing", - zap.String("namespace", a.changefeedID.Namespace()), - zap.String("changefeed", a.changefeedID.Name())) +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() - a.p.Close() - log.Info("Close kafka async producer success", - zap.String("namespace", a.changefeedID.Namespace()), - zap.String("changefeed", a.changefeedID.Name()), - zap.Duration("duration", time.Since(start))) + 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.Name()), + 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.Name()), + 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.Name()), + 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.Name()), + zap.Duration("duration", time.Since(start))) + } }() } -// 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 *asyncProducer) AsyncRunCallback(ctx context.Context) error { +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.Name())) return errors.Trace(ctx.Err()) - case event := <-a.p.Events(): - switch e := event.(type) { - case *kafka.Message: - if e != nil { - callback := e.Opaque.(func()) - if callback != nil { - callback() - } - } - case *kafka.Error: - if e == nil { - return nil + 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.Name()), + zap.Error(err)) + return errors.Trace(err) + case ack := <-p.producer.Successes(): + if ack != nil { + callback := ack.Metadata.(func()) + if callback != nil { + callback() } - return errors.WrapError(errors.ErrKafkaAsyncSendMessage, e) } + 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 (a *asyncProducer) AsyncSend(ctx context.Context, topic string, partition int32, message *common.Message) error { +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()) - default: + case p.producer.Input() <- msg: } - return a.p.Produce(&kafka.Message{ - TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: partition}, - Key: message.Key, - Value: message.Value, - Opaque: message.Callback, - }, nil) + return nil } diff --git a/pkg/sink/kafka/kafka.go b/pkg/sink/kafka/kafka.go deleted file mode 100644 index f94c4ee1b..000000000 --- a/pkg/sink/kafka/kafka.go +++ /dev/null @@ -1,123 +0,0 @@ -// Copyright 2025 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" - - commonType "github.com/pingcap/ticdc/pkg/common" - "github.com/pingcap/ticdc/pkg/sink/codec/common" -) - -// TopicDetail represent a topic's detail information. -type TopicDetail struct { - Name string - NumPartitions int32 - ReplicationFactor int16 -} - -// Broker represents a Kafka broker. -type Broker struct { - ID int32 -} - -// Factory is used to produce all kafka components. -type Factory interface { - // AdminClient return a kafka cluster admin client - AdminClient() (ClusterAdminClient, error) - // SyncProducer creates a sync producer to writer message to kafka - SyncProducer() (SyncProducer, error) - // AsyncProducer creates an async producer to writer message to kafka - AsyncProducer() (AsyncProducer, error) - // MetricsCollector returns the kafka metrics collector - MetricsCollector() MetricsCollector -} - -// FactoryCreator defines the type of factory creator. -type FactoryCreator func(*Options, commonType.ChangeFeedID) (Factory, error) - -// ClusterAdminClient is the administrative client for Kafka, -// which supports managing and inspecting topics, brokers, configurations and ACLs. -type ClusterAdminClient interface { - // GetAllBrokers return all brokers among the cluster - GetAllBrokers(ctx context.Context) ([]Broker, error) - - // GetBrokerConfig return the broker level configuration with the `configName` - GetBrokerConfig(ctx context.Context, configName string) (string, error) - - // GetTopicConfig return the topic level configuration with the `configName` - GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) - - // GetTopicsMeta return all target topics' metadata - // if `ignoreTopicError` is true, ignore the topic error and return the metadata of valid topics - GetTopicsMeta(ctx context.Context, - topics []string, ignoreTopicError bool) (map[string]TopicDetail, error) - - // GetTopicsPartitionsNum return the number of partitions of each topic. - GetTopicsPartitionsNum(ctx context.Context, topics []string) (map[string]int32, error) - - // CreateTopic creates a new topic. - CreateTopic(ctx context.Context, detail *TopicDetail, validateOnly bool) error - - // Close shuts down the admin client. - Close() - - // SetRemainingFetchesUntilTopicVisible for mock - SetRemainingFetchesUntilTopicVisible(topic string, num int) 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 -} - -// MetricsCollector is the interface for kafka metrics collector. -type MetricsCollector interface { - Run(ctx context.Context) -} diff --git a/pkg/sink/kafka/metrics.go b/pkg/sink/kafka/metrics.go index 835c14372..68ffa2ec4 100644 --- a/pkg/sink/kafka/metrics.go +++ b/pkg/sink/kafka/metrics.go @@ -74,6 +74,51 @@ var ( Name: "kafka_producer_response_rate", Help: "Responses/second received from all brokers.", }, []string{"namespace", "changefeed", "broker"}) + + // ClientRetryGauge only for kafka-go client to track internal retry count. + ClientRetryGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_retry_count", + Help: "Kafka client send request retry count", + }, []string{"namespace", "changefeed"}) + + // ClientErrorGauge only for kafka-go client to track internal error count. + ClientErrorGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_err_count", + Help: "Kafka client send request retry count", + }, []string{"namespace", "changefeed"}) + + // BatchDurationGauge only for kafka-go client to track internal batch duration. + BatchDurationGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_batch_duration", + Help: "Kafka client internal average batch message time cost in milliseconds", + }, []string{"namespace", "changefeed"}) + + // BatchMessageCountGauge only for kafka-go client to track each batch's messages count. + BatchMessageCountGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_batch_message_count", + Help: "Kafka client internal average batch message count", + }, []string{"namespace", "changefeed"}) + + // BatchSizeGauge only for kafka-go client to track each batch's size in bytes. + BatchSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_batch_size", + Help: "Kafka client internal average batch size in bytes", + }, []string{"namespace", "changefeed"}) ) // InitMetrics registers all metrics in this file. @@ -85,4 +130,11 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(RequestLatencyGauge) registry.MustRegister(requestsInFlightGauge) registry.MustRegister(responseRateGauge) + + // only used by kafka sink v2. + registry.MustRegister(BatchDurationGauge) + registry.MustRegister(BatchMessageCountGauge) + registry.MustRegister(BatchSizeGauge) + registry.MustRegister(ClientRetryGauge) + registry.MustRegister(ClientErrorGauge) } diff --git a/pkg/sink/kafka/metrics_collector.go b/pkg/sink/kafka/metrics_collector.go index 292311c87..117a94fc9 100644 --- a/pkg/sink/kafka/metrics_collector.go +++ b/pkg/sink/kafka/metrics_collector.go @@ -1,4 +1,4 @@ -// Copyright 2025 PingCAP, Inc. +// 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. @@ -15,127 +15,211 @@ package kafka import ( "context" - "encoding/json" + "strconv" "time" - "github.com/confluentinc/confluent-kafka-go/v2/kafka" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/common" + "github.com/rcrowley/go-metrics" "go.uber.org/zap" ) +// MetricsCollector is the interface for kafka metrics collector. +type MetricsCollector interface { + Run(ctx context.Context) +} + const ( // RefreshMetricsInterval specifies the interval of refresh kafka client metrics. RefreshMetricsInterval = 5 * time.Second + // refreshClusterMetaInterval specifies the interval of refresh kafka cluster meta. + // Do not set it too small, because it will cause too many requests to kafka cluster. + // Every request will get all topics and all brokers information. + refreshClusterMetaInterval = 30 * time.Minute ) -// The stats are provided as a JSON object string, see https://github.com/confluentinc/librdkafka/blob/master/STATISTICS.md -type kafkaMetrics struct { - // Instance type (producer or consumer) - Role string `json:"type"` - // Wall clock time in seconds since the epoch - Time int `json:"time"` - // Number of ops (callbacks, events, etc) waiting in queue for application to serve with rd_kafka_poll() - Ops int `json:"replyq"` - // Current number of messages in producer queues - MsgCount int `json:"msg_cnt"` - // Current total size of messages in producer queues - MsgSize int `json:"msg_size"` - - // Total number of requests sent to Kafka brokers - Tx int `json:"tx"` - // Total number of bytes transmitted to Kafka brokers - TxBytes int `json:"tx_bytes"` - // Total number of responses received from Kafka brokers - Rx int `json:"rx"` - // Total number of bytes received from Kafka brokers - RxBytes int `json:"rx_bytes"` - // Number of topics in the metadata cache. - MetadataCacheCnt int `json:"metadata_cache_cnt"` - - Brokers map[string]broker `json:"brokers"` -} - -type broker struct { - Name string `json:"name"` - Nodeid int `json:"nodeid"` // -1 for bootstraps - Nodename string `json:"nodename"` - State string `json:"state"` - Rtt window `json:"rtt"` -} - -type window struct { - Min int `json:"min"` - Max int `json:"max"` - Avg int `json:"avg"` - P99 int `json:"p99"` -} +// Sarama metrics names, see https://pkg.go.dev/github.com/IBM/sarama#pkg-overview. +const ( + // Producer level. + compressionRatioMetricName = "compression-ratio" + recordsPerRequestMetricName = "records-per-request" + + // Broker level. + outgoingByteRateMetricNamePrefix = "outgoing-byte-rate-for-broker-" + requestRateMetricNamePrefix = "request-rate-for-broker-" + requestLatencyInMsMetricNamePrefix = "request-latency-in-ms-for-broker-" + requestsInFlightMetricNamePrefix = "requests-in-flight-for-broker-" + responseRateMetricNamePrefix = "response-rate-for-broker-" + + p99 = "p99" + avg = "avg" +) -type metricsCollector struct { +type saramaMetricsCollector struct { changefeedID common.ChangeFeedID - config *kafka.ConfigMap + // adminClient is used to get broker infos from broker. + adminClient ClusterAdminClient + brokers map[int32]struct{} + registry metrics.Registry } -// NewMetricsCollector return a kafka metrics collector based on library. -func NewMetricsCollector( - changefeedID common.ChangeFeedID, - config *kafka.ConfigMap, -) MetricsCollector { - return &metricsCollector{changefeedID: changefeedID, config: config} -} +func (m *saramaMetricsCollector) Run(ctx context.Context) { + // Initialize brokers. + m.updateBrokers(ctx) + + refreshMetricsTicker := time.NewTicker(RefreshMetricsInterval) + refreshClusterMetaTicker := time.NewTicker(refreshClusterMetaInterval) + defer func() { + refreshMetricsTicker.Stop() + refreshClusterMetaTicker.Stop() + m.cleanupMetrics() + }() -func (m *metricsCollector) Run(ctx context.Context) { - _ = m.config.SetKey("statistics.interval.ms", int(RefreshMetricsInterval.Milliseconds())) - p, err := kafka.NewProducer(m.config) - if err != nil { - log.Error("create producer failed", zap.Error(err)) - return - } for { select { case <-ctx.Done(): - log.Warn("Kafka metrics collector stopped", - zap.String("namespace", m.changefeedID.String())) - p.Close() - m.cleanupMetrics() + log.Info("Kafka metrics collector stopped", + zap.String("namespace", m.changefeedID.Namespace()), + zap.String("changefeed", m.changefeedID.Name())) return - case event := <-p.Events(): - switch e := event.(type) { - case *kafka.Stats: - m.collect(e.String()) - } + case <-refreshMetricsTicker.C: + m.collectBrokerMetrics() + m.collectProducerMetrics() + case <-refreshClusterMetaTicker.C: + m.updateBrokers(ctx) } } } -func (m *metricsCollector) collect(data string) { - var statistics kafkaMetrics - if err := json.Unmarshal([]byte(data), &statistics); err != nil { - log.Error("kafka metrics collect failed", zap.Error(err)) +func (m *saramaMetricsCollector) updateBrokers(ctx context.Context) { + start := time.Now() + brokers, err := m.adminClient.GetAllBrokers(ctx) + if err != nil { + log.Warn("Get Kafka brokers failed, "+ + "use historical brokers to collect kafka broker level metrics", + zap.String("namespace", m.changefeedID.Namespace()), + zap.String("changefeed", m.changefeedID.Name()), + zap.Duration("duration", time.Since(start)), + zap.Error(err)) return } - recordsPerRequestGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg"). - Set(float64(statistics.Tx) / RefreshMetricsInterval.Seconds()) - requestsInFlightGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg"). - Set(float64(statistics.MsgCount) / RefreshMetricsInterval.Seconds()) - responseRateGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg"). - Set(float64(statistics.Rx) / RefreshMetricsInterval.Seconds()) - - for _, broker := range statistics.Brokers { - // latency is in milliseconds - RequestLatencyGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), broker.Name, "avg"). - Set(float64(broker.Rtt.Avg) * 1000 / RefreshMetricsInterval.Seconds()) + + for _, b := range brokers { + m.brokers[b.ID] = struct{}{} } } -func (m *metricsCollector) cleanupMetrics() { +func (m *saramaMetricsCollector) collectProducerMetrics() { + namespace := m.changefeedID.Namespace() + changefeedID := m.changefeedID.Name() + compressionRatioMetric := m.registry.Get(compressionRatioMetricName) + if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { + compressionRatioGauge. + WithLabelValues(namespace, changefeedID, avg). + Set(histogram.Snapshot().Mean()) + compressionRatioGauge.WithLabelValues(namespace, changefeedID, p99). + Set(histogram.Snapshot().Percentile(0.99)) + } + + recordsPerRequestMetric := m.registry.Get(recordsPerRequestMetricName) + if histogram, ok := recordsPerRequestMetric.(metrics.Histogram); ok { + recordsPerRequestGauge. + WithLabelValues(namespace, changefeedID, avg). + Set(histogram.Snapshot().Mean()) + recordsPerRequestGauge. + WithLabelValues(namespace, changefeedID, p99). + Set(histogram.Snapshot().Percentile(0.99)) + } +} + +func (m *saramaMetricsCollector) collectBrokerMetrics() { + namespace := m.changefeedID.Namespace() + changefeedID := m.changefeedID.Name() + for id := range m.brokers { + brokerID := strconv.Itoa(int(id)) + outgoingByteRateMetric := m.registry.Get( + getBrokerMetricName(outgoingByteRateMetricNamePrefix, brokerID)) + if meter, ok := outgoingByteRateMetric.(metrics.Meter); ok { + OutgoingByteRateGauge. + WithLabelValues(namespace, changefeedID, brokerID). + Set(meter.Snapshot().Rate1()) + } + + requestRateMetric := m.registry.Get( + getBrokerMetricName(requestRateMetricNamePrefix, brokerID)) + if meter, ok := requestRateMetric.(metrics.Meter); ok { + RequestRateGauge. + WithLabelValues(namespace, changefeedID, brokerID). + Set(meter.Snapshot().Rate1()) + } + + requestLatencyMetric := m.registry.Get( + getBrokerMetricName(requestLatencyInMsMetricNamePrefix, brokerID)) + if histogram, ok := requestLatencyMetric.(metrics.Histogram); ok { + RequestLatencyGauge. + WithLabelValues(namespace, changefeedID, brokerID, avg). + Set(histogram.Snapshot().Mean() / 1000) + RequestLatencyGauge. + WithLabelValues(namespace, changefeedID, brokerID, p99). + Set(histogram.Snapshot().Percentile(0.99) / 1000) + } + + requestsInFlightMetric := m.registry.Get(getBrokerMetricName( + requestsInFlightMetricNamePrefix, brokerID)) + if counter, ok := requestsInFlightMetric.(metrics.Counter); ok { + requestsInFlightGauge. + WithLabelValues(namespace, changefeedID, brokerID). + Set(float64(counter.Snapshot().Count())) + } + + responseRateMetric := m.registry.Get(getBrokerMetricName( + responseRateMetricNamePrefix, brokerID)) + if meter, ok := responseRateMetric.(metrics.Meter); ok { + responseRateGauge. + WithLabelValues(namespace, changefeedID, brokerID). + Set(meter.Snapshot().Rate1()) + } + } +} + +func getBrokerMetricName(prefix, brokerID string) string { + return prefix + brokerID +} + +func (m *saramaMetricsCollector) cleanupProducerMetrics() { + compressionRatioGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), avg) + compressionRatioGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), p99) + recordsPerRequestGauge. - DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg") - requestsInFlightGauge. - DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg") - responseRateGauge. - DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg") - - RequestLatencyGauge. - DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "avg") + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), avg) + recordsPerRequestGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), p99) +} + +func (m *saramaMetricsCollector) cleanupBrokerMetrics() { + namespace := m.changefeedID.Namespace() + changefeedID := m.changefeedID.Name() + for id := range m.brokers { + brokerID := strconv.Itoa(int(id)) + OutgoingByteRateGauge. + DeleteLabelValues(namespace, changefeedID, brokerID) + RequestRateGauge. + DeleteLabelValues(namespace, changefeedID, brokerID) + RequestLatencyGauge. + DeleteLabelValues(namespace, changefeedID, brokerID, avg) + RequestLatencyGauge. + DeleteLabelValues(namespace, changefeedID, brokerID, p99) + requestsInFlightGauge. + DeleteLabelValues(namespace, changefeedID, brokerID) + responseRateGauge. + DeleteLabelValues(namespace, changefeedID, brokerID) + + } +} + +func (m *saramaMetricsCollector) cleanupMetrics() { + m.cleanupProducerMetrics() + m.cleanupBrokerMetrics() } diff --git a/pkg/sink/kafka/mock_cluster_admin_client.go b/pkg/sink/kafka/mock_cluster_admin_client.go new file mode 100644 index 000000000..ef1dc104f --- /dev/null +++ b/pkg/sink/kafka/mock_cluster_admin_client.go @@ -0,0 +1,233 @@ +// Copyright 2024 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" + "fmt" + "strconv" + + "github.com/IBM/sarama" + "github.com/pingcap/ticdc/pkg/errors" +) + +const ( + // DefaultMockTopicName specifies the default mock topic name. + DefaultMockTopicName = "mock_topic" + // DefaultMockPartitionNum is the default partition number of default mock topic. + DefaultMockPartitionNum = 3 + // defaultMockControllerID specifies the default mock controller ID. + defaultMockControllerID = 1 + // topic replication factor must be 3 for Confluent Cloud Kafka. + defaultReplicationFactor = 3 +) + +const ( + // defaultMaxMessageBytes specifies the default max message bytes, + // default to 1048576, identical to kafka broker's `message.max.bytes` and topic's `max.message.bytes` + // see: https://kafka.apache.org/documentation/#brokerconfigs_message.max.bytes + // see: https://kafka.apache.org/documentation/#topicconfigs_max.message.bytes + defaultMaxMessageBytes = "1048588" + + // defaultMinInsyncReplicas specifies the default `min.insync.replicas` for broker and topic. + defaultMinInsyncReplicas = "1" +) + +var ( + // BrokerMessageMaxBytes is the broker's `message.max.bytes` + BrokerMessageMaxBytes = defaultMaxMessageBytes + // TopicMaxMessageBytes is the topic's `max.message.bytes` + TopicMaxMessageBytes = defaultMaxMessageBytes + // MinInSyncReplicas is the `min.insync.replicas` + MinInSyncReplicas = defaultMinInsyncReplicas +) + +type topicDetail struct { + TopicDetail + fetchesRemainingUntilVisible int +} + +// ClusterAdminClientMockImpl mock implements the admin client interface. +type ClusterAdminClientMockImpl struct { + topics map[string]*topicDetail + // Cluster controller ID. + controllerID int + brokerConfigs map[string]string + topicConfigs map[string]map[string]string +} + +// NewClusterAdminClientMockImpl news a ClusterAdminClientMockImpl struct with default configurations. +func NewClusterAdminClientMockImpl() *ClusterAdminClientMockImpl { + topics := make(map[string]*topicDetail) + topics[DefaultMockTopicName] = &topicDetail{ + fetchesRemainingUntilVisible: 0, + TopicDetail: TopicDetail{ + Name: DefaultMockTopicName, + NumPartitions: 3, + }, + } + + brokerConfigs := make(map[string]string) + brokerConfigs[BrokerMessageMaxBytesConfigName] = BrokerMessageMaxBytes + brokerConfigs[MinInsyncReplicasConfigName] = MinInSyncReplicas + + topicConfigs := make(map[string]map[string]string) + topicConfigs[DefaultMockTopicName] = make(map[string]string) + topicConfigs[DefaultMockTopicName][TopicMaxMessageBytesConfigName] = TopicMaxMessageBytes + topicConfigs[DefaultMockTopicName][MinInsyncReplicasConfigName] = MinInSyncReplicas + + return &ClusterAdminClientMockImpl{ + topics: topics, + controllerID: defaultMockControllerID, + brokerConfigs: brokerConfigs, + topicConfigs: topicConfigs, + } +} + +// GetAllBrokers implement the ClusterAdminClient interface +func (c *ClusterAdminClientMockImpl) GetAllBrokers(context.Context) ([]Broker, error) { + return nil, nil +} + +// GetBrokerConfig implement the ClusterAdminClient interface +func (c *ClusterAdminClientMockImpl) GetBrokerConfig( + _ context.Context, + configName string, +) (string, error) { + value, ok := c.brokerConfigs[configName] + if !ok { + return "", errors.ErrKafkaConfigNotFound.GenWithStack( + "cannot find the `%s` from the broker's configuration", configName) + } + return value, nil +} + +// GetTopicConfig implement the ClusterAdminClient interface +func (c *ClusterAdminClientMockImpl) GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) { + if _, ok := c.topics[topicName]; !ok { + return "", errors.ErrKafkaConfigNotFound.GenWithStack("cannot find the `%s` from the topic's configuration", topicName) + } + value, ok := c.topicConfigs[topicName][configName] + if !ok { + return "", errors.ErrKafkaConfigNotFound.GenWithStack( + "cannot find the `%s` from the topic's configuration", configName) + } + return value, nil +} + +// SetRemainingFetchesUntilTopicVisible is used to control the visibility of a specific topic. +// It is used to mock the topic creation delay. +func (c *ClusterAdminClientMockImpl) SetRemainingFetchesUntilTopicVisible( + topicName string, + fetchesRemainingUntilVisible int, +) error { + topic, ok := c.topics[topicName] + if !ok { + return fmt.Errorf("No such topic as %s", topicName) + } + + topic.fetchesRemainingUntilVisible = fetchesRemainingUntilVisible + return nil +} + +// GetTopicsMeta implement the ClusterAdminClient interface +func (c *ClusterAdminClientMockImpl) GetTopicsMeta( + _ context.Context, + topics []string, + _ bool, +) (map[string]TopicDetail, error) { + result := make(map[string]TopicDetail, len(topics)) + for _, topic := range topics { + details, ok := c.topics[topic] + if ok { + if details.fetchesRemainingUntilVisible > 0 { + details.fetchesRemainingUntilVisible-- + continue + } + result[topic] = details.TopicDetail + } + } + return result, nil +} + +// GetTopicsPartitionsNum implement the ClusterAdminClient interface +func (c *ClusterAdminClientMockImpl) GetTopicsPartitionsNum( + _ context.Context, topics []string, +) (map[string]int32, error) { + result := make(map[string]int32, len(topics)) + for _, topic := range topics { + result[topic] = c.topics[topic].NumPartitions + } + return result, nil +} + +// CreateTopic adds topic into map. +func (c *ClusterAdminClientMockImpl) CreateTopic( + _ context.Context, + detail *TopicDetail, + _ bool, +) error { + if detail.ReplicationFactor > defaultReplicationFactor { + return sarama.ErrInvalidReplicationFactor + } + + _, minInsyncReplicaConfigFound := c.brokerConfigs[MinInsyncReplicasConfigName] + // For Confluent Cloud, min.insync.replica is invisible and replication factor must be 3. + // Otherwise, ErrPolicyViolation is expected to be returned. + if !minInsyncReplicaConfigFound && + detail.ReplicationFactor != defaultReplicationFactor { + return sarama.ErrPolicyViolation + } + + c.topics[detail.Name] = &topicDetail{ + TopicDetail: *detail, + } + return nil +} + +// DeleteTopic deletes a topic, only used for testing. +func (c *ClusterAdminClientMockImpl) DeleteTopic(topicName string) { + delete(c.topics, topicName) +} + +// Close do nothing. +func (c *ClusterAdminClientMockImpl) Close() {} + +// SetMinInsyncReplicas sets the MinInsyncReplicas for broker and default topic. +func (c *ClusterAdminClientMockImpl) SetMinInsyncReplicas(minInsyncReplicas string) { + c.topicConfigs[DefaultMockTopicName][MinInsyncReplicasConfigName] = minInsyncReplicas + c.brokerConfigs[MinInsyncReplicasConfigName] = minInsyncReplicas +} + +// GetDefaultMockTopicName returns the default topic name +func (c *ClusterAdminClientMockImpl) GetDefaultMockTopicName() string { + return DefaultMockTopicName +} + +// GetBrokerMessageMaxBytes returns broker's `message.max.bytes` +func (c *ClusterAdminClientMockImpl) GetBrokerMessageMaxBytes() int { + messageMaxBytes, _ := strconv.Atoi(BrokerMessageMaxBytes) + return messageMaxBytes +} + +// GetTopicMaxMessageBytes returns topic's `max.message.bytes` +func (c *ClusterAdminClientMockImpl) GetTopicMaxMessageBytes() int { + maxMessageBytes, _ := strconv.Atoi(TopicMaxMessageBytes) + return maxMessageBytes +} + +// DropBrokerConfig remove all broker level configuration for test purpose. +func (c *ClusterAdminClientMockImpl) DropBrokerConfig(configName string) { + delete(c.brokerConfigs, configName) +} diff --git a/pkg/sink/kafka/mock_factory.go b/pkg/sink/kafka/mock_factory.go index 186f09034..b001956cb 100644 --- a/pkg/sink/kafka/mock_factory.go +++ b/pkg/sink/kafka/mock_factory.go @@ -1,4 +1,4 @@ -// Copyright 2025 PingCAP, Inc. +// Copyright 2022 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -15,349 +15,170 @@ package kafka import ( "context" - "fmt" - "strings" - "sync" - "time" - "github.com/confluentinc/confluent-kafka-go/v2/kafka" + "github.com/IBM/sarama" + "github.com/IBM/sarama/mocks" "github.com/pingcap/errors" - "github.com/pingcap/log" - commonType "github.com/pingcap/ticdc/pkg/common" + ticommon "github.com/pingcap/ticdc/pkg/common" + cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/sink/codec/common" - "go.uber.org/zap" ) -const ( - // DefaultMockTopicName specifies the default mock topic name. - DefaultMockTopicName = "mock_topic" - // DefaultMockPartitionNum is the default partition number of default mock topic. - DefaultMockPartitionNum = 3 - // defaultMockControllerID specifies the default mock controller ID. - defaultMockControllerID = 1 -) - -const ( - // defaultMaxMessageBytes specifies the default max message bytes, - // default to 1048576, identical to kafka broker's `message.max.bytes` and topic's `max.message.bytes` - // see: https://kafka.apache.org/documentation/#brokerconfigs_message.max.bytes - // see: https://kafka.apache.org/documentation/#topicconfigs_max.message.bytes - defaultMaxMessageBytes = "1048588" - - // defaultMinInsyncReplicas specifies the default `min.insync.replicas` for broker and topic. - defaultMinInsyncReplicas = "1" -) - -// var ( -// // BrokerMessageMaxBytes is the broker's `message.max.bytes` -// BrokerMessageMaxBytes = defaultMaxMessageBytes -// // TopicMaxMessageBytes is the topic's `max.message.bytes` -// TopicMaxMessageBytes = defaultMaxMessageBytes -// // MinInSyncReplicas is the `min.insync.replicas` -// MinInSyncReplicas = defaultMinInsyncReplicas -// ) - // MockFactory is a mock implementation of Factory interface. type MockFactory struct { - option *Options - changefeedID commonType.ChangeFeedID - mockCluster *kafka.MockCluster -} - -type mockTopicDetail struct { - TopicDetail - fetchesRemainingUntilVisible int + changefeedID ticommon.ChangeFeedID + config *sarama.Config + ErrorReporter mocks.ErrorReporter } // NewMockFactory constructs a Factory with mock implementation. func NewMockFactory( - o *Options, changefeedID commonType.ChangeFeedID, + _ context.Context, + o *Options, changefeedID ticommon.ChangeFeedID, ) (Factory, error) { - // The broker ids will start at 1 up to and including brokerCount. - mockCluster, err := kafka.NewMockCluster(defaultMockControllerID) + config, err := NewSaramaConfig(context.Background(), o) if err != nil { return nil, errors.Trace(err) } return &MockFactory{ - option: o, changefeedID: changefeedID, - mockCluster: mockCluster, + config: config, }, nil } // AdminClient return a mocked admin client func (f *MockFactory) AdminClient() (ClusterAdminClient, error) { - return &MockClusterAdmin{ - mockCluster: f.mockCluster, - topics: make(map[string]*mockTopicDetail), - }, nil + return NewClusterAdminClientMockImpl(), nil } -// SyncProducer creates a sync producer to Producer message to kafka +// SyncProducer creates a sync producer func (f *MockFactory) SyncProducer() (SyncProducer, error) { - return &MockSyncProducer{Producer: &MockProducer{}, changefeedID: f.changefeedID, option: f.option}, nil -} - -// AsyncProducer creates an async producer to Producer message to kafka -func (f *MockFactory) AsyncProducer() (AsyncProducer, error) { - return &MockAsyncProducer{Producer: &MockProducer{}, changefeedID: f.changefeedID, option: f.option, ch: make(chan *kafka.Message)}, nil -} - -// MetricsCollector returns a mocked metrics collector -func (f *MockFactory) MetricsCollector() MetricsCollector { - return &MockMetricsCollector{changefeedID: f.changefeedID} -} - -type MockClusterAdmin struct { - mockCluster *kafka.MockCluster - topics map[string]*mockTopicDetail -} - -func (c *MockClusterAdmin) GetAllBrokers(ctx context.Context) ([]Broker, error) { - bootstrapServers := c.mockCluster.BootstrapServers() - n := len(strings.Split(bootstrapServers, ",")) - brokers := make([]Broker, 0, n) - for i := 0; i < n; i++ { - brokers = append(brokers, Broker{ID: int32(i)}) - } - return brokers, nil -} - -func (c *MockClusterAdmin) GetTopicsMeta(ctx context.Context, - topics []string, ignoreTopicError bool, -) (map[string]TopicDetail, error) { - topicsMeta := make(map[string]TopicDetail) - for key, val := range c.topics { - topicsMeta[key] = val.TopicDetail - } - return topicsMeta, nil -} - -func (c *MockClusterAdmin) GetTopicsPartitionsNum( - ctx context.Context, topics []string, -) (map[string]int32, error) { - result := make(map[string]int32, len(topics)) - for _, topic := range topics { - msg, ok := c.topics[topic] - if !ok { - log.Warn("fetch topic meta failed", - zap.String("topic", topic), zap.Any("msg", msg)) - continue - } - result[topic] = msg.NumPartitions - } - return result, nil -} - -func (c *MockClusterAdmin) GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) { - _, ok := c.topics[topicName] - if !ok { - return "", nil - } - switch configName { - case "message.max.bytes", "max.message.bytes": - return defaultMaxMessageBytes, nil - case "min.insync.replicas": - return defaultMinInsyncReplicas, nil - } - return "0", nil -} - -func (c *MockClusterAdmin) CreateTopic(ctx context.Context, detail *TopicDetail, validateOnly bool) error { - bootstrapServers := c.mockCluster.BootstrapServers() - n := len(strings.Split(bootstrapServers, ",")) - if int(detail.ReplicationFactor) > n { - return kafka.NewError(kafka.ErrInvalidReplicationFactor, "kafka create topic failed: kafka server: Replication-factor is invalid", false) - } - err := c.mockCluster.CreateTopic(detail.Name, int(detail.NumPartitions), int(detail.ReplicationFactor)) - if err != nil { - return err - } - c.topics[detail.Name] = &mockTopicDetail{TopicDetail: *detail} - return nil -} - -func (c *MockClusterAdmin) GetBrokerConfig(ctx context.Context, configName string) (string, error) { - switch configName { - case "message.max.bytes", "max.message.bytes": - return defaultMaxMessageBytes, nil - case "min.insync.replicas": - return defaultMinInsyncReplicas, nil - } - return "0", nil + syncProducer := mocks.NewSyncProducer(f.ErrorReporter, f.config) + return &MockSaramaSyncProducer{ + Producer: syncProducer, + }, nil } -// SetRemainingFetchesUntilTopicVisible is used to control the visibility of a specific topic. -// It is used to mock the topic creation delay. -func (c *MockClusterAdmin) SetRemainingFetchesUntilTopicVisible( - topicName string, - fetchesRemainingUntilVisible int, -) error { - topic, ok := c.topics[topicName] - if !ok { - return fmt.Errorf("no such topic as %s", topicName) - } - topic.fetchesRemainingUntilVisible = fetchesRemainingUntilVisible - return nil +// AsyncProducer creates an async producer +func (f *MockFactory) AsyncProducer(ctx context.Context) (AsyncProducer, error) { + asyncProducer := mocks.NewAsyncProducer(f.ErrorReporter, f.config) + return &MockSaramaAsyncProducer{ + AsyncProducer: asyncProducer, + failpointCh: make(chan error, 1), + }, nil } -func (c *MockClusterAdmin) Close() { - c.mockCluster.Close() +// MetricsCollector returns the metric collector +func (f *MockFactory) MetricsCollector(_ ClusterAdminClient) MetricsCollector { + return &mockMetricsCollector{} } -type MockSyncProducer struct { - Producer *MockProducer - option *Options - changefeedID commonType.ChangeFeedID +// MockSaramaSyncProducer is a mock implementation of SyncProducer interface. +type MockSaramaSyncProducer struct { + Producer *mocks.SyncProducer } -// SendMessage produces message -func (s *MockSyncProducer) SendMessage( - ctx context.Context, - topic string, partition int32, +// SendMessage implement the SyncProducer interface. +func (m *MockSaramaSyncProducer) SendMessage( + _ context.Context, + topic string, partitionNum int32, message *common.Message, ) error { - select { - case <-ctx.Done(): - return ctx.Err() - default: - } - if s.Producer.getErr() != kafka.ErrNoError { - return kafka.NewError(s.Producer.getErr(), "", false) - } - if message.Length() > s.option.MaxMessageBytes { - s.Producer.setErr(kafka.ErrMsgSizeTooLarge) - return kafka.NewError(kafka.ErrMsgSizeTooLarge, "", false) - } - return nil + _, _, err := m.Producer.SendMessage(&sarama.ProducerMessage{ + Topic: topic, + Key: sarama.ByteEncoder(message.Key), + Value: sarama.ByteEncoder(message.Value), + Partition: partitionNum, + }) + return err } -// SendMessages produces a given set of messages -func (s *MockSyncProducer) SendMessages(ctx context.Context, topic string, partitionNum int32, message *common.Message) error { - var err error +// SendMessages implement the SyncProducer interface. +func (m *MockSaramaSyncProducer) SendMessages(_ context.Context, topic string, partitionNum int32, message *common.Message) error { + msgs := make([]*sarama.ProducerMessage, partitionNum) for i := 0; i < int(partitionNum); i++ { - e := s.SendMessage(ctx, topic, int32(i), message) - if e != nil { - err = e + msgs[i] = &sarama.ProducerMessage{ + Topic: topic, + Key: sarama.ByteEncoder(message.Key), + Value: sarama.ByteEncoder(message.Value), + Partition: int32(i), } } - return err + return m.Producer.SendMessages(msgs) } -// Close shuts down the mock producer -func (s *MockSyncProducer) Close() { +// Close implement the SyncProducer interface. +func (m *MockSaramaSyncProducer) Close() { + m.Producer.Close() } -// ExpectInputAndFail for test -func (s *MockSyncProducer) ExpectInputAndFail() { -} - -type MockAsyncProducer struct { - Producer *MockProducer - option *Options - changefeedID commonType.ChangeFeedID - ch chan *kafka.Message -} +// MockSaramaAsyncProducer is a mock implementation of AsyncProducer interface. +type MockSaramaAsyncProducer struct { + AsyncProducer *mocks.AsyncProducer + failpointCh chan error -// Close shuts down the producer -func (a *MockAsyncProducer) Close() { - a.Producer.setErr(kafka.ErrUnknown) - close(a.ch) + closed bool } -// AsyncRunCallback process the messages that has sent to kafka -func (a *MockAsyncProducer) AsyncRunCallback(ctx context.Context) error { - ticker := time.NewTicker(time.Millisecond) - defer ticker.Stop() +// AsyncRunCallback implement the AsyncProducer interface. +func (p *MockSaramaAsyncProducer) AsyncRunCallback( + ctx context.Context, +) error { for { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case msg := <-a.ch: - if msg != nil { - callback := msg.Opaque.(func()) + case err := <-p.failpointCh: + return errors.Trace(err) + case ack := <-p.AsyncProducer.Successes(): + if ack != nil { + callback := ack.Metadata.(func()) if callback != nil { callback() } } - case <-ticker.C: - if a.Producer.getErr() != kafka.ErrNoError { - return kafka.NewError(a.Producer.err, "", false) + case err := <-p.AsyncProducer.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 (a *MockAsyncProducer) AsyncSend(ctx context.Context, topic string, partition int32, message *common.Message) error { - msg := &kafka.Message{ - TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: partition}, - Key: message.Key, - Value: message.Value, - Opaque: message.Callback, +// AsyncSend implement the AsyncProducer interface. +func (p *MockSaramaAsyncProducer) 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 a.ch <- msg: - } - if message.Length() > a.option.MaxMessageBytes { - a.Producer.setErr(kafka.ErrMsgSizeTooLarge) + case p.AsyncProducer.Input() <- msg: } return nil } -type MockMetricsCollector struct { - changefeedID commonType.ChangeFeedID -} - -func (c *MockMetricsCollector) Run(_ context.Context) { -} - -type MockProducer struct { - err kafka.ErrorCode - mu sync.Mutex -} - -func (p *MockProducer) setErr(err kafka.ErrorCode) { - p.mu.Lock() - defer p.mu.Unlock() - p.err = err -} - -func (p *MockProducer) getErr() kafka.ErrorCode { - p.mu.Lock() - defer p.mu.Unlock() - return p.err -} - -// ExpectInputAndFail for test -func (p *MockProducer) ExpectInputAndFail(err kafka.ErrorCode) { - // TODO: mock send message - if err != p.err { - panic(fmt.Sprintf("expect input and fail. actual err: %s expected err: %s", p.err, err)) +// Close implement the AsyncProducer interface. +func (p *MockSaramaAsyncProducer) Close() { + if p.closed { + return } + _ = p.AsyncProducer.Close() + p.closed = true } -// ExpectSendMessageAndFail for test -func (p *MockProducer) ExpectSendMessageAndFail(err kafka.ErrorCode) { - if err != p.err { - panic(fmt.Sprintf("expect send message and succeed failed. actual err: %s expected err: %s", p.err, err)) - } -} +type mockMetricsCollector struct{} -// ExpectInputAndSucceed for test -func (p *MockProducer) ExpectInputAndSucceed() { - // TODO: mock send message - if p.err != kafka.ErrNoError { - panic(fmt.Sprintf("expect input and succeed failed. err: %s", p.err)) - } -} - -// ExpectInputAndSucceed for test -func (p *MockProducer) ExpectSendMessageAndSucceed() { - if p.err != kafka.ErrNoError { - panic(fmt.Sprintf("expect send message and succeed failed. err: %s", p.err)) - } +// Run implements the MetricsCollector interface. +func (m *mockMetricsCollector) Run(ctx context.Context) { } diff --git a/pkg/sink/kafka/oauth2_token_provider.go b/pkg/sink/kafka/oauth2_token_provider.go new file mode 100644 index 000000000..c447cd470 --- /dev/null +++ b/pkg/sink/kafka/oauth2_token_provider.go @@ -0,0 +1,84 @@ +// 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" +) + +// tokenProvider 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 + // client credentials library as defined by the spec, however non-compliant + // auth server implementations may want a custom type + endpointParams := url.Values{} + if o.SASL.OAuth2.GrantType != "" { + 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 != "" { + 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 index 189ee186f..3115db86a 100644 --- a/pkg/sink/kafka/options.go +++ b/pkg/sink/kafka/options.go @@ -167,7 +167,9 @@ type Options struct { SASL *security.SASL // Timeout for network configurations, default to `10s` - DialTimeout time.Duration + DialTimeout time.Duration + WriteTimeout time.Duration + ReadTimeout time.Duration } // NewOptions returns a default Kafka configuration @@ -184,6 +186,8 @@ func NewOptions() *Options { SASL: &security.SASL{}, AutoCreate: true, DialTimeout: 10 * time.Second, + WriteTimeout: 10 * time.Second, + ReadTimeout: 10 * time.Second, } } @@ -280,6 +284,22 @@ func (o *Options) Apply(changefeedID common.ChangeFeedID, 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 { @@ -316,6 +336,8 @@ func mergeConfig( 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 @@ -543,7 +565,7 @@ func NewKafkaClientID(captureAddr string, return } -// AdjustOptions adjust the `Options` and `Config` by condition. +// AdjustOptions adjust the `Options` and `sarama.Config` by condition. func AdjustOptions( ctx context.Context, admin ClusterAdminClient, diff --git a/pkg/sink/kafka/sarama.go b/pkg/sink/kafka/sarama.go new file mode 100644 index 000000000..4e271a4eb --- /dev/null +++ b/pkg/sink/kafka/sarama.go @@ -0,0 +1,253 @@ +// 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/ticdc/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 + + // According to the https://github.com/IBM/sarama/issues/2619, + // sarama may send message out of order even set the `config.Net.MaxOpenRequest` to 1, + // when the kafka cluster is unhealthy and trigger the internal retry mechanism. + config.Producer.Retry.Max = 0 + 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.MaxOpenRequests = 1 + 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 new file mode 100644 index 000000000..7637b8b21 --- /dev/null +++ b/pkg/sink/kafka/sarama_factory.go @@ -0,0 +1,135 @@ +// 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/ticdc/pkg/common" + "github.com/rcrowley/go-metrics" + "go.uber.org/zap" +) + +type saramaFactory struct { + changefeedID common.ChangeFeedID + config *sarama.Config + endpoints []string +} + +// NewSaramaFactory constructs a Factory with sarama implementation. +func NewSaramaFactory( + ctx context.Context, + o *Options, + changefeedID common.ChangeFeedID, +) (Factory, error) { + start := time.Now() + saramaConfig, err := NewSaramaConfig(ctx, o) + duration := time.Since(start).Seconds() + if duration > 2 { + log.Warn("new sarama config cost too much time", + zap.Any("duration", duration), zap.Stringer("changefeedID", changefeedID)) + } + if err != nil { + return nil, errors.Trace(err) + } + saramaConfig.MetricRegistry = metrics.NewRegistry() + + return &saramaFactory{ + changefeedID: changefeedID, + endpoints: o.BrokerEndpoints, + config: saramaConfig, + }, nil +} + +func (f *saramaFactory) AdminClient() (ClusterAdminClient, error) { + start := time.Now() + client, err := sarama.NewClient(f.endpoints, f.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() (SyncProducer, error) { + client, err := sarama.NewClient(f.endpoints, f.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(_ context.Context) (AsyncProducer, error) { + client, err := sarama.NewClient(f.endpoints, f.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: make(chan error, 1), + }, nil +} + +func (f *saramaFactory) MetricsCollector( + adminClient ClusterAdminClient, +) MetricsCollector { + return &saramaMetricsCollector{ + changefeedID: f.changefeedID, + adminClient: adminClient, + brokers: make(map[int32]struct{}), + registry: f.config.MetricRegistry, + } +} diff --git a/pkg/sink/kafka/v2/admin.go b/pkg/sink/kafka/v2/admin.go new file mode 100644 index 000000000..ee4c1f230 --- /dev/null +++ b/pkg/sink/kafka/v2/admin.go @@ -0,0 +1,269 @@ +// 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/errors" + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/common" + cerror "github.com/pingcap/ticdc/pkg/errors" + pkafka "github.com/pingcap/ticdc/pkg/sink/kafka" + "github.com/segmentio/kafka-go" + "go.uber.org/zap" +) + +type admin struct { + client Client + changefeedID common.ChangeFeedID +} + +func newClusterAdminClient( + endpoints []string, + transport *kafka.Transport, + changefeedID common.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 "", cerror.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 "", cerror.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 "", cerror.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.Name()), + 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 "", cerror.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.Cause(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.Name())) + 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.Name())) + + 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.Name())) + } + } + log.Info("kafka admin client is fully closed", + zap.String("namespace", a.changefeedID.Namespace()), + zap.String("changefeed", a.changefeedID.Name())) +} diff --git a/pkg/sink/kafka/v2/client.go b/pkg/sink/kafka/v2/client.go new file mode 100644 index 000000000..f9a1ed576 --- /dev/null +++ b/pkg/sink/kafka/v2/client.go @@ -0,0 +1,33 @@ +// 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 new file mode 100644 index 000000000..188ebdc57 --- /dev/null +++ b/pkg/sink/kafka/v2/factory.go @@ -0,0 +1,402 @@ +// 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" + + "github.com/jcmturner/gokrb5/v8/client" + "github.com/jcmturner/gokrb5/v8/config" + "github.com/jcmturner/gokrb5/v8/keytab" + "github.com/pingcap/errors" + "github.com/pingcap/log" + commonType "github.com/pingcap/ticdc/pkg/common" + cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/ticdc/pkg/sink/codec/common" + pkafka "github.com/pingcap/ticdc/pkg/sink/kafka" + "github.com/pingcap/tiflow/pkg/security" + tiv2 "github.com/pingcap/tiflow/pkg/sink/kafka/v2" + "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 commonType.ChangeFeedID + options *pkafka.Options + + writer *kafka.Writer +} + +// NewFactory returns a factory implemented based on kafka-go +func NewFactory( + _ context.Context, + options *pkafka.Options, + changefeedID commonType.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, cerror.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.Name()), + 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.Name())) + return w +} + +func (f *factory) AdminClient() (pkafka.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() (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, +) (pkafka.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: make(chan error, 1), + 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.Name()), + 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( + _ pkafka.ClusterAdminClient, +) pkafka.MetricsCollector { + return NewMetricsCollector(f.changefeedID, f.writer) +} + +type syncWriter struct { + changefeedID commonType.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.Name())) + 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.Name()), + 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.Name()), + zap.Duration("duration", time.Since(start))) + } +} + +type asyncWriter struct { + w tiv2.Writer + changefeedID commonType.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.Name())) + 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.Name()), + 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.Name()), + 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.Name()), + 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 cerror.WrapError(cerror.ErrKafkaAsyncSendMessage, err) + } +} diff --git a/pkg/sink/kafka/v2/gssapi.go b/pkg/sink/kafka/v2/gssapi.go new file mode 100644 index 000000000..d55063381 --- /dev/null +++ b/pkg/sink/kafka/v2/gssapi.go @@ -0,0 +1,252 @@ +// 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 new file mode 100644 index 000000000..ff852f1e7 --- /dev/null +++ b/pkg/sink/kafka/v2/manual_balancer.go @@ -0,0 +1,31 @@ +// 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/kafka/v2/metrics_collector.go b/pkg/sink/kafka/v2/metrics_collector.go new file mode 100644 index 000000000..70567562e --- /dev/null +++ b/pkg/sink/kafka/v2/metrics_collector.go @@ -0,0 +1,108 @@ +// 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" + "time" + + "github.com/pingcap/log" + "github.com/pingcap/ticdc/pkg/common" + "github.com/pingcap/ticdc/pkg/sink/kafka" + "go.uber.org/zap" +) + +// MetricsCollector is the kafka metrics collector based on kafka-go library. +type MetricsCollector struct { + changefeedID common.ChangeFeedID + writer Writer +} + +// NewMetricsCollector return a kafka metrics collector +func NewMetricsCollector( + changefeedID common.ChangeFeedID, + writer Writer, +) *MetricsCollector { + return &MetricsCollector{ + changefeedID: changefeedID, + writer: writer, + } +} + +// Run implement the MetricsCollector interface +func (m *MetricsCollector) Run(ctx context.Context) { + ticker := time.NewTicker(kafka.RefreshMetricsInterval) + defer func() { + ticker.Stop() + m.cleanupMetrics() + }() + + for { + select { + case <-ctx.Done(): + log.Info("Kafka metrics collector stopped", + zap.String("namespace", m.changefeedID.Namespace()), + zap.String("changefeed", m.changefeedID.Name())) + return + case <-ticker.C: + m.collectMetrics() + } + } +} + +func (m *MetricsCollector) collectMetrics() { + statistics := m.writer.Stats() + + // batch related metrics + kafka.BatchDurationGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()). + Set(statistics.BatchQueueTime.Avg.Seconds()) + kafka.BatchMessageCountGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()). + Set(float64(statistics.BatchSize.Avg)) + kafka.BatchSizeGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()). + Set(float64(statistics.BatchBytes.Avg)) + + // send request related metrics. + // metrics is collected each 5 seconds, divide by 5 to get per seconds average. + // since kafka-go does not support per broker metrics, so we add `v2` as the broker ID. + kafka.RequestRateGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2"). + Set(float64(statistics.Writes / 5)) + kafka.RequestLatencyGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2", "avg"). + Set(statistics.WriteTime.Avg.Seconds()) + kafka.OutgoingByteRateGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2"). + Set(float64(statistics.Bytes / 5)) + + kafka.ClientRetryGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()). + Set(float64(statistics.Retries)) + kafka.ClientErrorGauge.WithLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()). + Set(float64(statistics.Errors)) +} + +func (m *MetricsCollector) cleanupMetrics() { + kafka.BatchDurationGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()) + kafka.BatchMessageCountGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()) + kafka.BatchSizeGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()) + + kafka.RequestRateGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2") + kafka.RequestLatencyGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2", "avg") + kafka.OutgoingByteRateGauge. + DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name(), "v2") + + kafka.ClientRetryGauge.DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()) + kafka.ClientErrorGauge.DeleteLabelValues(m.changefeedID.Namespace(), m.changefeedID.Name()) +} diff --git a/pkg/sink/kafka/v2/writer.go b/pkg/sink/kafka/v2/writer.go new file mode 100644 index 000000000..aac8b6424 --- /dev/null +++ b/pkg/sink/kafka/v2/writer.go @@ -0,0 +1,29 @@ +// Copyright 2025 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" +) + +// Writer is the interface of the kafka-go writer, 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 Writer interface { + Stats() kafka.WriterStats + WriteMessages(ctx context.Context, msgs ...kafka.Message) error + Close() error +} diff --git a/tests/integration_tests/kafka_compression/run.sh b/tests/integration_tests/kafka_compression/run.sh index 113897010..aeb597753 100644 --- a/tests/integration_tests/kafka_compression/run.sh +++ b/tests/integration_tests/kafka_compression/run.sh @@ -18,12 +18,11 @@ function test_compression() { run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=canal-json&version=${KAFKA_VERSION}&enable-tidb-extension=true" run_sql_file $CUR/data/$1_data.sql ${UP_TIDB_HOST} ${UP_TIDB_PORT} - compression_algorithm=$(grep "kafka producer config.*compression.codec\\\\\":\\\\\"$1\\\\\".*" "$WORK_DIR/cdc.log") + compression_algorithm=$(grep "Kafka producer uses $1 compression algorithm" "$WORK_DIR/cdc.log") if [[ "$compression_algorithm" -ne 1 ]]; then echo "can't found producer compression algorithm" exit 1 fi - check_table_exists test.$1_finish_mark ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} 200 check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml run_cdc_cli changefeed pause -c $1 diff --git a/tests/integration_tests/open_protocol_claim_check/run.sh b/tests/integration_tests/open_protocol_claim_check/run.sh index 368f180ca..eaccdea6f 100644 --- a/tests/integration_tests/open_protocol_claim_check/run.sh +++ b/tests/integration_tests/open_protocol_claim_check/run.sh @@ -33,7 +33,7 @@ function run() { run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY - SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&max-message-bytes=1000&kafka-version=${KAFKA_VERSION}" + SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?protocol=open-protocol&max-message-bytes=800&kafka-version=${KAFKA_VERSION}" run_cdc_cli changefeed create --start-ts=$start_ts --target-ts=$target_ts --sink-uri="$SINK_URI" --config="$CUR/conf/changefeed.toml"