github.com/Jeffail/benthos/v3@v3.65.0/lib/input/reader/kafka.go (about)

     1  package reader
     2  
     3  import (
     4  	"context"
     5  	"crypto/tls"
     6  	"fmt"
     7  	"strconv"
     8  	"strings"
     9  	"sync"
    10  	"time"
    11  
    12  	"github.com/Jeffail/benthos/v3/lib/log"
    13  	"github.com/Jeffail/benthos/v3/lib/message"
    14  	"github.com/Jeffail/benthos/v3/lib/message/batch"
    15  	"github.com/Jeffail/benthos/v3/lib/metrics"
    16  	"github.com/Jeffail/benthos/v3/lib/types"
    17  	"github.com/Jeffail/benthos/v3/lib/util/kafka/sasl"
    18  	btls "github.com/Jeffail/benthos/v3/lib/util/tls"
    19  	"github.com/Shopify/sarama"
    20  	"gopkg.in/yaml.v3"
    21  )
    22  
    23  //------------------------------------------------------------------------------
    24  
    25  // KafkaConfig contains configuration fields for the Kafka input type.
    26  type KafkaConfig struct {
    27  	Addresses           []string                 `json:"addresses" yaml:"addresses"`
    28  	Topics              []string                 `json:"topics" yaml:"topics"`
    29  	ClientID            string                   `json:"client_id" yaml:"client_id"`
    30  	RackID              string                   `json:"rack_id" yaml:"rack_id"`
    31  	ConsumerGroup       string                   `json:"consumer_group" yaml:"consumer_group"`
    32  	Group               KafkaBalancedGroupConfig `json:"group" yaml:"group"`
    33  	CommitPeriod        string                   `json:"commit_period" yaml:"commit_period"`
    34  	CheckpointLimit     int                      `json:"checkpoint_limit" yaml:"checkpoint_limit"`
    35  	ExtractTracingMap   string                   `json:"extract_tracing_map" yaml:"extract_tracing_map"`
    36  	MaxProcessingPeriod string                   `json:"max_processing_period" yaml:"max_processing_period"`
    37  	FetchBufferCap      int                      `json:"fetch_buffer_cap" yaml:"fetch_buffer_cap"`
    38  	StartFromOldest     bool                     `json:"start_from_oldest" yaml:"start_from_oldest"`
    39  	TargetVersion       string                   `json:"target_version" yaml:"target_version"`
    40  	TLS                 btls.Config              `json:"tls" yaml:"tls"`
    41  	SASL                sasl.Config              `json:"sasl" yaml:"sasl"`
    42  	Batching            batch.PolicyConfig       `json:"batching" yaml:"batching"`
    43  
    44  	// TODO: V4 Remove this.
    45  	Topic         string `json:"topic" yaml:"topic"`
    46  	Partition     int32  `json:"partition" yaml:"partition"`
    47  	MaxBatchCount int    `json:"max_batch_count" yaml:"max_batch_count"`
    48  
    49  	deprecated bool
    50  }
    51  
    52  // IsDeprecated returns a boolean indicating whether this configuration uses the
    53  // old topic/partition fields.
    54  func (k KafkaConfig) IsDeprecated() bool {
    55  	return k.deprecated || k.Topic != "benthos_stream" || k.Partition != 0
    56  }
    57  
    58  // NewKafkaConfig creates a new KafkaConfig with default values.
    59  func NewKafkaConfig() KafkaConfig {
    60  	return KafkaConfig{
    61  		Addresses:           []string{"localhost:9092"},
    62  		Topics:              []string{},
    63  		ClientID:            "benthos_kafka_input",
    64  		RackID:              "",
    65  		ConsumerGroup:       "benthos_consumer_group",
    66  		Group:               NewKafkaBalancedGroupConfig(),
    67  		CommitPeriod:        "1s",
    68  		CheckpointLimit:     1,
    69  		MaxProcessingPeriod: "100ms",
    70  		FetchBufferCap:      256,
    71  		Topic:               "benthos_stream",
    72  		Partition:           0,
    73  		StartFromOldest:     true,
    74  		TargetVersion:       sarama.V1_0_0_0.String(),
    75  		MaxBatchCount:       1,
    76  		TLS:                 btls.NewConfig(),
    77  		SASL:                sasl.NewConfig(),
    78  		Batching:            batch.NewPolicyConfig(),
    79  	}
    80  }
    81  
    82  // UnmarshalYAML checks while parsing a Kafka config whether any deprecated
    83  // fields (topic, partition) have been specified.
    84  func (k *KafkaConfig) UnmarshalYAML(value *yaml.Node) error {
    85  	type confAlias KafkaConfig
    86  	aliased := confAlias(NewKafkaConfig())
    87  
    88  	if err := value.Decode(&aliased); err != nil {
    89  		return fmt.Errorf("line %v: %v", value.Line, err)
    90  	}
    91  
    92  	var raw interface{}
    93  	var deprecated bool
    94  	if err := value.Decode(&raw); err != nil {
    95  		return fmt.Errorf("line %v: %v", value.Line, err)
    96  	}
    97  	if m, ok := raw.(map[string]interface{}); ok {
    98  		if _, exists := m["topic"]; exists {
    99  			deprecated = true
   100  		}
   101  		if _, exists := m["partition"]; exists {
   102  			deprecated = true
   103  		}
   104  	}
   105  
   106  	*k = KafkaConfig(aliased)
   107  	k.deprecated = deprecated
   108  	return nil
   109  }
   110  
   111  //------------------------------------------------------------------------------
   112  
   113  // Kafka is an input type that reads from a Kafka instance.
   114  type Kafka struct {
   115  	client       sarama.Client
   116  	coordinator  *sarama.Broker
   117  	partConsumer sarama.PartitionConsumer
   118  	version      sarama.KafkaVersion
   119  
   120  	tlsConf *tls.Config
   121  
   122  	sMut sync.Mutex
   123  
   124  	offsetLastCommitted time.Time
   125  	commitPeriod        time.Duration
   126  	maxProcPeriod       time.Duration
   127  
   128  	mRcvErr metrics.StatCounter
   129  
   130  	offsetCommitted int64
   131  	offsetCommit    int64
   132  	offset          int64
   133  
   134  	addresses []string
   135  	conf      KafkaConfig
   136  	stats     metrics.Type
   137  	log       log.Modular
   138  	mgr       types.Manager
   139  
   140  	closeOnce  sync.Once
   141  	closedChan chan struct{}
   142  }
   143  
   144  // NewKafka creates a new Kafka input type.
   145  func NewKafka(
   146  	conf KafkaConfig, mgr types.Manager, log log.Modular, stats metrics.Type,
   147  ) (*Kafka, error) {
   148  	k := Kafka{
   149  		offset:     0,
   150  		conf:       conf,
   151  		stats:      stats,
   152  		mRcvErr:    stats.GetCounter("recv.error"),
   153  		log:        log,
   154  		mgr:        mgr,
   155  		closedChan: make(chan struct{}),
   156  	}
   157  
   158  	if conf.TLS.Enabled {
   159  		var err error
   160  		if k.tlsConf, err = conf.TLS.Get(); err != nil {
   161  			return nil, err
   162  		}
   163  	}
   164  
   165  	var err error
   166  	if k.version, err = sarama.ParseKafkaVersion(conf.TargetVersion); err != nil {
   167  		return nil, err
   168  	}
   169  
   170  	for _, addr := range conf.Addresses {
   171  		for _, splitAddr := range strings.Split(addr, ",") {
   172  			if trimmed := strings.TrimSpace(splitAddr); len(trimmed) > 0 {
   173  				k.addresses = append(k.addresses, trimmed)
   174  			}
   175  		}
   176  	}
   177  
   178  	if tout := conf.CommitPeriod; len(tout) > 0 {
   179  		var err error
   180  		if k.commitPeriod, err = time.ParseDuration(tout); err != nil {
   181  			return nil, fmt.Errorf("failed to parse commit period string: %v", err)
   182  		}
   183  	}
   184  
   185  	if tout := conf.MaxProcessingPeriod; len(tout) > 0 {
   186  		var err error
   187  		if k.maxProcPeriod, err = time.ParseDuration(tout); err != nil {
   188  			return nil, fmt.Errorf("failed to parse max processing period string: %v", err)
   189  		}
   190  	}
   191  	return &k, nil
   192  }
   193  
   194  //------------------------------------------------------------------------------
   195  
   196  // closeClients closes the kafka clients, this interrupts loop() out of the read
   197  // block.
   198  func (k *Kafka) closeClients() {
   199  	k.commit()
   200  
   201  	k.sMut.Lock()
   202  	defer k.sMut.Unlock()
   203  
   204  	if k.partConsumer != nil {
   205  		// NOTE: Needs draining before destroying.
   206  		k.partConsumer.AsyncClose()
   207  		defer func() {
   208  			// Drain both channels
   209  			for range k.partConsumer.Messages() {
   210  			}
   211  			for range k.partConsumer.Errors() {
   212  			}
   213  
   214  			k.partConsumer = nil
   215  		}()
   216  	}
   217  	if k.coordinator != nil {
   218  		k.coordinator.Close()
   219  		k.coordinator = nil
   220  	}
   221  	if k.client != nil {
   222  		k.client.Close()
   223  		k.client = nil
   224  	}
   225  	k.closeOnce.Do(func() {
   226  		close(k.closedChan)
   227  	})
   228  }
   229  
   230  //------------------------------------------------------------------------------
   231  
   232  // Connect establishes a Kafka connection.
   233  func (k *Kafka) Connect() error {
   234  	return k.ConnectWithContext(context.Background())
   235  }
   236  
   237  // ConnectWithContext establishes a Kafka connection.
   238  func (k *Kafka) ConnectWithContext(ctx context.Context) error {
   239  	var err error
   240  	defer func() {
   241  		if err != nil {
   242  			k.closeClients()
   243  		}
   244  	}()
   245  
   246  	k.sMut.Lock()
   247  	defer k.sMut.Unlock()
   248  
   249  	if k.client != nil {
   250  		return nil
   251  	}
   252  
   253  	config := sarama.NewConfig()
   254  	config.Version = k.version
   255  	config.ClientID = k.conf.ClientID
   256  	config.Net.DialTimeout = time.Second
   257  	config.Consumer.Return.Errors = true
   258  	config.Consumer.MaxProcessingTime = k.maxProcPeriod
   259  	config.ChannelBufferSize = k.conf.FetchBufferCap
   260  	config.Net.TLS.Enable = k.conf.TLS.Enabled
   261  	if k.conf.TLS.Enabled {
   262  		config.Net.TLS.Config = k.tlsConf
   263  	}
   264  	if err := k.conf.SASL.Apply(k.mgr, config); err != nil {
   265  		return err
   266  	}
   267  
   268  	k.client, err = sarama.NewClient(k.addresses, config)
   269  	if err != nil {
   270  		return err
   271  	}
   272  
   273  	k.coordinator, err = k.client.Coordinator(k.conf.ConsumerGroup)
   274  	if err != nil {
   275  		return err
   276  	}
   277  
   278  	var consumer sarama.Consumer
   279  	consumer, err = sarama.NewConsumerFromClient(k.client)
   280  	if err != nil {
   281  		return err
   282  	}
   283  
   284  	offsetReq := sarama.OffsetFetchRequest{}
   285  	offsetReq.ConsumerGroup = k.conf.ConsumerGroup
   286  	offsetReq.AddPartition(k.conf.Topic, k.conf.Partition)
   287  
   288  	if offsetRes, err := k.coordinator.FetchOffset(&offsetReq); err == nil {
   289  		offsetBlock := offsetRes.Blocks[k.conf.Topic][k.conf.Partition]
   290  		if offsetBlock.Err == sarama.ErrNoError {
   291  			k.offset = offsetBlock.Offset
   292  		} else {
   293  			k.log.Errorf("Failed to acquire offset: %v\n", offsetBlock.Err)
   294  		}
   295  		k.log.Debugf("Acquired stored offset: %v\n", k.offset)
   296  	} else {
   297  		k.log.Errorf("Failed to acquire offset from coordinator: %v\n", err)
   298  	}
   299  
   300  	var partConsumer sarama.PartitionConsumer
   301  	partConsumer, err = consumer.ConsumePartition(
   302  		k.conf.Topic, k.conf.Partition, k.offset,
   303  	)
   304  	if err != nil {
   305  		offsetTarget := sarama.OffsetOldest
   306  		if !k.conf.StartFromOldest {
   307  			offsetTarget = sarama.OffsetNewest
   308  			k.log.Warnln("Failed to read from stored offset, restarting from newest offset")
   309  		} else {
   310  			k.log.Warnln("Failed to read from stored offset, restarting from oldest offset")
   311  		}
   312  
   313  		k.log.Warnf(
   314  			"Attempting to obtain offset for topic %s, partition %v\n",
   315  			k.conf.Topic, k.conf.Partition,
   316  		)
   317  
   318  		// Get the new offset target
   319  		if k.offset, err = k.client.GetOffset(
   320  			k.conf.Topic, k.conf.Partition, offsetTarget,
   321  		); err == nil {
   322  			partConsumer, err = consumer.ConsumePartition(
   323  				k.conf.Topic, k.conf.Partition, k.offset,
   324  			)
   325  		}
   326  	}
   327  	if err != nil {
   328  		return err
   329  	}
   330  
   331  	k.partConsumer = partConsumer
   332  	k.log.Infof("Receiving Kafka messages from addresses: %s\n", k.addresses)
   333  
   334  	go func() {
   335  		for err := range partConsumer.Errors() {
   336  			if err != nil {
   337  				k.log.Errorf("Kafka message recv error: %v\n", err)
   338  				k.mRcvErr.Incr(1)
   339  			}
   340  		}
   341  	}()
   342  
   343  	return err
   344  }
   345  
   346  // Read attempts to read a message from a Kafka topic.
   347  func (k *Kafka) Read() (types.Message, error) {
   348  	return k.ReadNextWithContext(context.Background())
   349  }
   350  
   351  // ReadNextWithContext attempts to read a message from a Kafka topic.
   352  func (k *Kafka) ReadNextWithContext(ctx context.Context) (types.Message, error) {
   353  	var partConsumer sarama.PartitionConsumer
   354  
   355  	k.sMut.Lock()
   356  	partConsumer = k.partConsumer
   357  	k.sMut.Unlock()
   358  
   359  	if partConsumer == nil {
   360  		return nil, types.ErrNotConnected
   361  	}
   362  
   363  	hwm := partConsumer.HighWaterMarkOffset()
   364  
   365  	msg := message.New(nil)
   366  
   367  	addPart := func(data *sarama.ConsumerMessage) {
   368  		k.offset = data.Offset + 1
   369  		part := message.NewPart(data.Value)
   370  
   371  		meta := part.Metadata()
   372  		for _, hdr := range data.Headers {
   373  			meta.Set(string(hdr.Key), string(hdr.Value))
   374  		}
   375  
   376  		lag := hwm - data.Offset
   377  		if lag < 0 {
   378  			lag = 0
   379  		}
   380  
   381  		meta.Set("kafka_key", string(data.Key))
   382  		meta.Set("kafka_partition", strconv.Itoa(int(data.Partition)))
   383  		meta.Set("kafka_topic", data.Topic)
   384  		meta.Set("kafka_offset", strconv.FormatInt(data.Offset, 10))
   385  		meta.Set("kafka_lag", strconv.FormatInt(lag, 10))
   386  		meta.Set("kafka_timestamp_unix", strconv.FormatInt(data.Timestamp.Unix(), 10))
   387  
   388  		msg.Append(part)
   389  	}
   390  
   391  	select {
   392  	case data, open := <-partConsumer.Messages():
   393  		if !open {
   394  			return nil, types.ErrTypeClosed
   395  		}
   396  		addPart(data)
   397  	case <-ctx.Done():
   398  		return nil, types.ErrTimeout
   399  	}
   400  
   401  	if msg.Len() == 0 {
   402  		return nil, types.ErrTimeout
   403  	}
   404  	return msg, nil
   405  }
   406  
   407  // Acknowledge instructs whether the current offset should be committed.
   408  func (k *Kafka) Acknowledge(err error) error {
   409  	return k.AcknowledgeWithContext(context.Background(), err)
   410  }
   411  
   412  // AcknowledgeWithContext instructs whether the current offset should be
   413  // committed.
   414  func (k *Kafka) AcknowledgeWithContext(ctx context.Context, err error) error {
   415  	if err == nil {
   416  		k.offsetCommit = k.offset
   417  	}
   418  
   419  	if time.Since(k.offsetLastCommitted) < k.commitPeriod {
   420  		return nil
   421  	}
   422  
   423  	return k.commit()
   424  }
   425  
   426  func (k *Kafka) commit() error {
   427  	if k.offsetCommit == k.offsetCommitted {
   428  		return nil
   429  	}
   430  
   431  	var coordinator *sarama.Broker
   432  
   433  	k.sMut.Lock()
   434  	coordinator = k.coordinator
   435  	k.sMut.Unlock()
   436  
   437  	if coordinator == nil {
   438  		return types.ErrNotConnected
   439  	}
   440  
   441  	commitReq := sarama.OffsetCommitRequest{}
   442  	commitReq.ConsumerGroup = k.conf.ConsumerGroup
   443  	commitReq.AddBlock(k.conf.Topic, k.conf.Partition, k.offset, 0, "")
   444  
   445  	commitRes, err := coordinator.CommitOffset(&commitReq)
   446  	if err == nil {
   447  		err = commitRes.Errors[k.conf.Topic][k.conf.Partition]
   448  		if err == sarama.ErrNoError {
   449  			err = nil
   450  		}
   451  	}
   452  
   453  	if err != nil {
   454  		k.log.Errorf("Failed to commit offset: %v\n", err)
   455  
   456  		k.sMut.Lock()
   457  		defer k.sMut.Unlock()
   458  
   459  		if k.client == nil {
   460  			return nil
   461  		}
   462  
   463  		// Attempt to reconnect
   464  		if err := k.client.RefreshCoordinator(k.conf.ConsumerGroup); err != nil {
   465  			k.log.Errorf("Failed to refresh coordinator: %v\n", err)
   466  		}
   467  		if newCoord, err := k.client.Coordinator(k.conf.ConsumerGroup); err != nil {
   468  			k.log.Errorf("Failed to acquire new coordinator: %v\n", err)
   469  		} else if k.coordinator.ID() != newCoord.ID() {
   470  			k.coordinator.Close()
   471  			k.coordinator = newCoord
   472  		}
   473  	} else {
   474  		k.offsetCommitted = k.offsetCommit
   475  		k.offsetLastCommitted = time.Now()
   476  	}
   477  
   478  	return nil
   479  }
   480  
   481  // CloseAsync shuts down the Kafka input and stops processing requests.
   482  func (k *Kafka) CloseAsync() {
   483  	go k.closeClients()
   484  }
   485  
   486  // WaitForClose blocks until the Kafka input has closed down.
   487  func (k *Kafka) WaitForClose(timeout time.Duration) error {
   488  	select {
   489  	case <-k.closedChan:
   490  	case <-time.After(timeout):
   491  		return types.ErrTimeout
   492  	}
   493  	return nil
   494  }
   495  
   496  //------------------------------------------------------------------------------