github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/admin.go (about) 1 // Copyright 2023 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package kafka 15 16 import ( 17 "context" 18 "strconv" 19 "strings" 20 21 "github.com/IBM/sarama" 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "go.uber.org/zap" 27 ) 28 29 type saramaAdminClient struct { 30 changefeed model.ChangeFeedID 31 32 client sarama.Client 33 admin sarama.ClusterAdmin 34 } 35 36 func (a *saramaAdminClient) GetAllBrokers(_ context.Context) ([]Broker, error) { 37 brokers := a.client.Brokers() 38 result := make([]Broker, 0, len(brokers)) 39 for _, broker := range brokers { 40 result = append(result, Broker{ 41 ID: broker.ID(), 42 }) 43 } 44 45 return result, nil 46 } 47 48 func (a *saramaAdminClient) GetBrokerConfig( 49 _ context.Context, 50 configName string, 51 ) (string, error) { 52 _, controller, err := a.admin.DescribeCluster() 53 if err != nil { 54 return "", errors.Trace(err) 55 } 56 57 configEntries, err := a.admin.DescribeConfig(sarama.ConfigResource{ 58 Type: sarama.BrokerResource, 59 Name: strconv.Itoa(int(controller)), 60 ConfigNames: []string{configName}, 61 }) 62 if err != nil { 63 return "", errors.Trace(err) 64 } 65 66 // For compatibility with KOP, we checked all return values. 67 // 1. Kafka only returns requested configs. 68 // 2. Kop returns all configs. 69 for _, entry := range configEntries { 70 if entry.Name == configName { 71 return entry.Value, nil 72 } 73 } 74 75 log.Warn("Kafka config item not found", 76 zap.String("namespace", a.changefeed.Namespace), 77 zap.String("changefeed", a.changefeed.ID), 78 zap.String("configName", configName)) 79 return "", cerror.ErrKafkaConfigNotFound.GenWithStack( 80 "cannot find the `%s` from the broker's configuration", configName) 81 } 82 83 func (a *saramaAdminClient) GetTopicConfig( 84 _ context.Context, topicName string, configName string, 85 ) (string, error) { 86 configEntries, err := a.admin.DescribeConfig(sarama.ConfigResource{ 87 Type: sarama.TopicResource, 88 Name: topicName, 89 ConfigNames: []string{configName}, 90 }) 91 if err != nil { 92 return "", errors.Trace(err) 93 } 94 95 // For compatibility with KOP, we checked all return values. 96 // 1. Kafka only returns requested configs. 97 // 2. Kop returns all configs. 98 for _, entry := range configEntries { 99 if entry.Name == configName { 100 log.Info("Kafka config item found", 101 zap.String("namespace", a.changefeed.Namespace), 102 zap.String("changefeed", a.changefeed.ID), 103 zap.String("configName", configName), 104 zap.String("configValue", entry.Value)) 105 return entry.Value, nil 106 } 107 } 108 109 log.Warn("Kafka config item not found", 110 zap.String("namespace", a.changefeed.Namespace), 111 zap.String("changefeed", a.changefeed.ID), 112 zap.String("configName", configName)) 113 return "", cerror.ErrKafkaConfigNotFound.GenWithStack( 114 "cannot find the `%s` from the topic's configuration", configName) 115 } 116 117 func (a *saramaAdminClient) GetTopicsMeta( 118 _ context.Context, topics []string, ignoreTopicError bool, 119 ) (map[string]TopicDetail, error) { 120 result := make(map[string]TopicDetail, len(topics)) 121 122 metaList, err := a.admin.DescribeTopics(topics) 123 if err != nil { 124 return nil, errors.Trace(err) 125 } 126 127 for _, meta := range metaList { 128 if meta.Err != sarama.ErrNoError { 129 if !ignoreTopicError { 130 return nil, meta.Err 131 } 132 log.Warn("fetch topic meta failed", 133 zap.String("namespace", a.changefeed.Namespace), 134 zap.String("changefeed", a.changefeed.ID), 135 zap.String("topic", meta.Name), 136 zap.Error(meta.Err)) 137 continue 138 } 139 result[meta.Name] = TopicDetail{ 140 Name: meta.Name, 141 NumPartitions: int32(len(meta.Partitions)), 142 } 143 } 144 return result, nil 145 } 146 147 func (a *saramaAdminClient) GetTopicsPartitionsNum( 148 _ context.Context, topics []string, 149 ) (map[string]int32, error) { 150 result := make(map[string]int32, len(topics)) 151 for _, topic := range topics { 152 partition, err := a.client.Partitions(topic) 153 if err != nil { 154 return nil, errors.Trace(err) 155 } 156 result[topic] = int32(len(partition)) 157 } 158 159 return result, nil 160 } 161 162 func (a *saramaAdminClient) CreateTopic( 163 _ context.Context, detail *TopicDetail, validateOnly bool, 164 ) error { 165 request := &sarama.TopicDetail{ 166 NumPartitions: detail.NumPartitions, 167 ReplicationFactor: detail.ReplicationFactor, 168 } 169 170 err := a.admin.CreateTopic(detail.Name, request, validateOnly) 171 // Ignore the already exists error because it's not harmful. 172 if err != nil && !strings.Contains(err.Error(), sarama.ErrTopicAlreadyExists.Error()) { 173 return err 174 } 175 return nil 176 } 177 178 func (a *saramaAdminClient) Close() { 179 if err := a.admin.Close(); err != nil { 180 log.Warn("close admin client meet error", 181 zap.String("namespace", a.changefeed.Namespace), 182 zap.String("changefeed", a.changefeed.ID), 183 zap.Error(err)) 184 } 185 }