Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -209,6 +209,10 @@ private MQTT createClient() throws Exception {
String clientId = getClientId() + "-" + UUID.randomUUID().toString();
LOG.debug("MQTT client id set to {}", clientId);
client.setClientId(clientId);
} else {
String clientId = UUID.randomUUID().toString();
LOG.debug("MQTT client id set to random value {}", clientId);
client.setClientId(clientId);
}
return client;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@

import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.Connection;
import org.apache.beam.sdk.io.mqtt.MqttIO.Read;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.RunnableOnService;
import org.apache.beam.sdk.testing.TestPipeline;
Expand Down Expand Up @@ -77,6 +78,66 @@ public void startBroker() throws Exception {
brokerService.waitUntilStarted();
}

@Test(timeout = 60 * 1000)
@Category(RunnableOnService.class)
public void testReadNoClientId() throws Exception {
final String topicName = "READ_TOPIC_NO_CLIENT_ID";
Read mqttReader = MqttIO.read()
.withConnectionConfiguration(
MqttIO.ConnectionConfiguration.create(
"tcp://localhost:" + port,
topicName))
.withMaxNumRecords(10);
PCollection<byte[]> output = pipeline.apply(mqttReader);
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
MQTT client = new MQTT();
client.setHost("tcp://localhost:" + port);
final BlockingConnection publishConnection = client.blockingConnection();
publishConnection.connect();
Thread publisherThread = new Thread() {
public void run() {
try {
LOG.info("Waiting pipeline connected to the MQTT broker before sending "
+ "messages ...");
boolean pipelineConnected = false;
while (!pipelineConnected) {
Thread.sleep(1000);
for (Connection connection : brokerService.getBroker().getClients()) {
if (!connection.getConnectionId().isEmpty()) {
pipelineConnected = true;
}
}
}
for (int i = 0; i < 10; i++) {
publishConnection.publish(topicName, ("This is test " + i).getBytes(),
QoS.AT_LEAST_ONCE, false);
}
} catch (Exception e) {
// nothing to do
}
}
};
publisherThread.start();
pipeline.run();

publishConnection.disconnect();
publisherThread.join();
}

@Test(timeout = 60 * 1000)
@Category(RunnableOnService.class)
public void testRead() throws Exception {
Expand Down