-
Notifications
You must be signed in to change notification settings - Fork 397
/
TopicPartitionWriter.java
821 lines (758 loc) · 27 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
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
/**
* Copyright 2015 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License
* is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.hdfs;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
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.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 java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import io.confluent.common.utils.Time;
import io.confluent.connect.avro.AvroData;
import io.confluent.connect.hdfs.errors.HiveMetaStoreException;
import io.confluent.connect.hdfs.filter.CommittedFileFilter;
import io.confluent.connect.hdfs.filter.TopicPartitionCommittedFileFilter;
import io.confluent.connect.hdfs.hive.HiveMetaStore;
import io.confluent.connect.hdfs.hive.HiveUtil;
import io.confluent.connect.hdfs.partitioner.Partitioner;
import io.confluent.connect.hdfs.storage.HdfsStorage;
import io.confluent.connect.storage.common.StorageCommonConfig;
import io.confluent.connect.storage.hive.HiveConfig;
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.wal.WAL;
public class TopicPartitionWriter {
private static final Logger log = LoggerFactory.getLogger(TopicPartitionWriter.class);
private static final TimestampExtractor WALLCLOCK =
new TimeBasedPartitioner.WallclockTimestampExtractor();
private final io.confluent.connect.storage.format.RecordWriterProvider<HdfsSinkConnectorConfig>
newWriterProvider;
private final String zeroPadOffsetFormat;
private final boolean hiveIntegration;
private final Time time;
private final HdfsStorage storage;
private final WAL wal;
private final Map<String, String> tempFiles;
private final Map<String, io.confluent.connect.storage.format.RecordWriter> writers;
private final TopicPartition tp;
private final Partitioner partitioner;
private final TimestampExtractor timestampExtractor;
private final boolean isWallclockBased;
private final String url;
private final String topicsDir;
private State state;
private final Queue<SinkRecord> buffer;
private boolean recovered;
private final SinkTaskContext context;
private int recordCounter;
private final int flushSize;
private final long rotateIntervalMs;
private Long lastRotate;
private final long rotateScheduleIntervalMs;
private long nextScheduledRotate;
// This is one case where we cannot simply wrap the old or new RecordWriterProvider with the
// other because they have incompatible requirements for some methods -- one requires the Hadoop
// config + extra parameters, the other requires the ConnectorConfig and doesn't get the other
// extra parameters. Instead, we have to (optionally) store one of each and use whichever one is
// non-null.
private final RecordWriterProvider writerProvider;
private final HdfsSinkConnectorConfig connectorConfig;
private final AvroData avroData;
private final Set<String> appended;
private long offset;
private final Map<String, Long> startOffsets;
private final Map<String, Long> offsets;
private final long timeoutMs;
private long failureTime;
private final StorageSchemaCompatibility compatibility;
private Schema currentSchema;
private final String extension;
private final DateTimeZone timeZone;
private final String hiveDatabase;
private final HiveMetaStore hiveMetaStore;
private final io.confluent.connect.storage.format.SchemaFileReader<HdfsSinkConnectorConfig, Path>
schemaFileReader;
private final HiveUtil hive;
private final ExecutorService executorService;
private final Queue<Future<Void>> hiveUpdateFutures;
private final Set<String> hivePartitions;
public TopicPartitionWriter(
TopicPartition tp,
HdfsStorage storage,
RecordWriterProvider writerProvider,
io.confluent.connect.storage.format.RecordWriterProvider<HdfsSinkConnectorConfig>
newWriterProvider,
Partitioner partitioner,
HdfsSinkConnectorConfig connectorConfig,
SinkTaskContext context,
AvroData avroData,
Time time
) {
this(
tp,
storage,
writerProvider,
newWriterProvider,
partitioner,
connectorConfig,
context,
avroData,
null,
null,
null,
null,
null,
time
);
}
public TopicPartitionWriter(
TopicPartition tp,
HdfsStorage storage,
RecordWriterProvider writerProvider,
io.confluent.connect.storage.format.RecordWriterProvider<HdfsSinkConnectorConfig>
newWriterProvider,
Partitioner partitioner,
HdfsSinkConnectorConfig connectorConfig,
SinkTaskContext context,
AvroData avroData,
HiveMetaStore hiveMetaStore,
HiveUtil hive,
io.confluent.connect.storage.format.SchemaFileReader<HdfsSinkConnectorConfig, Path>
schemaFileReader,
ExecutorService executorService,
Queue<Future<Void>> hiveUpdateFutures,
Time time
) {
this.time = time;
this.tp = tp;
this.context = context;
this.avroData = avroData;
this.storage = storage;
this.writerProvider = writerProvider;
this.newWriterProvider = newWriterProvider;
this.partitioner = partitioner;
TimestampExtractor timestampExtractor = null;
if (partitioner instanceof DataWriter.PartitionerWrapper) {
io.confluent.connect.storage.partitioner.Partitioner<?> inner =
((DataWriter.PartitionerWrapper) partitioner).partitioner;
if (TimeBasedPartitioner.class.isAssignableFrom(inner.getClass())) {
timestampExtractor = ((TimeBasedPartitioner) inner).getTimestampExtractor();
}
}
this.timestampExtractor = timestampExtractor != null ? timestampExtractor : WALLCLOCK;
this.isWallclockBased = TimeBasedPartitioner.WallclockTimestampExtractor.class.isAssignableFrom(
this.timestampExtractor.getClass()
);
this.url = storage.url();
this.connectorConfig = storage.conf();
this.schemaFileReader = schemaFileReader;
topicsDir = connectorConfig.getString(StorageCommonConfig.TOPICS_DIR_CONFIG);
flushSize = connectorConfig.getInt(HdfsSinkConnectorConfig.FLUSH_SIZE_CONFIG);
rotateIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig.ROTATE_INTERVAL_MS_CONFIG);
rotateScheduleIntervalMs = connectorConfig.getLong(HdfsSinkConnectorConfig
.ROTATE_SCHEDULE_INTERVAL_MS_CONFIG);
timeoutMs = connectorConfig.getLong(HdfsSinkConnectorConfig.RETRY_BACKOFF_CONFIG);
compatibility = StorageSchemaCompatibility.getCompatibility(
connectorConfig.getString(HiveConfig.SCHEMA_COMPATIBILITY_CONFIG));
String logsDir = connectorConfig.getString(HdfsSinkConnectorConfig.LOGS_DIR_CONFIG);
wal = storage.wal(logsDir, tp);
buffer = new LinkedList<>();
writers = new HashMap<>();
tempFiles = new HashMap<>();
appended = new HashSet<>();
startOffsets = new HashMap<>();
offsets = new HashMap<>();
state = State.RECOVERY_STARTED;
failureTime = -1L;
offset = -1L;
if (writerProvider != null) {
extension = writerProvider.getExtension();
} else if (newWriterProvider != null) {
extension = newWriterProvider.getExtension();
} else {
throw new ConnectException(
"Invalid state: either old or new RecordWriterProvider must be provided"
);
}
zeroPadOffsetFormat = "%0"
+ connectorConfig.getInt(HdfsSinkConnectorConfig.FILENAME_OFFSET_ZERO_PAD_WIDTH_CONFIG)
+ "d";
hiveIntegration = connectorConfig.getBoolean(HiveConfig.HIVE_INTEGRATION_CONFIG);
if (hiveIntegration) {
hiveDatabase = connectorConfig.getString(HiveConfig.HIVE_DATABASE_CONFIG);
} else {
hiveDatabase = null;
}
this.hiveMetaStore = hiveMetaStore;
this.hive = hive;
this.executorService = executorService;
this.hiveUpdateFutures = hiveUpdateFutures;
hivePartitions = new HashSet<>();
if (rotateScheduleIntervalMs > 0) {
timeZone = DateTimeZone.forID(connectorConfig.getString(PartitionerConfig.TIMEZONE_CONFIG));
} else {
timeZone = null;
}
// Initialize rotation timers
updateRotationTimers(null);
}
@SuppressWarnings("fallthrough")
public boolean recover() {
try {
switch (state) {
case RECOVERY_STARTED:
log.info("Started recovery for topic partition {}", tp);
pause();
nextState();
case RECOVERY_PARTITION_PAUSED:
applyWAL();
nextState();
case WAL_APPLIED:
truncateWAL();
nextState();
case WAL_TRUNCATED:
resetOffsets();
nextState();
case OFFSET_RESET:
resume();
nextState();
log.info("Finished recovery for topic partition {}", tp);
break;
default:
log.error(
"{} is not a valid state to perform recovery for topic partition {}.",
state,
tp
);
}
} catch (ConnectException e) {
log.error("Recovery failed at state {}", state, e);
setRetryTimeout(timeoutMs);
return false;
}
return true;
}
private void updateRotationTimers(SinkRecord currentRecord) {
long now = time.milliseconds();
// Wallclock-based partitioners should be independent of the record argument.
lastRotate = isWallclockBased
? (Long) now
: currentRecord != null ? timestampExtractor.extract(currentRecord) : null;
if (log.isDebugEnabled() && rotateIntervalMs > 0) {
log.debug(
"Update last rotation timer. Next rotation for {} will be in {}ms",
tp,
rotateIntervalMs
);
}
if (rotateScheduleIntervalMs > 0) {
nextScheduledRotate = DateTimeUtils.getNextTimeAdjustedByDay(
now,
rotateScheduleIntervalMs,
timeZone
);
if (log.isDebugEnabled()) {
log.debug(
"Update scheduled rotation timer. Next rotation for {} will be at {}",
tp,
new DateTime(nextScheduledRotate).withZone(timeZone).toString()
);
}
}
}
@SuppressWarnings("fallthrough")
public void write() {
long now = time.milliseconds();
SinkRecord currentRecord = null;
if (failureTime > 0 && now - failureTime < timeoutMs) {
return;
}
if (state.compareTo(State.WRITE_STARTED) < 0) {
boolean success = recover();
if (!success) {
return;
}
updateRotationTimers(null);
}
while (!buffer.isEmpty()) {
try {
switch (state) {
case WRITE_STARTED:
pause();
nextState();
case WRITE_PARTITION_PAUSED:
if (currentSchema == null) {
if (compatibility != StorageSchemaCompatibility.NONE && offset != -1) {
String topicDir = FileUtils.topicDirectory(url, topicsDir, tp.topic());
CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp);
FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(
storage,
new Path(topicDir),
filter
);
if (fileStatusWithMaxOffset != null) {
currentSchema = schemaFileReader.getSchema(
connectorConfig,
fileStatusWithMaxOffset.getPath()
);
}
}
}
SinkRecord record = buffer.peek();
currentRecord = record;
Schema valueSchema = record.valueSchema();
if ((recordCounter <= 0 && currentSchema == null && valueSchema != null)
|| compatibility.shouldChangeSchema(record, null, currentSchema)) {
currentSchema = valueSchema;
if (hiveIntegration) {
createHiveTable();
alterHiveSchema();
}
if (recordCounter > 0) {
nextState();
} else {
break;
}
} else {
if (shouldRotateAndMaybeUpdateTimers(currentRecord, now)) {
log.info(
"Starting commit and rotation for topic partition {} with start offsets {} "
+ "and end offsets {}",
tp,
startOffsets,
offsets
);
nextState();
// Fall through and try to rotate immediately
} else {
SinkRecord projectedRecord = compatibility.project(record, null, currentSchema);
writeRecord(projectedRecord);
buffer.poll();
break;
}
}
case SHOULD_ROTATE:
updateRotationTimers(currentRecord);
closeTempFile();
nextState();
case TEMP_FILE_CLOSED:
appendToWAL();
nextState();
case WAL_APPENDED:
commitFile();
nextState();
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);
}
} catch (SchemaProjectorException | IllegalWorkerStateException | HiveMetaStoreException e) {
throw new RuntimeException(e);
} catch (ConnectException e) {
log.error("Exception on topic partition {}: ", tp, e);
failureTime = time.milliseconds();
setRetryTimeout(timeoutMs);
break;
}
}
if (buffer.isEmpty()) {
// committing files after waiting for rotateIntervalMs time but less than flush.size
// records available
if (recordCounter > 0 && shouldRotateAndMaybeUpdateTimers(currentRecord, now)) {
log.info(
"committing files after waiting for rotateIntervalMs time but less than flush.size "
+ "records available."
);
updateRotationTimers(currentRecord);
try {
closeTempFile();
appendToWAL();
commitFile();
} catch (ConnectException e) {
log.error("Exception on topic partition {}: ", tp, e);
failureTime = time.milliseconds();
setRetryTimeout(timeoutMs);
}
}
resume();
state = State.WRITE_STARTED;
}
}
public void close() throws ConnectException {
log.debug("Closing TopicPartitionWriter {}", tp);
List<Exception> exceptions = new ArrayList<>();
for (String encodedPartition : tempFiles.keySet()) {
try {
if (writers.containsKey(encodedPartition)) {
log.debug("Discarding in progress tempfile {} for {} {}",
tempFiles.get(encodedPartition), tp, encodedPartition
);
closeTempFile(encodedPartition);
deleteTempFile(encodedPartition);
}
} catch (DataException e) {
log.error(
"Error discarding temp file {} for {} {} when closing TopicPartitionWriter:",
tempFiles.get(encodedPartition),
tp,
encodedPartition,
e
);
}
}
writers.clear();
try {
wal.close();
} catch (ConnectException e) {
log.error("Error closing {}.", wal.getLogFile(), e);
exceptions.add(e);
}
startOffsets.clear();
offsets.clear();
if (exceptions.size() != 0) {
StringBuilder sb = new StringBuilder();
for (Exception exception : exceptions) {
sb.append(exception.getMessage());
sb.append("\n");
}
throw new ConnectException("Error closing writer: " + sb.toString());
}
}
public void buffer(SinkRecord sinkRecord) {
buffer.add(sinkRecord);
}
public long offset() {
return offset;
}
Map<String, io.confluent.connect.storage.format.RecordWriter> getWriters() {
return writers;
}
public Map<String, String> getTempFiles() {
return tempFiles;
}
private String getDirectory(String encodedPartition) {
return partitioner.generatePartitionedPath(tp.topic(), encodedPartition);
}
private void nextState() {
state = state.next();
}
private void setState(State state) {
this.state = state;
}
private boolean shouldRotateAndMaybeUpdateTimers(SinkRecord currentRecord, long now) {
Long currentTimestamp = null;
if (isWallclockBased) {
currentTimestamp = now;
} else if (currentRecord != null) {
currentTimestamp = timestampExtractor.extract(currentRecord);
lastRotate = lastRotate == null ? currentTimestamp : lastRotate;
}
boolean periodicRotation = rotateIntervalMs > 0
&& currentTimestamp != null
&& lastRotate != null
&& currentTimestamp - lastRotate >= rotateIntervalMs;
boolean scheduledRotation = rotateScheduleIntervalMs > 0 && now >= nextScheduledRotate;
boolean messageSizeRotation = recordCounter >= flushSize;
return periodicRotation || scheduledRotation || messageSizeRotation;
}
private void readOffset() throws ConnectException {
String path = FileUtils.topicDirectory(url, topicsDir, tp.topic());
CommittedFileFilter filter = new TopicPartitionCommittedFileFilter(tp);
FileStatus fileStatusWithMaxOffset = FileUtils.fileStatusWithMaxOffset(
storage,
new Path(path),
filter
);
if (fileStatusWithMaxOffset != null) {
offset = FileUtils.extractOffset(fileStatusWithMaxOffset.getPath().getName()) + 1;
}
}
private void pause() {
context.pause(tp);
}
private void resume() {
context.resume(tp);
}
private io.confluent.connect.storage.format.RecordWriter getWriter(
SinkRecord record,
String encodedPartition
) throws ConnectException {
if (writers.containsKey(encodedPartition)) {
return writers.get(encodedPartition);
}
String tempFile = getTempFile(encodedPartition);
final io.confluent.connect.storage.format.RecordWriter writer;
try {
if (writerProvider != null) {
writer = new OldRecordWriterWrapper(
writerProvider.getRecordWriter(
connectorConfig.getHadoopConfiguration(),
tempFile,
record,
avroData
)
);
} else if (newWriterProvider != null) {
writer = newWriterProvider.getRecordWriter(connectorConfig, tempFile);
} else {
throw new ConnectException(
"Invalid state: either old or new RecordWriterProvider must be provided"
);
}
} catch (IOException e) {
throw new ConnectException("Couldn't create RecordWriter", e);
}
writers.put(encodedPartition, writer);
if (hiveIntegration && !hivePartitions.contains(encodedPartition)) {
addHivePartition(encodedPartition);
hivePartitions.add(encodedPartition);
}
return writer;
}
private String getTempFile(String encodedPartition) {
String tempFile;
if (tempFiles.containsKey(encodedPartition)) {
tempFile = tempFiles.get(encodedPartition);
} else {
String directory = HdfsSinkConnectorConstants.TEMPFILE_DIRECTORY
+ getDirectory(encodedPartition);
tempFile = FileUtils.tempFileName(url, topicsDir, directory, extension);
tempFiles.put(encodedPartition, tempFile);
}
return tempFile;
}
private void applyWAL() throws ConnectException {
if (!recovered) {
wal.apply();
}
}
private void truncateWAL() throws ConnectException {
if (!recovered) {
wal.truncate();
}
}
private void resetOffsets() throws ConnectException {
if (!recovered) {
readOffset();
// Note that we must *always* request that we seek to an offset here. Currently the
// framework will still commit Kafka offsets even though we track our own (see KAFKA-3462),
// which can result in accidentally using that offset if one was committed but no files
// were rolled to their final location in HDFS (i.e. some data was accepted, written to a
// tempfile, but then that tempfile was discarded). To protect against this, even if we
// just want to start at offset 0 or reset to the earliest offset, we specify that
// explicitly to forcibly override any committed offsets.
if (offset > 0) {
log.debug("Resetting offset for {} to {}", tp, offset);
context.offset(tp, offset);
} else {
// The offset was not found, so rather than forcibly set the offset to 0 we let the
// consumer decide where to start based upon standard consumer offsets (if available)
// or the consumer's `auto.offset.reset` configuration
log.debug("Resetting offset for {} based upon existing consumer group offsets or, if "
+ "there are none, the consumer's 'auto.offset.reset' value.",
tp);
}
recovered = true;
}
}
private void writeRecord(SinkRecord record) {
if (offset == -1) {
offset = record.kafkaOffset();
}
String encodedPartition = partitioner.encodePartition(record);
io.confluent.connect.storage.format.RecordWriter writer = getWriter(record, encodedPartition);
writer.write(record);
if (!startOffsets.containsKey(encodedPartition)) {
startOffsets.put(encodedPartition, record.kafkaOffset());
}
offsets.put(encodedPartition, record.kafkaOffset());
recordCounter++;
}
private void closeTempFile(String encodedPartition) {
if (writers.containsKey(encodedPartition)) {
io.confluent.connect.storage.format.RecordWriter writer = writers.get(encodedPartition);
writer.close();
writers.remove(encodedPartition);
}
}
private void closeTempFile() {
for (String encodedPartition : tempFiles.keySet()) {
closeTempFile(encodedPartition);
}
}
private void appendToWAL(String encodedPartition) {
String tempFile = tempFiles.get(encodedPartition);
if (appended.contains(tempFile)) {
return;
}
if (!startOffsets.containsKey(encodedPartition)) {
return;
}
long startOffset = startOffsets.get(encodedPartition);
long endOffset = offsets.get(encodedPartition);
String directory = getDirectory(encodedPartition);
String committedFile = FileUtils.committedFileName(
url,
topicsDir,
directory,
tp,
startOffset,
endOffset,
extension,
zeroPadOffsetFormat
);
wal.append(tempFile, committedFile);
appended.add(tempFile);
}
private void appendToWAL() {
beginAppend();
for (String encodedPartition : tempFiles.keySet()) {
appendToWAL(encodedPartition);
}
endAppend();
}
private void beginAppend() {
if (!appended.contains(WAL.beginMarker)) {
wal.append(WAL.beginMarker, "");
}
}
private void endAppend() {
if (!appended.contains(WAL.endMarker)) {
wal.append(WAL.endMarker, "");
}
}
private void commitFile() {
appended.clear();
for (String encodedPartition : tempFiles.keySet()) {
commitFile(encodedPartition);
}
}
private void commitFile(String encodedPartition) {
if (!startOffsets.containsKey(encodedPartition)) {
return;
}
long startOffset = startOffsets.get(encodedPartition);
long endOffset = offsets.get(encodedPartition);
String tempFile = tempFiles.get(encodedPartition);
String directory = getDirectory(encodedPartition);
String committedFile = FileUtils.committedFileName(
url,
topicsDir,
directory,
tp,
startOffset,
endOffset,
extension,
zeroPadOffsetFormat
);
String directoryName = FileUtils.directoryName(url, topicsDir, directory);
if (!storage.exists(directoryName)) {
storage.create(directoryName);
}
storage.commit(tempFile, committedFile);
startOffsets.remove(encodedPartition);
offsets.remove(encodedPartition);
offset = offset + recordCounter;
recordCounter = 0;
log.info("Committed {} for {}", committedFile, tp);
}
private void deleteTempFile(String encodedPartition) {
storage.delete(tempFiles.get(encodedPartition));
}
private void setRetryTimeout(long timeoutMs) {
context.timeout(timeoutMs);
}
private void createHiveTable() {
Future<Void> future = executorService.submit(new Callable<Void>() {
@Override
public Void call() throws HiveMetaStoreException {
try {
hive.createTable(hiveDatabase, tp.topic(), currentSchema, partitioner);
} catch (Throwable e) {
log.error("Creating Hive table threw unexpected error", e);
}
return null;
}
});
hiveUpdateFutures.add(future);
}
private void alterHiveSchema() {
Future<Void> future = executorService.submit(new Callable<Void>() {
@Override
public Void call() throws HiveMetaStoreException {
try {
hive.alterSchema(hiveDatabase, tp.topic(), currentSchema);
} catch (Throwable e) {
log.error("Altering Hive schema threw unexpected error", e);
}
return null;
}
});
hiveUpdateFutures.add(future);
}
private void addHivePartition(final String location) {
Future<Void> future = executorService.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
try {
hiveMetaStore.addPartition(hiveDatabase, tp.topic(), location);
} catch (Throwable e) {
log.error("Adding Hive partition threw unexpected error", e);
}
return null;
}
});
hiveUpdateFutures.add(future);
}
private enum State {
RECOVERY_STARTED,
RECOVERY_PARTITION_PAUSED,
WAL_APPLIED,
WAL_TRUNCATED,
OFFSET_RESET,
WRITE_STARTED,
WRITE_PARTITION_PAUSED,
SHOULD_ROTATE,
TEMP_FILE_CLOSED,
WAL_APPENDED,
FILE_COMMITTED;
private static State[] vals = values();
public State next() {
return vals[(this.ordinal() + 1) % vals.length];
}
}
}