github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/v2/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 v2
    15  
    16  import (
    17  	"context"
    18  	"strconv"
    19  
    20  	"github.com/pingcap/log"
    21  	"github.com/pingcap/tiflow/cdc/model"
    22  	"github.com/pingcap/tiflow/pkg/errors"
    23  	pkafka "github.com/pingcap/tiflow/pkg/sink/kafka"
    24  	"github.com/segmentio/kafka-go"
    25  	"go.uber.org/zap"
    26  )
    27  
    28  type admin struct {
    29  	client       Client
    30  	changefeedID model.ChangeFeedID
    31  }
    32  
    33  func newClusterAdminClient(
    34  	endpoints []string,
    35  	transport *kafka.Transport,
    36  	changefeedID model.ChangeFeedID,
    37  ) pkafka.ClusterAdminClient {
    38  	client := newClient(endpoints, transport)
    39  	return &admin{
    40  		client:       client,
    41  		changefeedID: changefeedID,
    42  	}
    43  }
    44  
    45  func (a *admin) clusterMetadata(ctx context.Context) (*kafka.MetadataResponse, error) {
    46  	// request is not set, so it will return all metadata
    47  	result, err := a.client.Metadata(ctx, &kafka.MetadataRequest{})
    48  	if err != nil {
    49  		return nil, errors.Trace(err)
    50  	}
    51  	return result, nil
    52  }
    53  
    54  func (a *admin) GetAllBrokers(ctx context.Context) ([]pkafka.Broker, error) {
    55  	response, err := a.clusterMetadata(ctx)
    56  	if err != nil {
    57  		return nil, errors.Trace(err)
    58  	}
    59  
    60  	result := make([]pkafka.Broker, 0, len(response.Brokers))
    61  	for _, broker := range response.Brokers {
    62  		result = append(result, pkafka.Broker{
    63  			ID: int32(broker.ID),
    64  		})
    65  	}
    66  	return result, nil
    67  }
    68  
    69  func (a *admin) GetBrokerConfig(ctx context.Context, configName string) (string, error) {
    70  	response, err := a.clusterMetadata(ctx)
    71  	if err != nil {
    72  		return "", errors.Trace(err)
    73  	}
    74  
    75  	controllerID := response.Controller.ID
    76  	request := &kafka.DescribeConfigsRequest{
    77  		Resources: []kafka.DescribeConfigRequestResource{
    78  			{
    79  				ResourceType: kafka.ResourceTypeBroker,
    80  				ResourceName: strconv.Itoa(controllerID),
    81  				ConfigNames:  []string{configName},
    82  			},
    83  		},
    84  	}
    85  
    86  	resp, err := a.client.DescribeConfigs(ctx, request)
    87  	if err != nil {
    88  		return "", errors.Trace(err)
    89  	}
    90  
    91  	if len(resp.Resources) == 0 || len(resp.Resources[0].ConfigEntries) == 0 {
    92  		log.Warn("Kafka config item not found",
    93  			zap.String("configName", configName))
    94  		return "", errors.ErrKafkaConfigNotFound.GenWithStack(
    95  			"cannot find the `%s` from the broker's configuration", configName)
    96  	}
    97  
    98  	// For compatibility with KOP, we checked all return values.
    99  	// 1. Kafka only returns requested configs.
   100  	// 2. Kop returns all configs.
   101  	for _, entry := range resp.Resources[0].ConfigEntries {
   102  		if entry.ConfigName == configName {
   103  			return entry.ConfigValue, nil
   104  		}
   105  	}
   106  
   107  	log.Warn("Kafka config item not found",
   108  		zap.String("configName", configName))
   109  	return "", errors.ErrKafkaConfigNotFound.GenWithStack(
   110  		"cannot find the `%s` from the broker's configuration", configName)
   111  }
   112  
   113  func (a *admin) GetTopicConfig(ctx context.Context, topicName string, configName string) (string, error) {
   114  	request := &kafka.DescribeConfigsRequest{
   115  		Resources: []kafka.DescribeConfigRequestResource{
   116  			{
   117  				ResourceType: kafka.ResourceTypeTopic,
   118  				ResourceName: topicName,
   119  				ConfigNames:  []string{configName},
   120  			},
   121  		},
   122  	}
   123  
   124  	resp, err := a.client.DescribeConfigs(ctx, request)
   125  	if err != nil {
   126  		return "", errors.Trace(err)
   127  	}
   128  
   129  	if len(resp.Resources) == 0 || len(resp.Resources[0].ConfigEntries) == 0 {
   130  		log.Warn("Kafka config item not found",
   131  			zap.String("configName", configName))
   132  		return "", errors.ErrKafkaConfigNotFound.GenWithStack(
   133  			"cannot find the `%s` from the topic's configuration", configName)
   134  	}
   135  
   136  	// For compatibility with KOP, we checked all return values.
   137  	// 1. Kafka only returns requested configs.
   138  	// 2. Kop returns all configs.
   139  	for _, entry := range resp.Resources[0].ConfigEntries {
   140  		if entry.ConfigName == configName {
   141  			log.Info("Kafka config item found",
   142  				zap.String("namespace", a.changefeedID.Namespace),
   143  				zap.String("changefeed", a.changefeedID.ID),
   144  				zap.String("configName", configName),
   145  				zap.String("configValue", entry.ConfigValue))
   146  			return entry.ConfigValue, nil
   147  		}
   148  	}
   149  
   150  	log.Warn("Kafka config item not found",
   151  		zap.String("configName", configName))
   152  	return "", errors.ErrKafkaConfigNotFound.GenWithStack(
   153  		"cannot find the `%s` from the topic's configuration", configName)
   154  }
   155  
   156  func (a *admin) GetTopicsMeta(
   157  	ctx context.Context,
   158  	topics []string,
   159  	ignoreTopicError bool,
   160  ) (map[string]pkafka.TopicDetail, error) {
   161  	resp, err := a.client.Metadata(ctx, &kafka.MetadataRequest{
   162  		Topics: topics,
   163  	})
   164  	if err != nil {
   165  		return nil, errors.Trace(err)
   166  	}
   167  
   168  	result := make(map[string]pkafka.TopicDetail, len(resp.Topics))
   169  	for _, topic := range resp.Topics {
   170  		if topic.Error != nil {
   171  			if !ignoreTopicError {
   172  				return nil, errors.Trace(topic.Error)
   173  			}
   174  			log.Warn("fetch topic meta failed",
   175  				zap.String("topic", topic.Name), zap.Error(topic.Error))
   176  			continue
   177  		}
   178  		result[topic.Name] = pkafka.TopicDetail{
   179  			Name:          topic.Name,
   180  			NumPartitions: int32(len(topic.Partitions)),
   181  		}
   182  	}
   183  	return result, nil
   184  }
   185  
   186  func (a *admin) GetTopicsPartitionsNum(
   187  	ctx context.Context, topics []string,
   188  ) (map[string]int32, error) {
   189  	resp, err := a.client.Metadata(ctx, &kafka.MetadataRequest{
   190  		Topics: topics,
   191  	})
   192  	if err != nil {
   193  		return nil, errors.Trace(err)
   194  	}
   195  
   196  	result := make(map[string]int32, len(topics))
   197  	for _, topic := range resp.Topics {
   198  		result[topic.Name] = int32(len(topic.Partitions))
   199  	}
   200  	return result, nil
   201  }
   202  
   203  func (a *admin) CreateTopic(
   204  	ctx context.Context,
   205  	detail *pkafka.TopicDetail,
   206  	validateOnly bool,
   207  ) error {
   208  	request := &kafka.CreateTopicsRequest{
   209  		Topics: []kafka.TopicConfig{
   210  			{
   211  				Topic:             detail.Name,
   212  				NumPartitions:     int(detail.NumPartitions),
   213  				ReplicationFactor: int(detail.ReplicationFactor),
   214  			},
   215  		},
   216  		ValidateOnly: validateOnly,
   217  	}
   218  
   219  	response, err := a.client.CreateTopics(ctx, request)
   220  	if err != nil {
   221  		return errors.Trace(err)
   222  	}
   223  
   224  	for _, err := range response.Errors {
   225  		if err != nil && !errors.Is(err, kafka.TopicAlreadyExists) {
   226  			return errors.Trace(err)
   227  		}
   228  	}
   229  
   230  	return nil
   231  }
   232  
   233  func (a *admin) Close() {
   234  	log.Info("admin client start closing",
   235  		zap.String("namespace", a.changefeedID.Namespace),
   236  		zap.String("changefeed", a.changefeedID.ID))
   237  	client, ok := a.client.(*kafka.Client)
   238  	if !ok {
   239  		return
   240  	}
   241  
   242  	if client.Transport == nil {
   243  		return
   244  	}
   245  
   246  	transport, ok := client.Transport.(*kafka.Transport)
   247  	if !ok {
   248  		return
   249  	}
   250  
   251  	transport.CloseIdleConnections()
   252  	log.Info("admin client close idle connections",
   253  		zap.String("namespace", a.changefeedID.Namespace),
   254  		zap.String("changefeed", a.changefeedID.ID))
   255  
   256  	if transport.SASL != nil {
   257  		m, ok := transport.SASL.(mechanism)
   258  		if ok && m.client != nil {
   259  			m.client.Destroy()
   260  			log.Info("destroy sasl sessions",
   261  				zap.String("namespace", a.changefeedID.Namespace),
   262  				zap.String("changefeed", a.changefeedID.ID))
   263  		}
   264  	}
   265  	log.Info("kafka admin client is fully closed",
   266  		zap.String("namespace", a.changefeedID.Namespace),
   267  		zap.String("changefeed", a.changefeedID.ID))
   268  }