nrj179 opened a new issue, #30078:
URL: https://github.com/apache/beam/issues/30078
### What happened?
With Direct Runner it throws the below exception :
```
org.apache.beam.sdk.Pipeline$PipelineExecutionException:
java.lang.RuntimeException: org.apache.beam.sdk.coders.CoderException: cannot
encode a null String
at
org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:377)
at
org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:345)
at
org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:218)
at org.apache.beam.runners.direct.DirectRunner.run(DirectRunner.java:67)
Caused by: org.apache.beam.sdk.coders.CoderException: cannot encode a null
String
at
org.apache.beam.sdk.coders.StringUtf8Coder.encode(StringUtf8Coder.java:74)
at
org.apache.beam.sdk.coders.StringUtf8Coder.encode(StringUtf8Coder.java:68)
at
org.apache.beam.sdk.coders.StringUtf8Coder.encode(StringUtf8Coder.java:37)
at
org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.encode(PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.java:64)
```
With dataflow runner, it does not receive the message it all.
RCA:
Below method does not set "setNeedsAttributes(true)" and
"setNeedsMessageId(true)" like PubsubIO.readMessagesWithAttributesAndMessageId.
```
public static Read<PubsubMessage>
readMessagesWithAttributesAndMessageIdAndOrderingKey() {
return Read.newBuilder()
.setCoder(PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.of())
.setNeedsOrderingKey(true)
.build();
}
```
Due to this, in the **PubsubUnboundedSource.expand** method, function which
serializes only payload is picked (not the one which serializes message
attributes, ordering key and message Id) before passing it to
**_PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.encode_** method.
```
if (getNeedsAttributes() || getNeedsMessageId()) {
function = new PubsubMessages.ParsePubsubMessageProtoAsPayload();
} else {
function = new DeserializeBytesIntoPubsubMessagePayloadOnly();
}
```
So it always fail while trying to encode the MessageId in
PubsubMessageWithAttributesAndMessageIdAndOrderingKeyCoder.encode.
```
@Override
public void encode(PubsubMessage value, OutputStream outStream) throws
IOException {
PAYLOAD_CODER.encode(value.getPayload(), outStream);
ATTRIBUTES_CODER.encode(value.getAttributeMap(), outStream);
MESSAGE_ID_CODER.encode(value.getMessageId(), outStream); // fails here
// TODO(discuss what to do with publish_time field)
PUBLISH_TIME_CODER.encode(Timestamp.getDefaultInstance(), outStream);
ORDERING_KEY_CODER.encode(value.getOrderingKey(), outStream);
}
```
Tested with latest beam version 2.53.0
### Issue Priority
Priority: 1 (data loss / total loss of function)
### Issue Components
- [ ] Component: Python SDK
- [X] Component: Java SDK
- [ ] Component: Go SDK
- [ ] Component: Typescript SDK
- [ ] Component: IO connector
- [ ] Component: Beam YAML
- [ ] Component: Beam examples
- [ ] Component: Beam playground
- [ ] Component: Beam katas
- [ ] Component: Website
- [ ] Component: Spark Runner
- [ ] Component: Flink Runner
- [ ] Component: Samza Runner
- [ ] Component: Twister2 Runner
- [ ] Component: Hazelcast Jet Runner
- [ ] Component: Google Cloud Dataflow Runner
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]