From 5de6cf117ea6b10c0306263041518153ded1a01d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 12 Sep 2016 18:49:36 +0200 Subject: [PATCH 01/27] [BEAM-606] Create MqttIO --- sdks/java/io/mqtt/pom.xml | 141 ++++++ .../beam/sdk/io/mqtt/MqttCheckpointMark.java | 65 +++ .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 473 ++++++++++++++++++ .../apache/beam/sdk/io/mqtt/package-info.java | 22 + .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 162 ++++++ sdks/java/io/pom.xml | 1 + 6 files changed, 864 insertions(+) create mode 100644 sdks/java/io/mqtt/pom.xml create mode 100644 sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java create mode 100644 sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java create mode 100644 sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/package-info.java create mode 100644 sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml new file mode 100644 index 000000000000..b1efaddde617 --- /dev/null +++ b/sdks/java/io/mqtt/pom.xml @@ -0,0 +1,141 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-io-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-io-mqtt + Apache Beam :: SDKs :: Java :: IO :: MQTT + IO to read and write to a MQTT broker. + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + + + 1.0.2 + 5.13.1 + + + + + org.apache.beam + beam-sdks-java-core + + + + org.slf4j + slf4j-api + + + + joda-time + joda-time + + + + com.google.guava + guava + + + + com.google.code.findbugs + annotations + + + + org.eclipse.paho + org.eclipse.paho.client.mqttv3 + ${paho.version} + + + + + org.apache.activemq + activemq-broker + ${activemq.version} + test + + + org.apache.activemq + activemq-mqtt + ${activemq.version} + test + + + org.apache.activemq + activemq-kahadb-store + ${activemq.version} + test + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + + junit + junit + test + + + org.hamcrest + hamcrest-all + test + + + org.slf4j + slf4j-jdk14 + test + + + + \ No newline at end of file diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java new file mode 100644 index 000000000000..43d474a09e31 --- /dev/null +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mqtt; + +import java.io.Serializable; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.UnboundedSource; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.eclipse.paho.client.mqttv3.MqttMessage; + +import org.joda.time.Instant; + +/** + * Checkpoint for an unbounded {@link org.apache.beam.sdk.io.mqtt.MqttIO.Read}. + * It consists in a list of pending message (not possible to control the ACK in MQTT, all depends + * of the QoS) and the timestamp of the oldest message. + */ +@DefaultCoder(SerializableCoder.class) +public class MqttCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { + + private final List messages = new ArrayList<>(); + private Instant oldestTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + + public MqttCheckpointMark() { + } + + public Instant addMessage(MqttMessage message) { + if (messages.size() < 1) { + oldestTimestamp = Instant.now(); + } + messages.add(message); + return oldestTimestamp; + } + + public Instant getOldestTimestamp() { + return oldestTimestamp; + } + + @Override + public void finalizeCheckpoint() { + messages.clear(); + } + +} 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 new file mode 100644 index 000000000000..822c9e8a5944 --- /dev/null +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java @@ -0,0 +1,473 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mqtt; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; + +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; +import org.eclipse.paho.client.mqttv3.MqttCallback; +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttMessage; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An unbounded source for MQTT broker. + * + *

Reading from a MQTT broker

+ * + *

MqttIO source returns an unbounded collection of {@code byte[]} as + * {@code PCollection}, where {@code byte[]} is the message payload.

+ * + *

To configure a MQTT source, you have to provide a {@code ClientId}, a {@code ServerURI}, + * and eventually a {@code Topic} pattern. The following example illustrates various options + * for configuring the source:

