-
Notifications
You must be signed in to change notification settings - Fork 328
/
TopicPartitionWriter.java
745 lines (679 loc) · 25.7 KB
/
TopicPartitionWriter.java
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
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"); you may not use
* this file except in compliance with the License. You may obtain a copy of the
* License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package io.confluent.connect.s3;
import com.amazonaws.SdkClientException;
import io.confluent.connect.s3.storage.S3Storage;
import io.confluent.connect.s3.util.FileRotationTracker;
import io.confluent.connect.s3.util.RetryUtil;
import io.confluent.connect.s3.util.TombstoneTimestampExtractor;
import io.confluent.connect.storage.errors.PartitionException;
import io.confluent.connect.storage.schema.SchemaCompatibilityResult;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.DataException;
import org.apache.kafka.connect.errors.IllegalWorkerStateException;
import org.apache.kafka.connect.errors.SchemaProjectorException;
import org.apache.kafka.connect.sink.ErrantRecordReporter;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTaskContext;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.avro.SchemaParseException;
import org.apache.parquet.schema.InvalidSchemaException;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Queue;
import io.confluent.common.utils.SystemTime;
import io.confluent.common.utils.Time;
import io.confluent.connect.storage.StorageSinkConnectorConfig;
import io.confluent.connect.storage.common.StorageCommonConfig;
import io.confluent.connect.storage.common.util.StringUtils;
import io.confluent.connect.storage.format.RecordWriter;
import io.confluent.connect.storage.format.RecordWriterProvider;
import io.confluent.connect.storage.partitioner.Partitioner;
import io.confluent.connect.storage.partitioner.PartitionerConfig;
import io.confluent.connect.storage.partitioner.TimeBasedPartitioner;
import io.confluent.connect.storage.partitioner.TimestampExtractor;
import io.confluent.connect.storage.schema.StorageSchemaCompatibility;
import io.confluent.connect.storage.util.DateTimeUtils;
import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_PART_RETRIES_CONFIG;
import static io.confluent.connect.s3.S3SinkConnectorConfig.S3_RETRY_BACKOFF_CONFIG;
public class TopicPartitionWriter {
private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class);
private final Map<String, String> commitFiles;
private final Map<String, RecordWriter> writers;
private final Map<String, Schema> currentSchemas;
private final TopicPartition tp;
private final S3Storage storage;
private final Partitioner<?> partitioner;
private TimestampExtractor timestampExtractor;
private String topicsDir;
private State state;
private final Queue<SinkRecord> buffer;
private final SinkTaskContext context;
private final boolean isTaggingEnabled;
private final List<String> extraTagKeyValuePair;
private HashMap<String, String> hashMapTag;
private final boolean ignoreTaggingErrors;
private int recordCount;
private final int flushSize;
private final long rotateIntervalMs;
private final long rotateScheduleIntervalMs;
private long nextScheduledRotation;
private long currentOffset;
private Long currentTimestamp;
private String currentEncodedPartition;
private Long baseRecordTimestamp;
private Long offsetToCommit;
private final RecordWriterProvider<S3SinkConnectorConfig> writerProvider;
private final Map<String, Long> startOffsets;
private final Map<String, Long> endOffsets;
private final Map<String, Long> recordCounts;
private long timeoutMs;
private long failureTime;
private final StorageSchemaCompatibility compatibility;
private final String extension;
private final String zeroPadOffsetFormat;
private final String dirDelim;
private final String fileDelim;
private final Time time;
private DateTimeZone timeZone;
private final S3SinkConnectorConfig connectorConfig;
private static final Time SYSTEM_TIME = new SystemTime();
private ErrantRecordReporter reporter;
private final FileRotationTracker fileRotationTracker;
public TopicPartitionWriter(TopicPartition tp,
S3Storage storage,
RecordWriterProvider<S3SinkConnectorConfig> writerProvider,
Partitioner<?> partitioner,
S3SinkConnectorConfig connectorConfig,
SinkTaskContext context,
ErrantRecordReporter reporter) {
this(tp, storage, writerProvider, partitioner, connectorConfig, context, SYSTEM_TIME, reporter);
}
// Visible for testing
TopicPartitionWriter(TopicPartition tp,
S3Storage storage,
RecordWriterProvider<S3SinkConnectorConfig> writerProvider,
Partitioner<?> partitioner,
S3SinkConnectorConfig connectorConfig,
SinkTaskContext context,
Time time,
ErrantRecordReporter reporter
) {
this.connectorConfig = connectorConfig;
this.time = time;
this.tp = tp;
this.storage = storage;
this.context = context;
this.writerProvider = writerProvider;
this.partitioner = partitioner;
this.reporter = reporter;
this.timestampExtractor = null;
if (partitioner instanceof TimeBasedPartitioner) {
this.timestampExtractor = ((TimeBasedPartitioner) partitioner).getTimestampExtractor();
if (connectorConfig.isTombstoneWriteEnabled()) {
this.timestampExtractor = new TombstoneTimestampExtractor(timestampExtractor);
}
}
isTaggingEnabled = connectorConfig.getBoolean(S3SinkConnectorConfig.S3_OBJECT_TAGGING_CONFIG);
extraTagKeyValuePair =
connectorConfig.getList(S3SinkConnectorConfig.S3_OBJECT_TAGGING_EXTRA_KV);
getS3Tag();
ignoreTaggingErrors = connectorConfig.getString(
S3SinkConnectorConfig.S3_OBJECT_BEHAVIOR_ON_TAGGING_ERROR_CONFIG)
.equalsIgnoreCase(S3SinkConnectorConfig.IgnoreOrFailBehavior.IGNORE.toString());
flushSize = connectorConfig.getInt(S3SinkConnectorConfig.FLUSH_SIZE_CONFIG);
topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG);
rotateIntervalMs = connectorConfig.getLong(S3SinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG);
if (rotateIntervalMs > 0 && timestampExtractor == null) {
log.warn(
"Property '{}' is set to '{}ms' but partitioner is not an instance of '{}'. This property"
+ " is ignored.",
S3SinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG,
rotateIntervalMs,
TimeBasedPartitioner.class.getName()
);
}
rotateScheduleIntervalMs =
connectorConfig.getLong(S3SinkConnectorConfig.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG);
if (rotateScheduleIntervalMs > 0) {
timeZone = DateTimeZone.forID(connectorConfig.getString(PartitionerConfig.TIMEZONE_CONFIG));
}
timeoutMs = connectorConfig.getLong(S3SinkConnectorConfig.RETRY_BACKOFF_CONFIG);
compatibility = StorageSchemaCompatibility.getCompatibility(
connectorConfig.getString(StorageSinkConnectorConfig.SCHEMA_COMPATIBILITY_CONFIG));
buffer = new LinkedList<>();
commitFiles = new HashMap<>();
writers = new HashMap<>();
currentSchemas = new HashMap<>();
startOffsets = new HashMap<>();
endOffsets = new HashMap<>();
recordCounts = new HashMap<>();
state = State.WRITE_STARTED;
failureTime = -1L;
currentOffset = -1L;
dirDelim = connectorConfig.getString(StorageCommonConfig.DIRECTORY_DELIM_CONFIG);
fileDelim = connectorConfig.getString(StorageCommonConfig.FILE_DELIM_CONFIG);
extension = writerProvider.getExtension();
zeroPadOffsetFormat = "%0"
+ connectorConfig.getInt(S3SinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG)
+ "d";
fileRotationTracker = new FileRotationTracker();
// Initialize scheduled rotation timer if applicable
setNextScheduledRotation();
}
private void getS3Tag() {
hashMapTag = new HashMap<>();
if (extraTagKeyValuePair.size() != 0) {
for (int i = 0; i < extraTagKeyValuePair.size(); i++) {
String[] singleKv = extraTagKeyValuePair.get(i).split(":");
hashMapTag.put(singleKv[0], singleKv[1]);
}
}
}
private enum State {
WRITE_STARTED,
WRITE_PARTITION_PAUSED,
SHOULD_ROTATE,
FILE_COMMITTED;
private static final State[] VALS = values();
public State next() {
return VALS[(ordinal() + 1) % VALS.length];
}
}
public void write() {
long now = time.milliseconds();
if (failureTime > 0 && now - failureTime < timeoutMs) {
return;
} else {
failureTime = -1;
}
resetExpiredScheduledRotationIfNoPendingRecords(now);
while (!buffer.isEmpty()) {
try {
executeState(now);
} catch (IllegalWorkerStateException e) {
throw new ConnectException(e);
} catch (SchemaProjectorException e) {
if (reporter != null) {
reporter.report(buffer.poll(), e);
log.warn("Errant record written to DLQ due to: {}", e.getMessage());
} else {
throw e;
}
}
}
commitOnTimeIfNoData(now);
}
@SuppressWarnings("fallthrough")
private void executeState(long now) {
switch (state) {
case WRITE_STARTED:
pause();
nextState();
// fallthrough
case WRITE_PARTITION_PAUSED:
SinkRecord record = buffer.peek();
if (timestampExtractor != null) {
currentTimestamp = timestampExtractor.extract(record, now);
if (baseRecordTimestamp == null) {
baseRecordTimestamp = currentTimestamp;
}
}
Schema valueSchema = record.valueSchema();
String encodedPartition;
try {
encodedPartition = partitioner.encodePartition(record, now);
} catch (PartitionException e) {
if (reporter != null) {
reporter.report(record, e);
buffer.poll();
break;
} else {
throw e;
}
}
Schema currentValueSchema = currentSchemas.get(encodedPartition);
if (currentValueSchema == null) {
currentSchemas.put(encodedPartition, valueSchema);
currentValueSchema = valueSchema;
}
if (!checkRotationOrAppend(
record,
currentValueSchema,
valueSchema,
encodedPartition,
now
)) {
break;
}
// fallthrough
case SHOULD_ROTATE:
commitFiles();
nextState();
// fallthrough
case FILE_COMMITTED:
setState(State.WRITE_PARTITION_PAUSED);
break;
default:
log.error("{} is not a valid state to write record for topic partition {}.", state, tp);
}
}
/**
* Check if we should rotate the file (schema change, time-based).
* @returns true if rotation is being performed, false otherwise
*/
private boolean checkRotationOrAppend(
SinkRecord record,
Schema currentValueSchema,
Schema valueSchema,
String encodedPartition,
long now
) {
// rotateOnTime is safe to go before writeRecord, because it is acceptable
// even for a faulty record to trigger time-based rotation if it applies
if (rotateOnTime(encodedPartition, currentTimestamp, now)) {
setNextScheduledRotation();
nextState();
return true;
}
SchemaCompatibilityResult shouldChangeSchema =
compatibility.shouldChangeSchema(record, null, currentValueSchema);
if (shouldChangeSchema.isInCompatible() && recordCount > 0) {
fileRotationTracker.incrementRotationBySchemaChangeCount(encodedPartition,
shouldChangeSchema.getSchemaIncompatibilityType());
// This branch is never true for the first record read by this TopicPartitionWriter
log.trace(
"Incompatible change of schema detected for record '{}' with encoded partition "
+ "'{}' and current offset: '{}'",
record,
encodedPartition,
currentOffset
);
currentSchemas.put(encodedPartition, valueSchema);
nextState();
return true;
}
SinkRecord projectedRecord = compatibility.project(record, null, currentValueSchema);
boolean validRecord = writeRecord(projectedRecord, encodedPartition);
buffer.poll();
if (!validRecord) {
// skip the faulty record and don't rotate
return false;
}
if (rotateOnSize()) {
fileRotationTracker.incrementRotationByFlushSizeCount(encodedPartition);
log.info(
"Starting commit and rotation for topic partition {} with start offset {}",
tp,
startOffsets
);
nextState();
return true;
}
return false;
}
private void commitOnTimeIfNoData(long now) {
if (buffer.isEmpty()) {
// committing files after waiting for rotateIntervalMs time but less than flush.size
// records available
if (recordCount > 0 && rotateOnTime(currentEncodedPartition, currentTimestamp, now)) {
log.info(
"Committing files after waiting for rotateIntervalMs time but less than flush.size "
+ "records available."
);
setNextScheduledRotation();
commitFiles();
}
resume();
setState(State.WRITE_STARTED);
}
}
private void resetExpiredScheduledRotationIfNoPendingRecords(long now) {
if (recordCount == 0 && shouldApplyScheduledRotation(now)) {
setNextScheduledRotation();
}
}
public void close() throws ConnectException {
log.debug("Closing TopicPartitionWriter {}", tp);
for (RecordWriter writer : writers.values()) {
writer.close();
}
writers.clear();
startOffsets.clear();
}
public void buffer(SinkRecord sinkRecord) {
buffer.add(sinkRecord);
}
public Long getOffsetToCommitAndReset() {
Long latest = offsetToCommit;
offsetToCommit = null;
return latest;
}
public Long currentStartOffset() {
return minStartOffset();
}
public void failureTime(long when) {
this.failureTime = when;
}
public FileRotationTracker getFileRotationTracker() {
return fileRotationTracker;
}
public StorageSchemaCompatibility getSchemaCompatibility() {
return compatibility;
}
private Long minStartOffset() {
Optional<Long> minStartOffset = startOffsets.values().stream()
.min(Comparator.comparing(Long::valueOf));
return minStartOffset.isPresent() ? minStartOffset.get() : null;
}
private String getDirectoryPrefix(String encodedPartition) {
return partitioner.generatePartitionedPath(tp.topic(), encodedPartition);
}
private void nextState() {
state = state.next();
}
private void setState(State state) {
this.state = state;
}
private boolean rotateOnTime(String encodedPartition, Long recordTimestamp, long now) {
if (recordCount <= 0) {
return false;
}
// rotateIntervalMs > 0 implies timestampExtractor != null
boolean periodicRotation = rotateIntervalMs > 0
&& timestampExtractor != null
&& (
recordTimestamp - baseRecordTimestamp >= rotateIntervalMs
|| !encodedPartition.equals(currentEncodedPartition)
);
log.trace(
"Checking rotation on time for topic-partition '{}' "
+ "with recordCount '{}' and encodedPartition '{}'",
tp,
recordCount,
encodedPartition
);
log.trace(
"Should apply periodic time-based rotation for topic-partition '{}':"
+ " (rotateIntervalMs: '{}', baseRecordTimestamp: "
+ "'{}', timestamp: '{}', encodedPartition: '{}', currentEncodedPartition: '{}')? {}",
tp,
rotateIntervalMs,
baseRecordTimestamp,
recordTimestamp,
encodedPartition,
currentEncodedPartition,
periodicRotation
);
if (periodicRotation) {
fileRotationTracker.incrementRotationByRotationIntervalCount(encodedPartition);
} else if (shouldApplyScheduledRotation(now)) {
fileRotationTracker.incrementRotationByScheduledRotationIntervalCount(encodedPartition);
}
return periodicRotation || shouldApplyScheduledRotation(now);
}
private boolean shouldApplyScheduledRotation(long now) {
boolean scheduledRotation = rotateScheduleIntervalMs > 0 && now >= nextScheduledRotation;
log.trace(
"Should apply scheduled rotation for topic-partition '{}':"
+ " (rotateScheduleIntervalMs: '{}', nextScheduledRotation:"
+ " '{}', now: '{}')? {}",
tp,
rotateScheduleIntervalMs,
nextScheduledRotation,
now,
scheduledRotation
);
return scheduledRotation;
}
private void setNextScheduledRotation() {
if (rotateScheduleIntervalMs > 0) {
long now = time.milliseconds();
nextScheduledRotation = DateTimeUtils.getNextTimeAdjustedByDay(
now,
rotateScheduleIntervalMs,
timeZone
);
if (log.isDebugEnabled()) {
log.debug(
"Update scheduled rotation timer for topic-partition '{}': "
+ "(rotateScheduleIntervalMs: '{}', nextScheduledRotation: '{}', now: '{}'). "
+ "Next rotation will be at {}",
tp,
rotateScheduleIntervalMs,
nextScheduledRotation,
now,
new DateTime(nextScheduledRotation).withZone(timeZone).toString()
);
}
}
}
private boolean rotateOnSize() {
boolean messageSizeRotation = recordCount >= flushSize;
log.trace("Should apply size-based rotation for topic-partition '{}':"
+ " (count {} >= flush size {})? {}",
tp,
recordCount,
flushSize,
messageSizeRotation
);
return messageSizeRotation;
}
private void pause() {
log.trace("Pausing writer for topic-partition '{}'", tp);
context.pause(tp);
}
private void resume() {
log.trace("Resuming writer for topic-partition '{}'", tp);
context.resume(tp);
}
private RecordWriter newWriter(SinkRecord record, String encodedPartition)
throws ConnectException {
String commitFilename = getCommitFilename(encodedPartition);
log.debug(
"Creating new writer encodedPartition='{}' filename='{}'",
encodedPartition,
commitFilename
);
RecordWriter writer = writerProvider.getRecordWriter(connectorConfig, commitFilename);
writers.put(encodedPartition, writer);
return writer;
}
private String getCommitFilename(String encodedPartition) {
String commitFile;
if (commitFiles.containsKey(encodedPartition)) {
commitFile = commitFiles.get(encodedPartition);
} else {
long startOffset = startOffsets.get(encodedPartition);
String prefix = getDirectoryPrefix(encodedPartition);
commitFile = fileKeyToCommit(prefix, startOffset);
commitFiles.put(encodedPartition, commitFile);
}
return commitFile;
}
private String fileKey(String topicsPrefix, String keyPrefix, String name) {
String suffix = keyPrefix + dirDelim + name;
return StringUtils.isNotBlank(topicsPrefix)
? topicsPrefix + dirDelim + suffix
: suffix;
}
private String fileKeyToCommit(String dirPrefix, long startOffset) {
String name = tp.topic()
+ fileDelim
+ tp.partition()
+ fileDelim
+ String.format(zeroPadOffsetFormat, startOffset)
+ extension;
return fileKey(topicsDir, dirPrefix, name);
}
private boolean writeRecord(SinkRecord record, String encodedPartition) {
RecordWriter writer = writers.get(encodedPartition);
long currentOffsetIfSuccessful = record.kafkaOffset();
boolean shouldRemoveWriter = false;
boolean shouldRemoveStartOffset = false;
boolean shouldRemoveCommitFilename = false;
try {
if (!startOffsets.containsKey(encodedPartition)) {
log.trace(
"Setting writer's start offset for '{}' to {}",
encodedPartition,
currentOffsetIfSuccessful
);
startOffsets.put(encodedPartition, currentOffsetIfSuccessful);
shouldRemoveStartOffset = true;
}
if (writer == null) {
if (!commitFiles.containsKey(encodedPartition)) {
shouldRemoveCommitFilename = true;
}
writer = newWriter(record, encodedPartition);
shouldRemoveWriter = true;
}
writer.write(record);
} catch (DataException | SchemaParseException | InvalidSchemaException e) {
if (reporter != null) {
if (shouldRemoveStartOffset) {
startOffsets.remove(encodedPartition);
}
if (shouldRemoveWriter) {
writers.remove(encodedPartition);
}
if (shouldRemoveCommitFilename) {
commitFiles.remove(encodedPartition);
}
reporter.report(record, e);
log.warn("Errant record written to DLQ due to: {}", e.getMessage());
return false;
} else {
throw new ConnectException(e);
}
}
currentEncodedPartition = encodedPartition;
currentOffset = record.kafkaOffset();
if (shouldRemoveStartOffset) {
log.trace(
"Setting writer's start offset for '{}' to {}",
currentEncodedPartition,
currentOffset
);
// Once we have a "start offset" for a particular "encoded partition"
// value, we know that we have at least one record. This allows us
// to initialize all our maps at the same time, and saves future
// checks on the existence of keys
recordCounts.put(currentEncodedPartition, 0L);
endOffsets.put(currentEncodedPartition, 0L);
}
++recordCount;
recordCounts.put(currentEncodedPartition, recordCounts.get(currentEncodedPartition) + 1);
endOffsets.put(currentEncodedPartition, currentOffset);
return true;
}
private void commitFiles() {
for (Map.Entry<String, String> entry : commitFiles.entrySet()) {
String encodedPartition = entry.getKey();
commitFile(encodedPartition);
if (isTaggingEnabled) {
RetryUtil.exponentialBackoffRetry(() -> tagFile(
encodedPartition,
entry.getValue(),
hashMapTag
),
ConnectException.class,
connectorConfig.getInt(S3_PART_RETRIES_CONFIG),
connectorConfig.getLong(S3_RETRY_BACKOFF_CONFIG)
);
}
startOffsets.remove(encodedPartition);
endOffsets.remove(encodedPartition);
recordCounts.remove(encodedPartition);
log.debug("Committed {} for {}", entry.getValue(), tp);
}
offsetToCommit = currentOffset + 1;
commitFiles.clear();
currentSchemas.clear();
recordCount = 0;
baseRecordTimestamp = null;
log.info("Files committed to S3. Target commit offset for {} is {}", tp, offsetToCommit);
}
private void commitFile(String encodedPartition) {
if (!startOffsets.containsKey(encodedPartition)) {
log.warn("Tried to commit file with missing starting offset partition: {}. Ignoring.");
return;
}
if (writers.containsKey(encodedPartition)) {
RecordWriter writer = writers.get(encodedPartition);
// Commits the file and closes the underlying output stream.
writer.commit();
writers.remove(encodedPartition);
log.debug("Removed writer for '{}'", encodedPartition);
}
}
private void tagFile(
String encodedPartition,
String s3ObjectPath,
Map<String,String> extraHashMapTag) {
Long startOffset = startOffsets.get(encodedPartition);
Long endOffset = endOffsets.get(encodedPartition);
Long recordCount = recordCounts.get(encodedPartition);
if (startOffset == null || endOffset == null || recordCount == null) {
log.warn(
"Missing tags when attempting to tag file {}. "
+ "Starting offset tag: {}, "
+ "ending offset tag: {}, "
+ "record count tag: {}. Ignoring.",
encodedPartition,
startOffset == null ? "missing" : startOffset,
endOffset == null ? "missing" : endOffset,
recordCount == null ? "missing" : recordCount
);
return;
}
log.debug("Object to tag is: {}", s3ObjectPath);
Map<String, String> tags = new HashMap<>();
tags.put("startOffset", Long.toString(startOffset));
tags.put("endOffset", Long.toString(endOffset));
tags.put("recordCount", Long.toString(recordCount));
if (extraHashMapTag != null) {
tags.putAll(extraHashMapTag);
}
try {
storage.addTags(s3ObjectPath, tags);
log.info("Tagged S3 object {} with starting offset {}, ending offset {}, record count {}",
s3ObjectPath, startOffset, endOffset, recordCount);
} catch (SdkClientException e) {
if (ignoreTaggingErrors) {
log.warn("Unable to tag S3 object {}. Ignoring.", s3ObjectPath, e);
} else {
throw new ConnectException(String.format("Unable to tag S3 object %s", s3ObjectPath), e);
}
} catch (Exception e) {
if (ignoreTaggingErrors) {
log.warn("Unrecoverable exception while attempting to tag S3 object {}. Ignoring.",
s3ObjectPath, e);
} else {
throw new ConnectException(String.format("Unable to tag S3 object %s", s3ObjectPath), e);
}
}
}
}