[
https://issues.apache.org/jira/browse/BEAM-5496?focusedWorklogId=158437&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-158437
]
ASF GitHub Bot logged work on BEAM-5496:
----------------------------------------
Author: ASF GitHub Bot
Created on: 25/Oct/18 01:48
Start Date: 25/Oct/18 01:48
Worklog Time Spent: 10m
Work Description: lukecwik closed pull request #6701: [BEAM-5496] Fixes
bug of MqttIO fails to deserialize checkpoint
URL: https://github.com/apache/beam/pull/6701
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
index 2d7734eabfb..cc9c8adb61a 100644
--- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
+++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
@@ -28,6 +28,7 @@
import java.util.Collections;
import java.util.List;
import java.util.NoSuchElementException;
+import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import javax.annotation.Nullable;
@@ -305,12 +306,16 @@ public void populateDisplayData(DisplayData.Builder
builder) {
@VisibleForTesting
static class MqttCheckpointMark implements UnboundedSource.CheckpointMark,
Serializable {
- private String clientId;
- private Instant oldestMessageTimestamp = Instant.now();
- private transient List<Message> messages = new ArrayList<>();
+ @VisibleForTesting String clientId;
+ @VisibleForTesting Instant oldestMessageTimestamp = Instant.now();
+ @VisibleForTesting transient List<Message> messages = new ArrayList<>();
public MqttCheckpointMark() {}
+ public MqttCheckpointMark(String id) {
+ clientId = id;
+ }
+
public void add(Message message, Instant timestamp) {
if (timestamp.isBefore(oldestMessageTimestamp)) {
oldestMessageTimestamp = timestamp;
@@ -335,8 +340,26 @@ public void finalizeCheckpoint() {
// set an empty list to messages when deserialize
private void readObject(java.io.ObjectInputStream stream)
throws IOException, ClassNotFoundException {
+ stream.defaultReadObject();
messages = new ArrayList<>();
}
+
+ @Override
+ public boolean equals(Object other) {
+ if (other instanceof MqttCheckpointMark) {
+ MqttCheckpointMark that = (MqttCheckpointMark) other;
+ return Objects.equals(this.clientId, that.clientId)
+ && Objects.equals(this.oldestMessageTimestamp,
that.oldestMessageTimestamp)
+ && Objects.deepEquals(this.messages, that.messages);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(clientId, oldestMessageTimestamp, messages);
+ }
}
@VisibleForTesting
diff --git
a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
index 05c07c9a68e..b4fd2da9303 100644
---
a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
+++
b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
@@ -20,10 +20,15 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
import java.net.ServerSocket;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Set;
+import java.util.UUID;
import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.Connection;
@@ -263,6 +268,25 @@ public void testWrite() throws Exception {
}
}
+ @Test
+ public void testReadObject() throws Exception {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ ObjectOutputStream out = new ObjectOutputStream(bos);
+ MqttIO.MqttCheckpointMark cp1 = new
MqttIO.MqttCheckpointMark(UUID.randomUUID().toString());
+ out.writeObject(cp1);
+
+ ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
+ ObjectInputStream in = new ObjectInputStream(bis);
+ MqttIO.MqttCheckpointMark cp2 = (MqttIO.MqttCheckpointMark)
in.readObject();
+
+ // there should be no bytes left in the stream
+ assertEquals(0, in.available());
+ // the number of messages of the decoded checkpoint should be zero
+ assertEquals(0, cp2.messages.size());
+ assertEquals(cp1.clientId, cp2.clientId);
+ assertEquals(cp1.oldestMessageTimestamp, cp2.oldestMessageTimestamp);
+ }
+
@After
public void stopBroker() throws Exception {
if (brokerService != null) {
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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: 158437)
Time Spent: 1.5h (was: 1h 20m)
> MqttIO fails to deserialize checkpoint
> --------------------------------------
>
> Key: BEAM-5496
> URL: https://issues.apache.org/jira/browse/BEAM-5496
> Project: Beam
> Issue Type: Bug
> Components: io-java-mqtt
> Reporter: Luke Cwik
> Assignee: Island Chen
> Priority: Major
> Fix For: 2.9.0
>
> Time Spent: 1.5h
> Remaining Estimate: 0h
>
> Source of bug report:
> [https://lists.apache.org/thread.html/3de5a946bcb539dea9f18a31f712d6af5b66f9fbb6b01eed452c5afb@%3Cdev.beam.apache.org%3E]
>
> There is a bug of the built-in MqttIO, please check the
> <https://github.com/apache/beam/blob/master/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java#L336>,
> this readObject() method forget to invoke the "stream.defaultReadObject()"
> method.
>
> {code:java}
> // set an empty list to messages when deserialize
> private void readObject(java.io.ObjectInputStream stream)
> throws IOException, ClassNotFoundException {
> messages = new ArrayList<>();
> }{code}
>
> So there is an exception while the runner tried to deserialize the checkpoint
> object.
> {code:java}
> java.lang.RuntimeException: org.apache.beam.sdk.coders.CoderException: 95
> unexpected extra bytes after decoding
> org.apache.beam.sdk.io.mqtt.MqttIO$MqttCheckpointMark@6764e219
> at
> org.apache.beam.runners.direct.DirectRunner$DirectPipelineResult.waitUntilFinish(DirectRunner.java:340)
> ...{code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)