embulk-output-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.
Files changed (51) hide show
  1. checksums.yaml +7 -0
  2. data/.gitignore +12 -0
  3. data/LICENSE.txt +21 -0
  4. data/README.md +110 -0
  5. data/build.gradle +113 -0
  6. data/classpath/audience-annotations-0.5.0.jar +0 -0
  7. data/classpath/avro-1.9.0.jar +0 -0
  8. data/classpath/common-config-5.3.0.jar +0 -0
  9. data/classpath/common-utils-5.3.0.jar +0 -0
  10. data/classpath/commons-compress-1.18.jar +0 -0
  11. data/classpath/embulk-output-kafka-0.1.0.jar +0 -0
  12. data/classpath/jackson-annotations-2.9.0.jar +0 -0
  13. data/classpath/jackson-core-2.9.9.jar +0 -0
  14. data/classpath/jackson-databind-2.9.9.jar +0 -0
  15. data/classpath/jline-0.9.94.jar +0 -0
  16. data/classpath/jsr305-3.0.2.jar +0 -0
  17. data/classpath/kafka-avro-serializer-5.3.0.jar +0 -0
  18. data/classpath/kafka-clients-5.3.0-ccs.jar +0 -0
  19. data/classpath/kafka-schema-registry-client-5.3.0.jar +0 -0
  20. data/classpath/lz4-java-1.6.0.jar +0 -0
  21. data/classpath/netty-3.10.6.Final.jar +0 -0
  22. data/classpath/slf4j-api-1.7.26.jar +0 -0
  23. data/classpath/snappy-java-1.1.7.3.jar +0 -0
  24. data/classpath/spotbugs-annotations-3.1.9.jar +0 -0
  25. data/classpath/zkclient-0.10.jar +0 -0
  26. data/classpath/zookeeper-3.4.14.jar +0 -0
  27. data/classpath/zstd-jni-1.4.0-1.jar +0 -0
  28. data/config/checkstyle/checkstyle.xml +128 -0
  29. data/config/checkstyle/default.xml +108 -0
  30. data/gradle/wrapper/gradle-wrapper.jar +0 -0
  31. data/gradle/wrapper/gradle-wrapper.properties +5 -0
  32. data/gradlew +172 -0
  33. data/gradlew.bat +84 -0
  34. data/lib/embulk/output/kafka.rb +3 -0
  35. data/src/main/java/org/embulk/output/kafka/AvroFormatColumnVisitor.java +189 -0
  36. data/src/main/java/org/embulk/output/kafka/JsonFormatColumnVisitor.java +103 -0
  37. data/src/main/java/org/embulk/output/kafka/KafkaJsonSerializer.java +23 -0
  38. data/src/main/java/org/embulk/output/kafka/KafkaOutputColumnVisitor.java +53 -0
  39. data/src/main/java/org/embulk/output/kafka/KafkaOutputPlugin.java +323 -0
  40. data/src/main/java/org/embulk/output/kafka/RecordProducerFactory.java +105 -0
  41. data/src/test/java/org/embulk/output/kafka/TestKafkaOutputPlugin.java +5 -0
  42. data/src/test/resources/SimpleRecord.avsc +9 -0
  43. data/src/test/resources/config_complex.yml +26 -0
  44. data/src/test/resources/config_complex_avro.yml +43 -0
  45. data/src/test/resources/config_simple.yml +22 -0
  46. data/src/test/resources/config_simple_avro.yml +32 -0
  47. data/src/test/resources/config_simple_avro_avsc_file.yml +25 -0
  48. data/src/test/resources/config_with_key_column.yml +23 -0
  49. data/src/test/resources/in1.csv +4 -0
  50. data/src/test/resources/in_complex.csv +5 -0
  51. metadata +121 -0
