-
Notifications
You must be signed in to change notification settings - Fork 13
/
batch_consumer.go
321 lines (309 loc) · 11.6 KB
/
batch_consumer.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
package app
import (
"context"
"fmt"
"github.com/confluentinc/confluent-kafka-go/v2/kafka"
"github.com/jitsucom/bulker/bulkerapp/metrics"
bulker "github.com/jitsucom/bulker/bulkerlib"
"github.com/jitsucom/bulker/bulkerlib/types"
"github.com/jitsucom/bulker/eventslog"
"github.com/jitsucom/bulker/jitsubase/jsonorder"
"github.com/jitsucom/bulker/jitsubase/timestamp"
"github.com/jitsucom/bulker/jitsubase/utils"
"github.com/jitsucom/bulker/kafkabase"
"strconv"
"time"
)
type BatchConsumerImpl struct {
*AbstractBatchConsumer
eventsLogService eventslog.EventsLogService
}
func NewBatchConsumer(repository *Repository, destinationId string, batchPeriodSec int, topicId string, config *Config, kafkaConfig *kafka.ConfigMap, bulkerProducer *Producer, eventsLogService eventslog.EventsLogService) (*BatchConsumerImpl, error) {
base, err := NewAbstractBatchConsumer(repository, destinationId, batchPeriodSec, topicId, "batch", config, kafkaConfig, bulkerProducer)
if err != nil {
return nil, err
}
bc := BatchConsumerImpl{
AbstractBatchConsumer: base,
eventsLogService: eventsLogService,
}
bc.batchFunc = bc.processBatchImpl
bc.pause(false)
return &bc, nil
}
func (bc *BatchConsumerImpl) processBatchImpl(destination *Destination, batchNum, batchSize, retryBatchSize int, highOffset int64) (counters BatchCounters, nextBatch bool, err error) {
bc.Debugf("Starting batch #%d", batchNum)
counters.firstOffset = int64(kafka.OffsetBeginning)
startTime := time.Now()
var bulkerStream bulker.BulkerStream
ctx := context.WithValue(context.Background(), bulker.BatchNumberCtxKey, batchNum)
//position of last message in batch in case of failed. Needed for processFailed
var failedPosition *kafka.TopicPartition
var firstPosition *kafka.TopicPartition
var latestMessage *kafka.Message
defer func() {
if err != nil {
nextBatch = false
counters.failed = counters.consumed - counters.processed
if counters.failed > 0 {
// we separate original errors from retry errors
bc.SendMetrics(kafkabase.GetKafkaHeader(latestMessage, MetricsMetaHeader), "error", counters.failed-counters.retried)
bc.SendMetrics(kafkabase.GetKafkaHeader(latestMessage, MetricsMetaHeader), "retry_error", counters.retried)
}
if failedPosition != nil {
cnts, err2 := bc.processFailed(firstPosition, failedPosition, err)
counters.deadLettered = cnts.deadLettered
counters.retryScheduled = cnts.retryScheduled
if err2 != nil {
bc.errorMetric("PROCESS_FAILED_ERROR")
bc.SystemErrorf(err2.Error())
} else if counters.failed > 1 && int64(latestMessage.TopicPartition.Offset) < highOffset-1 {
// if we fail right on the first message - that probably means connection problems. No need to move further.
// otherwise we can try to consume next batch
nextBatch = true
}
}
} else if counters.processed > 0 {
bc.SendMetrics(kafkabase.GetKafkaHeader(latestMessage, MetricsMetaHeader), "success", counters.processed)
}
}()
var processedObjectSample types.Object
processed := 0
for i := 0; i < batchSize; i++ {
if bc.retired.Load() {
if bulkerStream != nil {
_, _ = bulkerStream.Abort(ctx)
}
return
}
if latestMessage != nil && int64(latestMessage.TopicPartition.Offset) >= highOffset-1 {
nextBatch = false
bc.Debugf("Reached watermark offset %d. Stopping batch", highOffset-1)
// we reached the end of the topic
break
}
message, err := bc.consumer.Load().ReadMessage(bc.waitForMessages)
if err != nil {
kafkaErr := err.(kafka.Error)
if kafkaErr.Code() == kafka.ErrTimedOut {
// waitForMessages period is over. it's ok. considering batch as full
break
}
bc.errorMetric("consumer_error:" + metrics.KafkaErrorCode(kafkaErr))
if bulkerStream != nil {
_, _ = bulkerStream.Abort(ctx)
}
return counters, false, bc.NewError("Failed to consume event from topic. Retryable: %t: %v", kafkaErr.IsRetriable(), kafkaErr)
}
counters.consumed++
retriesHeader := kafkabase.GetKafkaHeader(message, retriesCountHeader)
if retriesHeader != "" {
// we perform retries in smaller batches
//batchSize = retryBatchSize
counters.retried++
}
latestMessage = message
if firstPosition == nil {
firstPosition = &message.TopicPartition
counters.firstOffset = int64(message.TopicPartition.Offset)
}
var obj types.Object
err = jsonorder.Unmarshal(message.Value, &obj)
if err == nil {
if bulkerStream == nil {
destination.InitBulkerInstance()
bulkerStream, err = destination.bulker.CreateStream(bc.topicId, bc.tableName, bulker.Batch, destination.streamOptions.Options...)
if err != nil {
bc.errorMetric("failed to create bulker stream")
err = bc.NewError("Failed to create bulker stream: %v", err)
}
}
if err == nil {
//bc.Debugf("%d. Consumed Message ID: %s Offset: %s (Retries: %s) for: %s", i, obj.Id(), message.TopicPartition.Offset.String(), kafkabase.GetKafkaHeader(message, retriesCountHeader), destination.config.BulkerType)
_, processedObjectSample, err = bulkerStream.Consume(ctx, obj)
if err != nil {
bc.errorMetric("bulker_stream_error")
}
}
} else {
bc.errorMetric("parse_event_error")
}
if err != nil {
failedPosition = &latestMessage.TopicPartition
state := bulker.State{}
if bulkerStream != nil {
state, _ = bulkerStream.Abort(ctx)
}
//treat failed message as processed
state.ProcessedRows++
state.ProcessingTimeSec = time.Since(startTime).Seconds()
bc.postEventsLog(state, processedObjectSample, err)
return counters, false, bc.NewError("Failed to process event to bulker stream: %v", err)
} else {
processed++
}
}
//we've processed some messages. it is time to commit them
if processed > 0 {
if processed == batchSize {
nextBatch = true
}
pauseTimer := time.AfterFunc(time.Duration(bc.config.KafkaMaxPollIntervalMs)*time.Millisecond/2, func() {
// we need to pause consumer to avoid kafka session timeout while loading huge batches to slow destinations
bc.pause(true)
})
bc.Infof("Batch #%d Committing %d events to %s", batchNum, processed, destination.config.BulkerType)
var state bulker.State
//TODO: do we need to interrupt commit if consumer is retired?
state, err = bulkerStream.Complete(ctx)
pauseTimer.Stop()
state.ProcessingTimeSec = time.Since(startTime).Seconds()
bc.postEventsLog(state, processedObjectSample, err)
if err != nil {
failedPosition = &latestMessage.TopicPartition
return counters, false, bc.NewError("Failed to commit bulker stream to %s: %v", destination.config.BulkerType, err)
}
counters.processed = processed
_, err = bc.consumer.Load().CommitMessage(latestMessage)
if err != nil {
bc.errorMetric("KAFKA_COMMIT_ERR:" + metrics.KafkaErrorCode(err))
bc.SystemErrorf("Failed to commit kafka consumer after batch was successfully committed to the destination: %v", err)
err = bc.NewError("Failed to commit kafka consumer: %v", err)
return
}
} else if bulkerStream != nil {
_, _ = bulkerStream.Abort(ctx)
}
return
}
// processFailed consumes the latest failed batch of messages and sends them to the 'failed' topic
func (bc *BatchConsumerImpl) processFailed(firstPosition *kafka.TopicPartition, failedPosition *kafka.TopicPartition, originalErr error) (counters BatchCounters, err error) {
var producer *kafka.Producer
defer func() {
//recover
if r := recover(); r != nil {
err = bc.NewError("Recovered from panic: %v", r)
bc.SystemErrorf("Recovered from panic: %v", r)
}
if err != nil {
err = bc.NewError("Failed to put unsuccessful batch to 'failed' producer: %v", err)
}
if producer != nil {
producer.Close()
}
}()
producer, err = bc.initTransactionalProducer()
if err != nil {
return
}
bc.resume()
bc.Infof("Rolling back to first offset %d (failed at %d)", firstPosition.Offset, failedPosition.Offset)
//Rollback consumer to committed offset
_, err = bc.consumer.Load().SeekPartitions([]kafka.TopicPartition{*firstPosition})
if err != nil {
bc.errorMetric("SEEK_ERROR")
return BatchCounters{}, fmt.Errorf("failed to rollback kafka consumer offset: %v", err)
}
err = producer.BeginTransaction()
if err != nil {
return BatchCounters{}, fmt.Errorf("failed to begin kafka transaction: %v", err)
}
defer func() {
if err != nil {
//cleanup
_ = producer.AbortTransaction(context.Background())
_, err2 := bc.consumer.Load().SeekPartitions([]kafka.TopicPartition{*firstPosition})
if err2 != nil {
bc.errorMetric("SEEK_ERROR")
}
}
}()
for {
var message *kafka.Message
message, err = bc.consumer.Load().ReadMessage(bc.waitForMessages)
if err != nil {
kafkaErr := err.(kafka.Error)
if kafkaErr.Code() == kafka.ErrTimedOut {
err = fmt.Errorf("failed to consume message: %v", err)
return
}
if kafkaErr.IsRetriable() {
time.Sleep(10 * time.Second)
continue
} else {
bc.restartConsumer()
err = fmt.Errorf("failed to consume message: %v", err)
return
}
}
counters.consumed++
deadLettered := false
failedTopic, _ := MakeTopicId(bc.destinationId, retryTopicMode, allTablesToken, false)
retries, err := kafkabase.GetKafkaIntHeader(message, retriesCountHeader)
if err != nil {
bc.Errorf("failed to read retry header: %v", err)
}
if retries >= bc.config.MessagesRetryCount {
//no attempts left - send to dead-letter topic
deadLettered = true
failedTopic, _ = MakeTopicId(bc.destinationId, deadTopicMode, allTablesToken, false)
}
headers := message.Headers
kafkabase.PutKafkaHeader(&headers, errorHeader, utils.ShortenStringWithEllipsis(originalErr.Error(), 256))
kafkabase.PutKafkaHeader(&headers, originalTopicHeader, bc.topicId)
kafkabase.PutKafkaHeader(&headers, retriesCountHeader, strconv.Itoa(retries))
kafkabase.PutKafkaHeader(&headers, retryTimeHeader, timestamp.ToISOFormat(RetryBackOffTime(bc.config, retries+1).UTC()))
err = producer.Produce(&kafka.Message{
Key: message.Key,
TopicPartition: kafka.TopicPartition{Topic: &failedTopic, Partition: kafka.PartitionAny},
Headers: headers,
Value: message.Value,
}, nil)
if err != nil {
return counters, fmt.Errorf("failed to put message to producer: %v", err)
}
if deadLettered {
counters.deadLettered++
} else {
counters.retryScheduled++
}
//stop consuming on the latest message before failure
if message.TopicPartition.Offset == failedPosition.Offset {
break
}
}
groupMetadata, err := bc.consumer.Load().GetConsumerGroupMetadata()
if err != nil {
err = fmt.Errorf("failed to get consumer group metadata: %v", err)
return
}
offset := *failedPosition
offset.Offset++
//set consumer offset to the next message after failure. that happens atomically with whole producer transaction
err = producer.SendOffsetsToTransaction(context.Background(), []kafka.TopicPartition{offset}, groupMetadata)
if err != nil {
err = fmt.Errorf("failed to send consumer offset to producer transaction: %v", err)
return
}
err = producer.CommitTransaction(context.Background())
if err != nil {
err = fmt.Errorf("failed to commit kafka transaction for producer: %v", err)
return
}
return
}
func (bc *BatchConsumerImpl) postEventsLog(state bulker.State, processedObjectSample types.Object, batchErr error) {
if batchErr != nil && state.LastError == nil {
state.SetError(batchErr)
}
batchState := BatchState{State: state, LastMappedRow: processedObjectSample}
level := eventslog.LevelInfo
if batchErr != nil {
level = eventslog.LevelError
}
bc.eventsLogService.PostAsync(&eventslog.ActorEvent{EventType: eventslog.EventTypeBatch, Level: level, ActorId: bc.destinationId, Event: batchState})
}
type BatchState struct {
bulker.State `json:",inline"`
LastMappedRow types.Object `json:"lastMappedRow"`
}