/
AbstractKafkaConnectSource.java
295 lines (260 loc) · 11.3 KB
/
AbstractKafkaConnectSource.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
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.pulsar.io.kafka.connect;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.connect.runtime.TaskConfig;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.connect.source.SourceTaskContext;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.OffsetBackingStore;
import org.apache.kafka.connect.storage.OffsetStorageReader;
import org.apache.kafka.connect.storage.OffsetStorageReaderImpl;
import org.apache.kafka.connect.storage.OffsetStorageWriter;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.Source;
import org.apache.pulsar.io.core.SourceContext;
import org.apache.pulsar.io.kafka.connect.schema.KafkaSchemaWrappedSchema;
import org.apache.pulsar.kafka.shade.io.confluent.connect.avro.AvroConverter;
import org.apache.pulsar.kafka.shade.io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient;
import org.apache.pulsar.kafka.shade.io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
import java.util.concurrent.atomic.AtomicInteger;
import static org.apache.pulsar.io.kafka.connect.PulsarKafkaWorkerConfig.TOPIC_NAMESPACE_CONFIG;
/**
* A pulsar source that runs
*/
@Slf4j
public abstract class AbstractKafkaConnectSource<T> implements Source<T> {
// kafka connect related variables
private SourceTaskContext sourceTaskContext;
@Getter
private SourceTask sourceTask;
public Converter keyConverter;
public Converter valueConverter;
// pulsar io related variables
private Iterator<SourceRecord> currentBatch = null;
private CompletableFuture<Void> flushFuture;
private OffsetBackingStore offsetStore;
private OffsetStorageReader offsetReader;
private String topicNamespace;
@Getter
public OffsetStorageWriter offsetWriter;
// number of outstandingRecords that have been polled but not been acked
private AtomicInteger outstandingRecords = new AtomicInteger(0);
@Override
public void open(Map<String, Object> config, SourceContext sourceContext) throws Exception {
Map<String, String> stringConfig = new HashMap<>();
config.forEach((key, value) -> {
if (value instanceof String) {
stringConfig.put(key, (String) value);
}
});
// get the source class name from config and create source task from reflection
sourceTask = ((Class<? extends SourceTask>) Class.forName(stringConfig.get(TaskConfig.TASK_CLASS_CONFIG)))
.asSubclass(SourceTask.class)
.getDeclaredConstructor()
.newInstance();
topicNamespace = stringConfig.get(TOPIC_NAMESPACE_CONFIG);
// initialize the key and value converter
keyConverter = ((Class<? extends Converter>) Class.forName(stringConfig.get(PulsarKafkaWorkerConfig.KEY_CONVERTER_CLASS_CONFIG)))
.asSubclass(Converter.class)
.getDeclaredConstructor()
.newInstance();
valueConverter = ((Class<? extends Converter>) Class.forName(stringConfig.get(PulsarKafkaWorkerConfig.VALUE_CONVERTER_CLASS_CONFIG)))
.asSubclass(Converter.class)
.getDeclaredConstructor()
.newInstance();
if (keyConverter instanceof AvroConverter) {
keyConverter = new AvroConverter(new MockSchemaRegistryClient());
config.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "mock");
}
if (valueConverter instanceof AvroConverter) {
valueConverter = new AvroConverter(new MockSchemaRegistryClient());
config.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "mock");
}
keyConverter.configure(config, true);
valueConverter.configure(config, false);
offsetStore = new PulsarOffsetBackingStore(sourceContext.getPulsarClient());
PulsarKafkaWorkerConfig pulsarKafkaWorkerConfig = new PulsarKafkaWorkerConfig(stringConfig);
offsetStore.configure(pulsarKafkaWorkerConfig);
offsetStore.start();
offsetReader = new OffsetStorageReaderImpl(
offsetStore,
"pulsar-kafka-connect-adaptor",
keyConverter,
valueConverter
);
offsetWriter = new OffsetStorageWriter(
offsetStore,
"pulsar-kafka-connect-adaptor",
keyConverter,
valueConverter
);
sourceTaskContext = new PulsarIOSourceTaskContext(offsetReader, pulsarKafkaWorkerConfig);
sourceTask.initialize(sourceTaskContext);
sourceTask.start(stringConfig);
}
@Override
public synchronized Record<T> read() throws Exception {
while (true) {
if (currentBatch == null) {
flushFuture = new CompletableFuture<>();
List<SourceRecord> recordList = sourceTask.poll();
if (recordList == null || recordList.isEmpty()) {
Thread.sleep(1000);
continue;
}
outstandingRecords.addAndGet(recordList.size());
currentBatch = recordList.iterator();
}
if (currentBatch.hasNext()) {
AbstractKafkaSourceRecord<T> processRecord = processSourceRecord(currentBatch.next());
if (processRecord.isEmpty()) {
outstandingRecords.decrementAndGet();
continue;
} else {
return processRecord;
}
} else {
// there is no records any more, then waiting for the batch to complete writing
// to sink and the offsets are committed as well, then do next round read.
try {
flushFuture.get();
} catch (ExecutionException ex) {
// log the error, continue execution
log.error("execution exception while get flushFuture", ex);
} finally {
flushFuture = null;
currentBatch = null;
}
}
}
}
@Override
public void close() {
if (sourceTask != null) {
sourceTask.stop();
sourceTask = null;
}
if (offsetStore != null) {
offsetStore.stop();
offsetStore = null;
}
}
public abstract AbstractKafkaSourceRecord<T> processSourceRecord(final SourceRecord srcRecord);
private static Map<String, String> PROPERTIES = Collections.emptyMap();
private static Optional<Long> RECORD_SEQUENCE = Optional.empty();
private static long FLUSH_TIMEOUT_MS = 60000;
public abstract class AbstractKafkaSourceRecord<T> implements Record {
@Getter
Optional<String> key;
@Getter
T value;
@Getter
Optional<String> topicName;
@Getter
Optional<Long> eventTime;
@Getter
Optional<String> partitionId;
@Getter
Optional<String> destinationTopic;
@Getter
Optional<Integer> partitionIndex;
KafkaSchemaWrappedSchema keySchema;
KafkaSchemaWrappedSchema valueSchema;
AbstractKafkaSourceRecord(SourceRecord srcRecord) {
this.destinationTopic = Optional.of("persistent://"+topicNamespace + "/" + srcRecord.topic());
this.partitionIndex = Optional.ofNullable(srcRecord.kafkaPartition());
}
@Override
public Schema getSchema() {
return null;
}
@Override
public Optional<Long> getRecordSequence() {
return RECORD_SEQUENCE;
}
@Override
public Map<String, String> getProperties() {
return PROPERTIES;
}
public boolean isEmpty() {
return this.value == null;
}
private void completedFlushOffset(Throwable error, Void result) {
if (error != null) {
log.error("Failed to flush offsets to storage: ", error);
currentBatch = null;
offsetWriter.cancelFlush();
flushFuture.completeExceptionally(new Exception("No Offsets Added Error"));
} else {
try {
sourceTask.commit();
log.info("Finished flushing offsets to storage");
currentBatch = null;
flushFuture.complete(null);
} catch (InterruptedException exception) {
log.warn("Flush of {} offsets interrupted, cancelling", this);
offsetWriter.cancelFlush();
flushFuture.completeExceptionally(new Exception("Failed to commit offsets"));
}
}
}
@Override
public void ack() {
// TODO: should flush for each batch. not wait for a time for acked all.
// How to handle order between each batch. QueueList<pair<batch, automicInt>>. check if head is all acked.
boolean canFlush = (outstandingRecords.decrementAndGet() == 0);
// consumed all the records, flush the offsets
if (canFlush && flushFuture != null) {
if (!offsetWriter.beginFlush()) {
log.error("When beginFlush, No offsets to commit!");
flushFuture.completeExceptionally(new Exception("No Offsets Added Error when beginFlush"));
return;
}
Future<Void> doFlush = offsetWriter.doFlush(this::completedFlushOffset);
if (doFlush == null) {
// Offsets added in processSourceRecord, But here no offsets to commit
log.error("No offsets to commit!");
flushFuture.completeExceptionally(new Exception("No Offsets Added Error"));
return;
}
}
}
@Override
public void fail() {
if (flushFuture != null) {
flushFuture.completeExceptionally(new Exception("Sink Error"));
}
}
}
}