diff --git a/external/storm-eventhubs/pom.xml b/external/storm-eventhubs/pom.xml
index 21738a554cd..e44050dd071 100755
--- a/external/storm-eventhubs/pom.xml
+++ b/external/storm-eventhubs/pom.xml
@@ -94,6 +94,11 @@
eventhubs-client
${eventhubs.client.version}
+
+ com.microsoft.azure
+ azure-eventhubs
+ ${azure-eventhubs.version}
+
org.apache.storm
storm-core
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
index 3d64cc5624e..79fb52f4bbd 100755
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
+++ b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
@@ -17,20 +17,21 @@
*******************************************************************************/
package org.apache.storm.eventhubs.bolt;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.microsoft.eventhubs.client.EventHubClient;
+import com.microsoft.azure.eventhubs.EventData;
+import com.microsoft.azure.eventhubs.EventHubClient;
+import com.microsoft.azure.eventhubs.PartitionSender;
+import com.microsoft.azure.servicebus.ServiceBusException;
import com.microsoft.eventhubs.client.EventHubException;
-import com.microsoft.eventhubs.client.EventHubSender;
-
import org.apache.storm.task.OutputCollector;
import org.apache.storm.task.TopologyContext;
import org.apache.storm.topology.OutputFieldsDeclarer;
import org.apache.storm.topology.base.BaseRichBolt;
import org.apache.storm.tuple.Tuple;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
/**
* A bolt that writes event message to EventHub.
@@ -41,7 +42,8 @@ public class EventHubBolt extends BaseRichBolt {
.getLogger(EventHubBolt.class);
protected OutputCollector collector;
- protected EventHubSender sender;
+ protected PartitionSender sender=null;
+ protected EventHubClient ehClient=null;
protected EventHubBoltConfig boltConfig;
public EventHubBolt(String connectionString, String entityPath) {
@@ -70,10 +72,9 @@ public void prepare(Map config, TopologyContext context,
logger.info("creating sender: " + boltConfig.getConnectionString()
+ ", " + boltConfig.getEntityPath() + ", " + myPartitionId);
try {
- EventHubClient eventHubClient = EventHubClient.create(
- boltConfig.getConnectionString(),
- boltConfig.getEntityPath());
- sender = eventHubClient.createPartitionSender(myPartitionId);
+ ehClient = EventHubClient.createFromConnectionStringSync(boltConfig.getConnectionString());
+ if (boltConfig.getPartitionMode())
+ sender = ehClient.createPartitionSenderSync(Integer.toString(context.getThisTaskIndex()));
} catch (Exception ex) {
collector.reportError(ex);
throw new RuntimeException(ex);
@@ -84,11 +85,47 @@ public void prepare(Map config, TopologyContext context,
@Override
public void execute(Tuple tuple) {
try {
- sender.send(boltConfig.getEventDataFormat().serialize(tuple));
+ EventData sendEvent = new EventData(boltConfig.getEventDataFormat().serialize(tuple));
+ if(boltConfig.getPartitionMode() && sender!=null)
+ sender.sendSync(sendEvent);
+ else if(boltConfig.getPartitionMode() && sender==null)
+ throw new EventHubException("Sender is null");
+ else if(!boltConfig.getPartitionMode() && ehClient!=null)
+ ehClient.sendSync(sendEvent);
+ else if(!boltConfig.getPartitionMode() && ehClient==null)
+ throw new EventHubException("ehclient is null");
collector.ack(tuple);
- } catch (EventHubException ex) {
+ } catch (EventHubException ex ) {
collector.reportError(ex);
collector.fail(tuple);
+ }catch (ServiceBusException e){
+ collector.reportError(e);
+ collector.fail(tuple);
+ }
+ }
+
+ @Override
+ public void cleanup() {
+ if(sender != null) {
+ try {
+ sender.close().whenComplete((voidargs,error)->{
+ try{
+ if(error!=null){
+ logger.error("Exception during sender cleanup phase"+error.toString());
+ }
+ ehClient.closeSync();
+ }catch (Exception e){
+ logger.error("Exception during ehclient cleanup phase"+e.toString());
+ }
+ }).get();
+ }catch (InterruptedException e){
+ logger.error("Exception occured during cleanup phase"+e.toString());
+ }catch (ExecutionException e){
+ logger.error("Exception occured during cleanup phase"+e.toString());
+ }
+ logger.info("Eventhub Bolt cleaned up");
+ sender = null;
+ ehClient = null;
}
}
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBoltConfig.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBoltConfig.java
index 10b4e393e7c..6177c2e5341 100644
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBoltConfig.java
+++ b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBoltConfig.java
@@ -17,10 +17,10 @@
*******************************************************************************/
package org.apache.storm.eventhubs.bolt;
-import java.io.Serializable;
-
+import com.microsoft.azure.servicebus.ConnectionStringBuilder;
import org.apache.storm.eventhubs.spout.EventHubSpoutConfig;
-import com.microsoft.eventhubs.client.ConnectionStringBuilder;
+
+import java.io.Serializable;
/*
* EventHubs bolt configurations
@@ -81,8 +81,8 @@ public EventHubBoltConfig(String userName, String password, String namespace,
public EventHubBoltConfig(String userName, String password, String namespace,
String targetFqnAddress, String entityPath, boolean partitionMode,
IEventDataFormat dataFormat) {
- this.connectionString = new ConnectionStringBuilder(userName, password,
- namespace, targetFqnAddress).getConnectionString();
+ this.connectionString = new ConnectionStringBuilder(namespace,entityPath,
+ userName,password).toString();
this.entityPath = entityPath;
this.partitionMode = partitionMode;
this.dataFormat = dataFormat;
diff --git a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
index 7b0d7e54845..ef74f42c270 100644
--- a/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
+++ b/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/BinaryEventDataScheme.java
@@ -17,17 +17,17 @@
*******************************************************************************/
package org.apache.storm.eventhubs.spout;
-import org.apache.qpid.amqp_1_0.client.Message;
-import org.apache.qpid.amqp_1_0.type.Section;
-import org.apache.qpid.amqp_1_0.type.messaging.ApplicationProperties;
-import org.apache.qpid.amqp_1_0.type.messaging.Data;
+import com.microsoft.azure.eventhubs.EventData;
import org.apache.storm.tuple.Fields;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.IOException;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
+
/**
* An Event Data Scheme which deserializes message payload into the raw bytes.
*
@@ -37,23 +37,21 @@
*/
public class BinaryEventDataScheme implements IEventDataScheme {
+ private static final Logger logger = LoggerFactory.getLogger(BinaryEventDataScheme.class);
@Override
- public List