@@ -0,0 +1,323 @@
1
+ package org.embulk.output.kafka;
2
+
3
+ import com.fasterxml.jackson.annotation.JsonCreator;
4
+ import com.fasterxml.jackson.annotation.JsonValue;
5
+ import com.fasterxml.jackson.databind.ObjectMapper;
6
+ import com.fasterxml.jackson.databind.node.ObjectNode;
7
+ import org.apache.avro.generic.GenericData;
8
+ import org.apache.kafka.clients.producer.KafkaProducer;
9
+ import org.apache.kafka.clients.producer.ProducerRecord;
10
+ import org.embulk.config.Config;
11
+ import org.embulk.config.ConfigDefault;
12
+ import org.embulk.config.ConfigDiff;
13
+ import org.embulk.config.ConfigException;
14
+ import org.embulk.config.ConfigSource;
15
+ import org.embulk.config.Task;
16
+ import org.embulk.config.TaskReport;
17
+ import org.embulk.config.TaskSource;
18
+ import org.embulk.spi.Exec;
19
+ import org.embulk.spi.OutputPlugin;
20
+ import org.embulk.spi.Page;
21
+ import org.embulk.spi.PageReader;
22
+ import org.embulk.spi.Schema;
23
+ import org.embulk.spi.TransactionalPageOutput;
24
+ import org.slf4j.Logger;
25
+ import org.slf4j.LoggerFactory;
26
+
27
+ import java.io.File;
28
+ import java.io.IOException;
29
+ import java.util.List;
30
+ import java.util.Locale;
31
+ import java.util.Map;
32
+ import java.util.Optional;
33
+ import java.util.PrimitiveIterator;
34
+ import java.util.Random;
35
+ import java.util.concurrent.atomic.AtomicInteger;
36
+
37
+ public class KafkaOutputPlugin
38
+ implements OutputPlugin
39
+ {
40
+ public enum RecordSerializeFormat
41
+ {
42
+ JSON,
43
+ AVRO_WITH_SCHEMA_REGISTRY;
44
+
45
+ @JsonValue
46
+ public String toString()
47
+ {
48
+ return name().toLowerCase(Locale.ENGLISH);
49
+ }
50
+
51
+ @JsonCreator
52
+ public static RecordSerializeFormat ofString(String name)
53
+ {
54
+ switch (name.toLowerCase(Locale.ENGLISH)) {
55
+ case "json":
56
+ return JSON;
57
+ case "avro_with_schema_registry":
58
+ return AVRO_WITH_SCHEMA_REGISTRY;
59
+ default:
60
+ }
61
+
62
+ throw new ConfigException(String.format("Unknown serialize format '%s'. Supported modes are json, avro_with_schema_registry", name));
63
+ }
64
+ }
65
+
66
+ public interface PluginTask
67
+ extends Task
68
+ {
69
+ @Config("brokers")
70
+ public List<String> getBrokers();
71
+
72
+ @Config("topic")
73
+ public String getTopic();
74
+
75
+ @Config("topic_column")
76
+ @ConfigDefault("null")
77
+ public Optional<String> getTopicColumn();
78
+
79
+ @Config("schema_registry_url")
80
+ @ConfigDefault("null")
81
+ public Optional<String> getSchemaRegistryUrl();
82
+
83
+ @Config("serialize_format")
84
+ public RecordSerializeFormat getRecordSerializeFormat();
85
+
86
+ @Config("avsc_file")
87
+ @ConfigDefault("null")
88
+ public Optional<File> getAvscFile();
89
+
90
+ @Config("avsc")
91
+ @ConfigDefault("null")
92
+ public Optional<ObjectNode> getAvsc();
93
+
94
+ @Config("key_column_name")
95
+ @ConfigDefault("null")
96
+ public Optional<String> getKeyColumnName();
97
+
98
+ @Config("record_batch_size")
99
+ @ConfigDefault("1000")
100
+ public int getRecordBatchSize();
101
+
102
+ @Config("acks")
103
+ @ConfigDefault("\"1\"")
104
+ public String getAcks();
105
+
106
+ @Config("retries")
107
+ @ConfigDefault("1")
108
+ public int getRetries();
109
+
110
+ @Config("other_producer_configs")
111
+ @ConfigDefault("{}")
112
+ public Map<String, String> getOtherProducerConfigs();
113
+ }
114
+
115
+ private static ObjectMapper objectMapper = new ObjectMapper();
116
+ private Logger logger = LoggerFactory.getLogger(getClass());
117
+ private int recordLoggingCount = 1;
118
+
119
+ @Override
120
+ public ConfigDiff transaction(ConfigSource config,
121
+ Schema schema, int taskCount,
122
+ Control control)
123
+ {
124
+ PluginTask task = config.loadConfig(PluginTask.class);
125
+
126
+ // retryable (idempotent) output:
127
+ // return resume(task.dump(), schema, taskCount, control);
128
+
129
+ // non-retryable (non-idempotent) output:
130
+ control.run(task.dump());
131
+ return Exec.newConfigDiff();
132
+ }
133
+
134
+ @Override
135
+ public ConfigDiff resume(TaskSource taskSource,
136
+ Schema schema, int taskCount,
137
+ Control control)
138
+ {
139
+ throw new UnsupportedOperationException("kafka output plugin does not support resuming");
140
+ }
141
+
142
+ @Override
143
+ public void cleanup(TaskSource taskSource,
144
+ Schema schema, int taskCount,
145
+ List<TaskReport> successTaskReports)
146
+ {
147
+ }
148
+
149
+ @Override
150
+ public TransactionalPageOutput open(TaskSource taskSource, Schema schema, int taskIndex)
151
+ {
152
+ PluginTask task = taskSource.loadTask(PluginTask.class);
153
+
154
+ switch (task.getRecordSerializeFormat()) {
155
+ case JSON:
156
+ return buildPageOutputForJson(task, schema, taskIndex);
157
+ case AVRO_WITH_SCHEMA_REGISTRY:
158
+ return buildPageOutputForAvroWithSchemaRegistry(task, schema, taskIndex);
159
+ default:
160
+ throw new ConfigException("Unknow serialize format");
161
+ }
162
+ }
163
+
164
+ private TransactionalPageOutput buildPageOutputForJson(PluginTask task, Schema schema, int taskIndex)
165
+ {
166
+ KafkaProducer<Object, ObjectNode> producer = RecordProducerFactory.getForJson(task, schema, task.getOtherProducerConfigs());
167
+
168
+ PageReader pageReader = new PageReader(schema);
169
+ PrimitiveIterator.OfLong randomLong = new Random().longs(1, Long.MAX_VALUE).iterator();
170
+ AtomicInteger counter = new AtomicInteger(0);
171
+
172
+ return new TransactionalPageOutput() {
173
+ @Override
174
+ public void add(Page page)
175
+ {
176
+ pageReader.setPage(page);
177
+ while (pageReader.nextRecord()) {
178
+ JsonFormatColumnVisitor columnVisitor = new JsonFormatColumnVisitor(task, pageReader, objectMapper);
179
+
180
+ pageReader.getSchema().visitColumns(columnVisitor);
181
+
182
+ Object recordKey = columnVisitor.recordKey;
183
+ if (recordKey == null) {
184
+ recordKey = randomLong.next();
185
+ }
186
+
187
+ String targetTopic = columnVisitor.topicName != null ? columnVisitor.topicName : task.getTopic();
188
+ ProducerRecord<Object, ObjectNode> producerRecord = new ProducerRecord<>(targetTopic, recordKey, columnVisitor.jsonNode);
189
+ producer.send(producerRecord, (metadata, exception) -> {
190
+ if (exception != null) {
191
+ logger.error("produce error", exception);
192
+ }
193
+
194
+ logger.debug("sent record: {key: {}, value: {}}", producerRecord.key(), producerRecord.value());
195
+
196
+ int current = counter.incrementAndGet();
197
+ if (current >= recordLoggingCount) {
198
+ logger.info("[task-{}] Producer sent {} records", String.format("%04d", taskIndex), current);
199
+ recordLoggingCount = recordLoggingCount * 2;
200
+ }
201
+ });
202
+ }
203
+ }
204
+
205
+ @Override
206
+ public void finish()
207
+ {
208
+ producer.flush();
209
+ }
210
+
211
+ @Override
212
+ public void close()
213
+ {
214
+ producer.close();
215
+ }
216
+
217
+ @Override
218
+ public void abort()
219
+ {
220
+ producer.flush();
221
+ producer.close();
222
+ }
223
+
224
+ @Override
225
+ public TaskReport commit()
226
+ {
227
+ return null;
228
+ }
229
+ };
230
+ }
231
+
232
+ private TransactionalPageOutput buildPageOutputForAvroWithSchemaRegistry(PluginTask task, Schema schema, int taskIndex)
233
+ {
234
+ KafkaProducer<Object, Object> producer = RecordProducerFactory.getForAvroWithSchemaRegistry(task, schema, task.getOtherProducerConfigs());
235
+
236
+ PageReader pageReader = new PageReader(schema);
237
+
238
+ org.apache.avro.Schema avroSchema = null;
239
+ if (!task.getAvsc().isPresent() && !task.getAvscFile().isPresent() || task.getAvsc().isPresent() == task.getAvscFile().isPresent()) {
240
+ throw new ConfigException("avro_with_schema_registry format needs either one of avsc and avsc_file");
241
+ }
242
+ if (task.getAvsc().isPresent()) {
243
+ avroSchema = new org.apache.avro.Schema.Parser().parse(task.getAvsc().get().toString());
244
+ }
245
+ if (task.getAvscFile().isPresent()) {
246
+ try {
247
+ avroSchema = new org.apache.avro.Schema.Parser().parse(task.getAvscFile().get());
248
+ }
249
+ catch (IOException e) {
250
+ e.printStackTrace();
251
+ throw new ConfigException("avsc_file cannot read");
252
+ }
253
+ }
254
+
255
+ final Object[] key = new Object[1];
256
+ final String[] topicName = new String[1];
257
+ PrimitiveIterator.OfLong randomLong = new Random().longs(1, Long.MAX_VALUE).iterator();
258
+
259
+ AtomicInteger counter = new AtomicInteger(0);
260
+
261
+ final org.apache.avro.Schema finalAvroSchema = avroSchema;
262
+ return new TransactionalPageOutput()
263
+ {
264
+ @Override
265
+ public void add(Page page)
266
+ {
267
+ pageReader.setPage(page);
268
+ while (pageReader.nextRecord()) {
269
+ AvroFormatColumnVisitor columnVisitor = new AvroFormatColumnVisitor(task, pageReader, finalAvroSchema, new GenericData.Record(finalAvroSchema));
270
+
271
+ pageReader.getSchema().visitColumns(columnVisitor);
272
+
273
+ Object recordKey = columnVisitor.recordKey;
274
+ if (recordKey == null) {
275
+ recordKey = randomLong.next();
276
+ }
277
+
278
+ String targetTopic = columnVisitor.topicName != null ? columnVisitor.topicName : task.getTopic();
279
+
280
+ ProducerRecord<Object, Object> producerRecord = new ProducerRecord<>(targetTopic, recordKey, columnVisitor.genericRecord);
281
+ producer.send(producerRecord, (metadata, exception) -> {
282
+ if (exception != null) {
283
+ logger.error("produce error", exception);
284
+ }
285
+
286
+ logger.debug("sent record: {key: {}, value: {}}", producerRecord.key(), producerRecord.value());
287
+
288
+ int current = counter.incrementAndGet();
289
+ if (current >= recordLoggingCount) {
290
+ logger.info("[task-{}] Producer sent {} records", String.format("%04d", taskIndex), current);
291
+ recordLoggingCount = recordLoggingCount * 2;
292
+ }
293
+ });
294
+ }
295
+ }
296
+
297
+ @Override
298
+ public void finish()
299
+ {
300
+ producer.flush();
301
+ }
302
+
303
+ @Override
304
+ public void close()
305
+ {
306
+ producer.close();
307
+ }
308
+
309
+ @Override
310
+ public void abort()
311
+ {
312
+ producer.flush();
313
+ producer.close();
314
+ }
315
+
316
+ @Override
317
+ public TaskReport commit()
318
+ {
319
+ return null;
320
+ }
321
+ };
322
+ }
323
+ }
@@ -0,0 +1,105 @@
1
+ package org.embulk.output.kafka;
2
+
3
+ import com.fasterxml.jackson.databind.node.ObjectNode;
4
+ import com.google.common.collect.ImmutableMap;
5
+ import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig;
6
+ import io.confluent.kafka.serializers.KafkaAvroSerializer;
7
+ import org.apache.kafka.clients.producer.KafkaProducer;
8
+ import org.apache.kafka.clients.producer.ProducerConfig;
9
+ import org.apache.kafka.common.serialization.DoubleSerializer;
10
+ import org.apache.kafka.common.serialization.LongSerializer;
11
+ import org.apache.kafka.common.serialization.StringSerializer;
12
+ import org.embulk.config.ConfigException;
13
+ import org.embulk.spi.Column;
14
+ import org.embulk.spi.ColumnVisitor;
15
+ import org.embulk.spi.Schema;
16
+
17
+ import java.util.Map;
18
+ import java.util.Properties;
19
+
20
+ class RecordProducerFactory
21
+ {
22
+ private RecordProducerFactory() {}
23
+
24
+ private static Properties buildProperties(KafkaOutputPlugin.PluginTask task, Schema schema, Map<String, String> configs)
25
+ {
26
+ Properties kafkaProps = new Properties();
27
+
28
+ kafkaProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, task.getBrokers());
29
+ kafkaProps.put(ProducerConfig.ACKS_CONFIG, task.getAcks());
30
+ kafkaProps.put(ProducerConfig.RETRIES_CONFIG, task.getRetries());
31
+ kafkaProps.put(ProducerConfig.BATCH_SIZE_CONFIG, task.getRecordBatchSize());
32
+
33
+ configs.forEach(kafkaProps::setProperty);
34
+
35
+ if (task.getKeyColumnName().isPresent()) {
36
+ String keyColumnName = task.getKeyColumnName().get();
37
+ Column column = schema.getColumns().stream()
38
+ .filter(c -> c.getName().equals(keyColumnName))
39
+ .findFirst()
40
+ .orElseThrow(() -> new ConfigException("key column is not found"));
41
+
42
+ column.visit(new ColumnVisitor()
43
+ {
44
+ @Override
45
+ public void booleanColumn(Column column)
46
+ {
47
+ throw new RuntimeException("boolean column is not supported for key_column_name");
48
+ }
49
+
50
+ @Override
51
+ public void longColumn(Column column)
52
+ {
53
+ kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class);
54
+ }
55
+
56
+ @Override
57
+ public void doubleColumn(Column column)
58
+ {
59
+ kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, DoubleSerializer.class);
60
+ }
61
+
62
+ @Override
63
+ public void stringColumn(Column column)
64
+ {
65
+ kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
66
+ }
67
+
68
+ @Override
69
+ public void timestampColumn(Column column)
70
+ {
71
+ throw new RuntimeException("timestamp column is not supported for key_column_name");
72
+ }
73
+
74
+ @Override
75
+ public void jsonColumn(Column column)
76
+ {
77
+ throw new RuntimeException("json column is not supported for key_column_name");
78
+ }
79
+ });
80
+ }
81
+ else {
82
+ kafkaProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class);
83
+ }
84
+
85
+ return kafkaProps;
86
+ }
87
+
88
+ static KafkaProducer<Object, ObjectNode> getForJson(KafkaOutputPlugin.PluginTask task, Schema schema, Map<String, String> configs)
89
+ {
90
+ return new KafkaProducer<>(buildProperties(task, schema, configs), null, new KafkaJsonSerializer());
91
+ }
92
+
93
+ static KafkaProducer<Object, Object> getForAvroWithSchemaRegistry(KafkaOutputPlugin.PluginTask task, Schema schema, Map<String, String> configs)
94
+ {
95
+ KafkaAvroSerializer kafkaAvroSerializer = new KafkaAvroSerializer();
96
+ String schemaRegistryUrl = task.getSchemaRegistryUrl().orElseThrow(() -> new ConfigException("avro_with_schema_registry format needs schema_registry_url"));
97
+
98
+ Map<String, String> avroSerializerConfigs = ImmutableMap.<String, String>builder()
99
+ .put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl)
100
+ .build();
101
+ kafkaAvroSerializer.configure(avroSerializerConfigs, false);
102
+
103
+ return new KafkaProducer<>(buildProperties(task, schema, configs), null, kafkaAvroSerializer);
104
+ }
105
+ }
@@ -0,0 +1,5 @@
1
+ package org.embulk.output.kafka;
2
+
3
+ public class TestKafkaOutputPlugin
4
+ {
5
+ }
@@ -0,0 +1,9 @@
1
+ {
2
+ "type": "record",
3
+ "name": "SimpleRecord",
4
+ "fields": [
5
+ {"name": "id", "type": "string"},
6
+ {"name": "int_item", "type": "long"},
7
+ {"name": "varchar_item", "type": "string"}
8
+ ]
9
+ }
@@ -0,0 +1,26 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in_complex
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: "\t"
9
+ quote: "\0"
10
+ escape: "\0"
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'time', type: timestamp, format: "%Y-%m-%dT%H:%M:%S"}
17
+ - {name: 'array', type: json}
18
+ - {name: 'data', type: json}
19
+ out:
20
+ type: kafka
21
+ topic: "json-topic"
22
+ serialize_format: json
23
+ brokers:
24
+ - "localhost:9092"
25
+ other_producer_configs:
26
+ buffer.memory: "67108864"
@@ -0,0 +1,43 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in_complex
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: "\t"
9
+ quote: "\0"
10
+ escape: "\0"
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'time', type: timestamp, format: "%Y-%m-%dT%H:%M:%S"}
17
+ - {name: 'array', type: json}
18
+ - {name: 'data', type: json}
19
+ out:
20
+ type: kafka
21
+ topic: "avro-complex-topic"
22
+ acks: all
23
+ retries: 3
24
+ brokers:
25
+ - "localhost:9092"
26
+ schema_registry_url: "http://localhost:48081/"
27
+ serialize_format: avro_with_schema_registry
28
+ other_producer_configs:
29
+ buffer.memory: "67108864"
30
+ avsc:
31
+ type: record
32
+ name: ComplexRecord
33
+ fields: [
34
+ {name: "id", type: "string"},
35
+ {name: "int_item", type: "long"},
36
+ {name: "time", type: "long", logicalType: "timestamp-milli"},
37
+ {name: "array", type: {type: "array", items: "long"}},
38
+ {name: "data", type: {type: "record", name: "InnerData", fields: [
39
+ {name: "hoge", type: "string"},
40
+ {name: "aaa", type: ["null", "string"]},
41
+ {name: "array", type: {type: "array", items: "long"}},
42
+ ]}},
43
+ ]
@@ -0,0 +1,22 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in1
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: ','
9
+ quote: '"'
10
+ escape: '"'
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'varchar_item', type: string}
17
+ out:
18
+ type: kafka
19
+ topic: "json-topic"
20
+ serialize_format: json
21
+ brokers:
22
+ - "localhost:9092"
@@ -0,0 +1,32 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in1
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: ','
9
+ quote: '"'
10
+ escape: '"'
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'varchar_item', type: string}
17
+
18
+ out:
19
+ type: kafka
20
+ topic: "avro-simple-topic"
21
+ brokers:
22
+ - "localhost:9092"
23
+ schema_registry_url: "http://localhost:48081/"
24
+ serialize_format: avro_with_schema_registry
25
+ avsc:
26
+ type: record
27
+ name: SimpleRecord
28
+ fields: [
29
+ {name: "id", type: "string"},
30
+ {name: "int_item", type: "long"},
31
+ {name: "varchar_item", type: "string"},
32
+ ]
@@ -0,0 +1,25 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in1
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: ','
9
+ quote: '"'
10
+ escape: '"'
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'varchar_item', type: string}
17
+
18
+ out:
19
+ type: kafka
20
+ topic: "avro-simple-topic"
21
+ brokers:
22
+ - "localhost:9092"
23
+ schema_registry_url: "http://localhost:48081/"
24
+ serialize_format: avro_with_schema_registry
25
+ avsc_file: ./src/test/resources/SimpleRecord.avsc
@@ -0,0 +1,23 @@
1
+ in:
2
+ type: file
3
+ path_prefix: ./src/test/resources/in1
4
+ parser:
5
+ charset: UTF-8
6
+ newline: CRLF
7
+ type: csv
8
+ delimiter: ','
9
+ quote: '"'
10
+ escape: '"'
11
+ null_string: 'NULL'
12
+ skip_header_lines: 1
13
+ columns:
14
+ - {name: 'id', type: string}
15
+ - {name: 'int_item', type: long}
16
+ - {name: 'varchar_item', type: string}
17
+ out:
18
+ type: kafka
19
+ topic: "json-topic"
20
+ serialize_format: json
21
+ brokers:
22
+ - "localhost:9092"
23
+ key_column_name: id
@@ -0,0 +1,4 @@
1
+ id,int_item,varchar_item
2
+ A001,9,a
3
+ A002,0,b
4
+ A003,9,c
@@ -0,0 +1,5 @@
1
+ id int_item time array data
2
+ A001 9 2018-02-01T12:15:18 [1,2,3] {"hoge": "fuga1", "aaa": "bbb1", "array": [1,2,3]}
3
+ A002 0 2018-02-02T12:15:18 [1,2,3] {"hoge": "fuga2", "aaa": null, "array": [4,5,6]}
4
+ A003 9 2018-02-03T12:15:18 [1,2,3] {"hoge": "fuga3", "aaa": "bbb3", "array": [7,8,9]}
5
+