[
https://issues.apache.org/jira/browse/BEAM-7029?focusedWorklogId=286048&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-286048
]
ASF GitHub Bot logged work on BEAM-7029:
----------------------------------------
Author: ASF GitHub Bot
Created on: 31/Jul/19 16:44
Start Date: 31/Jul/19 16:44
Worklog Time Spent: 10m
Work Description: manuelaguilar commented on issue #8251: [BEAM-7029] Add
KafkaIO.Read as external transform
URL: https://github.com/apache/beam/pull/8251#issuecomment-516928565
@mxm I've been trying to use the ReadFromKafka tranform (kafka.py) via Flink
with the portable runner, but it seems the value coder resolves to
ByteArrayCoder instead of KafkaRecordCoder. Am I missing a registration step?
My transform is similar to the one in the unit test (flink_runner_test.py).
Here are some logs with some extra INFO I added:
`
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(1/4)] INFO org.apache.beam.sdk.coders.LengthPrefixCoder - Creating length
prefix coder with value coder class org.apache.beam.sdk.coders.ByteArrayCoder
[grpc-default-executor-1] INFO org.apache.beam.fn.harness.FnHarness - Fn
Harness started
[grpc-default-executor-1] INFO org.apache.beam.fn.harness.FnHarness -
Entering instruction processing loop
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(2/4)] INFO org.apache.beam.runners.core.construction.CoderTranslators -
Getting length prefix coder for type class
org.apache.beam.sdk.coders.ByteArrayCoder
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(2/4)] INFO org.apache.beam.sdk.coders.LengthPrefixCoder - Creating length
prefix coder with value coder class org.apache.beam.sdk.coders.ByteArrayCoder
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(3/4)] INFO org.apache.beam.runners.core.construction.CoderTranslators -
Getting length prefix coder for type class
org.apache.beam.sdk.coders.ByteArrayCoder
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(3/4)] INFO org.apache.beam.sdk.coders.LengthPrefixCoder - Creating length
prefix coder with value coder class org.apache.beam.sdk.coders.ByteArrayCoder
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(4/4)] INFO org.apache.beam.sdk.coders.LengthPrefixCoder - Creating length
prefix coder with value coder class org.apache.beam.sdk.coders.ByteArrayCoder
...
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(4/4)] INFO org.apache.beam.sdk.coders.LengthPrefixCoder - Encoding value of
type class org.apache.beam.sdk.io.kafka.KafkaRecord with coder of type class
org.apache.beam.sdk.coders.ByteArrayCoder
[pool-13-thread-1] INFO org.apache.beam.sdk.io.kafka.KafkaUnboundedSource -
Reader-3: Returning from consumer pool loop
[Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(4/4)] INFO org.apache.flink.runtime.taskmanager.Task - Source: Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/KafkaIO.Read/Read(KafkaUnboundedSource)
-> Flat Map -> [1]Read From
Kafka/ExternalTransform(beam:external:java:kafka:read:v1)/Remove Kafka Metadata
(4/4) (6eabba22b9485eeaf271d72fffdc08b1) switched from RUNNING to FAILED.
java.lang.ClassCastException: org.apache.beam.sdk.io.kafka.KafkaRecord
cannot be cast to [B
at
org.apache.beam.sdk.coders.ByteArrayCoder.encode(ByteArrayCoder.java:44)
at
org.apache.beam.sdk.coders.LengthPrefixCoder.encode(LengthPrefixCoder.java:66)
at
org.apache.beam.sdk.values.ValueWithRecordId$ValueWithRecordIdCoder.encode(ValueWithRecordId.java:112)
at
org.apache.beam.sdk.values.ValueWithRecordId$ValueWithRecordIdCoder.encode(ValueWithRecordId.java:87)
at
org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.encode(WindowedValue.java:537)
at
org.apache.beam.sdk.util.WindowedValue$FullWindowedValueCoder.encode(WindowedValue.java:486)
at
org.apache.beam.sdk.util.CoderUtils.encodeToSafeStream(CoderUtils.java:82)
at
org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:66)
at
org.apache.beam.sdk.util.CoderUtils.encodeToByteArray(CoderUtils.java:51)
at org.apache.beam.sdk.util.CoderUtils.clone(CoderUtils.java:141)
at
org.apache.beam.runners.flink.translation.types.CoderTypeSerializer.copy(CoderTypeSerializer.java:89)
at
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.pushToOperator(OperatorChain.java:577)
at
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:554)
at
org.apache.flink.streaming.runtime.tasks.OperatorChain$CopyingChainingOutput.collect(OperatorChain.java:534)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:718)
at
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.collect(AbstractStreamOperator.java:696)
at
org.apache.flink.streaming.api.operators.StreamSourceContexts$ManualWatermarkContext.processAndCollectWithTimestamp(StreamSourceContexts.java:310)
at
org.apache.flink.streaming.api.operators.StreamSourceContexts$WatermarkContext.collectWithTimestamp(StreamSourceContexts.java:409)
at
org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.emitElement(UnboundedSourceWrapper.java:334)
at
org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper.run(UnboundedSourceWrapper.java:239)
at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:93)
at
org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:57)
at
org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:97)
at
org.apache.flink.streaming.runtime.tasks.StoppableSourceStreamTask.run(StoppableSourceStreamTask.java:45)
at
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:300)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:711)
at java.lang.Thread.run(Thread.java:748)
`
----------------------------------------------------------------
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: 286048)
Time Spent: 13h 40m (was: 13.5h)
> 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
> Fix For: 2.13.0
>
> Time Spent: 13h 40m
> 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.14#76016)