embulk-input-kafka 0.1.0
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- checksums.yaml +7 -0
- data/.circleci/config.yml +44 -0
- data/.gitignore +12 -0
- data/LICENSE.txt +21 -0
- data/README.md +53 -0
- data/build.gradle +149 -0
- data/config/checkstyle/checkstyle.xml +128 -0
- data/config/checkstyle/default.xml +108 -0
- data/gradle/wrapper/gradle-wrapper.jar +0 -0
- data/gradle/wrapper/gradle-wrapper.properties +5 -0
- data/gradlew +172 -0
- data/gradlew.bat +84 -0
- data/lib/embulk/input/kafka.rb +3 -0
- data/src/main/java/org/embulk/input/kafka/AbstractKafkaInputColumnVisitor.java +135 -0
- data/src/main/java/org/embulk/input/kafka/AvroFormatColumnVisitor.java +207 -0
- data/src/main/java/org/embulk/input/kafka/JsonFormatColumnVisitor.java +168 -0
- data/src/main/java/org/embulk/input/kafka/KafkaInputPlugin.java +513 -0
- data/src/main/java/org/embulk/input/kafka/KafkaJsonDeserializer.java +38 -0
- data/src/test/avro/ComplexRecordAvro.avsc +18 -0
- data/src/test/avro/SimpleRecordAvro.avsc +11 -0
- data/src/test/java/org/embulk/input/kafka/ComplexRecord.java +75 -0
- data/src/test/java/org/embulk/input/kafka/SimpleRecord.java +39 -0
- data/src/test/java/org/embulk/input/kafka/TestKafkaInputPlugin.java +353 -0
- data/src/test/resources/config_complex.yml +16 -0
- data/src/test/resources/config_complex_avro.yml +17 -0
- data/src/test/resources/config_simple.yml +11 -0
- data/src/test/resources/config_simple_avro.yml +12 -0
- metadata +118 -0
@@ -0,0 +1,168 @@
|
|
1
|
+
package org.embulk.input.kafka;
|
2
|
+
|
3
|
+
import com.fasterxml.jackson.databind.JsonNode;
|
4
|
+
import com.fasterxml.jackson.databind.node.ArrayNode;
|
5
|
+
import com.fasterxml.jackson.databind.node.ObjectNode;
|
6
|
+
import org.embulk.input.kafka.KafkaInputPlugin.PluginTask;
|
7
|
+
import org.embulk.spi.Column;
|
8
|
+
import org.embulk.spi.ColumnVisitor;
|
9
|
+
import org.embulk.spi.PageBuilder;
|
10
|
+
import org.embulk.spi.time.Timestamp;
|
11
|
+
import org.embulk.spi.time.TimestampParser;
|
12
|
+
import org.msgpack.value.Value;
|
13
|
+
import org.msgpack.value.ValueFactory;
|
14
|
+
|
15
|
+
import java.util.ArrayList;
|
16
|
+
import java.util.HashMap;
|
17
|
+
import java.util.List;
|
18
|
+
import java.util.Map;
|
19
|
+
|
20
|
+
public class JsonFormatColumnVisitor extends AbstractKafkaInputColumnVisitor<ObjectNode> implements ColumnVisitor
|
21
|
+
{
|
22
|
+
public JsonFormatColumnVisitor(PluginTask task, PageBuilder pageBuilder, TimestampParser[] timestampParsers)
|
23
|
+
{
|
24
|
+
super(task, pageBuilder, timestampParsers);
|
25
|
+
}
|
26
|
+
|
27
|
+
@Override
|
28
|
+
public void booleanColumn(Column column)
|
29
|
+
{
|
30
|
+
JsonNode value = recordValue.get(column.getName());
|
31
|
+
if (value.isNull()) {
|
32
|
+
pageBuilder.setNull(column);
|
33
|
+
return;
|
34
|
+
}
|
35
|
+
|
36
|
+
pageBuilder.setBoolean(column, value.booleanValue());
|
37
|
+
}
|
38
|
+
|
39
|
+
@Override
|
40
|
+
public void longColumn(Column column)
|
41
|
+
{
|
42
|
+
if (super.isKeyColumn(column)) {
|
43
|
+
super.longColumnForKey(column);
|
44
|
+
return;
|
45
|
+
}
|
46
|
+
|
47
|
+
if (isPartitionColumn(column)) {
|
48
|
+
super.longColumnForPartition(column);
|
49
|
+
return;
|
50
|
+
}
|
51
|
+
|
52
|
+
JsonNode value = recordValue.get(column.getName());
|
53
|
+
if (value.isNull()) {
|
54
|
+
pageBuilder.setNull(column);
|
55
|
+
return;
|
56
|
+
}
|
57
|
+
|
58
|
+
pageBuilder.setLong(column, value.longValue());
|
59
|
+
}
|
60
|
+
|
61
|
+
@Override
|
62
|
+
public void doubleColumn(Column column)
|
63
|
+
{
|
64
|
+
if (super.isKeyColumn(column)) {
|
65
|
+
super.doubleColumnForKey(column);
|
66
|
+
return;
|
67
|
+
}
|
68
|
+
|
69
|
+
JsonNode value = recordValue.get(column.getName());
|
70
|
+
if (value.isNull()) {
|
71
|
+
pageBuilder.setNull(column);
|
72
|
+
return;
|
73
|
+
}
|
74
|
+
|
75
|
+
pageBuilder.setDouble(column, value.doubleValue());
|
76
|
+
}
|
77
|
+
|
78
|
+
@Override
|
79
|
+
public void stringColumn(Column column)
|
80
|
+
{
|
81
|
+
if (super.isKeyColumn(column)) {
|
82
|
+
super.stringColumnForKey(column);
|
83
|
+
return;
|
84
|
+
}
|
85
|
+
|
86
|
+
JsonNode value = recordValue.get(column.getName());
|
87
|
+
if (value.isNull()) {
|
88
|
+
pageBuilder.setNull(column);
|
89
|
+
return;
|
90
|
+
}
|
91
|
+
|
92
|
+
pageBuilder.setString(column, value.textValue());
|
93
|
+
}
|
94
|
+
|
95
|
+
@Override
|
96
|
+
public void timestampColumn(Column column)
|
97
|
+
{
|
98
|
+
if (super.isKeyColumn(column)) {
|
99
|
+
super.timestampColumnForKey(column);
|
100
|
+
return;
|
101
|
+
}
|
102
|
+
|
103
|
+
JsonNode value = recordValue.get(column.getName());
|
104
|
+
if (value.isNull()) {
|
105
|
+
pageBuilder.setNull(column);
|
106
|
+
return;
|
107
|
+
}
|
108
|
+
|
109
|
+
Timestamp timestamp = timestampParsers[column.getIndex()].parse(value.textValue());
|
110
|
+
pageBuilder.setTimestamp(column, timestamp);
|
111
|
+
}
|
112
|
+
|
113
|
+
@Override
|
114
|
+
public void jsonColumn(Column column)
|
115
|
+
{
|
116
|
+
JsonNode jsonNode = recordValue.get(column.getName());
|
117
|
+
if (jsonNode.isNull()) {
|
118
|
+
pageBuilder.setNull(column);
|
119
|
+
return;
|
120
|
+
}
|
121
|
+
|
122
|
+
pageBuilder.setJson(column, convertJsonValueToMsgpackValue(jsonNode));
|
123
|
+
}
|
124
|
+
|
125
|
+
private Value convertArrayToMsgpackValue(ArrayNode node)
|
126
|
+
{
|
127
|
+
List<Value> values = new ArrayList<>();
|
128
|
+
node.forEach(item -> values.add(convertJsonValueToMsgpackValue(item)));
|
129
|
+
return ValueFactory.newArray(values);
|
130
|
+
}
|
131
|
+
|
132
|
+
private Value convertObjectToMsgpackValue(ObjectNode node)
|
133
|
+
{
|
134
|
+
Map<Value, Value> values = new HashMap<>();
|
135
|
+
node.fields().forEachRemaining(field -> {
|
136
|
+
values.put(ValueFactory.newString(field.getKey()),
|
137
|
+
convertJsonValueToMsgpackValue(field.getValue()));
|
138
|
+
});
|
139
|
+
return ValueFactory.newMap(values);
|
140
|
+
}
|
141
|
+
|
142
|
+
private Value convertJsonValueToMsgpackValue(JsonNode node)
|
143
|
+
{
|
144
|
+
if (node.isFloatingPointNumber()) {
|
145
|
+
return ValueFactory.newFloat(node.doubleValue());
|
146
|
+
}
|
147
|
+
else if (node.isIntegralNumber()) {
|
148
|
+
return ValueFactory.newInteger(node.longValue());
|
149
|
+
}
|
150
|
+
else if (node.isIntegralNumber()) {
|
151
|
+
return ValueFactory.newInteger(node.longValue());
|
152
|
+
}
|
153
|
+
else if (node.isTextual()) {
|
154
|
+
return ValueFactory.newString(node.textValue());
|
155
|
+
}
|
156
|
+
else if (node.isNull()) {
|
157
|
+
return ValueFactory.newNil();
|
158
|
+
}
|
159
|
+
else if (node.isArray()) {
|
160
|
+
return convertArrayToMsgpackValue((ArrayNode) node);
|
161
|
+
}
|
162
|
+
else if (node.isObject()) {
|
163
|
+
return convertObjectToMsgpackValue((ObjectNode) node);
|
164
|
+
}
|
165
|
+
|
166
|
+
throw new RuntimeException("unknown json node");
|
167
|
+
}
|
168
|
+
}
|
@@ -0,0 +1,513 @@
|
|
1
|
+
package org.embulk.input.kafka;
|
2
|
+
|
3
|
+
import com.fasterxml.jackson.annotation.JsonCreator;
|
4
|
+
import com.fasterxml.jackson.annotation.JsonValue;
|
5
|
+
import com.fasterxml.jackson.databind.node.ObjectNode;
|
6
|
+
import com.google.common.collect.ImmutableMap;
|
7
|
+
import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig;
|
8
|
+
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
|
9
|
+
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
10
|
+
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
11
|
+
import org.apache.kafka.clients.consumer.ConsumerRecords;
|
12
|
+
import org.apache.kafka.clients.consumer.KafkaConsumer;
|
13
|
+
import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
14
|
+
import org.apache.kafka.common.PartitionInfo;
|
15
|
+
import org.apache.kafka.common.TopicPartition;
|
16
|
+
import org.apache.kafka.common.serialization.BytesDeserializer;
|
17
|
+
import org.apache.kafka.common.utils.Bytes;
|
18
|
+
import org.embulk.config.Config;
|
19
|
+
import org.embulk.config.ConfigDefault;
|
20
|
+
import org.embulk.config.ConfigDiff;
|
21
|
+
import org.embulk.config.ConfigException;
|
22
|
+
import org.embulk.config.ConfigSource;
|
23
|
+
import org.embulk.config.Task;
|
24
|
+
import org.embulk.config.TaskReport;
|
25
|
+
import org.embulk.config.TaskSource;
|
26
|
+
import org.embulk.spi.BufferAllocator;
|
27
|
+
import org.embulk.spi.Exec;
|
28
|
+
import org.embulk.spi.InputPlugin;
|
29
|
+
import org.embulk.spi.PageBuilder;
|
30
|
+
import org.embulk.spi.PageOutput;
|
31
|
+
import org.embulk.spi.Schema;
|
32
|
+
import org.embulk.spi.SchemaConfig;
|
33
|
+
import org.embulk.spi.time.TimestampParser;
|
34
|
+
import org.embulk.spi.util.Timestamps;
|
35
|
+
import org.slf4j.Logger;
|
36
|
+
import org.slf4j.LoggerFactory;
|
37
|
+
|
38
|
+
import java.time.Duration;
|
39
|
+
import java.util.ArrayList;
|
40
|
+
import java.util.List;
|
41
|
+
import java.util.Locale;
|
42
|
+
import java.util.Map;
|
43
|
+
import java.util.Optional;
|
44
|
+
import java.util.Properties;
|
45
|
+
import java.util.concurrent.CopyOnWriteArrayList;
|
46
|
+
import java.util.stream.Collectors;
|
47
|
+
import java.util.stream.IntStream;
|
48
|
+
|
49
|
+
public class KafkaInputPlugin
|
50
|
+
implements InputPlugin
|
51
|
+
{
|
52
|
+
static final String MOCK_SCHEMA_REGISTRY_SCOPE = "embulk-input-kafka";
|
53
|
+
|
54
|
+
public enum RecordSerializeFormat
|
55
|
+
{
|
56
|
+
JSON,
|
57
|
+
AVRO_WITH_SCHEMA_REGISTRY;
|
58
|
+
|
59
|
+
@JsonValue
|
60
|
+
public String toString()
|
61
|
+
{
|
62
|
+
return name().toLowerCase(Locale.ENGLISH);
|
63
|
+
}
|
64
|
+
|
65
|
+
@JsonCreator
|
66
|
+
public static RecordSerializeFormat ofString(String name)
|
67
|
+
{
|
68
|
+
switch (name.toLowerCase(Locale.ENGLISH)) {
|
69
|
+
case "json":
|
70
|
+
return JSON;
|
71
|
+
case "avro_with_schema_registry":
|
72
|
+
return AVRO_WITH_SCHEMA_REGISTRY;
|
73
|
+
default:
|
74
|
+
}
|
75
|
+
|
76
|
+
throw new ConfigException(String.format(
|
77
|
+
"Unknown serialize format '%s'. Supported modes are json, avro_with_schema_registry",
|
78
|
+
name));
|
79
|
+
}
|
80
|
+
}
|
81
|
+
|
82
|
+
public enum SeekMode
|
83
|
+
{
|
84
|
+
EARLIEST {
|
85
|
+
@Override
|
86
|
+
public void seek(KafkaConsumer<?, ?> consumer,
|
87
|
+
List<TopicPartition> topicPartitions, Optional<Long> timestamp)
|
88
|
+
{
|
89
|
+
consumer.seekToBeginning(topicPartitions);
|
90
|
+
}
|
91
|
+
},
|
92
|
+
TIMESTAMP {
|
93
|
+
@Override
|
94
|
+
public void seek(KafkaConsumer<?, ?> consumer,
|
95
|
+
List<TopicPartition> topicPartitions, Optional<Long> timestamp)
|
96
|
+
{
|
97
|
+
if (timestamp.isPresent()) {
|
98
|
+
Map<TopicPartition, Long> topicPartitionWithTimestamp = topicPartitions.stream()
|
99
|
+
.collect(Collectors
|
100
|
+
.toMap(topicPartition -> topicPartition,
|
101
|
+
topicPartition -> timestamp.get()));
|
102
|
+
Map<TopicPartition, OffsetAndTimestamp> topicPartitionOffsetAndTimestamp = consumer
|
103
|
+
.offsetsForTimes(topicPartitionWithTimestamp);
|
104
|
+
topicPartitionOffsetAndTimestamp.forEach(((topicPartition, offsetAndTimestamp) -> {
|
105
|
+
if (offsetAndTimestamp != null) {
|
106
|
+
consumer.seek(topicPartition, offsetAndTimestamp.offset());
|
107
|
+
}
|
108
|
+
}));
|
109
|
+
}
|
110
|
+
}
|
111
|
+
};
|
112
|
+
|
113
|
+
@JsonValue
|
114
|
+
public String toString()
|
115
|
+
{
|
116
|
+
return name().toLowerCase(Locale.ENGLISH);
|
117
|
+
}
|
118
|
+
|
119
|
+
@JsonCreator
|
120
|
+
public static SeekMode ofString(String name)
|
121
|
+
{
|
122
|
+
switch (name.toLowerCase(Locale.ENGLISH)) {
|
123
|
+
case "earliest":
|
124
|
+
return EARLIEST;
|
125
|
+
case "timestamp":
|
126
|
+
return TIMESTAMP;
|
127
|
+
default:
|
128
|
+
}
|
129
|
+
|
130
|
+
throw new ConfigException(String
|
131
|
+
.format("Unknown seek mode '%s'. Supported modes are earliest, timestamp",
|
132
|
+
name));
|
133
|
+
}
|
134
|
+
|
135
|
+
public abstract void seek(KafkaConsumer<?, ?> consumer, List<TopicPartition> topicPartitions,
|
136
|
+
Optional<Long> timestamp);
|
137
|
+
}
|
138
|
+
|
139
|
+
public enum TerminationMode {
|
140
|
+
OFFSET_AT_START {
|
141
|
+
@Override
|
142
|
+
public Map<TopicPartition, Long> getOffsetsForTermination(
|
143
|
+
KafkaConsumer<?, ?> consumer,
|
144
|
+
List<TopicPartition> topicPartitions)
|
145
|
+
{
|
146
|
+
return consumer.endOffsets(topicPartitions);
|
147
|
+
}
|
148
|
+
},
|
149
|
+
ENDLESS {
|
150
|
+
@Override
|
151
|
+
public Map<TopicPartition, Long> getOffsetsForTermination(
|
152
|
+
KafkaConsumer<?, ?> consumer,
|
153
|
+
List<TopicPartition> topicPartitions)
|
154
|
+
{
|
155
|
+
return ImmutableMap.of();
|
156
|
+
}
|
157
|
+
};
|
158
|
+
|
159
|
+
@JsonCreator
|
160
|
+
public static TerminationMode ofString(String name)
|
161
|
+
{
|
162
|
+
switch (name.toLowerCase(Locale.ENGLISH)) {
|
163
|
+
case "offset_at_start":
|
164
|
+
return OFFSET_AT_START;
|
165
|
+
case "endless":
|
166
|
+
return ENDLESS;
|
167
|
+
default:
|
168
|
+
}
|
169
|
+
|
170
|
+
throw new ConfigException(String
|
171
|
+
.format("Unknown seek mode '%s'. Supported modes are offset_at_start, endless",
|
172
|
+
name));
|
173
|
+
}
|
174
|
+
|
175
|
+
public abstract Map<TopicPartition, Long> getOffsetsForTermination(
|
176
|
+
KafkaConsumer<?, ?> consumer,
|
177
|
+
List<TopicPartition> topicPartitions);
|
178
|
+
}
|
179
|
+
|
180
|
+
public interface PluginTask
|
181
|
+
extends Task, TimestampParser.Task
|
182
|
+
{
|
183
|
+
@Config("brokers")
|
184
|
+
public List<String> getBrokers();
|
185
|
+
|
186
|
+
@Config("topics")
|
187
|
+
public List<String> getTopics();
|
188
|
+
|
189
|
+
@Config("schema_registry_url")
|
190
|
+
@ConfigDefault("null")
|
191
|
+
public Optional<String> getSchemaRegistryUrl();
|
192
|
+
|
193
|
+
@Config("serialize_format")
|
194
|
+
public RecordSerializeFormat getRecordSerializeFormat();
|
195
|
+
|
196
|
+
@Config("seek_mode")
|
197
|
+
@ConfigDefault("\"earliest\"")
|
198
|
+
public SeekMode getSeekMode();
|
199
|
+
|
200
|
+
@Config("termination_mode")
|
201
|
+
@ConfigDefault("\"offset_at_start\"")
|
202
|
+
public TerminationMode getTerminationMode();
|
203
|
+
|
204
|
+
@Config("timestamp_for_seeking")
|
205
|
+
@ConfigDefault("null")
|
206
|
+
public Optional<Long> getTimestampForSeeking();
|
207
|
+
|
208
|
+
@Config("key_column_name")
|
209
|
+
@ConfigDefault("\"_key\"")
|
210
|
+
public String getKeyColumnName();
|
211
|
+
|
212
|
+
@Config("partition_column_name")
|
213
|
+
@ConfigDefault("\"_partition\"")
|
214
|
+
public String getPartitionColumnName();
|
215
|
+
|
216
|
+
@Config("fetch_max_wait_ms")
|
217
|
+
@ConfigDefault("30000")
|
218
|
+
public int getFetchMaxWaitMs();
|
219
|
+
|
220
|
+
@Config("max_empty_pollings")
|
221
|
+
@ConfigDefault("2")
|
222
|
+
public int getMaxEmptyPollings();
|
223
|
+
|
224
|
+
@Config("other_consumer_configs")
|
225
|
+
@ConfigDefault("{}")
|
226
|
+
public Map<String, String> getOtherConsumerConfigs();
|
227
|
+
|
228
|
+
@Config("value_subject_name_strategy")
|
229
|
+
@ConfigDefault("null")
|
230
|
+
public java.util.Optional<String> getValueSubjectNameStrategy();
|
231
|
+
|
232
|
+
@Config("columns")
|
233
|
+
public SchemaConfig getColumns();
|
234
|
+
|
235
|
+
@Config("assignments")
|
236
|
+
@ConfigDefault("[]")
|
237
|
+
public List<List<String>> getAssignments();
|
238
|
+
|
239
|
+
public void setAssignments(List<List<String>> assignments);
|
240
|
+
}
|
241
|
+
|
242
|
+
private static Logger logger = LoggerFactory.getLogger(KafkaInputPlugin.class);
|
243
|
+
|
244
|
+
@Override
|
245
|
+
public ConfigDiff transaction(ConfigSource config,
|
246
|
+
InputPlugin.Control control)
|
247
|
+
{
|
248
|
+
PluginTask task = config.loadConfig(PluginTask.class);
|
249
|
+
|
250
|
+
Schema schema = task.getColumns().toSchema();
|
251
|
+
|
252
|
+
Properties props = new Properties();
|
253
|
+
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, task.getBrokers());
|
254
|
+
KafkaConsumer<Bytes, Bytes> consumer = new KafkaConsumer<>(props, new BytesDeserializer(),
|
255
|
+
new BytesDeserializer());
|
256
|
+
int maxTaskCount = Runtime.getRuntime().availableProcessors() * 2;
|
257
|
+
|
258
|
+
List<List<String>> assignments = buildAssignments(consumer, task.getTopics(), maxTaskCount);
|
259
|
+
int taskCount = Math.min(assignments.size(), maxTaskCount);
|
260
|
+
|
261
|
+
task.setAssignments(assignments);
|
262
|
+
|
263
|
+
return resume(task.dump(), schema, taskCount, control);
|
264
|
+
}
|
265
|
+
|
266
|
+
private List<List<String>> buildAssignments(KafkaConsumer<?, ?> consumer, List<String> topics,
|
267
|
+
int maxTaskCount)
|
268
|
+
{
|
269
|
+
List<List<String>> assignments = IntStream.range(0, maxTaskCount)
|
270
|
+
.mapToObj(n -> new ArrayList<String>()).collect(Collectors.toList());
|
271
|
+
int taskIndex = 0;
|
272
|
+
for (String topic : topics) {
|
273
|
+
for (PartitionInfo partitionInfo : consumer.partitionsFor(topic)) {
|
274
|
+
List<String> list = assignments.get(taskIndex);
|
275
|
+
if (list == null) {
|
276
|
+
list = new ArrayList<>();
|
277
|
+
}
|
278
|
+
list.add(String.format("%s:%d", partitionInfo.topic(), partitionInfo.partition()));
|
279
|
+
taskIndex += 1;
|
280
|
+
taskIndex = taskIndex % maxTaskCount;
|
281
|
+
}
|
282
|
+
}
|
283
|
+
|
284
|
+
return assignments;
|
285
|
+
}
|
286
|
+
|
287
|
+
private List<TopicPartition> buildTopicPartitions(List<List<String>> assignments, int taskIndex)
|
288
|
+
{
|
289
|
+
List<TopicPartition> topicPartitions = new CopyOnWriteArrayList<>();
|
290
|
+
assignments.get(taskIndex).forEach(assignmentInfo -> {
|
291
|
+
String[] assignmentInfoArray = assignmentInfo.split(":");
|
292
|
+
TopicPartition topicPartition = new TopicPartition(assignmentInfoArray[0],
|
293
|
+
Integer.parseInt(assignmentInfoArray[1]));
|
294
|
+
topicPartitions.add(topicPartition);
|
295
|
+
});
|
296
|
+
|
297
|
+
return topicPartitions;
|
298
|
+
}
|
299
|
+
|
300
|
+
@Override
|
301
|
+
public ConfigDiff resume(TaskSource taskSource,
|
302
|
+
Schema schema, int taskCount,
|
303
|
+
InputPlugin.Control control)
|
304
|
+
{
|
305
|
+
control.run(taskSource, schema, taskCount);
|
306
|
+
return Exec.newConfigDiff();
|
307
|
+
}
|
308
|
+
|
309
|
+
@Override
|
310
|
+
public void cleanup(TaskSource taskSource,
|
311
|
+
Schema schema, int taskCount,
|
312
|
+
List<TaskReport> successTaskReports)
|
313
|
+
{
|
314
|
+
}
|
315
|
+
|
316
|
+
@Override
|
317
|
+
public TaskReport run(TaskSource taskSource,
|
318
|
+
Schema schema, int taskIndex,
|
319
|
+
PageOutput output)
|
320
|
+
{
|
321
|
+
PluginTask task = taskSource.loadTask(PluginTask.class);
|
322
|
+
|
323
|
+
Properties props = new Properties();
|
324
|
+
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, task.getBrokers());
|
325
|
+
task.getOtherConsumerConfigs().forEach(props::setProperty);
|
326
|
+
List<TopicPartition> topicPartitions = buildTopicPartitions(task.getAssignments(), taskIndex);
|
327
|
+
switch (task.getRecordSerializeFormat()) {
|
328
|
+
case JSON:
|
329
|
+
JsonInputProcess jsonInputProcess = new JsonInputProcess(task, schema, output, props,
|
330
|
+
topicPartitions);
|
331
|
+
jsonInputProcess.run();
|
332
|
+
break;
|
333
|
+
case AVRO_WITH_SCHEMA_REGISTRY:
|
334
|
+
AvroInputProcess avroInputProcess = new AvroInputProcess(task, schema, output, props,
|
335
|
+
topicPartitions);
|
336
|
+
avroInputProcess.run();
|
337
|
+
break;
|
338
|
+
default:
|
339
|
+
throw new ConfigException("Unknown record_serialization_format");
|
340
|
+
}
|
341
|
+
|
342
|
+
TaskReport taskReport = Exec.newTaskReport();
|
343
|
+
return taskReport;
|
344
|
+
}
|
345
|
+
|
346
|
+
abstract static class AbstractInputProcess<V>
|
347
|
+
{
|
348
|
+
protected final PluginTask task;
|
349
|
+
private final Schema schema;
|
350
|
+
private final PageOutput output;
|
351
|
+
protected final Properties props;
|
352
|
+
private final List<TopicPartition> topicPartitions;
|
353
|
+
protected final TimestampParser[] timestampParsers;
|
354
|
+
|
355
|
+
protected AbstractInputProcess(PluginTask task, Schema schema,
|
356
|
+
PageOutput output, Properties props,
|
357
|
+
List<TopicPartition> topicPartitions)
|
358
|
+
{
|
359
|
+
this.task = task;
|
360
|
+
this.schema = schema;
|
361
|
+
this.output = output;
|
362
|
+
this.props = props;
|
363
|
+
this.topicPartitions = topicPartitions;
|
364
|
+
this.timestampParsers = Timestamps.newTimestampColumnParsers(task, task.getColumns());
|
365
|
+
}
|
366
|
+
|
367
|
+
public abstract KafkaConsumer<Bytes, V> getConsumer();
|
368
|
+
|
369
|
+
public abstract AbstractKafkaInputColumnVisitor<V> getColumnVisitor(PageBuilder pageBuilder);
|
370
|
+
|
371
|
+
public void run()
|
372
|
+
{
|
373
|
+
try (KafkaConsumer<Bytes, V> consumer = getConsumer()) {
|
374
|
+
Map<TopicPartition, Long> offsetsForTermination = task
|
375
|
+
.getTerminationMode()
|
376
|
+
.getOffsetsForTermination(consumer, topicPartitions);
|
377
|
+
|
378
|
+
assignAndSeek(task, topicPartitions, offsetsForTermination, consumer);
|
379
|
+
|
380
|
+
BufferAllocator allocator = Exec.getBufferAllocator();
|
381
|
+
try (PageBuilder pageBuilder = new PageBuilder(allocator, schema, output)) {
|
382
|
+
final AbstractKafkaInputColumnVisitor<V> columnVisitor = getColumnVisitor(pageBuilder);
|
383
|
+
|
384
|
+
boolean reassign = false;
|
385
|
+
int emptyPollingCount = 0;
|
386
|
+
|
387
|
+
while (!topicPartitions.isEmpty()) {
|
388
|
+
if (reassign) {
|
389
|
+
consumer.assign(topicPartitions);
|
390
|
+
}
|
391
|
+
|
392
|
+
ConsumerRecords<Bytes, V> records = consumer
|
393
|
+
.poll(Duration.ofMillis(task.getFetchMaxWaitMs()));
|
394
|
+
|
395
|
+
if (records.isEmpty()) {
|
396
|
+
emptyPollingCount += 1;
|
397
|
+
logger.info("polling results are empty. remaining count is {}",
|
398
|
+
task.getMaxEmptyPollings() - emptyPollingCount);
|
399
|
+
if (emptyPollingCount >= task.getMaxEmptyPollings()) {
|
400
|
+
break;
|
401
|
+
}
|
402
|
+
}
|
403
|
+
|
404
|
+
for (ConsumerRecord<Bytes, V> record : records) {
|
405
|
+
if (record.value() != null) {
|
406
|
+
columnVisitor.reset(record);
|
407
|
+
schema.visitColumns(columnVisitor);
|
408
|
+
pageBuilder.addRecord();
|
409
|
+
}
|
410
|
+
|
411
|
+
TopicPartition topicPartition = new TopicPartition(record.topic(),
|
412
|
+
record.partition());
|
413
|
+
if (task.getTerminationMode() == TerminationMode.OFFSET_AT_START
|
414
|
+
&& record.offset() >= offsetsForTermination.get(topicPartition) - 1) {
|
415
|
+
reassign = true;
|
416
|
+
topicPartitions.remove(topicPartition);
|
417
|
+
}
|
418
|
+
}
|
419
|
+
}
|
420
|
+
|
421
|
+
pageBuilder.finish();
|
422
|
+
}
|
423
|
+
}
|
424
|
+
}
|
425
|
+
|
426
|
+
private void assignAndSeek(PluginTask task,
|
427
|
+
List<TopicPartition> topicPartitions, Map<TopicPartition, Long> offsetsForTermination,
|
428
|
+
KafkaConsumer<?, ?> consumer)
|
429
|
+
{
|
430
|
+
consumer.assign(topicPartitions);
|
431
|
+
|
432
|
+
task.getSeekMode().seek(consumer, topicPartitions, task.getTimestampForSeeking());
|
433
|
+
|
434
|
+
for (TopicPartition topicPartition : topicPartitions) {
|
435
|
+
long position = consumer.position(topicPartition);
|
436
|
+
if (position >= offsetsForTermination.get(topicPartition)) {
|
437
|
+
topicPartitions.remove(topicPartition);
|
438
|
+
}
|
439
|
+
}
|
440
|
+
|
441
|
+
consumer.assign(topicPartitions);
|
442
|
+
}
|
443
|
+
}
|
444
|
+
|
445
|
+
static class JsonInputProcess extends AbstractInputProcess<ObjectNode>
|
446
|
+
{
|
447
|
+
JsonInputProcess(PluginTask task, Schema schema,
|
448
|
+
PageOutput output, Properties props,
|
449
|
+
List<TopicPartition> topicPartitions)
|
450
|
+
{
|
451
|
+
super(task, schema, output, props, topicPartitions);
|
452
|
+
}
|
453
|
+
|
454
|
+
@Override
|
455
|
+
public KafkaConsumer<Bytes, ObjectNode> getConsumer()
|
456
|
+
{
|
457
|
+
return new KafkaConsumer<>(props, new BytesDeserializer(), new KafkaJsonDeserializer());
|
458
|
+
}
|
459
|
+
|
460
|
+
@Override
|
461
|
+
public AbstractKafkaInputColumnVisitor<ObjectNode> getColumnVisitor(PageBuilder pageBuilder)
|
462
|
+
{
|
463
|
+
return new JsonFormatColumnVisitor(task, pageBuilder, timestampParsers);
|
464
|
+
}
|
465
|
+
}
|
466
|
+
|
467
|
+
static class AvroInputProcess extends AbstractInputProcess<Object>
|
468
|
+
{
|
469
|
+
protected AvroInputProcess(PluginTask task, Schema schema, PageOutput output,
|
470
|
+
Properties props, List<TopicPartition> topicPartitions)
|
471
|
+
{
|
472
|
+
super(task, schema, output, props, topicPartitions);
|
473
|
+
}
|
474
|
+
|
475
|
+
private KafkaAvroDeserializer buildKafkaAvroDeserializer()
|
476
|
+
{
|
477
|
+
KafkaAvroDeserializer kafkaAvroDeserializer = new KafkaAvroDeserializer();
|
478
|
+
|
479
|
+
String schemaRegistryUrl = task.getSchemaRegistryUrl().orElseThrow(
|
480
|
+
() -> new ConfigException("avro_with_schema_registry format needs schema_registry_url"));
|
481
|
+
|
482
|
+
ImmutableMap.Builder<String, String> builder = ImmutableMap.<String, String>builder()
|
483
|
+
.put(AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl);
|
484
|
+
|
485
|
+
if (task.getValueSubjectNameStrategy().isPresent()) {
|
486
|
+
builder.put(AbstractKafkaSchemaSerDeConfig.VALUE_SUBJECT_NAME_STRATEGY,
|
487
|
+
task.getValueSubjectNameStrategy().get());
|
488
|
+
}
|
489
|
+
Map<String, String> avroDeserializerConfig = builder.build();
|
490
|
+
kafkaAvroDeserializer.configure(avroDeserializerConfig, false);
|
491
|
+
|
492
|
+
return kafkaAvroDeserializer;
|
493
|
+
}
|
494
|
+
|
495
|
+
@Override
|
496
|
+
public KafkaConsumer<Bytes, Object> getConsumer()
|
497
|
+
{
|
498
|
+
return new KafkaConsumer<>(props, new BytesDeserializer(), buildKafkaAvroDeserializer());
|
499
|
+
}
|
500
|
+
|
501
|
+
@Override
|
502
|
+
public AbstractKafkaInputColumnVisitor<Object> getColumnVisitor(PageBuilder pageBuilder)
|
503
|
+
{
|
504
|
+
return new AvroFormatColumnVisitor(task, pageBuilder, timestampParsers);
|
505
|
+
}
|
506
|
+
}
|
507
|
+
|
508
|
+
@Override
|
509
|
+
public ConfigDiff guess(ConfigSource config)
|
510
|
+
{
|
511
|
+
return Exec.newConfigDiff();
|
512
|
+
}
|
513
|
+
}
|