[
https://issues.apache.org/jira/browse/BEAM-7029?focusedWorklogId=225547&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-225547
]
ASF GitHub Bot logged work on BEAM-7029:
----------------------------------------
Author: ASF GitHub Bot
Created on: 10/Apr/19 13:22
Start Date: 10/Apr/19 13:22
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #8251: [BEAM-7029] Add
KafkaIO.Read as external transform
URL: https://github.com/apache/beam/pull/8251#discussion_r273932432
##########
File path:
sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
##########
@@ -386,6 +397,114 @@
abstract Builder<K, V> setOffsetConsumerConfig(Map<String, Object>
offsetConsumerConfig);
abstract Read<K, V> build();
+
+ @Override
+ public PTransform<PBegin, PCollection<KV<K, V>>> buildExternal(
+ External.Configuration config) {
+ ImmutableMap.Builder<String, Object> mapBuilder =
ImmutableMap.builder();
+ for (KV<byte[], byte[]> kv : config.consumerConfig) {
+ String key = utf8String(kv.getKey());
+ String value = utf8String(kv.getValue());
+ mapBuilder.put(key, value);
+ }
+ setConsumerConfig(mapBuilder.build());
+
+ ImmutableList.Builder<String> listBuilder = ImmutableList.builder();
+ for (byte[] topic : config.topics) {
+ listBuilder.add(utf8String(topic));
+ }
+ setTopics(listBuilder.build());
+
+ String keyDeserializerClassName = utf8String(config.keyDeserializer);
+ Class keyDeserializer = resolveClass(keyDeserializerClassName);
+ setKeyDeserializer(keyDeserializer);
+ setKeyCoder(resolveCoder(keyDeserializer));
+
+ String valueDeserializerClassName =
utf8String(config.valueDeserializer);
+ Class valueDeserializer = resolveClass(valueDeserializerClassName);
+ setValueDeserializer(valueDeserializer);
+ setValueCoder(resolveCoder(valueDeserializer));
+
+ // Set required defaults
+ setTopicPartitions(Collections.emptyList());
+ setConsumerFactoryFn(Read.KAFKA_CONSUMER_FACTORY_FN);
+ setMaxNumRecords(Long.MAX_VALUE);
+ setCommitOffsetsInFinalizeEnabled(false);
+ setTimestampPolicyFactory(TimestampPolicyFactory.withProcessingTime());
+ // We do not include Metadata until we can encode KafkaRecords
cross-language
+ return build().withoutMetadata();
+ }
+
+ private static Coder resolveCoder(Class deserializer) {
+ for (Method method : deserializer.getDeclaredMethods()) {
+ if (method.getName().equals("deserialize")) {
+ Class<?> returnType = method.getReturnType();
+ if (returnType.equals(Object.class)) {
+ continue;
+ }
+ if (returnType.equals(Bytes.class)) {
+ return ByteArrayCoder.of();
+ } else if (returnType.equals(Integer.class)) {
+ return VarIntCoder.of();
+ } else if (returnType.equals(Long.class)) {
+ return VarLongCoder.of();
+ } else {
+ throw new RuntimeException("Couldn't infer Coder from " +
deserializer);
+ }
+ }
+ }
+ throw new RuntimeException("Couldn't resolve coder for Deserializer: "
+ deserializer);
+ }
+
+ private static Class resolveClass(String className) {
+ try {
+ return Class.forName(className);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not find deserializer class: " +
className);
+ }
+ }
+
+ private static String utf8String(byte[] bytes) {
+ return new String(bytes, Charsets.UTF_8);
+ }
+ }
+
+ /** Exposes GenerateSequence as an external transform for cross-language
usage. */
Review comment:
Fixed.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 225547)
Time Spent: 5h 50m (was: 5h 40m)
> Support KafkaIO to be configured externally for use with other SDKs
> -------------------------------------------------------------------
>
> Key: BEAM-7029
> URL: https://issues.apache.org/jira/browse/BEAM-7029
> Project: Beam
> Issue Type: New Feature
> Components: io-java-kafka, runner-flink, sdk-py-core
> Reporter: Maximilian Michels
> Assignee: Maximilian Michels
> Priority: Major
> Time Spent: 5h 50m
> Remaining Estimate: 0h
>
> As of BEAM-6730, we can externally configure existing transforms from SDKs.
> We should add more useful transforms then just {{GenerateSequence}}.
> {{KafkaIO}} is a good candidate.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)