github.com/Jeffail/benthos/v3@v3.65.0/lib/input/kafka_parts.go (about) 1 package input 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "io" 8 "strings" 9 "sync" 10 "time" 11 12 "github.com/Jeffail/benthos/v3/lib/message/batch" 13 "github.com/Jeffail/benthos/v3/lib/types" 14 "github.com/Shopify/sarama" 15 ) 16 17 type closureOffsetTracker struct { 18 fn func(string, int32, int64, string) 19 } 20 21 func (c *closureOffsetTracker) MarkOffset(topic string, partition int32, offset int64, metadata string) { 22 c.fn(topic, partition, offset, metadata) 23 } 24 25 func (k *kafkaReader) runPartitionConsumer( 26 ctx context.Context, 27 wg *sync.WaitGroup, 28 topic string, 29 partition int32, 30 consumer sarama.PartitionConsumer, 31 ) { 32 k.log.Debugf("Consuming messages from topic '%v' partition '%v'\n", topic, partition) 33 defer k.log.Debugf("Stopped consuming messages from topic '%v' partition '%v'\n", topic, partition) 34 defer wg.Done() 35 36 batchPolicy, err := batch.NewPolicy(k.conf.Batching, k.mgr, k.log, k.stats) 37 if err != nil { 38 k.log.Errorf("Failed to initialise batch policy: %v, falling back to no policy.\n", err) 39 conf := batch.NewPolicyConfig() 40 conf.Count = 1 41 if batchPolicy, err = batch.NewPolicy(conf, k.mgr, k.log, k.stats); err != nil { 42 panic(err) 43 } 44 } 45 defer batchPolicy.CloseAsync() 46 47 var nextTimedBatchChan <-chan time.Time 48 var flushBatch func(context.Context, chan<- asyncMessage, types.Message, int64) bool 49 if k.conf.CheckpointLimit > 1 { 50 flushBatch = k.asyncCheckpointer(topic, partition) 51 } else { 52 flushBatch = k.syncCheckpointer(topic, partition) 53 } 54 55 var latestOffset int64 56 57 partMsgLoop: 58 for { 59 if nextTimedBatchChan == nil { 60 if tNext := batchPolicy.UntilNext(); tNext >= 0 { 61 nextTimedBatchChan = time.After(tNext) 62 } 63 } 64 select { 65 case <-nextTimedBatchChan: 66 nextTimedBatchChan = nil 67 if !flushBatch(ctx, k.msgChan, batchPolicy.Flush(), latestOffset+1) { 68 break partMsgLoop 69 } 70 case data, open := <-consumer.Messages(): 71 if !open { 72 break partMsgLoop 73 } 74 k.log.Tracef("Received message from topic %v partition %v\n", topic, partition) 75 76 latestOffset = data.Offset 77 part := dataToPart(consumer.HighWaterMarkOffset(), data) 78 79 if batchPolicy.Add(part) { 80 nextTimedBatchChan = nil 81 if !flushBatch(ctx, k.msgChan, batchPolicy.Flush(), latestOffset+1) { 82 break partMsgLoop 83 } 84 } 85 case err, open := <-consumer.Errors(): 86 if !open { 87 break partMsgLoop 88 } 89 if err != nil && !strings.HasSuffix(err.Error(), "EOF") { 90 k.log.Errorf("Kafka message recv error: %v\n", err) 91 } 92 case <-ctx.Done(): 93 break partMsgLoop 94 } 95 } 96 // Drain everything that's left. 97 for range consumer.Messages() { 98 } 99 for range consumer.Errors() { 100 } 101 } 102 103 func (k *kafkaReader) offsetVersion() int16 { 104 // - 0 (kafka 0.8.1 and later) 105 // - 1 (kafka 0.8.2 and later) 106 // - 2 (kafka 0.9.0 and later) 107 // - 3 (kafka 0.11.0 and later) 108 // - 4 (kafka 2.0.0 and later) 109 var v int16 = 1 110 // TODO: Increase this if we drop support for v0.8.2, or if we allow a 111 // custom retention period. 112 return v 113 } 114 115 func (k *kafkaReader) offsetPartitionPutRequest(consumerGroup string) *sarama.OffsetCommitRequest { 116 v := k.offsetVersion() 117 req := &sarama.OffsetCommitRequest{ 118 ConsumerGroup: consumerGroup, 119 Version: v, 120 ConsumerGroupGeneration: sarama.GroupGenerationUndefined, 121 ConsumerID: "", 122 } 123 return req 124 } 125 126 func (k *kafkaReader) connectExplicitTopics(ctx context.Context, config *sarama.Config) error { 127 var coordinator *sarama.Broker 128 var consumer sarama.Consumer 129 var client sarama.Client 130 var err error 131 132 defer func() { 133 if err != nil { 134 if consumer != nil { 135 consumer.Close() 136 } 137 if coordinator != nil { 138 coordinator.Close() 139 } 140 if client != nil { 141 client.Close() 142 } 143 } 144 }() 145 146 if client, err = sarama.NewClient(k.addresses, config); err != nil { 147 return err 148 } 149 if len(k.conf.ConsumerGroup) > 0 { 150 if coordinator, err = client.Coordinator(k.conf.ConsumerGroup); err != nil { 151 return err 152 } 153 } 154 if consumer, err = sarama.NewConsumerFromClient(client); err != nil { 155 return err 156 } 157 158 offsetGetReq := sarama.OffsetFetchRequest{ 159 Version: k.offsetVersion(), 160 ConsumerGroup: k.conf.ConsumerGroup, 161 } 162 for topic, parts := range k.topicPartitions { 163 for _, part := range parts { 164 offsetGetReq.AddPartition(topic, part) 165 } 166 } 167 168 var offsetRes *sarama.OffsetFetchResponse 169 if coordinator != nil { 170 if offsetRes, err = coordinator.FetchOffset(&offsetGetReq); err != nil { 171 if errors.Is(err, io.EOF) { 172 offsetRes = &sarama.OffsetFetchResponse{} 173 } else { 174 return fmt.Errorf("failed to acquire offsets from broker: %v", err) 175 } 176 } 177 } else { 178 offsetRes = &sarama.OffsetFetchResponse{} 179 } 180 181 offsetPutReq := k.offsetPartitionPutRequest(k.conf.ConsumerGroup) 182 offsetTracker := &closureOffsetTracker{ 183 // Note: We don't need to wrap this call in a mutex lock because the 184 // checkpointer that uses it already does this, but it's not 185 // particularly clear, hence this comment. 186 fn: func(topic string, partition int32, offset int64, metadata string) { 187 offsetPutReq.AddBlock(topic, partition, offset, time.Now().Unix(), metadata) 188 }, 189 } 190 191 partConsumers := []sarama.PartitionConsumer{} 192 consumerWG := sync.WaitGroup{} 193 msgChan := make(chan asyncMessage) 194 ctx, doneFn := context.WithCancel(context.Background()) 195 196 for topic, partitions := range k.topicPartitions { 197 for _, partition := range partitions { 198 offset := sarama.OffsetNewest 199 if k.conf.StartFromOldest { 200 offset = sarama.OffsetOldest 201 } 202 if block := offsetRes.GetBlock(topic, partition); block != nil { 203 if block.Err == sarama.ErrNoError { 204 if block.Offset > 0 { 205 offset = block.Offset 206 } 207 } else { 208 k.log.Debugf("Failed to acquire offset for topic %v partition %v: %v\n", topic, partition, block.Err) 209 } 210 } else { 211 k.log.Debugf("Failed to acquire offset for topic %v partition %v\n", topic, partition) 212 } 213 214 var partConsumer sarama.PartitionConsumer 215 if partConsumer, err = consumer.ConsumePartition(topic, partition, offset); err != nil { 216 // TODO: Actually verify the error was caused by a non-existent offset 217 if k.conf.StartFromOldest { 218 offset = sarama.OffsetOldest 219 k.log.Warnf("Failed to read from stored offset, restarting from oldest offset: %v\n", err) 220 } else { 221 offset = sarama.OffsetNewest 222 k.log.Warnf("Failed to read from stored offset, restarting from newest offset: %v\n", err) 223 } 224 if partConsumer, err = consumer.ConsumePartition(topic, partition, offset); err != nil { 225 doneFn() 226 return fmt.Errorf("failed to consume topic %v partition %v: %v", topic, partition, err) 227 } 228 } 229 230 consumerWG.Add(1) 231 partConsumers = append(partConsumers, partConsumer) 232 go k.runPartitionConsumer(ctx, &consumerWG, topic, partition, partConsumer) 233 } 234 235 k.log.Infof("Consuming kafka topic %v, partitions %v from brokers %s as group '%v'\n", topic, partitions, k.addresses, k.conf.ConsumerGroup) 236 } 237 238 doneCtx, doneFn := context.WithCancel(context.Background()) 239 go func() { 240 defer doneFn() 241 looping := true 242 for looping { 243 select { 244 case <-ctx.Done(): 245 looping = false 246 case <-time.After(k.commitPeriod): 247 } 248 k.cMut.Lock() 249 putReq := offsetPutReq 250 offsetPutReq = k.offsetPartitionPutRequest(k.conf.ConsumerGroup) 251 k.cMut.Unlock() 252 if coordinator != nil { 253 if _, err := coordinator.CommitOffset(putReq); err != nil { 254 k.log.Errorf("Failed to commit offsets: %v\n", err) 255 } 256 } 257 } 258 for _, consumer := range partConsumers { 259 consumer.AsyncClose() 260 } 261 consumerWG.Done() 262 263 k.cMut.Lock() 264 if k.msgChan != nil { 265 close(k.msgChan) 266 k.msgChan = nil 267 } 268 k.cMut.Unlock() 269 270 if coordinator != nil { 271 coordinator.Close() 272 } 273 client.Close() 274 }() 275 276 k.consumerCloseFn = doneFn 277 k.consumerDoneCtx = doneCtx 278 k.session = offsetTracker 279 k.msgChan = msgChan 280 return nil 281 }