-
Notifications
You must be signed in to change notification settings - Fork 46
/
BackupSinkTask.java
228 lines (200 loc) · 9.22 KB
/
BackupSinkTask.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
package de.azapps.kafkabackup.sink;
import de.azapps.kafkabackup.common.offset.EndOffsetReader;
import de.azapps.kafkabackup.common.offset.OffsetSink;
import de.azapps.kafkabackup.common.partition.PartitionIndex;
import de.azapps.kafkabackup.common.partition.PartitionWriter;
import de.azapps.kafkabackup.common.record.Record;
import de.azapps.kafkabackup.common.segment.SegmentIndex;
import de.azapps.kafkabackup.common.segment.SegmentWriter;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.apache.kafka.connect.sink.SinkTaskContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.*;
public class BackupSinkTask extends SinkTask {
private static final Logger log = LoggerFactory.getLogger(BackupSinkTask.class);
private Path targetDir;
private Map<TopicPartition, PartitionWriter> partitionWriters = new HashMap<>();
private long maxSegmentSizeBytes;
private OffsetSink offsetSink;
private BackupSinkConfig config;
private Map<TopicPartition, Long> endOffsets;
private Map<TopicPartition, Long> currentOffsets = new HashMap<>();
private EndOffsetReader endOffsetReader;
private java.util.function.Consumer<Integer> exitFunction;
@Override
public String version() {
return "0.1";
}
@Override
public void start(Map<String, String> props) {
start(props, null, null, null);
}
public void start(
Map<String, String> props,
OffsetSink overrideOffsetSink,
EndOffsetReader overrideEndOffsetReader,
java.util.function.Consumer<Integer> overrideExitFunction
) {
this.config = new BackupSinkConfig(props);
try {
maxSegmentSizeBytes = config.maxSegmentSizeBytes();
targetDir = Paths.get(config.targetDir());
Files.createDirectories(targetDir);
// Allow tests to use mock offset sync
if(overrideOffsetSink != null) {
offsetSink = overrideOffsetSink;
} else {
AdminClient adminClient = AdminClient.create(config.adminConfig());
offsetSink = new OffsetSink(adminClient, targetDir);
}
if (overrideEndOffsetReader != null) {
this.endOffsetReader = overrideEndOffsetReader;
} else {
endOffsetReader = new EndOffsetReader(config.consumerConfig());
}
if (overrideExitFunction != null) {
this.exitFunction = overrideExitFunction;
} else {
this.exitFunction = System::exit;
}
log.debug("Initialized BackupSinkTask with target dir {}", targetDir);
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* Check for end-offsets. Terminate if all offsets >= end-offsets
*/
private void terminateIfCompleted() {
boolean terminate = true;
for (Map.Entry<TopicPartition, Long> partitionOffset : endOffsets.entrySet()) {
Long endOffset = partitionOffset.getValue();
Long currentOffset = currentOffsets.getOrDefault(partitionOffset.getKey(), -1L);
if (currentOffset < endOffset - 1) {
return;
}
}
if (terminate) {
log.debug("Snapshot complete. Terminating kafka connect.");
stop(); // seems that this is not called when using System.exit()
exitFunction.accept(0);
}
}
@Override
public void put(Collection<SinkRecord> records) {
try {
for (SinkRecord sinkRecord : records) {
TopicPartition topicPartition = new TopicPartition(sinkRecord.topic(), sinkRecord.kafkaPartition());
PartitionWriter partition = partitionWriters.get(topicPartition);
partition.append(Record.fromSinkRecord(sinkRecord));
if (sinkRecord.kafkaOffset() % 100 == 0) {
log.debug("Backed up Topic {}, Partition {}, up to offset {}", sinkRecord.topic(), sinkRecord.kafkaPartition(), sinkRecord.kafkaOffset());
}
if (config.snapShotMode()) {
currentOffsets.put(topicPartition, sinkRecord.kafkaOffset());
}
}
// Todo: refactor to own worker. E.g. using the scheduler of MM2
offsetSink.syncConsumerGroups();
offsetSink.syncOffsets();
if (config.snapShotMode()) {
terminateIfCompleted();
}
} catch (IOException | SegmentIndex.IndexException | PartitionIndex.IndexException | SegmentWriter.SegmentException e) {
throw new RuntimeException(e);
}
}
public void open(Collection<TopicPartition> partitions) {
super.open(partitions);
try {
for (TopicPartition topicPartition : partitions) {
Path topicDir = Paths.get(targetDir.toString(), topicPartition.topic());
Files.createDirectories(topicDir);
PartitionWriter partitionWriter = new PartitionWriter(topicPartition.topic(), topicPartition.partition(), topicDir, maxSegmentSizeBytes);
long lastWrittenOffset = partitionWriter.lastWrittenOffset();
// 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 written to disk. 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 (lastWrittenOffset > 0) {
context.offset(topicPartition, lastWrittenOffset + 1);
log.debug("Initialized Topic {}, Partition {}. Last written offset: {}"
, topicPartition.topic(), topicPartition.partition(), lastWrittenOffset);
} 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
// if we are in snapshot mode, then just start at zero.
if (config.snapShotMode()) {
context.offset(topicPartition, 0);
}
log.info("Resetting offset for {} based upon existing consumer group offsets or, if "
+ "there are none, the consumer's 'auto.offset.reset' value.", topicPartition);
}
this.partitionWriters.put(topicPartition, partitionWriter);
this.currentOffsets.put(topicPartition, lastWrittenOffset);
}
if ( config.snapShotMode() ) {
this.endOffsets = endOffsetReader.getEndOffsets(partitions);
this.terminateIfCompleted();
}
if (partitions.isEmpty()) {
log.info("No partitions assigned to BackupSinkTask");
}
} catch (IOException | SegmentIndex.IndexException | PartitionIndex.IndexException e) {
throw new RuntimeException(e);
}
}
public void close(Collection<TopicPartition> partitions) {
super.close(partitions);
try {
for (TopicPartition topicPartition : partitions) {
PartitionWriter partitionWriter = partitionWriters.get(topicPartition);
if (partitionWriter != null) {
partitionWriter.close();
}
partitionWriters.remove(topicPartition);
log.debug("Closed BackupSinkTask for Topic {}, Partition {}"
, topicPartition.topic(), topicPartition.partition());
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public void stop() {
try {
for (PartitionWriter partition : partitionWriters.values()) {
partition.close();
}
offsetSink.close();
log.info("Stopped BackupSinkTask");
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public void flush(Map<TopicPartition, OffsetAndMetadata> currentOffsets) {
try {
for (PartitionWriter partitionWriter : partitionWriters.values()) {
partitionWriter.flush();
log.debug("Flushed Topic {}, Partition {}"
, partitionWriter.topic(), partitionWriter.partition());
}
offsetSink.flush();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}