+ * + *
{@code
+ *
+ * pipeline.apply(MqttIO.read().withClientId("my_client").withServerUri("tcp://host:11883")
+ *   .withTopic("topic")
+ *
+ * }
+ * + *

Writing to a MQTT broker

+ * + *

MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker.

+ * + *

To configure a MQTT sink, you must specify the {@code ClientId}, {@code ServerURI}, + * {@code Topic}. Eventually, you can also specify the {@code Retained} and {@code QoS} of the + * message.

+ * + *

For instance:

+ * + *
{@code
+ *
+ * pipeline
+ *   .apply(...) // provide PCollection
+ *   .MqttIO.write()
+ *     .withClientId("my-client")
+ *     .withServerUri("tcp://host:11883")
+ *     .withTopic("my-topic")
+ *     .withRetained(true)
+ *     .withQoS(2)
+ *
+ * }
+ */ +public class MqttIO { + + private static final Logger LOGGER = LoggerFactory.getLogger(MqttIO.class); + + public static Read read() { + return new Read(new UnboundedMqttSource(null, null, null, + Long.MAX_VALUE, null)); + } + + public static Write write() { + return new Write(new Write.MqttWriter(null, null, null, 1, false)); + } + + private MqttIO() { + } + + /** + * A {@link PTransform>} to read from a MQTT broker. + */ + public static class Read extends PTransform> { + + public Read withServerUri(String serverUri) { + return new Read(source.withServerUri(serverUri)); + } + + public Read withClientId(String clientId) { + return new Read(source.withClientId(clientId)); + } + + public Read withTopic(String topic) { + return new Read(source.withTopic(topic)); + } + + public Read withMaxNumRecords(long maxNumRecords) { + return new Read(source.withMaxNumRecords(maxNumRecords)); + } + + public Read withMaxReadTime(Duration maxReadTime) { + return new Read(source.withMaxReadTime(maxReadTime)); + } + + private final UnboundedMqttSource source; + + private Read(UnboundedMqttSource source) { + this.source = source; + } + + @Override + public PCollection apply(PBegin input) { + + org.apache.beam.sdk.io.Read.Unbounded unbounded = + org.apache.beam.sdk.io.Read.from(getSource()); + + PTransform> transform = unbounded; + + if (source.maxNumRecords != Long.MAX_VALUE) { + transform = unbounded.withMaxNumRecords(source.maxNumRecords); + } else if (source.maxReadTime != null) { + transform = unbounded.withMaxReadTime(source.maxReadTime); + } + + return input.getPipeline().apply(transform); + } + + @VisibleForTesting + public UnboundedMqttSource getSource() { + return source; + } + + @Override + public void validate(PBegin input) { + source.validate(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + source.populateDisplayData(builder); + } + + } + + private static class UnboundedMqttSource extends UnboundedSource { + + public UnboundedMqttSource withServerUri(String serverUri) { + return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); + } + + public UnboundedMqttSource withClientId(String clientId) { + return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); + } + + public UnboundedMqttSource withTopic(String topic) { + return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); + } + + public UnboundedMqttSource withMaxNumRecords(long maxNumRecords) { + return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); + } + + public UnboundedMqttSource withMaxReadTime(Duration maxReadTime) { + return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); + } + + private final String serverUri; + private final String clientId; + @Nullable + private final String topic; + + private final long maxNumRecords; + private final Duration maxReadTime; + + public UnboundedMqttSource(String serverUri, String clientId, String topic, + long maxNumRecords, Duration maxReadTime) { + this.serverUri = serverUri; + this.clientId = clientId; + this.topic = topic; + this.maxNumRecords = maxNumRecords; + this.maxReadTime = maxReadTime; + } + + @Override + public UnboundedReader createReader(PipelineOptions options, + MqttCheckpointMark checkpointMark) { + return new UnboundedMqttReader(this, checkpointMark); + } + + @Override + public List generateInitialSplits(int desiredNumSplits, + PipelineOptions options) { + List sources = new ArrayList<>(); + for (int i = 0; i < desiredNumSplits; i++) { + // NB: it's important that user understand the impact of MQTT message QoS + sources.add(new UnboundedMqttSource(serverUri, clientId + "-" + i, topic, maxNumRecords, + maxReadTime)); + } + return sources; + } + + @Override + public void validate() { + Preconditions.checkNotNull(serverUri, "serverUri"); + Preconditions.checkNotNull(clientId, "clientId"); + } + + @Override + public Coder getCheckpointMarkCoder() { + return SerializableCoder.of(MqttCheckpointMark.class); + } + + @Override + public Coder getDefaultOutputCoder() { + return SerializableCoder.of(byte[].class); + } + + } + + private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader + implements MqttCallback { + + private UnboundedMqttSource source; + + private MqttCheckpointMark checkpointMark; + + private MqttClient client; + private byte[] current; + private Instant currentTimestamp; + + private UnboundedMqttReader(UnboundedMqttSource source, + MqttCheckpointMark checkpointMark) { + this.source = source; + if (checkpointMark != null) { + this.checkpointMark = checkpointMark; + } else { + this.checkpointMark = new MqttCheckpointMark(); + } + this.current = null; + } + + @Override + public boolean start() throws IOException { + try { + client = new MqttClient(source.serverUri, source.clientId); + client.connect(); + if (source.topic != null) { + client.subscribe(source.topic); + } + client.setCallback(this); + return advance(); + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public boolean advance() throws IOException { + if (current == null) { + return false; + } else { + return true; + } + } + + @Override + public void close() throws IOException { + try { + if (client != null) { + client.disconnect(); + client.close(); + } + } catch (Exception e) { + throw new IOException(e); + } + } + + @Override + public Instant getWatermark() { + return checkpointMark.getOldestTimestamp(); + } + + @Override + public MqttCheckpointMark getCheckpointMark() { + return checkpointMark; + } + + @Override + public byte[] getCurrent() { + return current; + } + + @Override + public UnboundedMqttSource getCurrentSource() { + return source; + } + + @Override + public Instant getCurrentTimestamp() { + if (current == null) { + throw new NoSuchElementException(); + } + return currentTimestamp; + } + + @Override + public void connectionLost(Throwable t) { + LOGGER.warn("Lost connection to MQTT server", t); + } + + @Override + public void messageArrived(String topic, MqttMessage message) { + this.current = message.getPayload(); + this.currentTimestamp = checkpointMark.addMessage(message); + } + + @Override + public void deliveryComplete(IMqttDeliveryToken token) { + } + + } + + /** + * A {@link PTransform} to write and send a message to a MQTT server. + */ + public static class Write extends PTransform, PDone> { + + public Write withServerUri(String serverUri) { + return new Write(writer.withServerUri(serverUri)); + } + + public Write withClientId(String clientId) { + return new Write(writer.withClientId(clientId)); + } + + public Write withTopic(String topic) { + return new Write(writer.withTopic(topic)); + } + + public Write withQoS(int qos) { + return new Write(writer.withQoS(qos)); + } + + public Write withRetained(boolean retained) { + return new Write(writer.withRetained(retained)); + } + + private final MqttWriter writer; + + private Write(MqttWriter writer) { + this.writer = writer; + } + + @Override + public PDone apply(PCollection input) { + input.apply(ParDo.of(writer)); + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + writer.validate(); + } + + /** + * MQTT writer. + */ + private static class MqttWriter extends DoFn { + + private final String serverUri; + private final String clientId; + private final String topic; + private final int qos; + private final boolean retained; + + private MqttClient client; + + public MqttWriter(String serverUri, String clientId, String topic, int qos, + boolean retained) { + this.serverUri = serverUri; + this.clientId = clientId; + this.topic = topic; + this.qos = qos; + this.retained = retained; + } + + public MqttWriter withServerUri(String serverUri) { + return new MqttWriter(serverUri, clientId, topic, qos, retained); + } + + public MqttWriter withClientId(String clientId) { + return new MqttWriter(serverUri, clientId, topic, qos, retained); + } + + public MqttWriter withTopic(String topic) { + return new MqttWriter(serverUri, clientId, topic, qos, retained); + } + + public MqttWriter withQoS(int qos) { + return new MqttWriter(serverUri, clientId, topic, qos, retained); + } + + public MqttWriter withRetained(boolean retained) { + return new MqttWriter(serverUri, clientId, topic, qos, retained); + } + + public void validate() { + Preconditions.checkNotNull(serverUri, "serverUri"); + Preconditions.checkNotNull(clientId, "clientId"); + } + + @Setup + public void createMqttClient() throws Exception { + client = new MqttClient(serverUri, clientId); + client.connect(); + } + + @ProcessElement + public void processElement(ProcessContext context) throws Exception { + byte[] payload = context.element(); + MqttMessage message = new MqttMessage(); + message.setQos(qos); + message.setRetained(retained); + message.setPayload(payload); + client.publish(topic, message); + } + + @Teardown + public void closeMqttClient() throws Exception { + if (client != null) { + client.disconnect(); + client.close(); + } + } + + } + + } + +} diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/package-info.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/package-info.java new file mode 100644 index 000000000000..a50eac104d11 --- /dev/null +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/package-info.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * Transforms for reading and writing from MQTT. + */ +package org.apache.beam.sdk.io.mqtt; 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 new file mode 100644 index 000000000000..855aff721a6f --- /dev/null +++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.mqtt; + +import java.io.Serializable; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; + +import org.apache.activemq.broker.BrokerService; +import org.apache.activemq.store.memory.MemoryPersistenceAdapter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; + +import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; +import org.eclipse.paho.client.mqttv3.MqttCallback; +import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttException; +import org.eclipse.paho.client.mqttv3.MqttMessage; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests of {@link MqttIO}. + */ +@RunWith(JUnit4.class) +public class MqttIOTest implements Serializable { + + private transient BrokerService broker; + + @Before + public void startBroker() throws Exception { + broker = new BrokerService(); + broker.setUseJmx(false); + broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); + broker.addConnector(new URI("mqtt://localhost:11883")); + broker.start(); + } + + @Test + @Category(NeedsRunner.class) + public void testRead() throws Exception { + // produce message on the broker + MqttClient client = new MqttClient("tcp://localhost:11883", "test"); + client.connect(); + for (int i = 0; i < 10; i++) { + MqttMessage message = new MqttMessage(); + message.setQos(2); + message.setRetained(true); + message.setPayload("This is a test".getBytes()); + client.publish("BEAM", message); + } + client.disconnect(); + client.close(); + + Pipeline pipeline = TestPipeline.create(); + + PCollection output = pipeline.apply( + MqttIO.read() + .withClientId("BEAM_PIPELINE") + .withServerUri("tcp://localhost:11883") + .withTopic("BEAM") + .withMaxNumRecords(10)); + + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(10L); + PAssert.that(output).satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + for (byte[] element : input) { + String inputString = new String(element); + Assert.assertEquals("This is a test", inputString); + } + return null; + } + }); + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testWrite() throws Exception { + List messages = new ArrayList<>(); + MqttClient client = receive(messages); + + Pipeline pipeline = TestPipeline.create(); + + ArrayList data = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + data.add("Test".getBytes()); + } + pipeline.apply(Create.of(data)) + .apply(MqttIO.write().withClientId("BEAM_PIPELINE") + .withServerUri("tcp://localhost:11883").withTopic("BEAM_TOPIC") + .withRetained(true).withQoS(2)); + pipeline.run(); + + Assert.assertEquals(100, messages.size()); + + client.disconnect(); + client.close(); + } + + private MqttClient receive(final List messages) throws MqttException { + MqttClient client = new MqttClient("tcp://localhost:11883", "test"); + MqttCallback callback = new MqttCallback() { + @Override + public void connectionLost(Throwable cause) { + cause.printStackTrace(); + } + + @Override + public void messageArrived(String topic, MqttMessage message) throws Exception { + messages.add(message); + } + + @Override + public void deliveryComplete(IMqttDeliveryToken token) { + System.out.println("Delivery complete"); + } + }; + client.connect(); + client.subscribe("BEAM_TOPIC"); + client.setCallback(callback); + return client; + } + + @After + public void stopBroker() throws Exception { + if (broker != null) { + broker.stop(); + } + } + +} diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 12ede66660c3..4a59b1ad884f 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -40,6 +40,7 @@ kinesis mongodb jdbc + mqtt From 9cd1d06502fdb8a8a0ecf55e95fa598586357237 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 23 Sep 2016 23:27:40 +0200 Subject: [PATCH 02/27] [BEAM-606] Use AutoValue, remove checkpoint, introduce BlockingQueue and improve tests. --- sdks/java/io/mqtt/pom.xml | 7 + .../beam/sdk/io/mqtt/MqttCheckpointMark.java | 65 --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 403 +++++++++--------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 76 ++-- 4 files changed, 267 insertions(+), 284 deletions(-) delete mode 100644 sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index b1efaddde617..1e383362ade1 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -96,6 +96,13 @@ ${paho.version} + + + com.google.auto.value + auto-value + provided + + org.apache.activemq diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java deleted file mode 100644 index 43d474a09e31..000000000000 --- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttCheckpointMark.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.io.mqtt; - -import java.io.Serializable; - -import java.util.ArrayList; -import java.util.List; - -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.io.UnboundedSource; - -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.eclipse.paho.client.mqttv3.MqttMessage; - -import org.joda.time.Instant; - -/** - * Checkpoint for an unbounded {@link org.apache.beam.sdk.io.mqtt.MqttIO.Read}. - * It consists in a list of pending message (not possible to control the ACK in MQTT, all depends - * of the QoS) and the timestamp of the oldest message. - */ -@DefaultCoder(SerializableCoder.class) -public class MqttCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { - - private final List messages = new ArrayList<>(); - private Instant oldestTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - - public MqttCheckpointMark() { - } - - public Instant addMessage(MqttMessage message) { - if (messages.size() < 1) { - oldestTimestamp = Instant.now(); - } - messages.add(message); - return oldestTimestamp; - } - - public Instant getOldestTimestamp() { - return oldestTimestamp; - } - - @Override - public void finalizeCheckpoint() { - messages.clear(); - } - -} 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 822c9e8a5944..9e5d23ab48a8 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 @@ -17,24 +17,30 @@ */ package org.apache.beam.sdk.io.mqtt; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.google.auto.value.AutoValue; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -56,16 +62,20 @@ *

Reading from a MQTT broker

* *

MqttIO source returns an unbounded collection of {@code byte[]} as - * {@code PCollection}, where {@code byte[]} is the message payload.

+ * {@code PCollection}, where {@code byte[]} is the MQTT message payload.

* - *

To configure a MQTT source, you have to provide a {@code ClientId}, a {@code ServerURI}, - * and eventually a {@code Topic} pattern. The following example illustrates various options - * for configuring the source:

+ *

To configure a MQTT source, you have to provide a MQTT connection configuration including + * {@code ClientId}, a {@code ServerURI}, and eventually a {@code Topic} pattern. The following + * example illustrates various options for configuring the source:

* *
{@code
  *
- * pipeline.apply(MqttIO.read().withClientId("my_client").withServerUri("tcp://host:11883")
- *   .withTopic("topic")
+ * pipeline.apply(
+ *   MqttIO.read()
+ *    .withMqttConnectionConfiguration(MqttIO.MqttConnectionConfiguration.create(
+ *      "tcp://host:11883",
+ *      "my_client_id",
+ *      "my_topic"))
  *
  * }
* @@ -73,8 +83,10 @@ * *

MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker.

* - *

To configure a MQTT sink, you must specify the {@code ClientId}, {@code ServerURI}, - * {@code Topic}. Eventually, you can also specify the {@code Retained} and {@code QoS} of the + *

To configure a MQTT sink, as for the read, you have to specify a MQTT connection + * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}.

+ * + *

Eventually, you can also specify the {@code Retained} and {@code QoS} of the MQTT * message.

* *

For instance:

@@ -84,9 +96,10 @@ * pipeline * .apply(...) // provide PCollection * .MqttIO.write() - * .withClientId("my-client") - * .withServerUri("tcp://host:11883") - * .withTopic("my-topic") + * .withMqttConnectionConfiguration(MqttIO.MqttConnectionConfiguration.create( + * "tcp://host:11883", + * "my_client_id", + * "my_topic")) * .withRetained(true) * .withQoS(2) * @@ -97,127 +110,143 @@ public class MqttIO { private static final Logger LOGGER = LoggerFactory.getLogger(MqttIO.class); public static Read read() { - return new Read(new UnboundedMqttSource(null, null, null, - Long.MAX_VALUE, null)); + return new AutoValue_MqttIO_Read.Builder() + .setMaxReadTime(null).setMaxNumRecords(Long.MAX_VALUE).build(); } public static Write write() { - return new Write(new Write.MqttWriter(null, null, null, 1, false)); + return new AutoValue_MqttIO_Write.Builder() + .setRetained(false) + .setQos(0) + .build(); } private MqttIO() { } /** - * A {@link PTransform>} to read from a MQTT broker. + * A POJO describing a MQTT connection. */ - public static class Read extends PTransform> { + @AutoValue + abstract static class MqttConnectionConfiguration implements Serializable { + + @Nullable abstract String serverUri(); + @Nullable abstract String clientId(); + @Nullable abstract String topic(); - public Read withServerUri(String serverUri) { - return new Read(source.withServerUri(serverUri)); + public static MqttConnectionConfiguration create(String serverUri, String clientId) { + checkNotNull(serverUri, "serverUri"); + checkNotNull(clientId, "clientId"); + return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, null); } - public Read withClientId(String clientId) { - return new Read(source.withClientId(clientId)); + public static MqttConnectionConfiguration create(String serverUri, String clientId, + String topic) { + checkNotNull(serverUri, "serverUri"); + checkNotNull(clientId, "clientId"); + return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, topic); } - public Read withTopic(String topic) { - return new Read(source.withTopic(topic)); + private void validate() { + checkNotNull(serverUri()); + checkNotNull(clientId()); } - public Read withMaxNumRecords(long maxNumRecords) { - return new Read(source.withMaxNumRecords(maxNumRecords)); + private void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("serverUri", serverUri())); + builder.add(DisplayData.item("clientId", clientId())); + builder.add(DisplayData.item("topic", topic())); } - public Read withMaxReadTime(Duration maxReadTime) { - return new Read(source.withMaxReadTime(maxReadTime)); + MqttClient getClient() throws Exception { + MqttClient client = new MqttClient(serverUri(), clientId()); + client.connect(); + if (topic() != null) { + client.subscribe(topic()); + } + return client; } - private final UnboundedMqttSource source; + } - private Read(UnboundedMqttSource source) { - this.source = source; + /** + * A {@link PTransform} to read from a MQTT broker. + */ + @AutoValue + public abstract static class Read extends PTransform> { + + @Nullable abstract MqttConnectionConfiguration mqttConnectionConfiguration(); + @Nullable abstract long maxNumRecords(); + @Nullable abstract Duration maxReadTime(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMqttConnectionConfiguration(MqttConnectionConfiguration config); + abstract Builder setMaxNumRecords(long maxNumRecords); + abstract Builder setMaxReadTime(Duration maxReadTime); + abstract Read build(); + } + + public Read withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { + checkNotNull(configuration, "MqttConnectionConfiguration"); + return toBuilder().setMqttConnectionConfiguration(configuration).build(); + } + + public Read withMaxNumRecords(long maxNumRecords) { + return toBuilder().setMaxNumRecords(maxNumRecords).build(); + } + + public Read withMaxReadTime(Duration maxReadTime) { + return toBuilder().setMaxReadTime(maxReadTime).build(); } @Override public PCollection apply(PBegin input) { org.apache.beam.sdk.io.Read.Unbounded unbounded = - org.apache.beam.sdk.io.Read.from(getSource()); + org.apache.beam.sdk.io.Read.from(new UnboundedMqttSource(this)); PTransform> transform = unbounded; - if (source.maxNumRecords != Long.MAX_VALUE) { - transform = unbounded.withMaxNumRecords(source.maxNumRecords); - } else if (source.maxReadTime != null) { - transform = unbounded.withMaxReadTime(source.maxReadTime); + if (maxNumRecords() != Long.MAX_VALUE) { + transform = unbounded.withMaxNumRecords(maxNumRecords()); + } else if (maxReadTime() != null) { + transform = unbounded.withMaxReadTime(maxReadTime()); } return input.getPipeline().apply(transform); } - @VisibleForTesting - public UnboundedMqttSource getSource() { - return source; - } - @Override public void validate(PBegin input) { - source.validate(); + mqttConnectionConfiguration().validate(); } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - source.populateDisplayData(builder); + mqttConnectionConfiguration().populateDisplayData(builder); + builder.add(DisplayData.item("maxNumRecords", maxNumRecords())); + builder.addIfNotNull(DisplayData.item("maxReadTime", maxReadTime())); } } - private static class UnboundedMqttSource extends UnboundedSource { + private static class UnboundedMqttSource + extends UnboundedSource { - public UnboundedMqttSource withServerUri(String serverUri) { - return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); - } + private Read spec; - public UnboundedMqttSource withClientId(String clientId) { - return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); - } - - public UnboundedMqttSource withTopic(String topic) { - return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); - } - - public UnboundedMqttSource withMaxNumRecords(long maxNumRecords) { - return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); - } - - public UnboundedMqttSource withMaxReadTime(Duration maxReadTime) { - return new UnboundedMqttSource(serverUri, clientId, topic, maxNumRecords, maxReadTime); - } - - private final String serverUri; - private final String clientId; - @Nullable - private final String topic; - - private final long maxNumRecords; - private final Duration maxReadTime; - - public UnboundedMqttSource(String serverUri, String clientId, String topic, - long maxNumRecords, Duration maxReadTime) { - this.serverUri = serverUri; - this.clientId = clientId; - this.topic = topic; - this.maxNumRecords = maxNumRecords; - this.maxReadTime = maxReadTime; + public UnboundedMqttSource(Read spec) { + this.spec = spec; } @Override public UnboundedReader createReader(PipelineOptions options, - MqttCheckpointMark checkpointMark) { - return new UnboundedMqttReader(this, checkpointMark); + CheckpointMark checkpointMark) { + return new UnboundedMqttReader(this); } @Override @@ -226,21 +255,24 @@ public List generateInitialSplits(int desiredNumSplits, List sources = new ArrayList<>(); for (int i = 0; i < desiredNumSplits; i++) { // NB: it's important that user understand the impact of MQTT message QoS - sources.add(new UnboundedMqttSource(serverUri, clientId + "-" + i, topic, maxNumRecords, - maxReadTime)); + sources.add(new UnboundedMqttSource(spec)); } return sources; } @Override public void validate() { - Preconditions.checkNotNull(serverUri, "serverUri"); - Preconditions.checkNotNull(clientId, "clientId"); + spec.validate(null); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + spec.populateDisplayData(builder); } @Override public Coder getCheckpointMarkCoder() { - return SerializableCoder.of(MqttCheckpointMark.class); + return VoidCoder.of(); } @Override @@ -250,37 +282,46 @@ public Coder getDefaultOutputCoder() { } - private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader - implements MqttCallback { - - private UnboundedMqttSource source; + private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { - private MqttCheckpointMark checkpointMark; + private final UnboundedMqttSource source; - private MqttClient client; - private byte[] current; - private Instant currentTimestamp; + private transient MqttClient client; + private transient byte[] current; + private transient Instant currentTimestamp; + private transient Instant watermark; + private transient BlockingQueue> queue; - private UnboundedMqttReader(UnboundedMqttSource source, - MqttCheckpointMark checkpointMark) { + private UnboundedMqttReader(UnboundedMqttSource source) { this.source = source; - if (checkpointMark != null) { - this.checkpointMark = checkpointMark; - } else { - this.checkpointMark = new MqttCheckpointMark(); - } this.current = null; + this.queue = new LinkedBlockingQueue<>(); } @Override public boolean start() throws IOException { + LOGGER.info("Starting MQTT reader"); + Read spec = source.spec; try { - client = new MqttClient(source.serverUri, source.clientId); - client.connect(); - if (source.topic != null) { - client.subscribe(source.topic); - } - client.setCallback(this); + client = spec.mqttConnectionConfiguration().getClient(); + client.setCallback(new MqttCallback() { + @Override + public void connectionLost(Throwable cause) { + LOGGER.warn("MQTT connection lost", cause); + } + + @Override + public void messageArrived(String topic, MqttMessage message) throws Exception { + LOGGER.info("Message arrived"); + currentTimestamp = Instant.now(); + queue.put(KV.of(message, currentTimestamp)); + } + + @Override + public void deliveryComplete(IMqttDeliveryToken token) { + // nothing to do + } + }); return advance(); } catch (Exception e) { throw new IOException(e); @@ -289,19 +330,31 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { - if (current == null) { - return false; - } else { - return true; + LOGGER.info("Taking from the pending queue ({})", queue.size()); + try { + KV message = queue.take(); + current = message.getKey().getPayload(); + watermark = message.getValue(); + if (current == null) { + return false; + } else { + return true; + } + } catch (InterruptedException e) { + throw new IOException(e); } } @Override public void close() throws IOException { + LOGGER.info("Closing MQTT reader"); try { if (client != null) { - client.disconnect(); - client.close(); + try { + client.disconnect(); + } finally { + client.close(); + } } } catch (Exception e) { throw new IOException(e); @@ -310,16 +363,24 @@ public void close() throws IOException { @Override public Instant getWatermark() { - return checkpointMark.getOldestTimestamp(); + return watermark; } @Override - public MqttCheckpointMark getCheckpointMark() { - return checkpointMark; + public UnboundedSource.CheckpointMark getCheckpointMark() { + return new UnboundedSource.CheckpointMark() { + @Override + public void finalizeCheckpoint() throws IOException { + // nothing to do + } + }; } @Override public byte[] getCurrent() { + if (current == null) { + throw new NoSuchElementException(); + } return current; } @@ -336,63 +397,56 @@ public Instant getCurrentTimestamp() { return currentTimestamp; } - @Override - public void connectionLost(Throwable t) { - LOGGER.warn("Lost connection to MQTT server", t); - } - - @Override - public void messageArrived(String topic, MqttMessage message) { - this.current = message.getPayload(); - this.currentTimestamp = checkpointMark.addMessage(message); - } - - @Override - public void deliveryComplete(IMqttDeliveryToken token) { - } - } /** * A {@link PTransform} to write and send a message to a MQTT server. */ - public static class Write extends PTransform, PDone> { + @AutoValue + public abstract static class Write extends PTransform, PDone> { - public Write withServerUri(String serverUri) { - return new Write(writer.withServerUri(serverUri)); - } + @Nullable abstract MqttConnectionConfiguration mqttConnectionConfiguration(); + @Nullable abstract int qos(); + @Nullable abstract boolean retained(); + + abstract Builder toBuilder(); - public Write withClientId(String clientId) { - return new Write(writer.withClientId(clientId)); + @AutoValue.Builder + abstract static class Builder { + abstract Builder setMqttConnectionConfiguration(MqttConnectionConfiguration configuration); + abstract Builder setQos(int qos); + abstract Builder setRetained(boolean retained); + abstract Write build(); } - public Write withTopic(String topic) { - return new Write(writer.withTopic(topic)); + public Write withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { + return toBuilder().setMqttConnectionConfiguration(configuration).build(); } public Write withQoS(int qos) { - return new Write(writer.withQoS(qos)); + return toBuilder().setQos(qos).build(); } public Write withRetained(boolean retained) { - return new Write(writer.withRetained(retained)); - } - - private final MqttWriter writer; - - private Write(MqttWriter writer) { - this.writer = writer; + return toBuilder().setRetained(retained).build(); } @Override public PDone apply(PCollection input) { - input.apply(ParDo.of(writer)); + input.apply(ParDo.of(new MqttWriter(this))); return PDone.in(input.getPipeline()); } @Override public void validate(PCollection input) { - writer.validate(); + mqttConnectionConfiguration().validate(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + mqttConnectionConfiguration().populateDisplayData(builder); + builder.add(DisplayData.item("qos", qos())); + builder.add(DisplayData.item("retained", retained())); } /** @@ -400,62 +454,27 @@ public void validate(PCollection input) { */ private static class MqttWriter extends DoFn { - private final String serverUri; - private final String clientId; - private final String topic; - private final int qos; - private final boolean retained; - - private MqttClient client; - - public MqttWriter(String serverUri, String clientId, String topic, int qos, - boolean retained) { - this.serverUri = serverUri; - this.clientId = clientId; - this.topic = topic; - this.qos = qos; - this.retained = retained; - } - - public MqttWriter withServerUri(String serverUri) { - return new MqttWriter(serverUri, clientId, topic, qos, retained); - } - - public MqttWriter withClientId(String clientId) { - return new MqttWriter(serverUri, clientId, topic, qos, retained); - } + Write spec; - public MqttWriter withTopic(String topic) { - return new MqttWriter(serverUri, clientId, topic, qos, retained); - } - - public MqttWriter withQoS(int qos) { - return new MqttWriter(serverUri, clientId, topic, qos, retained); - } - - public MqttWriter withRetained(boolean retained) { - return new MqttWriter(serverUri, clientId, topic, qos, retained); - } + private transient MqttClient client; - public void validate() { - Preconditions.checkNotNull(serverUri, "serverUri"); - Preconditions.checkNotNull(clientId, "clientId"); + public MqttWriter(Write spec) { + this.spec = spec; } @Setup public void createMqttClient() throws Exception { - client = new MqttClient(serverUri, clientId); - client.connect(); + client = spec.mqttConnectionConfiguration().getClient(); } @ProcessElement public void processElement(ProcessContext context) throws Exception { byte[] payload = context.element(); MqttMessage message = new MqttMessage(); - message.setQos(qos); - message.setRetained(retained); + message.setQos(spec.qos()); + message.setRetained(spec.retained()); message.setPayload(payload); - client.publish(topic, message); + client.publish(spec.mqttConnectionConfiguration().topic(), message); } @Teardown 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 855aff721a6f..02ded4bb8f37 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 @@ -39,9 +39,9 @@ import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; -import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -53,10 +53,10 @@ @RunWith(JUnit4.class) public class MqttIOTest implements Serializable { - private transient BrokerService broker; + private static transient BrokerService broker; - @Before - public void startBroker() throws Exception { + @BeforeClass + public static void startBroker() throws Exception { broker = new BrokerService(); broker.setUseJmx(false); broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); @@ -67,26 +67,15 @@ public void startBroker() throws Exception { @Test @Category(NeedsRunner.class) public void testRead() throws Exception { - // produce message on the broker - MqttClient client = new MqttClient("tcp://localhost:11883", "test"); - client.connect(); - for (int i = 0; i < 10; i++) { - MqttMessage message = new MqttMessage(); - message.setQos(2); - message.setRetained(true); - message.setPayload("This is a test".getBytes()); - client.publish("BEAM", message); - } - client.disconnect(); - client.close(); - Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline.apply( MqttIO.read() - .withClientId("BEAM_PIPELINE") - .withServerUri("tcp://localhost:11883") - .withTopic("BEAM") + .withMqttConnectionConfiguration( + MqttIO.MqttConnectionConfiguration.create( + "tcp://localhost:11883", + "BEAM_PIPELINE", + "READ_TOPIC")) .withMaxNumRecords(10)); PAssert.thatSingleton(output.apply("Count", Count.globally())) @@ -101,6 +90,35 @@ public Void apply(Iterable input) { return null; } }); + + // produce messages on the broker in another thread + Thread thread = new Thread() { + public void run() { + try { + // gives time to the pipeline to start + Thread.sleep(2000); + } catch (Exception e) { + // nothing to do + } + try { + MqttClient client = new MqttClient("tcp://localhost:11883", "publisher"); + client.connect(); + for (int i = 0; i < 10; i++) { + MqttMessage message = new MqttMessage(); + message.setQos(0); + message.setRetained(false); + message.setPayload("This is a test".getBytes()); + client.publish("READ_TOPIC", message); + } + client.disconnect(); + client.close(); + } catch (Exception e) { + // nothing to do + } + } + }; + thread.start(); + pipeline.run(); } @@ -117,8 +135,12 @@ public void testWrite() throws Exception { data.add("Test".getBytes()); } pipeline.apply(Create.of(data)) - .apply(MqttIO.write().withClientId("BEAM_PIPELINE") - .withServerUri("tcp://localhost:11883").withTopic("BEAM_TOPIC") + .apply(MqttIO.write() + .withMqttConnectionConfiguration( + MqttIO.MqttConnectionConfiguration.create( + "tcp://localhost:11883", + "BEAM_PIPELINE", + "WRITE_TOPIC")) .withRetained(true).withQoS(2)); pipeline.run(); @@ -129,7 +151,7 @@ public void testWrite() throws Exception { } private MqttClient receive(final List messages) throws MqttException { - MqttClient client = new MqttClient("tcp://localhost:11883", "test"); + MqttClient client = new MqttClient("tcp://localhost:11883", "receiver"); MqttCallback callback = new MqttCallback() { @Override public void connectionLost(Throwable cause) { @@ -147,13 +169,13 @@ public void deliveryComplete(IMqttDeliveryToken token) { } }; client.connect(); - client.subscribe("BEAM_TOPIC"); + client.subscribe("WRITE_TOPIC"); client.setCallback(callback); return client; } - @After - public void stopBroker() throws Exception { + @AfterClass + public static void stopBroker() throws Exception { if (broker != null) { broker.stop(); } From 58e77098003b784ed7509c66ec2ac014102502cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 5 Oct 2016 12:28:44 +0200 Subject: [PATCH 03/27] [BEAM-606] Set topic required. Move some subscriber specific MQTT code on the reader. Improve tests. --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 81 +++++++++---------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 23 ++++-- 2 files changed, 53 insertions(+), 51 deletions(-) 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 9e5d23ab48a8..25541c4b08af 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 @@ -48,6 +48,7 @@ import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; import org.eclipse.paho.client.mqttv3.MqttClient; +import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; import org.joda.time.Duration; @@ -134,16 +135,11 @@ abstract static class MqttConnectionConfiguration implements Serializable { @Nullable abstract String clientId(); @Nullable abstract String topic(); - public static MqttConnectionConfiguration create(String serverUri, String clientId) { - checkNotNull(serverUri, "serverUri"); - checkNotNull(clientId, "clientId"); - return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, null); - } - public static MqttConnectionConfiguration create(String serverUri, String clientId, String topic) { checkNotNull(serverUri, "serverUri"); checkNotNull(clientId, "clientId"); + checkNotNull(topic, "topic"); return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, topic); } @@ -161,9 +157,6 @@ private void populateDisplayData(DisplayData.Builder builder) { MqttClient getClient() throws Exception { MqttClient client = new MqttClient(serverUri(), clientId()); client.connect(); - if (topic() != null) { - client.subscribe(topic()); - } return client; } @@ -279,18 +272,16 @@ public Coder getCheckpointMarkCoder() { public Coder getDefaultOutputCoder() { return SerializableCoder.of(byte[].class); } - } private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { private final UnboundedMqttSource source; - private transient MqttClient client; - private transient byte[] current; - private transient Instant currentTimestamp; - private transient Instant watermark; - private transient BlockingQueue> queue; + private MqttClient client; + private byte[] current; + private Instant currentTimestamp; + private BlockingQueue> queue; private UnboundedMqttReader(UnboundedMqttSource source) { this.source = source; @@ -300,21 +291,26 @@ private UnboundedMqttReader(UnboundedMqttSource source) { @Override public boolean start() throws IOException { - LOGGER.info("Starting MQTT reader"); + LOGGER.debug("Starting MQTT reader"); Read spec = source.spec; try { client = spec.mqttConnectionConfiguration().getClient(); + client.subscribe(spec.mqttConnectionConfiguration().topic()); client.setCallback(new MqttCallback() { @Override public void connectionLost(Throwable cause) { LOGGER.warn("MQTT connection lost", cause); + try { + close(); + } catch (Exception e) { + // nothing to do + } } @Override public void messageArrived(String topic, MqttMessage message) throws Exception { - LOGGER.info("Message arrived"); - currentTimestamp = Instant.now(); - queue.put(KV.of(message, currentTimestamp)); + LOGGER.trace("Message arrived"); + queue.put(KV.of(message, Instant.now())); } @Override @@ -330,16 +326,12 @@ public void deliveryComplete(IMqttDeliveryToken token) { @Override public boolean advance() throws IOException { - LOGGER.info("Taking from the pending queue ({})", queue.size()); + LOGGER.debug("Taking from the pending queue ({})", queue.size()); try { KV message = queue.take(); current = message.getKey().getPayload(); - watermark = message.getValue(); - if (current == null) { - return false; - } else { - return true; - } + currentTimestamp = message.getValue(); + return true; } catch (InterruptedException e) { throw new IOException(e); } @@ -347,7 +339,7 @@ public boolean advance() throws IOException { @Override public void close() throws IOException { - LOGGER.info("Closing MQTT reader"); + LOGGER.debug("Closing MQTT reader"); try { if (client != null) { try { @@ -356,14 +348,15 @@ public void close() throws IOException { client.close(); } } - } catch (Exception e) { - throw new IOException(e); + } catch (MqttException mqttException) { + throw new IOException(mqttException); } } @Override public Instant getWatermark() { - return watermark; + // TODO use custom or better watermark + return Instant.now(); } @Override @@ -384,11 +377,6 @@ public byte[] getCurrent() { return current; } - @Override - public UnboundedMqttSource getCurrentSource() { - return source; - } - @Override public Instant getCurrentTimestamp() { if (current == null) { @@ -397,6 +385,11 @@ public Instant getCurrentTimestamp() { return currentTimestamp; } + @Override + public UnboundedMqttSource getCurrentSource() { + return source; + } + } /** @@ -433,7 +426,7 @@ public Write withRetained(boolean retained) { @Override public PDone apply(PCollection input) { - input.apply(ParDo.of(new MqttWriter(this))); + input.apply(ParDo.of(new WriteFn(this))); return PDone.in(input.getPipeline()); } @@ -449,16 +442,13 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("retained", retained())); } - /** - * MQTT writer. - */ - private static class MqttWriter extends DoFn { + private static class WriteFn extends DoFn { - Write spec; + private final Write spec; private transient MqttClient client; - public MqttWriter(Write spec) { + public WriteFn(Write spec) { this.spec = spec; } @@ -480,8 +470,11 @@ public void processElement(ProcessContext context) throws Exception { @Teardown public void closeMqttClient() throws Exception { if (client != null) { - client.disconnect(); - client.close(); + try { + client.disconnect(); + } finally { + client.close(); + } } } 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 02ded4bb8f37..263a317e8e14 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 @@ -32,13 +32,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; - import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; - import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -81,17 +79,22 @@ public void testRead() throws Exception { PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(10L); PAssert.that(output).satisfies(new SerializableFunction, Void>() { + + int count = 0; + @Override public Void apply(Iterable input) { for (byte[] element : input) { String inputString = new String(element); - Assert.assertEquals("This is a test", inputString); + Assert.assertEquals("This is test " + count, inputString); + count++; } return null; } }); // produce messages on the broker in another thread + // This thread prevents to block the pipeline waiting for new messages Thread thread = new Thread() { public void run() { try { @@ -107,7 +110,7 @@ public void run() { MqttMessage message = new MqttMessage(); message.setQos(0); message.setRetained(false); - message.setPayload("This is a test".getBytes()); + message.setPayload(("This is test " + i).getBytes()); client.publish("READ_TOPIC", message); } client.disconnect(); @@ -134,6 +137,10 @@ public void testWrite() throws Exception { for (int i = 0; i < 100; i++) { data.add("Test".getBytes()); } + // we use QoS 2 here to be sure the subscriber completely receive all messages before + // shutting down the MQTT broker. + // Quality of Service 2 indicates that a message should be delivered once. The message will + // be persisted to disk, and will be subject to a two-phase ack. pipeline.apply(Create.of(data)) .apply(MqttIO.write() .withMqttConnectionConfiguration( @@ -141,7 +148,7 @@ public void testWrite() throws Exception { "tcp://localhost:11883", "BEAM_PIPELINE", "WRITE_TOPIC")) - .withRetained(true).withQoS(2)); + .withQoS(2)); pipeline.run(); Assert.assertEquals(100, messages.size()); @@ -160,12 +167,14 @@ public void connectionLost(Throwable cause) { @Override public void messageArrived(String topic, MqttMessage message) throws Exception { - messages.add(message); + synchronized (messages) { + messages.add(message); + } } @Override public void deliveryComplete(IMqttDeliveryToken token) { - System.out.println("Delivery complete"); + // nothing to do } }; client.connect(); From 2a27bff9304617e18a44a39f0fc6fb16865a25b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 5 Oct 2016 16:46:24 +0200 Subject: [PATCH 04/27] [BEAM-606] Replace KV internal use with a regular POJO --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 44 +++++++++++++++---- 1 file changed, 35 insertions(+), 9 deletions(-) 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 25541c4b08af..346f49405d7d 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 @@ -40,20 +40,16 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; import org.eclipse.paho.client.mqttv3.MqttClient; import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; - import org.joda.time.Duration; import org.joda.time.Instant; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -274,6 +270,36 @@ public Coder getDefaultOutputCoder() { } } + /** + * POJO used to store MQTT message and its timestamp. + */ + private static class MqttMessageWithTimestamp { + + private MqttMessage message; + private Instant timestamp; + + public MqttMessageWithTimestamp(MqttMessage message, Instant timestamp) { + this.message = message; + this.timestamp = timestamp; + } + + public MqttMessage getMessage() { + return message; + } + + public void setMessage(MqttMessage message) { + this.message = message; + } + + public Instant getTimestamp() { + return timestamp; + } + + public void setTimestamp(Instant timestamp) { + this.timestamp = timestamp; + } + } + private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { private final UnboundedMqttSource source; @@ -281,7 +307,7 @@ private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader private MqttClient client; private byte[] current; private Instant currentTimestamp; - private BlockingQueue> queue; + private BlockingQueue queue; private UnboundedMqttReader(UnboundedMqttSource source) { this.source = source; @@ -310,7 +336,7 @@ public void connectionLost(Throwable cause) { @Override public void messageArrived(String topic, MqttMessage message) throws Exception { LOGGER.trace("Message arrived"); - queue.put(KV.of(message, Instant.now())); + queue.put(new MqttMessageWithTimestamp(message, Instant.now())); } @Override @@ -328,9 +354,9 @@ public void deliveryComplete(IMqttDeliveryToken token) { public boolean advance() throws IOException { LOGGER.debug("Taking from the pending queue ({})", queue.size()); try { - KV message = queue.take(); - current = message.getKey().getPayload(); - currentTimestamp = message.getValue(); + MqttMessageWithTimestamp message = queue.take(); + current = message.getMessage().getPayload(); + currentTimestamp = message.getTimestamp(); return true; } catch (InterruptedException e) { throw new IOException(e); From b212efc76ade7e7b37f3b91abc3c19fdb903bdb2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 5 Oct 2016 16:55:43 +0200 Subject: [PATCH 05/27] [BEAM-606] Use a better value for watermark --- .../main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) 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 346f49405d7d..507bde18100c 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 @@ -40,6 +40,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -381,8 +382,14 @@ public void close() throws IOException { @Override public Instant getWatermark() { - // TODO use custom or better watermark - return Instant.now(); + MqttMessageWithTimestamp message = queue.peek(); + if (message == null) { + // no message yet in the queue, returning the min possible timestamp value + return BoundedWindow.TIMESTAMP_MIN_VALUE; + } else { + // watermark is the timestamp of the oldest message in the queue + return message.getTimestamp(); + } } @Override From dd48a7d5bd7948264feeaa5cee7de7b01979d4f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 10 Oct 2016 21:12:37 +0200 Subject: [PATCH 06/27] [BEAM-606] Validation is now performed only in create(), maxNumRecords and maxReadTime are now exclusive, use ByteArrayCoder instead of SerializableCoder, use poll instead of take on the blocking queue, add javadoc on the with* methods, code cleanup --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 133 +++++++++++++----- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 10 +- 2 files changed, 103 insertions(+), 40 deletions(-) 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 507bde18100c..d36b9ddcec86 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.mqtt; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; @@ -28,11 +29,12 @@ import java.util.NoSuchElementException; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -126,7 +128,7 @@ private MqttIO() { * A POJO describing a MQTT connection. */ @AutoValue - abstract static class MqttConnectionConfiguration implements Serializable { + public abstract static class MqttConnectionConfiguration implements Serializable { @Nullable abstract String serverUri(); @Nullable abstract String clientId(); @@ -140,18 +142,13 @@ public static MqttConnectionConfiguration create(String serverUri, String client return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, topic); } - private void validate() { - checkNotNull(serverUri()); - checkNotNull(clientId()); - } - private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("serverUri", serverUri())); builder.add(DisplayData.item("clientId", clientId())); builder.add(DisplayData.item("topic", topic())); } - MqttClient getClient() throws Exception { + private MqttClient getClient() throws Exception { MqttClient client = new MqttClient(serverUri(), clientId()); client.connect(); return client; @@ -179,16 +176,45 @@ abstract static class Builder { abstract Read build(); } + /** + * Define the MQTT connection configuration used to connect to the MQTT broker. + * + * @param configuration A {@link MqttConnectionConfiguration} instance. + * @return The {@link Read} {@link PTransform} with the corresponding MQTT connection + * configuration. + */ public Read withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { checkNotNull(configuration, "MqttConnectionConfiguration"); return toBuilder().setMqttConnectionConfiguration(configuration).build(); } + /** + * Define the max number of records received by the {@link Read}. + * When this max number of records is lower then {@code Long.MAX_VALUE}, the {@link Read} + * will provide a bounded {@link PCollection}. + * + * @param maxNumRecords The max number of records received. + * @return The {@link Read} {@link PTransform} with the corresponding max num records + * configuration. + */ public Read withMaxNumRecords(long maxNumRecords) { + checkArgument(maxReadTime() == null, + "maxNumRecord and maxReadTime are exclusive"); return toBuilder().setMaxNumRecords(maxNumRecords).build(); } + /** + * Define the max read time (duration) while the {@link Read} will receive messages. + * When this max read time is not null, the {@link Read} will provide a bounded + * {@link PCollection}. + * + * @param maxReadTime The max read time duration. + * @return The {@link Read} {@link PTransform} with the corresponding max read time + * configuration. + */ public Read withMaxReadTime(Duration maxReadTime) { + checkArgument(maxNumRecords() == Long.MAX_VALUE, + "maxNumRecord and maxReadTime are exclusive"); return toBuilder().setMaxReadTime(maxReadTime).build(); } @@ -211,7 +237,7 @@ public PCollection apply(PBegin input) { @Override public void validate(PBegin input) { - mqttConnectionConfiguration().validate(); + // validation is performed in the MqttConnectionConfiguration create() } @Override @@ -227,7 +253,7 @@ public void populateDisplayData(DisplayData.Builder builder) { private static class UnboundedMqttSource extends UnboundedSource { - private Read spec; + private final Read spec; public UnboundedMqttSource(Read spec) { this.spec = spec; @@ -244,7 +270,6 @@ public List generateInitialSplits(int desiredNumSplits, PipelineOptions options) { List sources = new ArrayList<>(); for (int i = 0; i < desiredNumSplits; i++) { - // NB: it's important that user understand the impact of MQTT message QoS sources.add(new UnboundedMqttSource(spec)); } return sources; @@ -267,7 +292,7 @@ public Coder getCheckpointMarkCoder() { @Override public Coder getDefaultOutputCoder() { - return SerializableCoder.of(byte[].class); + return ByteArrayCoder.of(); } } @@ -283,22 +308,6 @@ public MqttMessageWithTimestamp(MqttMessage message, Instant timestamp) { this.message = message; this.timestamp = timestamp; } - - public MqttMessage getMessage() { - return message; - } - - public void setMessage(MqttMessage message) { - this.message = message; - } - - public Instant getTimestamp() { - return timestamp; - } - - public void setTimestamp(Instant timestamp) { - this.timestamp = timestamp; - } } private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { @@ -355,9 +364,12 @@ public void deliveryComplete(IMqttDeliveryToken token) { public boolean advance() throws IOException { LOGGER.debug("Taking from the pending queue ({})", queue.size()); try { - MqttMessageWithTimestamp message = queue.take(); - current = message.getMessage().getPayload(); - currentTimestamp = message.getTimestamp(); + MqttMessageWithTimestamp message = queue.poll(5, TimeUnit.SECONDS); + if (message == null) { + return false; + } + current = message.message.getPayload(); + currentTimestamp = message.timestamp; return true; } catch (InterruptedException e) { throw new IOException(e); @@ -388,7 +400,7 @@ public Instant getWatermark() { return BoundedWindow.TIMESTAMP_MIN_VALUE; } else { // watermark is the timestamp of the oldest message in the queue - return message.getTimestamp(); + return message.timestamp; } } @@ -445,14 +457,69 @@ abstract static class Builder { abstract Write build(); } + /** + * Define MQTT connection configuration used to connect to the MQTT broker. + * + * @param configuration The {@link MqttConnectionConfiguration} instance. + * @return The {@link Write} {@link PTransform} with the corresponding connection configuration. + */ public Write withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { return toBuilder().setMqttConnectionConfiguration(configuration).build(); } + /** + * Define the MQTT message quality of service. + * + *
    + *
  • Quality of Service 0 - indicates that a message should + * be delivered at most once (zero or one times). The message will not be persisted to disk, + * and will not be acknowledged across the network. This QoS is the fastest, + * but should only be used for messages which are not valuable - note that + * if the server cannot process the message (for example, there + * is an authorization problem), then an + * {@link MqttCallback#deliveryComplete(IMqttDeliveryToken)}. + * Also known as "fire and forget".
  • + * + *
  • Quality of Service 1 - indicates that a message should + * be delivered at least once (one or more times). The message can only be delivered safely if + * it can be persisted, so the application must supply a means of + * persistence using {@code MqttConnectOptions}. + * If a persistence mechanism is not specified, the message will not be + * delivered in the event of a client failure. + * The message will be acknowledged across the network. + * This is the default QoS.
  • + * + *
  • Quality of Service 2 - indicates that a message should + * be delivered once. The message will be persisted to disk, and will + * be subject to a two-phase acknowledgement across the network. + * The message can only be delivered safely if + * it can be persisted, so the application must supply a means of + * persistence using {@code MqttConnectOptions}. + * If a persistence mechanism is not specified, the message will not be + * delivered in the event of a client failure.
  • + *
+ * + * If persistence is not configured, QoS 1 and 2 messages will still be delivered + * in the event of a network or server problem as the client will hold state in memory. + * If the MQTT client is shutdown or fails and persistence is not configured then + * delivery of QoS 1 and 2 messages can not be maintained as client-side state will + * be lost. + * + * @param qos The quality of service value. + * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. + */ public Write withQoS(int qos) { return toBuilder().setQos(qos).build(); } + /** + * Whether or not the publish message should be retained by the messaging engine. + * Sending a message with the retained set to {@code false} will clear the + * retained message from the server. The default value is {@code false}. + * + * @param retained Whether or not the messaging engine should retain the message. + * @return The {@link Write} {@link PTransform} with the corresponding retained configuration. + */ public Write withRetained(boolean retained) { return toBuilder().setRetained(retained).build(); } @@ -465,7 +532,7 @@ public PDone apply(PCollection input) { @Override public void validate(PCollection input) { - mqttConnectionConfiguration().validate(); + // validation is performed in the MqttConnectionConfiguration create() } @Override 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 263a317e8e14..308ba4df6139 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 @@ -79,11 +79,9 @@ public void testRead() throws Exception { PAssert.thatSingleton(output.apply("Count", Count.globally())) .isEqualTo(10L); PAssert.that(output).satisfies(new SerializableFunction, Void>() { - - int count = 0; - @Override public Void apply(Iterable input) { + int count = 0; for (byte[] element : input) { String inputString = new String(element); Assert.assertEquals("This is test " + count, inputString); @@ -108,8 +106,8 @@ public void run() { client.connect(); for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); - message.setQos(0); - message.setRetained(false); + message.setQos(2); + message.setRetained(true); message.setPayload(("This is test " + i).getBytes()); client.publish("READ_TOPIC", message); } @@ -167,9 +165,7 @@ public void connectionLost(Throwable cause) { @Override public void messageArrived(String topic, MqttMessage message) throws Exception { - synchronized (messages) { messages.add(message); - } } @Override From 52c3cdeec9f89621fc51e030c6b86528ffc5ffaa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 13 Oct 2016 09:44:00 +0200 Subject: [PATCH 07/27] [BEAM-606] Rename connection configuration, improve javadoc --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 120 ++++++++---------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 8 +- 2 files changed, 57 insertions(+), 71 deletions(-) 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 d36b9ddcec86..614012ad8199 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 @@ -61,18 +61,18 @@ * *

Reading from a MQTT broker

* - *

MqttIO source returns an unbounded collection of {@code byte[]} as - * {@code PCollection}, where {@code byte[]} is the MQTT message payload.

+ *

MqttIO source returns an unbounded {@link PCollection} containing MQTT message + * payloads (as {@code byte[]}).

* *

To configure a MQTT source, you have to provide a MQTT connection configuration including - * {@code ClientId}, a {@code ServerURI}, and eventually a {@code Topic} pattern. The following + * {@code ClientId}, a {@code ServerURI}, and a {@code Topic} pattern. The following * example illustrates various options for configuring the source:

* *
{@code
  *
  * pipeline.apply(
  *   MqttIO.read()
- *    .withMqttConnectionConfiguration(MqttIO.MqttConnectionConfiguration.create(
+ *    .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create(
  *      "tcp://host:11883",
  *      "my_client_id",
  *      "my_topic"))
@@ -86,7 +86,7 @@
  * 

To configure a MQTT sink, as for the read, you have to specify a MQTT connection * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}.

* - *

Eventually, you can also specify the {@code Retained} and {@code QoS} of the MQTT + *

Optionally, you can also specify the {@code Retained} and {@code QoS} of the MQTT * message.

* *

For instance:

@@ -96,7 +96,7 @@ * pipeline * .apply(...) // provide PCollection * .MqttIO.write() - * .withMqttConnectionConfiguration(MqttIO.MqttConnectionConfiguration.create( + * .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create( * "tcp://host:11883", * "my_client_id", * "my_topic")) @@ -128,18 +128,18 @@ private MqttIO() { * A POJO describing a MQTT connection. */ @AutoValue - public abstract static class MqttConnectionConfiguration implements Serializable { + public abstract static class ConnectionConfiguration implements Serializable { @Nullable abstract String serverUri(); @Nullable abstract String clientId(); @Nullable abstract String topic(); - public static MqttConnectionConfiguration create(String serverUri, String clientId, - String topic) { + public static ConnectionConfiguration create(String serverUri, String clientId, + String topic) { checkNotNull(serverUri, "serverUri"); checkNotNull(clientId, "clientId"); checkNotNull(topic, "topic"); - return new AutoValue_MqttIO_MqttConnectionConfiguration(serverUri, clientId, topic); + return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, clientId, topic); } private void populateDisplayData(DisplayData.Builder builder) { @@ -148,7 +148,7 @@ private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("topic", topic())); } - private MqttClient getClient() throws Exception { + private MqttClient getClient() throws MqttException { MqttClient client = new MqttClient(serverUri(), clientId()); client.connect(); return client; @@ -162,15 +162,15 @@ private MqttClient getClient() throws Exception { @AutoValue public abstract static class Read extends PTransform> { - @Nullable abstract MqttConnectionConfiguration mqttConnectionConfiguration(); - @Nullable abstract long maxNumRecords(); + @Nullable abstract ConnectionConfiguration connectionConfiguration(); + abstract long maxNumRecords(); @Nullable abstract Duration maxReadTime(); abstract Builder toBuilder(); @AutoValue.Builder abstract static class Builder { - abstract Builder setMqttConnectionConfiguration(MqttConnectionConfiguration config); + abstract Builder setConnectionConfiguration(ConnectionConfiguration config); abstract Builder setMaxNumRecords(long maxNumRecords); abstract Builder setMaxReadTime(Duration maxReadTime); abstract Read build(); @@ -178,24 +178,16 @@ abstract static class Builder { /** * Define the MQTT connection configuration used to connect to the MQTT broker. - * - * @param configuration A {@link MqttConnectionConfiguration} instance. - * @return The {@link Read} {@link PTransform} with the corresponding MQTT connection - * configuration. */ - public Read withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { - checkNotNull(configuration, "MqttConnectionConfiguration"); - return toBuilder().setMqttConnectionConfiguration(configuration).build(); + public Read withConnectionConfiguration(ConnectionConfiguration configuration) { + checkNotNull(configuration, "ConnectionConfiguration"); + return toBuilder().setConnectionConfiguration(configuration).build(); } /** * Define the max number of records received by the {@link Read}. - * When this max number of records is lower then {@code Long.MAX_VALUE}, the {@link Read} + * When this max number of records is lower than {@code Long.MAX_VALUE}, the {@link Read} * will provide a bounded {@link PCollection}. - * - * @param maxNumRecords The max number of records received. - * @return The {@link Read} {@link PTransform} with the corresponding max num records - * configuration. */ public Read withMaxNumRecords(long maxNumRecords) { checkArgument(maxReadTime() == null, @@ -207,10 +199,6 @@ public Read withMaxNumRecords(long maxNumRecords) { * Define the max read time (duration) while the {@link Read} will receive messages. * When this max read time is not null, the {@link Read} will provide a bounded * {@link PCollection}. - * - * @param maxReadTime The max read time duration. - * @return The {@link Read} {@link PTransform} with the corresponding max read time - * configuration. */ public Read withMaxReadTime(Duration maxReadTime) { checkArgument(maxNumRecords() == Long.MAX_VALUE, @@ -237,13 +225,13 @@ public PCollection apply(PBegin input) { @Override public void validate(PBegin input) { - // validation is performed in the MqttConnectionConfiguration create() + // validation is performed in the ConnectionConfiguration create() } @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - mqttConnectionConfiguration().populateDisplayData(builder); + connectionConfiguration().populateDisplayData(builder); builder.add(DisplayData.item("maxNumRecords", maxNumRecords())); builder.addIfNotNull(DisplayData.item("maxReadTime", maxReadTime())); } @@ -260,7 +248,7 @@ public UnboundedMqttSource(Read spec) { } @Override - public UnboundedReader createReader(PipelineOptions options, + public UnboundedReader createReader(PipelineOptions options, CheckpointMark checkpointMark) { return new UnboundedMqttReader(this); } @@ -301,8 +289,8 @@ public Coder getDefaultOutputCoder() { */ private static class MqttMessageWithTimestamp { - private MqttMessage message; - private Instant timestamp; + private final MqttMessage message; + private final Instant timestamp; public MqttMessageWithTimestamp(MqttMessage message, Instant timestamp) { this.message = message; @@ -330,15 +318,15 @@ public boolean start() throws IOException { LOGGER.debug("Starting MQTT reader"); Read spec = source.spec; try { - client = spec.mqttConnectionConfiguration().getClient(); - client.subscribe(spec.mqttConnectionConfiguration().topic()); + client = spec.connectionConfiguration().getClient(); + client.subscribe(spec.connectionConfiguration().topic()); client.setCallback(new MqttCallback() { @Override public void connectionLost(Throwable cause) { LOGGER.warn("MQTT connection lost", cause); try { close(); - } catch (Exception e) { + } catch (IOException e) { // nothing to do } } @@ -355,7 +343,7 @@ public void deliveryComplete(IMqttDeliveryToken token) { } }); return advance(); - } catch (Exception e) { + } catch (MqttException e) { throw new IOException(e); } } @@ -363,17 +351,20 @@ public void deliveryComplete(IMqttDeliveryToken token) { @Override public boolean advance() throws IOException { LOGGER.debug("Taking from the pending queue ({})", queue.size()); + MqttMessageWithTimestamp message = null; try { - MqttMessageWithTimestamp message = queue.poll(5, TimeUnit.SECONDS); - if (message == null) { - return false; - } - current = message.message.getPayload(); - currentTimestamp = message.timestamp; - return true; + message = queue.poll(5, TimeUnit.SECONDS); } catch (InterruptedException e) { throw new IOException(e); } + if (message == null) { + current = null; + currentTimestamp = null; + return false; + } + current = message.message.getPayload(); + currentTimestamp = message.timestamp; + return true; } @Override @@ -443,7 +434,7 @@ public UnboundedMqttSource getCurrentSource() { @AutoValue public abstract static class Write extends PTransform, PDone> { - @Nullable abstract MqttConnectionConfiguration mqttConnectionConfiguration(); + @Nullable abstract ConnectionConfiguration connectionConfiguration(); @Nullable abstract int qos(); @Nullable abstract boolean retained(); @@ -451,7 +442,7 @@ public abstract static class Write extends PTransform, PDone @AutoValue.Builder abstract static class Builder { - abstract Builder setMqttConnectionConfiguration(MqttConnectionConfiguration configuration); + abstract Builder setConnectionConfiguration(ConnectionConfiguration configuration); abstract Builder setQos(int qos); abstract Builder setRetained(boolean retained); abstract Write build(); @@ -459,12 +450,9 @@ abstract static class Builder { /** * Define MQTT connection configuration used to connect to the MQTT broker. - * - * @param configuration The {@link MqttConnectionConfiguration} instance. - * @return The {@link Write} {@link PTransform} with the corresponding connection configuration. */ - public Write withMqttConnectionConfiguration(MqttConnectionConfiguration configuration) { - return toBuilder().setMqttConnectionConfiguration(configuration).build(); + public Write withConnectionConfiguration(ConnectionConfiguration configuration) { + return toBuilder().setConnectionConfiguration(configuration).build(); } /** @@ -477,33 +465,31 @@ public Write withMqttConnectionConfiguration(MqttConnectionConfiguration configu * but should only be used for messages which are not valuable - note that * if the server cannot process the message (for example, there * is an authorization problem), then an - * {@link MqttCallback#deliveryComplete(IMqttDeliveryToken)}. + * {@link MqttCallback#deliveryComplete(IMqttDeliveryToken)} won't be called. * Also known as "fire and forget". * *
  • Quality of Service 1 - indicates that a message should * be delivered at least once (one or more times). The message can only be delivered safely if - * it can be persisted, so the application must supply a means of - * persistence using {@code MqttConnectOptions}. - * If a persistence mechanism is not specified, the message will not be + * it can be persisted on the broker. + * If the broker can't persist the message, the message will not be * delivered in the event of a client failure. * The message will be acknowledged across the network. * This is the default QoS.
  • * *
  • Quality of Service 2 - indicates that a message should - * be delivered once. The message will be persisted to disk, and will + * be delivered once. The message will be persisted to disk on the broker, and will * be subject to a two-phase acknowledgement across the network. * The message can only be delivered safely if - * it can be persisted, so the application must supply a means of - * persistence using {@code MqttConnectOptions}. - * If a persistence mechanism is not specified, the message will not be + * it can be persisted on the broker. + * If a persistence mechanism is not specified on the broker, the message will not be * delivered in the event of a client failure.
  • * * - * If persistence is not configured, QoS 1 and 2 messages will still be delivered + *

    If persistence is not configured, QoS 1 and 2 messages will still be delivered * in the event of a network or server problem as the client will hold state in memory. * If the MQTT client is shutdown or fails and persistence is not configured then * delivery of QoS 1 and 2 messages can not be maintained as client-side state will - * be lost. + * be lost.

    * * @param qos The quality of service value. * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. @@ -532,12 +518,12 @@ public PDone apply(PCollection input) { @Override public void validate(PCollection input) { - // validation is performed in the MqttConnectionConfiguration create() + // validation is performed in the ConnectionConfiguration create() } @Override public void populateDisplayData(DisplayData.Builder builder) { - mqttConnectionConfiguration().populateDisplayData(builder); + connectionConfiguration().populateDisplayData(builder); builder.add(DisplayData.item("qos", qos())); builder.add(DisplayData.item("retained", retained())); } @@ -554,7 +540,7 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { - client = spec.mqttConnectionConfiguration().getClient(); + client = spec.connectionConfiguration().getClient(); } @ProcessElement @@ -564,7 +550,7 @@ public void processElement(ProcessContext context) throws Exception { message.setQos(spec.qos()); message.setRetained(spec.retained()); message.setPayload(payload); - client.publish(spec.mqttConnectionConfiguration().topic(), message); + client.publish(spec.connectionConfiguration().topic(), message); } @Teardown 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 308ba4df6139..e5e36d981b23 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 @@ -69,8 +69,8 @@ public void testRead() throws Exception { PCollection output = pipeline.apply( MqttIO.read() - .withMqttConnectionConfiguration( - MqttIO.MqttConnectionConfiguration.create( + .withConnectionConfiguration( + MqttIO.ConnectionConfiguration.create( "tcp://localhost:11883", "BEAM_PIPELINE", "READ_TOPIC")) @@ -141,8 +141,8 @@ public void testWrite() throws Exception { // be persisted to disk, and will be subject to a two-phase ack. pipeline.apply(Create.of(data)) .apply(MqttIO.write() - .withMqttConnectionConfiguration( - MqttIO.MqttConnectionConfiguration.create( + .withConnectionConfiguration( + MqttIO.ConnectionConfiguration.create( "tcp://localhost:11883", "BEAM_PIPELINE", "WRITE_TOPIC")) From 6483580bbd88346eb32364e1fbbe82971aa7393e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 13 Oct 2016 10:41:52 +0200 Subject: [PATCH 08/27] [BEAM-606] Refactore read test to be more reliable --- .../test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 e5e36d981b23..5c6aef3528cc 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 @@ -81,11 +81,12 @@ public void testRead() throws Exception { PAssert.that(output).satisfies(new SerializableFunction, Void>() { @Override public Void apply(Iterable input) { - int count = 0; for (byte[] element : input) { String inputString = new String(element); - Assert.assertEquals("This is test " + count, inputString); - count++; + Assert.assertTrue(inputString.startsWith("This is test ")); + int count = Integer.parseInt(inputString.substring("This is test ".length())); + Assert.assertTrue(count < 10); + Assert.assertTrue(count >= 0); } return null; } From 3b443127b810bc7cafc7e246f0dbc2d839eaaeeb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 21 Oct 2016 15:50:44 +0200 Subject: [PATCH 09/27] [BEAM-606] Update documentation, change the QoS in the test --- .../main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java | 11 +++++------ .../java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) 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 614012ad8199..7bf356f2d7a3 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 @@ -279,7 +279,7 @@ public Coder getCheckpointMarkCoder() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getDefaultOutputCoder() { return ByteArrayCoder.of(); } } @@ -324,11 +324,6 @@ public boolean start() throws IOException { @Override public void connectionLost(Throwable cause) { LOGGER.warn("MQTT connection lost", cause); - try { - close(); - } catch (IOException e) { - // nothing to do - } } @Override @@ -491,6 +486,10 @@ public Write withConnectionConfiguration(ConnectionConfiguration configuration) * delivery of QoS 1 and 2 messages can not be maintained as client-side state will * be lost.

    * + *

    For now, MqttIO fully supports QoS 0 and 1 (delivery at least once). QoS 2 is for now + * limited and use with care, as it can result to duplication of message (delivery exactly + * once).

    + * * @param qos The quality of service value. * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. */ 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 5c6aef3528cc..69f9e559a8eb 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 @@ -107,7 +107,7 @@ public void run() { client.connect(); for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); - message.setQos(2); + message.setQos(1); message.setRetained(true); message.setPayload(("This is test " + i).getBytes()); client.publish("READ_TOPIC", message); From eb31df8dfc11e6581135792220f8fe07752f1d0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 2 Nov 2016 07:01:39 +0100 Subject: [PATCH 10/27] [BEAM-606] Start ActiveMQ broker on free network port in the tests --- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) 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 69f9e559a8eb..891667140475 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 @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.mqtt; import java.io.Serializable; +import java.net.ServerSocket; import java.net.URI; import java.util.ArrayList; import java.util.List; @@ -44,6 +45,8 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Tests of {@link MqttIO}. @@ -51,14 +54,24 @@ @RunWith(JUnit4.class) public class MqttIOTest implements Serializable { + private static final Logger LOGGER = LoggerFactory.getLogger(MqttIOTest.class); + private static transient BrokerService broker; + private static int port; + @BeforeClass public static void startBroker() throws Exception { + LOGGER.info("Finding free network port"); + ServerSocket socket = new ServerSocket(0); + port = socket.getLocalPort(); + socket.close(); + + LOGGER.info("Starting ActiveMQ broker on {}", port); broker = new BrokerService(); broker.setUseJmx(false); broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); - broker.addConnector(new URI("mqtt://localhost:11883")); + broker.addConnector(new URI("mqtt://localhost:" + port)); broker.start(); } @@ -71,7 +84,7 @@ public void testRead() throws Exception { MqttIO.read() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( - "tcp://localhost:11883", + "tcp://localhost:" + port, "BEAM_PIPELINE", "READ_TOPIC")) .withMaxNumRecords(10)); @@ -103,7 +116,7 @@ public void run() { // nothing to do } try { - MqttClient client = new MqttClient("tcp://localhost:11883", "publisher"); + MqttClient client = new MqttClient("tcp://localhost:" + port, "publisher"); client.connect(); for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); @@ -144,7 +157,7 @@ public void testWrite() throws Exception { .apply(MqttIO.write() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( - "tcp://localhost:11883", + "tcp://localhost:" + port, "BEAM_PIPELINE", "WRITE_TOPIC")) .withQoS(2)); @@ -157,7 +170,7 @@ public void testWrite() throws Exception { } private MqttClient receive(final List messages) throws MqttException { - MqttClient client = new MqttClient("tcp://localhost:11883", "receiver"); + MqttClient client = new MqttClient("tcp://localhost:" + port, "receiver"); MqttCallback callback = new MqttCallback() { @Override public void connectionLost(Throwable cause) { From ec93b77c24ca8e57fb3bc242fb07f7719c16054e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 2 Nov 2016 10:38:55 +0100 Subject: [PATCH 11/27] [BEAM-606] Fix parent pom version --- sdks/java/io/mqtt/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index 1e383362ade1..a515a00860ae 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.3.0-incubating-SNAPSHOT + 0.4.0-incubating-SNAPSHOT ../pom.xml From 911e98b317f63f54f3f95927299ef6aae0ec8fe2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 3 Nov 2016 09:50:00 +0100 Subject: [PATCH 12/27] [BEAM-606] Exclude support of QoS 2, cleanup in javadoc --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 62 +++++++++++++------ .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 7 +-- 2 files changed, 44 insertions(+), 25 deletions(-) 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 7bf356f2d7a3..befde12e3afc 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 @@ -23,6 +23,8 @@ import com.google.auto.value.AutoValue; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -33,8 +35,10 @@ import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -62,11 +66,11 @@ *

    Reading from a MQTT broker

    * *

    MqttIO source returns an unbounded {@link PCollection} containing MQTT message - * payloads (as {@code byte[]}).

    + * payloads (as {@code byte[]}). * *

    To configure a MQTT source, you have to provide a MQTT connection configuration including * {@code ClientId}, a {@code ServerURI}, and a {@code Topic} pattern. The following - * example illustrates various options for configuring the source:

    + * example illustrates various options for configuring the source: * *
    {@code
      *
    @@ -81,15 +85,15 @@
      *
      * 

    Writing to a MQTT broker

    * - *

    MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker.

    + *

    MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker. * *

    To configure a MQTT sink, as for the read, you have to specify a MQTT connection - * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}.

    + * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}. * *

    Optionally, you can also specify the {@code Retained} and {@code QoS} of the MQTT - * message.

    + * message. * - *

    For instance:

    + *

    For instance: * *

    {@code
      *
    @@ -100,8 +104,6 @@
      *       "tcp://host:11883",
      *       "my_client_id",
      *       "my_topic"))
    - *     .withRetained(true)
    - *     .withQoS(2)
      *
      * }
    */ @@ -232,7 +234,9 @@ public void validate(PBegin input) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); connectionConfiguration().populateDisplayData(builder); - builder.add(DisplayData.item("maxNumRecords", maxNumRecords())); + if (maxNumRecords() != Long.MAX_VALUE) { + builder.add(DisplayData.item("maxNumRecords", maxNumRecords())); + } builder.addIfNotNull(DisplayData.item("maxReadTime", maxReadTime())); } @@ -274,8 +278,8 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getCheckpointMarkCoder() { - return VoidCoder.of(); + public Coder getCheckpointMarkCoder() { + return new CheckpointCoder(); } @Override @@ -284,6 +288,25 @@ public Coder getDefaultOutputCoder() { } } + /** + * Checkpoint coder acting as a {@link VoidCoder}. + */ + private static class CheckpointCoder extends AtomicCoder { + + @Override + public void encode(UnboundedSource.CheckpointMark value, OutputStream outStream, Context + context) throws CoderException, IOException { + // nothing to write + } + + @Override + public UnboundedSource.CheckpointMark decode(InputStream inStream, Context context) throws + CoderException, IOException { + // nothing to read + return null; + } + } + /** * POJO used to store MQTT message and its timestamp. */ @@ -459,16 +482,16 @@ public Write withConnectionConfiguration(ConnectionConfiguration configuration) * and will not be acknowledged across the network. This QoS is the fastest, * but should only be used for messages which are not valuable - note that * if the server cannot process the message (for example, there - * is an authorization problem), then an - * {@link MqttCallback#deliveryComplete(IMqttDeliveryToken)} won't be called. + * is an authorization problem), then the message will be silently dropped. * Also known as "fire and forget". * *
  • Quality of Service 1 - indicates that a message should * be delivered at least once (one or more times). The message can only be delivered safely if * it can be persisted on the broker. * If the broker can't persist the message, the message will not be - * delivered in the event of a client failure. - * The message will be acknowledged across the network. + * delivered in the event of a subscriber failure. + * The {@link MqttIO.Write} with this QoS guarantee that all messages written to it will be + * delivered to subscribers at least once. * This is the default QoS.
  • * *
  • Quality of Service 2 - indicates that a message should @@ -477,18 +500,19 @@ public Write withConnectionConfiguration(ConnectionConfiguration configuration) * The message can only be delivered safely if * it can be persisted on the broker. * If a persistence mechanism is not specified on the broker, the message will not be - * delivered in the event of a client failure.
  • + * delivered in the event of a client failure. + * This QoS is not supported by {@link MqttIO}. * * *

    If persistence is not configured, QoS 1 and 2 messages will still be delivered * in the event of a network or server problem as the client will hold state in memory. * If the MQTT client is shutdown or fails and persistence is not configured then * delivery of QoS 1 and 2 messages can not be maintained as client-side state will - * be lost.

    + * be lost. * *

    For now, MqttIO fully supports QoS 0 and 1 (delivery at least once). QoS 2 is for now * limited and use with care, as it can result to duplication of message (delivery exactly - * once).

    + * once). * * @param qos The quality of service value. * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. @@ -517,7 +541,7 @@ public PDone apply(PCollection input) { @Override public void validate(PCollection input) { - // validation is performed in the ConnectionConfiguration create() + checkArgument(qos() != 0 || qos() != 1, "Supported QoS are 0 and 1"); } @Override 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 891667140475..72b6a9e7fa5d 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 @@ -120,8 +120,7 @@ public void run() { client.connect(); for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); - message.setQos(1); - message.setRetained(true); + message.setQos(0); message.setPayload(("This is test " + i).getBytes()); client.publish("READ_TOPIC", message); } @@ -149,10 +148,6 @@ public void testWrite() throws Exception { for (int i = 0; i < 100; i++) { data.add("Test".getBytes()); } - // we use QoS 2 here to be sure the subscriber completely receive all messages before - // shutting down the MQTT broker. - // Quality of Service 2 indicates that a message should be delivered once. The message will - // be persisted to disk, and will be subject to a two-phase ack. pipeline.apply(Create.of(data)) .apply(MqttIO.write() .withConnectionConfiguration( From 655b53e6024a5a7cc3efa4c416c484753a936685 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 14 Nov 2016 13:07:01 +0100 Subject: [PATCH 13/27] [BEAM-606] Add core MQTT test --- sdks/java/io/mqtt/pom.xml | 2 +- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 48 +++++++++++++++++++ 2 files changed, 49 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index a515a00860ae..24ff8e6c6b8e 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -60,7 +60,7 @@ - 1.0.2 + 1.1.0 5.13.1 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 72b6a9e7fa5d..a67c9afb75aa 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 @@ -136,6 +136,54 @@ public void run() { pipeline.run(); } + public class MyMqttCallback implements MqttCallback { + private String client; + + public MyMqttCallback(String client) { + this.client = client; + } + + @Override + public void connectionLost(Throwable cause) { + LOGGER.warn("Connection lost client {}", client, cause); + } + + @Override + public void messageArrived(String topic, MqttMessage message) throws Exception { + LOGGER.info("Message arrived client {} on topic {}: {}", client, topic, + new String(message.getPayload())); + } + + @Override + public void deliveryComplete(IMqttDeliveryToken token) { + LOGGER.info("Delivery complete client {}", client); + } + } + + @Test + public void severalSubscriber() throws Exception { + MqttClient sub1 = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); + sub1.setCallback(new MyMqttCallback("sub1")); + sub1.connect(); + sub1.subscribe("test"); + + MqttClient sub2 = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); + sub2.setCallback(new MyMqttCallback("sub2")); + sub2.connect(); + sub2.subscribe("test"); + + MqttClient pub = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); + pub.connect(); + MqttMessage message = new MqttMessage("foo".getBytes()); + message.setQos(0); + pub.publish("test", message); + + pub.disconnect(); + + sub1.disconnect(); + sub2.disconnect(); + } + @Test @Category(NeedsRunner.class) public void testWrite() throws Exception { From 134cca0868672ac3b5a1594bd0ff1dbc8ce7e0a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 22 Nov 2016 17:58:14 +0100 Subject: [PATCH 14/27] [BEAM-606] Improved checkpoint with manual ack, update comments about QoS (relationship between publisher and subsribers), auto generate clientId (reliability) --- sdks/java/io/mqtt/pom.xml | 10 +- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 214 ++++++++++-------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 57 +---- 3 files changed, 125 insertions(+), 156 deletions(-) diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index 24ff8e6c6b8e..5f496a77f24c 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -85,17 +85,17 @@ guava - - com.google.code.findbugs - annotations - - org.eclipse.paho org.eclipse.paho.client.mqttv3 ${paho.version} + + com.google.code.findbugs + jsr305 + + com.google.auto.value 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 befde12e3afc..d3c398ffefee 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 @@ -18,15 +18,13 @@ package org.apache.beam.sdk.io.mqtt; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; import com.google.auto.value.AutoValue; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.io.Serializable; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.BlockingQueue; @@ -35,11 +33,10 @@ import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -78,7 +75,6 @@ * MqttIO.read() * .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create( * "tcp://host:11883", - * "my_client_id", * "my_topic")) * * }
    @@ -102,7 +98,6 @@ * .MqttIO.write() * .withConnectionConfiguration(MqttIO.ConnectionConfiguration.create( * "tcp://host:11883", - * "my_client_id", * "my_topic")) * * }
    @@ -133,25 +128,25 @@ private MqttIO() { public abstract static class ConnectionConfiguration implements Serializable { @Nullable abstract String serverUri(); - @Nullable abstract String clientId(); @Nullable abstract String topic(); - public static ConnectionConfiguration create(String serverUri, String clientId, - String topic) { - checkNotNull(serverUri, "serverUri"); - checkNotNull(clientId, "clientId"); - checkNotNull(topic, "topic"); - return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, clientId, topic); + public static ConnectionConfiguration create(String serverUri, String topic) { + checkArgument(serverUri != null, + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "serverUri"); + checkArgument(topic != null, + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "topic"); + return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic); } private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("serverUri", serverUri())); - builder.add(DisplayData.item("clientId", clientId())); builder.add(DisplayData.item("topic", topic())); } private MqttClient getClient() throws MqttException { - MqttClient client = new MqttClient(serverUri(), clientId()); + MqttClient client = new MqttClient(serverUri(), MqttClient.generateClientId()); client.connect(); return client; } @@ -168,7 +163,7 @@ public abstract static class Read extends PTransform abstract long maxNumRecords(); @Nullable abstract Duration maxReadTime(); - abstract Builder toBuilder(); + abstract Builder builder(); @AutoValue.Builder abstract static class Builder { @@ -182,8 +177,10 @@ abstract static class Builder { * Define the MQTT connection configuration used to connect to the MQTT broker. */ public Read withConnectionConfiguration(ConnectionConfiguration configuration) { - checkNotNull(configuration, "ConnectionConfiguration"); - return toBuilder().setConnectionConfiguration(configuration).build(); + checkArgument(configuration != null, + "MqttIO.read().withConnectionConfiguration(configuration) called with null " + + "configuration or not called at all"); + return builder().setConnectionConfiguration(configuration).build(); } /** @@ -194,7 +191,7 @@ public Read withConnectionConfiguration(ConnectionConfiguration configuration) { public Read withMaxNumRecords(long maxNumRecords) { checkArgument(maxReadTime() == null, "maxNumRecord and maxReadTime are exclusive"); - return toBuilder().setMaxNumRecords(maxNumRecords).build(); + return builder().setMaxNumRecords(maxNumRecords).build(); } /** @@ -205,7 +202,7 @@ public Read withMaxNumRecords(long maxNumRecords) { public Read withMaxReadTime(Duration maxReadTime) { checkArgument(maxNumRecords() == Long.MAX_VALUE, "maxNumRecord and maxReadTime are exclusive"); - return toBuilder().setMaxReadTime(maxReadTime).build(); + return builder().setMaxReadTime(maxReadTime).build(); } @Override @@ -242,8 +239,53 @@ public void populateDisplayData(DisplayData.Builder builder) { } + /** + * Checkpoint for an unbounded MQTT source. Consists of the MQTT messages waiting to be + * acknowledged and oldest pending message timestamp. + */ + @DefaultCoder(AvroCoder.class) + private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark { + + private MqttClient client; + + private final List messages = new ArrayList<>(); + private Instant oldestPendingTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + + public MqttCheckpointMark() { + } + + public void setClient(MqttClient client) { + this.client = client; + } + + public void add(MqttMessageWithTimestamp message) { + if (message.getTimestamp().isBefore(oldestPendingTimestamp)) { + oldestPendingTimestamp = message.getTimestamp(); + } + messages.add(message); + } + + @Override + public void finalizeCheckpoint() { + for (MqttMessageWithTimestamp message : messages) { + try { + client.messageArrivedComplete(message.getMessage().getId(), + message.getMessage().getQos()); + if (message.getTimestamp().isAfter(oldestPendingTimestamp)) { + oldestPendingTimestamp = message.getTimestamp(); + } + } catch (Exception e) { + LOGGER.warn("Can't ack message {} with QoS {}", message.getMessage().getId(), + message.getMessage().getQos()); + } + } + messages.clear(); + } + + } + private static class UnboundedMqttSource - extends UnboundedSource { + extends UnboundedSource { private final Read spec; @@ -253,18 +295,18 @@ public UnboundedMqttSource(Read spec) { @Override public UnboundedReader createReader(PipelineOptions options, - CheckpointMark checkpointMark) { - return new UnboundedMqttReader(this); + MqttCheckpointMark checkpointMark) { + return new UnboundedMqttReader(this, checkpointMark); } @Override public List generateInitialSplits(int desiredNumSplits, PipelineOptions options) { - List sources = new ArrayList<>(); - for (int i = 0; i < desiredNumSplits; i++) { - sources.add(new UnboundedMqttSource(spec)); - } - return sources; + // MQTT is based on a pub/sub pattern + // so, if we create several subscribers on the same topic, they all will receive the same + // message, resulting to duplicate messages in the PCollection. + // So, for MQTT, we limit to number of split ot 1 (unique source). + return Collections.singletonList(new UnboundedMqttSource(spec)); } @Override @@ -278,8 +320,8 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getCheckpointMarkCoder() { - return new CheckpointCoder(); + public Coder getCheckpointMarkCoder() { + return AvroCoder.of(MqttCheckpointMark.class); } @Override @@ -288,25 +330,6 @@ public Coder getDefaultOutputCoder() { } } - /** - * Checkpoint coder acting as a {@link VoidCoder}. - */ - private static class CheckpointCoder extends AtomicCoder { - - @Override - public void encode(UnboundedSource.CheckpointMark value, OutputStream outStream, Context - context) throws CoderException, IOException { - // nothing to write - } - - @Override - public UnboundedSource.CheckpointMark decode(InputStream inStream, Context context) throws - CoderException, IOException { - // nothing to read - return null; - } - } - /** * POJO used to store MQTT message and its timestamp. */ @@ -319,6 +342,14 @@ public MqttMessageWithTimestamp(MqttMessage message, Instant timestamp) { this.message = message; this.timestamp = timestamp; } + + public MqttMessage getMessage() { + return message; + } + + public Instant getTimestamp() { + return timestamp; + } } private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { @@ -328,11 +359,17 @@ private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader private MqttClient client; private byte[] current; private Instant currentTimestamp; + private MqttCheckpointMark checkpointMark; private BlockingQueue queue; - private UnboundedMqttReader(UnboundedMqttSource source) { + public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkpointMark) { this.source = source; this.current = null; + if (checkpointMark != null) { + this.checkpointMark = checkpointMark; + } else { + this.checkpointMark = new MqttCheckpointMark(); + } this.queue = new LinkedBlockingQueue<>(); } @@ -343,6 +380,7 @@ public boolean start() throws IOException { try { client = spec.connectionConfiguration().getClient(); client.subscribe(spec.connectionConfiguration().topic()); + client.setManualAcks(true); client.setCallback(new MqttCallback() { @Override public void connectionLost(Throwable cause) { @@ -360,6 +398,7 @@ public void deliveryComplete(IMqttDeliveryToken token) { // nothing to do } }); + checkpointMark.setClient(client); return advance(); } catch (MqttException e) { throw new IOException(e); @@ -380,8 +419,12 @@ public boolean advance() throws IOException { currentTimestamp = null; return false; } + + checkpointMark.add(message); + current = message.message.getPayload(); currentTimestamp = message.timestamp; + return true; } @@ -403,24 +446,12 @@ public void close() throws IOException { @Override public Instant getWatermark() { - MqttMessageWithTimestamp message = queue.peek(); - if (message == null) { - // no message yet in the queue, returning the min possible timestamp value - return BoundedWindow.TIMESTAMP_MIN_VALUE; - } else { - // watermark is the timestamp of the oldest message in the queue - return message.timestamp; - } + return checkpointMark.oldestPendingTimestamp; } @Override public UnboundedSource.CheckpointMark getCheckpointMark() { - return new UnboundedSource.CheckpointMark() { - @Override - public void finalizeCheckpoint() throws IOException { - // nothing to do - } - }; + return checkpointMark; } @Override @@ -456,7 +487,7 @@ public abstract static class Write extends PTransform, PDone @Nullable abstract int qos(); @Nullable abstract boolean retained(); - abstract Builder toBuilder(); + abstract Builder builder(); @AutoValue.Builder abstract static class Builder { @@ -470,38 +501,27 @@ abstract static class Builder { * Define MQTT connection configuration used to connect to the MQTT broker. */ public Write withConnectionConfiguration(ConnectionConfiguration configuration) { - return toBuilder().setConnectionConfiguration(configuration).build(); + checkArgument(configuration != null, + "MqttIO.write().withConnectionConfiguration(configuration) called with null " + + "configuration or not called at all"); + return builder().setConnectionConfiguration(configuration).build(); } /** * Define the MQTT message quality of service. * *
      - *
    • Quality of Service 0 - indicates that a message should - * be delivered at most once (zero or one times). The message will not be persisted to disk, - * and will not be acknowledged across the network. This QoS is the fastest, - * but should only be used for messages which are not valuable - note that - * if the server cannot process the message (for example, there - * is an authorization problem), then the message will be silently dropped. - * Also known as "fire and forget".
    • + *
    • Quality of Service 0 (at most once) - a message won't be ack by the receiver or + * stored and redelivered by the sender. This is "fire and forget" and provides the same + * guarantee as the underlying TCP protocol.
    • * - *
    • Quality of Service 1 - indicates that a message should - * be delivered at least once (one or more times). The message can only be delivered safely if - * it can be persisted on the broker. - * If the broker can't persist the message, the message will not be - * delivered in the event of a subscriber failure. - * The {@link MqttIO.Write} with this QoS guarantee that all messages written to it will be - * delivered to subscribers at least once. - * This is the default QoS.
    • + *
    • Quality of Service 1 (at least once) - a message will be delivered at least once to + * the receiver. But the message can also be delivered more than once. The sender will + * store the message until it gets an ack from the receiver.
    • * - *
    • Quality of Service 2 - indicates that a message should - * be delivered once. The message will be persisted to disk on the broker, and will - * be subject to a two-phase acknowledgement across the network. - * The message can only be delivered safely if - * it can be persisted on the broker. - * If a persistence mechanism is not specified on the broker, the message will not be - * delivered in the event of a client failure. - * This QoS is not supported by {@link MqttIO}.
    • + *
    • Quality of Service 2 (exactly one) - each message is received only once by the + * counterpart. It is the safest and also the slowest quality of service level. The + * guarantee is provided by two flows there and back between sender and receiver.
    • *
    * *

    If persistence is not configured, QoS 1 and 2 messages will still be delivered @@ -510,27 +530,25 @@ public Write withConnectionConfiguration(ConnectionConfiguration configuration) * delivery of QoS 1 and 2 messages can not be maintained as client-side state will * be lost. * - *

    For now, MqttIO fully supports QoS 0 and 1 (delivery at least once). QoS 2 is for now - * limited and use with care, as it can result to duplication of message (delivery exactly - * once). - * * @param qos The quality of service value. * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. */ public Write withQoS(int qos) { - return toBuilder().setQos(qos).build(); + return builder().setQos(qos).build(); } /** * Whether or not the publish message should be retained by the messaging engine. * Sending a message with the retained set to {@code false} will clear the * retained message from the server. The default value is {@code false}. + * When a subscriber connects, he gets the latest retained message (else it doesn't get any + * existing message, he will have to wait a new incoming message). * * @param retained Whether or not the messaging engine should retain the message. * @return The {@link Write} {@link PTransform} with the corresponding retained configuration. */ public Write withRetained(boolean retained) { - return toBuilder().setRetained(retained).build(); + return builder().setRetained(retained).build(); } @Override @@ -541,7 +559,7 @@ public PDone apply(PCollection input) { @Override public void validate(PCollection input) { - checkArgument(qos() != 0 || qos() != 1, "Supported QoS are 0 and 1"); + // validate is done in connection configuration } @Override 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 a67c9afb75aa..ae49bceb13e1 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 @@ -85,7 +85,6 @@ public void testRead() throws Exception { .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "BEAM_PIPELINE", "READ_TOPIC")) .withMaxNumRecords(10)); @@ -116,7 +115,8 @@ public void run() { // nothing to do } try { - MqttClient client = new MqttClient("tcp://localhost:" + port, "publisher"); + MqttClient client = new MqttClient("tcp://localhost:" + port, + MqttClient.generateClientId()); client.connect(); for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); @@ -136,54 +136,6 @@ public void run() { pipeline.run(); } - public class MyMqttCallback implements MqttCallback { - private String client; - - public MyMqttCallback(String client) { - this.client = client; - } - - @Override - public void connectionLost(Throwable cause) { - LOGGER.warn("Connection lost client {}", client, cause); - } - - @Override - public void messageArrived(String topic, MqttMessage message) throws Exception { - LOGGER.info("Message arrived client {} on topic {}: {}", client, topic, - new String(message.getPayload())); - } - - @Override - public void deliveryComplete(IMqttDeliveryToken token) { - LOGGER.info("Delivery complete client {}", client); - } - } - - @Test - public void severalSubscriber() throws Exception { - MqttClient sub1 = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); - sub1.setCallback(new MyMqttCallback("sub1")); - sub1.connect(); - sub1.subscribe("test"); - - MqttClient sub2 = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); - sub2.setCallback(new MyMqttCallback("sub2")); - sub2.connect(); - sub2.subscribe("test"); - - MqttClient pub = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); - pub.connect(); - MqttMessage message = new MqttMessage("foo".getBytes()); - message.setQos(0); - pub.publish("test", message); - - pub.disconnect(); - - sub1.disconnect(); - sub2.disconnect(); - } - @Test @Category(NeedsRunner.class) public void testWrite() throws Exception { @@ -201,9 +153,8 @@ public void testWrite() throws Exception { .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "BEAM_PIPELINE", "WRITE_TOPIC")) - .withQoS(2)); + .withQoS(1)); pipeline.run(); Assert.assertEquals(100, messages.size()); @@ -213,7 +164,7 @@ public void testWrite() throws Exception { } private MqttClient receive(final List messages) throws MqttException { - MqttClient client = new MqttClient("tcp://localhost:" + port, "receiver"); + MqttClient client = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); MqttCallback callback = new MqttCallback() { @Override public void connectionLost(Throwable cause) { From 2d4307211a2ed56f4e22fd2bfdd2715b521a164b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 5 Dec 2016 11:14:41 +0100 Subject: [PATCH 15/27] [BEAM-606] Fix checkpoint mark --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 68 ++++++++++++------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 2 +- 2 files changed, 45 insertions(+), 25 deletions(-) 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 d3c398ffefee..e0bf2a5c9dd7 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 @@ -36,14 +36,12 @@ import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -243,43 +241,63 @@ public void populateDisplayData(DisplayData.Builder builder) { * Checkpoint for an unbounded MQTT source. Consists of the MQTT messages waiting to be * acknowledged and oldest pending message timestamp. */ - @DefaultCoder(AvroCoder.class) private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark { - private MqttClient client; + private Read spec; + private Instant oldestMessageTimestamp = Instant.now(); + + private static class MessageIdWithQos { + + private int id; + private int qos; + + MessageIdWithQos(int id, int qos) { + this.id = id; + this.qos = qos; + } + + } - private final List messages = new ArrayList<>(); - private Instant oldestPendingTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + private final List messages = new ArrayList<>(); public MqttCheckpointMark() { } - public void setClient(MqttClient client) { - this.client = client; + public void setSpec(Read spec) { + this.spec = spec; } - public void add(MqttMessageWithTimestamp message) { - if (message.getTimestamp().isBefore(oldestPendingTimestamp)) { - oldestPendingTimestamp = message.getTimestamp(); + public void add(int id, int qos, Instant timestamp) { + if (timestamp.isBefore(oldestMessageTimestamp)) { + oldestMessageTimestamp = timestamp; } + MessageIdWithQos message = new MessageIdWithQos(id, qos); messages.add(message); } @Override public void finalizeCheckpoint() { - for (MqttMessageWithTimestamp message : messages) { + MqttClient client; + try { + client = spec.connectionConfiguration().getClient(); + } catch (Exception e) { + throw new IllegalStateException("Can't finalize checkpoint", e); + } + for (MessageIdWithQos message : messages) { try { - client.messageArrivedComplete(message.getMessage().getId(), - message.getMessage().getQos()); - if (message.getTimestamp().isAfter(oldestPendingTimestamp)) { - oldestPendingTimestamp = message.getTimestamp(); - } + client.messageArrivedComplete(message.id, message.qos); } catch (Exception e) { - LOGGER.warn("Can't ack message {} with QoS {}", message.getMessage().getId(), - message.getMessage().getQos()); + LOGGER.warn("Can't ack message {} with QoS {}", message.id, message.qos); } } + oldestMessageTimestamp = Instant.now(); messages.clear(); + try { + client.disconnect(); + client.close(); + } catch (Exception e) { + // nothing to do + } } } @@ -398,7 +416,7 @@ public void deliveryComplete(IMqttDeliveryToken token) { // nothing to do } }); - checkpointMark.setClient(client); + checkpointMark.setSpec(source.spec); return advance(); } catch (MqttException e) { throw new IOException(e); @@ -420,7 +438,9 @@ public boolean advance() throws IOException { return false; } - checkpointMark.add(message); + checkpointMark.add(message.getMessage().getId(), + message.getMessage().getQos(), + message.timestamp); current = message.message.getPayload(); currentTimestamp = message.timestamp; @@ -446,7 +466,7 @@ public void close() throws IOException { @Override public Instant getWatermark() { - return checkpointMark.oldestPendingTimestamp; + return checkpointMark.oldestMessageTimestamp; } @Override @@ -541,8 +561,8 @@ public Write withQoS(int qos) { * Whether or not the publish message should be retained by the messaging engine. * Sending a message with the retained set to {@code false} will clear the * retained message from the server. The default value is {@code false}. - * When a subscriber connects, he gets the latest retained message (else it doesn't get any - * existing message, he will have to wait a new incoming message). + * When a subscriber connects, it gets the latest retained message (else it doesn't get any + * existing message, it will have to wait a new incoming message). * * @param retained Whether or not the messaging engine should retain the message. * @return The {@link Write} {@link PTransform} with the corresponding retained configuration. 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 ae49bceb13e1..03ea2d435084 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 @@ -132,8 +132,8 @@ public void run() { } }; thread.start(); - pipeline.run(); + thread.join(); } @Test From b3c3f3c8acf0b646ce120258c7e3954746f7544a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 9 Dec 2016 18:14:07 +0100 Subject: [PATCH 16/27] [BEAM-606] Only support QoS 1 in Write, updated to last SDK changes --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 44 +++---------------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 3 +- 2 files changed, 6 insertions(+), 41 deletions(-) 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 e0bf2a5c9dd7..1b5f5b39e694 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 @@ -84,8 +84,8 @@ *

    To configure a MQTT sink, as for the read, you have to specify a MQTT connection * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}. * - *

    Optionally, you can also specify the {@code Retained} and {@code QoS} of the MQTT - * message. + *

    The MqttIO only fully supports QoS 1 (at least once). It's the only QoS level guaranteed + * due to potential retries on bundles. * *

    For instance: * @@ -112,7 +112,6 @@ public static Read read() { public static Write write() { return new AutoValue_MqttIO_Write.Builder() .setRetained(false) - .setQos(0) .build(); } @@ -204,7 +203,7 @@ public Read withMaxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PBegin input) { + public PCollection expand(PBegin input) { org.apache.beam.sdk.io.Read.Unbounded unbounded = org.apache.beam.sdk.io.Read.from(new UnboundedMqttSource(this)); @@ -504,7 +503,6 @@ public UnboundedMqttSource getCurrentSource() { public abstract static class Write extends PTransform, PDone> { @Nullable abstract ConnectionConfiguration connectionConfiguration(); - @Nullable abstract int qos(); @Nullable abstract boolean retained(); abstract Builder builder(); @@ -512,7 +510,6 @@ public abstract static class Write extends PTransform, PDone @AutoValue.Builder abstract static class Builder { abstract Builder setConnectionConfiguration(ConnectionConfiguration configuration); - abstract Builder setQos(int qos); abstract Builder setRetained(boolean retained); abstract Write build(); } @@ -527,36 +524,6 @@ public Write withConnectionConfiguration(ConnectionConfiguration configuration) return builder().setConnectionConfiguration(configuration).build(); } - /** - * Define the MQTT message quality of service. - * - *

      - *
    • Quality of Service 0 (at most once) - a message won't be ack by the receiver or - * stored and redelivered by the sender. This is "fire and forget" and provides the same - * guarantee as the underlying TCP protocol.
    • - * - *
    • Quality of Service 1 (at least once) - a message will be delivered at least once to - * the receiver. But the message can also be delivered more than once. The sender will - * store the message until it gets an ack from the receiver.
    • - * - *
    • Quality of Service 2 (exactly one) - each message is received only once by the - * counterpart. It is the safest and also the slowest quality of service level. The - * guarantee is provided by two flows there and back between sender and receiver.
    • - *
    - * - *

    If persistence is not configured, QoS 1 and 2 messages will still be delivered - * in the event of a network or server problem as the client will hold state in memory. - * If the MQTT client is shutdown or fails and persistence is not configured then - * delivery of QoS 1 and 2 messages can not be maintained as client-side state will - * be lost. - * - * @param qos The quality of service value. - * @return The {@link Write} {@link PTransform} with the corresponding QoS configuration. - */ - public Write withQoS(int qos) { - return builder().setQos(qos).build(); - } - /** * Whether or not the publish message should be retained by the messaging engine. * Sending a message with the retained set to {@code false} will clear the @@ -572,7 +539,7 @@ public Write withRetained(boolean retained) { } @Override - public PDone apply(PCollection input) { + public PDone expand(PCollection input) { input.apply(ParDo.of(new WriteFn(this))); return PDone.in(input.getPipeline()); } @@ -585,7 +552,6 @@ public void validate(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { connectionConfiguration().populateDisplayData(builder); - builder.add(DisplayData.item("qos", qos())); builder.add(DisplayData.item("retained", retained())); } @@ -608,7 +574,7 @@ public void createMqttClient() throws Exception { public void processElement(ProcessContext context) throws Exception { byte[] payload = context.element(); MqttMessage message = new MqttMessage(); - message.setQos(spec.qos()); + message.setQos(1); message.setRetained(spec.retained()); message.setPayload(payload); client.publish(spec.connectionConfiguration().topic(), message); 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 03ea2d435084..eaaac1d229cb 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 @@ -153,8 +153,7 @@ public void testWrite() throws Exception { .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "WRITE_TOPIC")) - .withQoS(1)); + "WRITE_TOPIC"))); pipeline.run(); Assert.assertEquals(100, messages.size()); From ca3eec8c051bf3d565b552edc8e46ba89b680d60 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sat, 10 Dec 2016 16:56:35 +0100 Subject: [PATCH 17/27] [BEAM-606] Use ActiveMQ KahaDB in test for persistence adapter --- .../test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 eaaac1d229cb..e876b56ae7f5 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.mqtt; +import java.io.File; import java.io.Serializable; import java.net.ServerSocket; import java.net.URI; @@ -24,7 +25,7 @@ import java.util.List; import org.apache.activemq.broker.BrokerService; -import org.apache.activemq.store.memory.MemoryPersistenceAdapter; +import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -70,7 +71,9 @@ public static void startBroker() throws Exception { LOGGER.info("Starting ActiveMQ broker on {}", port); broker = new BrokerService(); broker.setUseJmx(false); - broker.setPersistenceAdapter(new MemoryPersistenceAdapter()); + KahaDBPersistenceAdapter kahaDb = new KahaDBPersistenceAdapter(); + kahaDb.setDirectory(new File("target/kahadb")); + broker.setPersistenceAdapter(kahaDb); broker.addConnector(new URI("mqtt://localhost:" + port)); broker.start(); } From 8730e0a87ecdda45d6f94bcb06dc74f3a7170f9f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 14 Dec 2016 14:17:20 +0100 Subject: [PATCH 18/27] [BEAM-606] Update parent version --- sdks/java/io/mqtt/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index 5f496a77f24c..015c0138424a 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-sdks-java-io-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml From 0bbfe7ef254122e23ddd3a658cb3673293aa860b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 14 Dec 2016 21:22:48 +0100 Subject: [PATCH 19/27] [BEAM-606] Fix checkpoint mark reusing the same MQTT client as in the source, fix on the tests --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 52 ++++-- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 176 ++++++++++-------- 2 files changed, 128 insertions(+), 100 deletions(-) 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 1b5f5b39e694..6c023dee24bc 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 @@ -126,6 +126,7 @@ public abstract static class ConnectionConfiguration implements Serializable { @Nullable abstract String serverUri(); @Nullable abstract String topic(); + @Nullable abstract String clientId(); public static ConnectionConfiguration create(String serverUri, String topic) { checkArgument(serverUri != null, @@ -134,20 +135,42 @@ public static ConnectionConfiguration create(String serverUri, String topic) { checkArgument(topic != null, "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + "topic"); - return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic); + return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic, + MqttClient.generateClientId()); + } + + public static ConnectionConfiguration create(String serverUri, String topic, String clientId) { + checkArgument(serverUri != null, + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "serverUri"); + checkArgument(topic != null, + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "topic"); + checkArgument(clientId != null, "MqttIO.ConnectionConfiguration.create(serverUri, topic, " + + "clientId) called with null clientId"); + return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic, clientId); } private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("serverUri", serverUri())); builder.add(DisplayData.item("topic", topic())); + builder.add(DisplayData.item("clientId", clientId())); } - private MqttClient getClient() throws MqttException { - MqttClient client = new MqttClient(serverUri(), MqttClient.generateClientId()); + private MqttClient getClient(boolean random) throws MqttException { + String id = clientId(); + if (random) { + id = clientId() + "-" + MqttClient.generateClientId(); + } + MqttClient client = new MqttClient(serverUri(), id); client.connect(); return client; } + private MqttClient getClient() throws MqttException { + return getClient(false); + } + } /** @@ -242,7 +265,7 @@ public void populateDisplayData(DisplayData.Builder builder) { */ private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark { - private Read spec; + private transient MqttClient client; private Instant oldestMessageTimestamp = Instant.now(); private static class MessageIdWithQos { @@ -262,8 +285,8 @@ private static class MessageIdWithQos { public MqttCheckpointMark() { } - public void setSpec(Read spec) { - this.spec = spec; + public void setClient(MqttClient client) { + this.client = client; } public void add(int id, int qos, Instant timestamp) { @@ -276,12 +299,6 @@ public void add(int id, int qos, Instant timestamp) { @Override public void finalizeCheckpoint() { - MqttClient client; - try { - client = spec.connectionConfiguration().getClient(); - } catch (Exception e) { - throw new IllegalStateException("Can't finalize checkpoint", e); - } for (MessageIdWithQos message : messages) { try { client.messageArrivedComplete(message.id, message.qos); @@ -291,12 +308,6 @@ public void finalizeCheckpoint() { } oldestMessageTimestamp = Instant.now(); messages.clear(); - try { - client.disconnect(); - client.close(); - } catch (Exception e) { - // nothing to do - } } } @@ -402,6 +413,7 @@ public boolean start() throws IOException { @Override public void connectionLost(Throwable cause) { LOGGER.warn("MQTT connection lost", cause); + cause.printStackTrace(); } @Override @@ -415,7 +427,7 @@ public void deliveryComplete(IMqttDeliveryToken token) { // nothing to do } }); - checkpointMark.setSpec(source.spec); + checkpointMark.setClient(client); return advance(); } catch (MqttException e) { throw new IOException(e); @@ -567,7 +579,7 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { - client = spec.connectionConfiguration().getClient(); + client = spec.connectionConfiguration().getClient(true); } @ProcessElement 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 e876b56ae7f5..71c162166ed6 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 @@ -17,31 +17,33 @@ */ package org.apache.beam.sdk.io.mqtt; +import static org.junit.Assert.assertEquals; + import java.io.File; import java.io.Serializable; import java.net.ServerSocket; import java.net.URI; import java.util.ArrayList; -import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import org.apache.activemq.broker.BrokerService; -import org.apache.activemq.store.kahadb.KahaDBPersistenceAdapter; +import org.apache.activemq.broker.Connection; +import org.apache.activemq.broker.TransportConnector; +import org.apache.activemq.store.kahadb.KahaDBStore; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; import org.eclipse.paho.client.mqttv3.MqttClient; -import org.eclipse.paho.client.mqttv3.MqttException; import org.eclipse.paho.client.mqttv3.MqttMessage; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; +import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; @@ -57,70 +59,84 @@ public class MqttIOTest implements Serializable { private static final Logger LOGGER = LoggerFactory.getLogger(MqttIOTest.class); - private static transient BrokerService broker; + private static transient BrokerService brokerService; private static int port; - @BeforeClass - public static void startBroker() throws Exception { + @Before + public void startBroker() throws Exception { LOGGER.info("Finding free network port"); ServerSocket socket = new ServerSocket(0); port = socket.getLocalPort(); socket.close(); - LOGGER.info("Starting ActiveMQ broker on {}", port); - broker = new BrokerService(); - broker.setUseJmx(false); - KahaDBPersistenceAdapter kahaDb = new KahaDBPersistenceAdapter(); - kahaDb.setDirectory(new File("target/kahadb")); - broker.setPersistenceAdapter(kahaDb); - broker.addConnector(new URI("mqtt://localhost:" + port)); - broker.start(); + LOGGER.info("Starting ActiveMQ brokerService on {}", port); + brokerService = new BrokerService(); + brokerService.setDeleteAllMessagesOnStartup(true); + brokerService.setPersistent(true); + brokerService.setDataDirectory("target/activemq-data/"); + KahaDBStore kahaDBStore = new KahaDBStore(); + kahaDBStore.setDirectory(new File("target/activemq-data/")); + brokerService.setPersistenceAdapter(kahaDBStore); + brokerService.setAdvisorySupport(false); + brokerService.setUseJmx(true); + brokerService.getManagementContext().setCreateConnector(false); + brokerService.setSchedulerSupport(true); + brokerService.setPopulateJMSXUserID(true); + TransportConnector mqttConnector = new TransportConnector(); + mqttConnector.setName("mqtt"); + mqttConnector.setUri(new URI("mqtt://localhost:" + port)); + mqttConnector.setAllowLinkStealing(true); + brokerService.addConnector(mqttConnector); + brokerService.start(); + brokerService.waitUntilStarted(); } - @Test + @Test(timeout = 120 * 1000) @Category(NeedsRunner.class) public void testRead() throws Exception { - Pipeline pipeline = TestPipeline.create(); + final Pipeline pipeline = TestPipeline.create(); PCollection output = pipeline.apply( MqttIO.read() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "READ_TOPIC")) + "READ_TOPIC", + "READ_PIPELINE")) .withMaxNumRecords(10)); - - PAssert.thatSingleton(output.apply("Count", Count.globally())) - .isEqualTo(10L); - PAssert.that(output).satisfies(new SerializableFunction, Void>() { - @Override - public Void apply(Iterable input) { - for (byte[] element : input) { - String inputString = new String(element); - Assert.assertTrue(inputString.startsWith("This is test ")); - int count = Integer.parseInt(inputString.substring("This is test ".length())); - Assert.assertTrue(count < 10); - Assert.assertTrue(count >= 0); - } - return null; - } - }); - - // produce messages on the broker in another thread + PAssert.that(output).containsInAnyOrder( + "This is test 0".getBytes(), + "This is test 1".getBytes(), + "This is test 2".getBytes(), + "This is test 3".getBytes(), + "This is test 4".getBytes(), + "This is test 5".getBytes(), + "This is test 6".getBytes(), + "This is test 7".getBytes(), + "This is test 8".getBytes(), + "This is test 9".getBytes() + ); + + // produce messages on the brokerService in another thread // This thread prevents to block the pipeline waiting for new messages Thread thread = new Thread() { public void run() { - try { - // gives time to the pipeline to start - Thread.sleep(2000); - } catch (Exception e) { - // nothing to do - } try { MqttClient client = new MqttClient("tcp://localhost:" + port, - MqttClient.generateClientId()); + MqttClient.generateClientId(), new MqttDefaultFilePersistence("target/paho/")); client.connect(); + LOGGER.info("Waiting pipeline connected to the MQTT broker before sending " + + "messages ..."); + boolean pipelineConnected = false; + while (!pipelineConnected) { + Thread.sleep(50); + for (Connection connection : brokerService.getBroker().getClients()) { + if (connection.getConnectionId().equals("READ_PIPELINE")) { + pipelineConnected = true; + } + } + } for (int i = 0; i < 10; i++) { MqttMessage message = new MqttMessage(); message.setQos(0); @@ -142,13 +158,34 @@ public void run() { @Test @Category(NeedsRunner.class) public void testWrite() throws Exception { - List messages = new ArrayList<>(); - MqttClient client = receive(messages); + final CountDownLatch latch = new CountDownLatch(200); + + MqttClient client = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId(), + new MqttDefaultFilePersistence("target/paho/")); + client.connect(); + client.subscribe("WRITE_TOPIC"); + client.setCallback(new MqttCallback() { + @Override + public void connectionLost(Throwable throwable) { + LOGGER.warn("Connection Lost", throwable); + throwable.printStackTrace(); + } + + @Override + public void messageArrived(String s, MqttMessage mqttMessage) throws Exception { + latch.countDown(); + } + + @Override + public void deliveryComplete(IMqttDeliveryToken iMqttDeliveryToken) { + // nothing to do + } + }); Pipeline pipeline = TestPipeline.create(); ArrayList data = new ArrayList<>(); - for (int i = 0; i < 100; i++) { + for (int i = 0; i < 200; i++) { data.add("Test".getBytes()); } pipeline.apply(Create.of(data)) @@ -156,43 +193,22 @@ public void testWrite() throws Exception { .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "WRITE_TOPIC"))); + "WRITE_TOPIC", "TEST"))); pipeline.run(); - Assert.assertEquals(100, messages.size()); + latch.await(30, TimeUnit.SECONDS); + assertEquals(0, latch.getCount()); client.disconnect(); client.close(); } - private MqttClient receive(final List messages) throws MqttException { - MqttClient client = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId()); - MqttCallback callback = new MqttCallback() { - @Override - public void connectionLost(Throwable cause) { - cause.printStackTrace(); - } - - @Override - public void messageArrived(String topic, MqttMessage message) throws Exception { - messages.add(message); - } - - @Override - public void deliveryComplete(IMqttDeliveryToken token) { - // nothing to do - } - }; - client.connect(); - client.subscribe("WRITE_TOPIC"); - client.setCallback(callback); - return client; - } - - @AfterClass - public static void stopBroker() throws Exception { - if (broker != null) { - broker.stop(); + @After + public void stopBroker() throws Exception { + if (brokerService != null) { + brokerService.stop(); + brokerService.waitUntilStopped(); + brokerService = null; } } From 463e0a3c1cf452e020c4974f0818e87f4e6c944d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sun, 18 Dec 2016 08:29:40 +0100 Subject: [PATCH 20/27] [BEAM-606] Replace paho MQTT client lib by fusesource one providing ~ 30% better performances. --- sdks/java/io/mqtt/pom.xml | 12 +- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 261 ++++++++---------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 124 ++++----- 3 files changed, 170 insertions(+), 227 deletions(-) diff --git a/sdks/java/io/mqtt/pom.xml b/sdks/java/io/mqtt/pom.xml index 015c0138424a..ab0fa27ced83 100644 --- a/sdks/java/io/mqtt/pom.xml +++ b/sdks/java/io/mqtt/pom.xml @@ -60,7 +60,6 @@ - 1.1.0 5.13.1 @@ -86,9 +85,14 @@ - org.eclipse.paho - org.eclipse.paho.client.mqttv3 - ${paho.version} + org.fusesource.mqtt-client + mqtt-client + 1.14 + + + org.fusesource.hawtbuf + hawtbuf + 1.11 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 6c023dee24bc..407f31c92439 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 @@ -27,9 +27,6 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; @@ -45,11 +42,11 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; -import org.eclipse.paho.client.mqttv3.MqttCallback; -import org.eclipse.paho.client.mqttv3.MqttClient; -import org.eclipse.paho.client.mqttv3.MqttException; -import org.eclipse.paho.client.mqttv3.MqttMessage; +import org.fusesource.mqtt.client.BlockingConnection; +import org.fusesource.mqtt.client.MQTT; +import org.fusesource.mqtt.client.Message; +import org.fusesource.mqtt.client.QoS; +import org.fusesource.mqtt.client.Topic; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -64,7 +61,8 @@ * payloads (as {@code byte[]}). * *

    To configure a MQTT source, you have to provide a MQTT connection configuration including - * {@code ClientId}, a {@code ServerURI}, and a {@code Topic} pattern. The following + * {@code ClientId}, a {@code ServerURI}, a {@code Topic} pattern, and optionally {@code + * username} and {@code password} to connect to the MQTT broker. The following * example illustrates various options for configuring the source: * *

    {@code
    @@ -79,10 +77,10 @@
      *
      * 

    Writing to a MQTT broker

    * - *

    MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker. + *

    MqttIO sink supports writing {@code byte[]} to a getTopic on a MQTT broker. * *

    To configure a MQTT sink, as for the read, you have to specify a MQTT connection - * configuration with {@code ClientId}, {@code ServerURI}, {@code Topic}. + * configuration with {@code ServerURI}, {@code Topic}, ... * *

    The MqttIO only fully supports QoS 1 (at least once). It's the only QoS level guaranteed * due to potential retries on bundles. @@ -124,51 +122,90 @@ private MqttIO() { @AutoValue public abstract static class ConnectionConfiguration implements Serializable { - @Nullable abstract String serverUri(); - @Nullable abstract String topic(); - @Nullable abstract String clientId(); + @Nullable abstract String getServerUri(); + @Nullable abstract String getTopic(); + @Nullable abstract String getClientId(); + @Nullable abstract String getUsername(); + @Nullable abstract String getPassword(); + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setServerUri(String serverUri); + abstract Builder setTopic(String topic); + abstract Builder setClientId(String clientId); + abstract Builder setUsername(String username); + abstract Builder setPassword(String password); + abstract ConnectionConfiguration build(); + } + + /** + * Describe a connection configuration to the MQTT broker. This method creates an unique random + * MQTT client ID. + * + * @param serverUri The MQTT broker URI. + * @param topic The MQTT getTopic pattern. + * @return A connection configuration to the MQTT broker. + */ public static ConnectionConfiguration create(String serverUri, String topic) { checkArgument(serverUri != null, - "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " - + "serverUri"); + "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " + + "getServerUri"); checkArgument(topic != null, - "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " - + "topic"); - return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic, - MqttClient.generateClientId()); + "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " + + "getTopic"); + return new AutoValue_MqttIO_ConnectionConfiguration.Builder().setServerUri(serverUri) + .setTopic(topic).build(); } + /** + * Describe a connection configuration to the MQTT broker. + * + * @param serverUri The MQTT broker URI. + * @param topic The MQTT getTopic pattern. + * @param clientId A client ID prefix, used to construct an unique client ID. + * @return A connection configuration to the MQTT broker. + */ public static ConnectionConfiguration create(String serverUri, String topic, String clientId) { checkArgument(serverUri != null, - "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " - + "serverUri"); + "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " + + "getServerUri"); checkArgument(topic != null, - "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " - + "topic"); - checkArgument(clientId != null, "MqttIO.ConnectionConfiguration.create(serverUri, topic, " - + "clientId) called with null clientId"); - return new AutoValue_MqttIO_ConnectionConfiguration(serverUri, topic, clientId); + "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " + + "getTopic"); + checkArgument(clientId != null, "MqttIO.ConnectionConfiguration.create(getServerUri," + + "getTopic, getClientId) called with null getClientId"); + return new AutoValue_MqttIO_ConnectionConfiguration.Builder().setServerUri(serverUri) + .setTopic(topic).setClientId(clientId).build(); } - private void populateDisplayData(DisplayData.Builder builder) { - builder.add(DisplayData.item("serverUri", serverUri())); - builder.add(DisplayData.item("topic", topic())); - builder.add(DisplayData.item("clientId", clientId())); + public ConnectionConfiguration withUsername(String username) { + return builder().setUsername(username).build(); } - private MqttClient getClient(boolean random) throws MqttException { - String id = clientId(); - if (random) { - id = clientId() + "-" + MqttClient.generateClientId(); - } - MqttClient client = new MqttClient(serverUri(), id); - client.connect(); - return client; + public ConnectionConfiguration withPassword(String password) { + return builder().setPassword(password).build(); } - private MqttClient getClient() throws MqttException { - return getClient(false); + private void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("serverUri", getServerUri())); + builder.add(DisplayData.item("topic", getTopic())); + builder.addIfNotNull(DisplayData.item("clientId", getClientId())); + builder.addIfNotNull(DisplayData.item("username", getUsername())); + } + + private MQTT getClient() throws Exception { + MQTT client = new MQTT(); + client.setHost(getServerUri()); + if (getUsername() != null) { + client.setUserName(getUsername()); + client.setPassword(getPassword()); + } + if (getClientId() != null) { + client.setClientId(getClientId() + "-" + System.currentTimeMillis()); + } + return client; } } @@ -265,45 +302,27 @@ public void populateDisplayData(DisplayData.Builder builder) { */ private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark { - private transient MqttClient client; private Instant oldestMessageTimestamp = Instant.now(); - private static class MessageIdWithQos { - - private int id; - private int qos; - - MessageIdWithQos(int id, int qos) { - this.id = id; - this.qos = qos; - } - - } - - private final List messages = new ArrayList<>(); + private final List messages = new ArrayList<>(); public MqttCheckpointMark() { } - public void setClient(MqttClient client) { - this.client = client; - } - - public void add(int id, int qos, Instant timestamp) { + public void add(Message message, Instant timestamp) { if (timestamp.isBefore(oldestMessageTimestamp)) { oldestMessageTimestamp = timestamp; } - MessageIdWithQos message = new MessageIdWithQos(id, qos); messages.add(message); } @Override public void finalizeCheckpoint() { - for (MessageIdWithQos message : messages) { + for (Message message : messages) { try { - client.messageArrivedComplete(message.id, message.qos); + message.ack(); } catch (Exception e) { - LOGGER.warn("Can't ack message {} with QoS {}", message.id, message.qos); + LOGGER.warn("Can't ack message", e); } } oldestMessageTimestamp = Instant.now(); @@ -331,7 +350,7 @@ public UnboundedReader createReader(PipelineOptions options, public List generateInitialSplits(int desiredNumSplits, PipelineOptions options) { // MQTT is based on a pub/sub pattern - // so, if we create several subscribers on the same topic, they all will receive the same + // so, if we create several subscribers on the same getTopic, they all will receive the same // message, resulting to duplicate messages in the PCollection. // So, for MQTT, we limit to number of split ot 1 (unique source). return Collections.singletonList(new UnboundedMqttSource(spec)); @@ -358,37 +377,15 @@ public Coder getDefaultOutputCoder() { } } - /** - * POJO used to store MQTT message and its timestamp. - */ - private static class MqttMessageWithTimestamp { - - private final MqttMessage message; - private final Instant timestamp; - - public MqttMessageWithTimestamp(MqttMessage message, Instant timestamp) { - this.message = message; - this.timestamp = timestamp; - } - - public MqttMessage getMessage() { - return message; - } - - public Instant getTimestamp() { - return timestamp; - } - } - private static class UnboundedMqttReader extends UnboundedSource.UnboundedReader { private final UnboundedMqttSource source; - private MqttClient client; + private MQTT client; + private BlockingConnection connection; private byte[] current; private Instant currentTimestamp; private MqttCheckpointMark checkpointMark; - private BlockingQueue queue; public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkpointMark) { this.source = source; @@ -398,7 +395,6 @@ public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkp } else { this.checkpointMark = new MqttCheckpointMark(); } - this.queue = new LinkedBlockingQueue<>(); } @Override @@ -407,55 +403,26 @@ public boolean start() throws IOException { Read spec = source.spec; try { client = spec.connectionConfiguration().getClient(); - client.subscribe(spec.connectionConfiguration().topic()); - client.setManualAcks(true); - client.setCallback(new MqttCallback() { - @Override - public void connectionLost(Throwable cause) { - LOGGER.warn("MQTT connection lost", cause); - cause.printStackTrace(); - } - - @Override - public void messageArrived(String topic, MqttMessage message) throws Exception { - LOGGER.trace("Message arrived"); - queue.put(new MqttMessageWithTimestamp(message, Instant.now())); - } - - @Override - public void deliveryComplete(IMqttDeliveryToken token) { - // nothing to do - } - }); - checkpointMark.setClient(client); + connection = client.blockingConnection(); + connection.connect(); + connection.subscribe(new Topic[]{ + new Topic(spec.connectionConfiguration().getTopic(), QoS.AT_LEAST_ONCE)}); return advance(); - } catch (MqttException e) { + } catch (Exception e) { throw new IOException(e); } } @Override public boolean advance() throws IOException { - LOGGER.debug("Taking from the pending queue ({})", queue.size()); - MqttMessageWithTimestamp message = null; try { - message = queue.poll(5, TimeUnit.SECONDS); - } catch (InterruptedException e) { + Message message = connection.receive(); + current = message.getPayload(); + currentTimestamp = Instant.now(); + checkpointMark.add(message, currentTimestamp); + } catch (Exception e) { throw new IOException(e); } - if (message == null) { - current = null; - currentTimestamp = null; - return false; - } - - checkpointMark.add(message.getMessage().getId(), - message.getMessage().getQos(), - message.timestamp); - - current = message.message.getPayload(); - currentTimestamp = message.timestamp; - return true; } @@ -463,15 +430,11 @@ public boolean advance() throws IOException { public void close() throws IOException { LOGGER.debug("Closing MQTT reader"); try { - if (client != null) { - try { - client.disconnect(); - } finally { - client.close(); - } + if (connection != null) { + connection.disconnect(); } - } catch (MqttException mqttException) { - throw new IOException(mqttException); + } catch (Exception e) { + throw new IOException(e); } } @@ -571,7 +534,8 @@ private static class WriteFn extends DoFn { private final Write spec; - private transient MqttClient client; + private transient MQTT client; + private transient BlockingConnection connection; public WriteFn(Write spec) { this.spec = spec; @@ -579,27 +543,24 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { - client = spec.connectionConfiguration().getClient(true); + client = new MQTT(); + client.setHost(spec.connectionConfiguration().getServerUri()); + // client ID is auto generated + connection = client.blockingConnection(); + connection.connect(); } @ProcessElement public void processElement(ProcessContext context) throws Exception { byte[] payload = context.element(); - MqttMessage message = new MqttMessage(); - message.setQos(1); - message.setRetained(spec.retained()); - message.setPayload(payload); - client.publish(spec.connectionConfiguration().topic(), message); + connection.publish(spec.connectionConfiguration().getTopic(), payload, QoS.AT_LEAST_ONCE, + false); } @Teardown public void closeMqttClient() throws Exception { - if (client != null) { - try { - client.disconnect(); - } finally { - client.close(); - } + if (connection != null) { + connection.disconnect(); } } 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 71c162166ed6..166dfa6cff0f 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 @@ -18,30 +18,29 @@ package org.apache.beam.sdk.io.mqtt; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; -import java.io.File; -import java.io.Serializable; import java.net.ServerSocket; -import java.net.URI; import java.util.ArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; +import java.util.HashSet; +import java.util.Set; import org.apache.activemq.broker.BrokerService; import org.apache.activemq.broker.Connection; -import org.apache.activemq.broker.TransportConnector; -import org.apache.activemq.store.kahadb.KahaDBStore; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; -import org.eclipse.paho.client.mqttv3.MqttCallback; -import org.eclipse.paho.client.mqttv3.MqttClient; -import org.eclipse.paho.client.mqttv3.MqttMessage; -import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence; +import org.fusesource.hawtbuf.Buffer; +import org.fusesource.mqtt.client.BlockingConnection; +import org.fusesource.mqtt.client.Future; +import org.fusesource.mqtt.client.FutureConnection; +import org.fusesource.mqtt.client.MQTT; +import org.fusesource.mqtt.client.Message; +import org.fusesource.mqtt.client.QoS; +import org.fusesource.mqtt.client.Topic; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -55,7 +54,7 @@ * Tests of {@link MqttIO}. */ @RunWith(JUnit4.class) -public class MqttIOTest implements Serializable { +public class MqttIOTest { private static final Logger LOGGER = LoggerFactory.getLogger(MqttIOTest.class); @@ -73,26 +72,13 @@ public void startBroker() throws Exception { LOGGER.info("Starting ActiveMQ brokerService on {}", port); brokerService = new BrokerService(); brokerService.setDeleteAllMessagesOnStartup(true); - brokerService.setPersistent(true); - brokerService.setDataDirectory("target/activemq-data/"); - KahaDBStore kahaDBStore = new KahaDBStore(); - kahaDBStore.setDirectory(new File("target/activemq-data/")); - brokerService.setPersistenceAdapter(kahaDBStore); - brokerService.setAdvisorySupport(false); - brokerService.setUseJmx(true); - brokerService.getManagementContext().setCreateConnector(false); - brokerService.setSchedulerSupport(true); - brokerService.setPopulateJMSXUserID(true); - TransportConnector mqttConnector = new TransportConnector(); - mqttConnector.setName("mqtt"); - mqttConnector.setUri(new URI("mqtt://localhost:" + port)); - mqttConnector.setAllowLinkStealing(true); - brokerService.addConnector(mqttConnector); + brokerService.setPersistent(false); + brokerService.addConnector("mqtt://localhost:" + port); brokerService.start(); brokerService.waitUntilStarted(); } - @Test(timeout = 120 * 1000) + @Test(timeout = 60 * 1000) @Category(NeedsRunner.class) public void testRead() throws Exception { final Pipeline pipeline = TestPipeline.create(); @@ -120,87 +106,79 @@ public void testRead() throws Exception { // produce messages on the brokerService in another thread // This thread prevents to block the pipeline waiting for new messages - Thread thread = new Thread() { + MQTT client = new MQTT(); + client.setHost("tcp://localhost:" + port); + final BlockingConnection publishConnection = client.blockingConnection(); + publishConnection.connect(); + Thread publisherThread = new Thread() { public void run() { try { - MqttClient client = new MqttClient("tcp://localhost:" + port, - MqttClient.generateClientId(), new MqttDefaultFilePersistence("target/paho/")); - client.connect(); LOGGER.info("Waiting pipeline connected to the MQTT broker before sending " + "messages ..."); boolean pipelineConnected = false; while (!pipelineConnected) { - Thread.sleep(50); + Thread.sleep(1000); for (Connection connection : brokerService.getBroker().getClients()) { - if (connection.getConnectionId().equals("READ_PIPELINE")) { + if (connection.getConnectionId().startsWith("READ_PIPELINE")) { pipelineConnected = true; } } } for (int i = 0; i < 10; i++) { - MqttMessage message = new MqttMessage(); - message.setQos(0); - message.setPayload(("This is test " + i).getBytes()); - client.publish("READ_TOPIC", message); + publishConnection.publish("READ_TOPIC", ("This is test " + i).getBytes(), + QoS.AT_LEAST_ONCE, false); } - client.disconnect(); - client.close(); } catch (Exception e) { // nothing to do } } }; - thread.start(); + publisherThread.start(); pipeline.run(); - thread.join(); + + publishConnection.disconnect(); + publisherThread.join(); } @Test - @Category(NeedsRunner.class) public void testWrite() throws Exception { - final CountDownLatch latch = new CountDownLatch(200); - - MqttClient client = new MqttClient("tcp://localhost:" + port, MqttClient.generateClientId(), - new MqttDefaultFilePersistence("target/paho/")); - client.connect(); - client.subscribe("WRITE_TOPIC"); - client.setCallback(new MqttCallback() { - @Override - public void connectionLost(Throwable throwable) { - LOGGER.warn("Connection Lost", throwable); - throwable.printStackTrace(); - } - - @Override - public void messageArrived(String s, MqttMessage mqttMessage) throws Exception { - latch.countDown(); - } - - @Override - public void deliveryComplete(IMqttDeliveryToken iMqttDeliveryToken) { - // nothing to do - } - }); + MQTT client = new MQTT(); + client.setHost("tcp://localhost:" + port); + FutureConnection connection = client.futureConnection(); + Future f1 = connection.connect(); + Future receive = connection.receive(); + connection.subscribe(new Topic[]{new Topic(Buffer.utf8("WRITE_TOPIC"), QoS.EXACTLY_ONCE)}); Pipeline pipeline = TestPipeline.create(); ArrayList data = new ArrayList<>(); for (int i = 0; i < 200; i++) { - data.add("Test".getBytes()); + data.add(("Test " + i).getBytes()); } pipeline.apply(Create.of(data)) .apply(MqttIO.write() .withConnectionConfiguration( MqttIO.ConnectionConfiguration.create( "tcp://localhost:" + port, - "WRITE_TOPIC", "TEST"))); + "WRITE_TOPIC"))); pipeline.run(); - latch.await(30, TimeUnit.SECONDS); - assertEquals(0, latch.getCount()); + Set messages = new HashSet<>(); - client.disconnect(); - client.close(); + for (int i = 0; i < 200; i++) { + Message message = receive.await(); + messages.add(new String(message.getPayload())); + message.ack(); + receive = connection.receive(); + } + + Future f4 = connection.disconnect(); + f4.await(); + + assertEquals(200, messages.size()); + for (int i = 0; i < 200; i++) { + assertTrue(messages.contains("Test " + i)); + } } @After From 8d63a79388dce00da680b6db209f151a0c54b594 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sun, 18 Dec 2016 10:23:45 +0100 Subject: [PATCH 21/27] [BEAM-606] Use spec to create MQTT client in write --- .../src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 407f31c92439..e92e93b8ed72 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 @@ -543,9 +543,7 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { - client = new MQTT(); - client.setHost(spec.connectionConfiguration().getServerUri()); - // client ID is auto generated + client = spec.connectionConfiguration().getClient(); connection = client.blockingConnection(); connection.connect(); } From bef017615f5649b58731344e900004d8867b4191 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 19 Dec 2016 17:26:21 +0100 Subject: [PATCH 22/27] [BEAM-606] Rename LOGGER to LOG and add debug logging messages --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) 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 e92e93b8ed72..e18b1425e97d 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 @@ -100,7 +100,7 @@ */ public class MqttIO { - private static final Logger LOGGER = LoggerFactory.getLogger(MqttIO.class); + private static final Logger LOG = LoggerFactory.getLogger(MqttIO.class); public static Read read() { return new AutoValue_MqttIO_Read.Builder() @@ -196,14 +196,18 @@ private void populateDisplayData(DisplayData.Builder builder) { } private MQTT getClient() throws Exception { + LOG.debug("Creating MQTT client to {}", getServerUri()); MQTT client = new MQTT(); client.setHost(getServerUri()); if (getUsername() != null) { + LOG.debug("MQTT client uses username {}", getUsername()); client.setUserName(getUsername()); client.setPassword(getPassword()); } if (getClientId() != null) { - client.setClientId(getClientId() + "-" + System.currentTimeMillis()); + String clientId = getClientId() + "-" + System.currentTimeMillis(); + LOG.debug("MQTT client id set to {}", clientId); + client.setClientId(clientId); } return client; } @@ -271,9 +275,13 @@ public PCollection expand(PBegin input) { PTransform> transform = unbounded; if (maxNumRecords() != Long.MAX_VALUE) { + LOG.debug("Bound MqttIO.Read to {} records", maxNumRecords()); transform = unbounded.withMaxNumRecords(maxNumRecords()); } else if (maxReadTime() != null) { + LOG.debug("Bound MqttIO.Read to {} max read time", maxReadTime()); transform = unbounded.withMaxReadTime(maxReadTime()); + } else { + LOG.debug("Unbounded MqttIO.Read ..."); } return input.getPipeline().apply(transform); @@ -318,11 +326,12 @@ public void add(Message message, Instant timestamp) { @Override public void finalizeCheckpoint() { + LOG.debug("Finalizing checkpoint acknowledging pending messages"); for (Message message : messages) { try { message.ack(); } catch (Exception e) { - LOGGER.warn("Can't ack message", e); + LOG.warn("Can't ack message", e); } } oldestMessageTimestamp = Instant.now(); @@ -399,7 +408,7 @@ public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkp @Override public boolean start() throws IOException { - LOGGER.debug("Starting MQTT reader"); + LOG.debug("Starting MQTT reader"); Read spec = source.spec; try { client = spec.connectionConfiguration().getClient(); @@ -416,6 +425,7 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { try { + LOG.debug("MQTT reading waiting message ..."); Message message = connection.receive(); current = message.getPayload(); currentTimestamp = Instant.now(); @@ -428,7 +438,7 @@ public boolean advance() throws IOException { @Override public void close() throws IOException { - LOGGER.debug("Closing MQTT reader"); + LOG.debug("Closing MQTT reader"); try { if (connection != null) { connection.disconnect(); @@ -544,6 +554,7 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { client = spec.connectionConfiguration().getClient(); + LOG.debug("Start MQTT connection"); connection = client.blockingConnection(); connection.connect(); } @@ -551,6 +562,7 @@ public void createMqttClient() throws Exception { @ProcessElement public void processElement(ProcessContext context) throws Exception { byte[] payload = context.element(); + LOG.debug("Sending message {}", new String(payload)); connection.publish(spec.connectionConfiguration().getTopic(), payload, QoS.AT_LEAST_ONCE, false); } @@ -558,6 +570,7 @@ public void processElement(ProcessContext context) throws Exception { @Teardown public void closeMqttClient() throws Exception { if (connection != null) { + LOG.debug("Disconnecting MQTT connection"); connection.disconnect(); } } From a12c708f7b14af82157c3b4bbeda0a37dd9906cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 20 Dec 2016 06:13:31 +0100 Subject: [PATCH 23/27] [BEAM-606] Set messages transient in the MqttCheckpointMark --- .../java/org/apache/beam/sdk/io/mqtt/MqttIO.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) 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 e18b1425e97d..fa6bd6e40ccb 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 @@ -30,9 +30,9 @@ import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -308,11 +308,11 @@ public void populateDisplayData(DisplayData.Builder builder) { * Checkpoint for an unbounded MQTT source. Consists of the MQTT messages waiting to be * acknowledged and oldest pending message timestamp. */ - private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark { + private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { private Instant oldestMessageTimestamp = Instant.now(); - private final List messages = new ArrayList<>(); + private transient List messages = new ArrayList<>(); public MqttCheckpointMark() { } @@ -338,6 +338,12 @@ public void finalizeCheckpoint() { messages.clear(); } + // set an empty list to messages when deserialize + private void readObject(java.io.ObjectInputStream stream) + throws java.io.IOException, ClassNotFoundException { + messages = new ArrayList<>(); + } + } private static class UnboundedMqttSource @@ -377,7 +383,7 @@ public void populateDisplayData(DisplayData.Builder builder) { @Override public Coder getCheckpointMarkCoder() { - return AvroCoder.of(MqttCheckpointMark.class); + return SerializableCoder.of(MqttCheckpointMark.class); } @Override From 0116624b0dacf070d73737b8925c2646ceccccaa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 20 Dec 2016 06:31:27 +0100 Subject: [PATCH 24/27] [BEAM-606] Use UUID to generate unique MQTT client ID, use blocking connection in write test --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 32 ++++++------- .../apache/beam/sdk/io/mqtt/MqttIOTest.java | 48 +++++++++++-------- 2 files changed, 43 insertions(+), 37 deletions(-) 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 fa6bd6e40ccb..2cac39421243 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 @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; +import java.util.UUID; import javax.annotation.Nullable; @@ -195,7 +196,7 @@ private void populateDisplayData(DisplayData.Builder builder) { builder.addIfNotNull(DisplayData.item("username", getUsername())); } - private MQTT getClient() throws Exception { + private MQTT createClient() throws Exception { LOG.debug("Creating MQTT client to {}", getServerUri()); MQTT client = new MQTT(); client.setHost(getServerUri()); @@ -205,7 +206,7 @@ private MQTT getClient() throws Exception { client.setPassword(getPassword()); } if (getClientId() != null) { - String clientId = getClientId() + "-" + System.currentTimeMillis(); + String clientId = getClientId() + "-" + UUID.randomUUID().toString(); LOG.debug("MQTT client id set to {}", clientId); client.setClientId(clientId); } @@ -275,13 +276,9 @@ public PCollection expand(PBegin input) { PTransform> transform = unbounded; if (maxNumRecords() != Long.MAX_VALUE) { - LOG.debug("Bound MqttIO.Read to {} records", maxNumRecords()); transform = unbounded.withMaxNumRecords(maxNumRecords()); } else if (maxReadTime() != null) { - LOG.debug("Bound MqttIO.Read to {} max read time", maxReadTime()); transform = unbounded.withMaxReadTime(maxReadTime()); - } else { - LOG.debug("Unbounded MqttIO.Read ..."); } return input.getPipeline().apply(transform); @@ -310,8 +307,8 @@ public void populateDisplayData(DisplayData.Builder builder) { */ private static class MqttCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { + private String clientId; private Instant oldestMessageTimestamp = Instant.now(); - private transient List messages = new ArrayList<>(); public MqttCheckpointMark() { @@ -326,12 +323,12 @@ public void add(Message message, Instant timestamp) { @Override public void finalizeCheckpoint() { - LOG.debug("Finalizing checkpoint acknowledging pending messages"); + LOG.debug("Finalizing checkpoint acknowledging pending messages for client ID {}", clientId); for (Message message : messages) { try { message.ack(); } catch (Exception e) { - LOG.warn("Can't ack message", e); + LOG.warn("Can't ack message for client ID {}", clientId, e); } } oldestMessageTimestamp = Instant.now(); @@ -414,10 +411,12 @@ public UnboundedMqttReader(UnboundedMqttSource source, MqttCheckpointMark checkp @Override public boolean start() throws IOException { - LOG.debug("Starting MQTT reader"); + LOG.debug("Starting MQTT reader ..."); Read spec = source.spec; try { - client = spec.connectionConfiguration().getClient(); + client = spec.connectionConfiguration().createClient(); + LOG.debug("Reader client ID is {}", client.getClientId()); + checkpointMark.clientId = client.getClientId().toString(); connection = client.blockingConnection(); connection.connect(); connection.subscribe(new Topic[]{ @@ -431,7 +430,7 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { try { - LOG.debug("MQTT reading waiting message ..."); + LOG.debug("MQTT reader (client ID {}) waiting message ...", client.getClientId()); Message message = connection.receive(); current = message.getPayload(); currentTimestamp = Instant.now(); @@ -444,7 +443,7 @@ public boolean advance() throws IOException { @Override public void close() throws IOException { - LOG.debug("Closing MQTT reader"); + LOG.debug("Closing MQTT reader (client ID {})", client.getClientId()); try { if (connection != null) { connection.disconnect(); @@ -559,8 +558,9 @@ public WriteFn(Write spec) { @Setup public void createMqttClient() throws Exception { - client = spec.connectionConfiguration().getClient(); - LOG.debug("Start MQTT connection"); + LOG.debug("Starting MQTT writer"); + client = spec.connectionConfiguration().createClient(); + LOG.debug("MQTT writer client ID is {}", client.getClientId()); connection = client.blockingConnection(); connection.connect(); } @@ -576,7 +576,7 @@ public void processElement(ProcessContext context) throws Exception { @Teardown public void closeMqttClient() throws Exception { if (connection != null) { - LOG.debug("Disconnecting MQTT connection"); + LOG.debug("Disconnecting MQTT connection (client ID {})", client.getClientId()); connection.disconnect(); } } 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 166dfa6cff0f..8ba12139fd3d 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 @@ -35,8 +35,6 @@ import org.apache.beam.sdk.values.PCollection; import org.fusesource.hawtbuf.Buffer; import org.fusesource.mqtt.client.BlockingConnection; -import org.fusesource.mqtt.client.Future; -import org.fusesource.mqtt.client.FutureConnection; import org.fusesource.mqtt.client.MQTT; import org.fusesource.mqtt.client.Message; import org.fusesource.mqtt.client.QoS; @@ -56,7 +54,7 @@ @RunWith(JUnit4.class) public class MqttIOTest { - private static final Logger LOGGER = LoggerFactory.getLogger(MqttIOTest.class); + private static final Logger LOG = LoggerFactory.getLogger(MqttIOTest.class); private static transient BrokerService brokerService; @@ -64,14 +62,15 @@ public class MqttIOTest { @Before public void startBroker() throws Exception { - LOGGER.info("Finding free network port"); + LOG.info("Finding free network port"); ServerSocket socket = new ServerSocket(0); port = socket.getLocalPort(); socket.close(); - LOGGER.info("Starting ActiveMQ brokerService on {}", port); + LOG.info("Starting ActiveMQ brokerService on {}", port); brokerService = new BrokerService(); brokerService.setDeleteAllMessagesOnStartup(true); + // use memory persistence for the test: it's faster and don't pollute test folder with KahaDB brokerService.setPersistent(false); brokerService.addConnector("mqtt://localhost:" + port); brokerService.start(); @@ -113,7 +112,7 @@ public void testRead() throws Exception { Thread publisherThread = new Thread() { public void run() { try { - LOGGER.info("Waiting pipeline connected to the MQTT broker before sending " + LOG.info("Waiting pipeline connected to the MQTT broker before sending " + "messages ..."); boolean pipelineConnected = false; while (!pipelineConnected) { @@ -144,10 +143,26 @@ public void run() { public void testWrite() throws Exception { MQTT client = new MQTT(); client.setHost("tcp://localhost:" + port); - FutureConnection connection = client.futureConnection(); - Future f1 = connection.connect(); - Future receive = connection.receive(); - connection.subscribe(new Topic[]{new Topic(Buffer.utf8("WRITE_TOPIC"), QoS.EXACTLY_ONCE)}); + final BlockingConnection connection = client.blockingConnection(); + connection.connect(); + connection.subscribe(new Topic[]{new Topic(Buffer.utf8("WRITE_TOPIC"), QoS.AT_LEAST_ONCE)}); + + final Set messages = new HashSet<>(); + + Thread subscriber = new Thread() { + public void run() { + try { + for (int i = 0; i < 200; i++) { + Message message = connection.receive(); + messages.add(new String(message.getPayload())); + message.ack(); + } + } catch (Exception e) { + LOG.error("Can't receive message", e); + } + } + }; + subscriber.start(); Pipeline pipeline = TestPipeline.create(); @@ -162,18 +177,9 @@ public void testWrite() throws Exception { "tcp://localhost:" + port, "WRITE_TOPIC"))); pipeline.run(); + subscriber.join(); - Set messages = new HashSet<>(); - - for (int i = 0; i < 200; i++) { - Message message = receive.await(); - messages.add(new String(message.getPayload())); - message.ack(); - receive = connection.receive(); - } - - Future f4 = connection.disconnect(); - f4.await(); + connection.disconnect(); assertEquals(200, messages.size()); for (int i = 0; i < 200; i++) { From 14ae4eaa351946b103fb275b8c9c5b836d8e6084 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 20 Dec 2016 15:38:34 +0100 Subject: [PATCH 25/27] [BEAM-606] Tests cleanup --- .../test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 8ba12139fd3d..98fc8d6b7899 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 @@ -30,6 +30,7 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; @@ -43,15 +44,12 @@ import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Tests of {@link MqttIO}. */ -@RunWith(JUnit4.class) public class MqttIOTest { private static final Logger LOG = LoggerFactory.getLogger(MqttIOTest.class); @@ -78,7 +76,7 @@ public void startBroker() throws Exception { } @Test(timeout = 60 * 1000) - @Category(NeedsRunner.class) + @Category(RunnableOnService.class) public void testRead() throws Exception { final Pipeline pipeline = TestPipeline.create(); @@ -140,6 +138,7 @@ public void run() { } @Test + @Category(RunnableOnService.class) public void testWrite() throws Exception { MQTT client = new MQTT(); client.setHost("tcp://localhost:" + port); From 2554928f023c07425def20c51ebab0e59e459ccf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Wed, 21 Dec 2016 19:19:16 +0100 Subject: [PATCH 26/27] [BEAM-606] Fix checkstyle --- .../src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java | 1 - 1 file changed, 1 deletion(-) 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 98fc8d6b7899..be8fbc737a5e 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 @@ -28,7 +28,6 @@ import org.apache.activemq.broker.BrokerService; import org.apache.activemq.broker.Connection; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; From 2c5152428abc5ad6f715c7abd06ad5a9f341b665 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 23 Dec 2016 07:17:13 +0100 Subject: [PATCH 27/27] =?UTF-8?q?[BEAM-606]=C2=A0Fix=20preconditions=20mes?= =?UTF-8?q?sages?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/apache/beam/sdk/io/mqtt/MqttIO.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) 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 2cac39421243..221f9b1ee828 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 @@ -78,7 +78,7 @@ * *

    Writing to a MQTT broker

    * - *

    MqttIO sink supports writing {@code byte[]} to a getTopic on a MQTT broker. + *

    MqttIO sink supports writing {@code byte[]} to a topic on a MQTT broker. * *

    To configure a MQTT sink, as for the read, you have to specify a MQTT connection * configuration with {@code ServerURI}, {@code Topic}, ... @@ -151,11 +151,11 @@ abstract static class Builder { */ public static ConnectionConfiguration create(String serverUri, String topic) { checkArgument(serverUri != null, - "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " - + "getServerUri"); + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "serverUri"); checkArgument(topic != null, - "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " - + "getTopic"); + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "topic"); return new AutoValue_MqttIO_ConnectionConfiguration.Builder().setServerUri(serverUri) .setTopic(topic).build(); } @@ -170,13 +170,13 @@ public static ConnectionConfiguration create(String serverUri, String topic) { */ public static ConnectionConfiguration create(String serverUri, String topic, String clientId) { checkArgument(serverUri != null, - "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " - + "getServerUri"); + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "serverUri"); checkArgument(topic != null, - "MqttIO.ConnectionConfiguration.create(getServerUri, getTopic) called with null " - + "getTopic"); - checkArgument(clientId != null, "MqttIO.ConnectionConfiguration.create(getServerUri," - + "getTopic, getClientId) called with null getClientId"); + "MqttIO.ConnectionConfiguration.create(serverUri, topic) called with null " + + "topic"); + checkArgument(clientId != null, "MqttIO.ConnectionConfiguration.create(serverUri," + + "topic, clientId) called with null clientId"); return new AutoValue_MqttIO_ConnectionConfiguration.Builder().setServerUri(serverUri) .setTopic(topic).setClientId(clientId).build(); } @@ -362,7 +362,7 @@ public UnboundedReader createReader(PipelineOptions options, public List generateInitialSplits(int desiredNumSplits, PipelineOptions options) { // MQTT is based on a pub/sub pattern - // so, if we create several subscribers on the same getTopic, they all will receive the same + // so, if we create several subscribers on the same topic, they all will receive the same // message, resulting to duplicate messages in the PCollection. // So, for MQTT, we limit to number of split ot 1 (unique source). return Collections.singletonList(new UnboundedMqttSource(spec));