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
10 changes: 10 additions & 0 deletions external/storm-eventhubs/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -94,6 +94,16 @@
<artifactId>eventhubs-client</artifactId>
<version>${eventhubs.client.version}</version>
</dependency>
<dependency>
<groupId>com.microsoft.azure</groupId>
<artifactId>azure-eventhubs</artifactId>
<version>0.11.0</version>
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

make it a property

</dependency>
<dependency>
<groupId>com.microsoft.azure</groupId>
<artifactId>azure-eventhubs</artifactId>
<version>${azure-eventhubs.version}</version>
</dependency>
<dependency>
<groupId>org.apache.storm</groupId>
<artifactId>storm-core</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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) {
Expand Down Expand Up @@ -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);
Expand All @@ -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;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,15 @@
*******************************************************************************/
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 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.
*
Expand All @@ -38,22 +36,10 @@
public class BinaryEventDataScheme implements IEventDataScheme {

@Override
public List<Object> deserialize(Message message) {
public List<Object> deserialize(EventData eventData) {
final List<Object> fieldContents = new ArrayList<Object>();

Map metaDataMap = new HashMap();
byte[] messageData = new byte[0];

for (Section section : message.getPayload()) {
if (section instanceof Data) {
Data data = (Data) section;
messageData = data.getValue().getArray();
} else if (section instanceof ApplicationProperties) {
final ApplicationProperties applicationProperties = (ApplicationProperties) section;
metaDataMap = applicationProperties.getValue();
}
}

byte[] messageData = eventData.getBody();
Map metaDataMap = eventData.getProperties();
fieldContents.add(messageData);
fieldContents.add(metaDataMap);
return fieldContents;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,18 +17,14 @@
*******************************************************************************/
package org.apache.storm.eventhubs.spout;

import com.microsoft.azure.eventhubs.EventData;
import org.apache.storm.tuple.Fields;

import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

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.AmqpValue;
import org.apache.qpid.amqp_1_0.type.messaging.ApplicationProperties;
import org.apache.qpid.amqp_1_0.type.messaging.Data;

/**
* An Event Data Scheme which deserializes message payload into the Strings. No
* encoding is assumed. The receiver will need to handle parsing of the string
Expand All @@ -44,27 +40,13 @@
public class EventDataScheme implements IEventDataScheme {

private static final long serialVersionUID = 1L;

@Override
public List<Object> deserialize(Message message) {
public List<Object> deserialize(EventData eventData) {
final List<Object> fieldContents = new ArrayList<Object>();

Map metaDataMap = new HashMap();
String messageData = "";

for (Section section : message.getPayload()) {
if (section instanceof Data) {
Data data = (Data) section;
messageData = new String(data.getValue().getArray());
} else if (section instanceof AmqpValue) {
AmqpValue amqpValue = (AmqpValue) section;
messageData = amqpValue.getValue().toString();
} else if (section instanceof ApplicationProperties) {
final ApplicationProperties applicationProperties = (ApplicationProperties) section;
metaDataMap = applicationProperties.getValue();
}
}

if(eventData.getBody()!=null)
messageData = new String (eventData.getBody(),eventData.getBodyOffset(),eventData.getBodyLength(),Charset.defaultCharset());
Map metaDataMap = eventData.getProperties();
fieldContents.add(messageData);
fieldContents.add(metaDataMap);
return fieldContents;
Expand Down
96 changes: 48 additions & 48 deletions ...ache/storm/eventhubs/spout/EventData.java → .../storm/eventhubs/spout/EventDataWrap.java
100755 → 100644
Original file line number Diff line number Diff line change
@@ -1,48 +1,48 @@
/*******************************************************************************
* 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.storm.eventhubs.spout;
import org.apache.qpid.amqp_1_0.client.Message;
public class EventData implements Comparable<EventData> {
private final Message message;
private final MessageId messageId;
public EventData(Message message, MessageId messageId) {
this.message = message;
this.messageId = messageId;
}
public static EventData create(Message message, MessageId messageId) {
return new EventData(message, messageId);
}
public Message getMessage() {
return this.message;
}
public MessageId getMessageId() {
return this.messageId;
}
@Override
public int compareTo(EventData ed) {
return messageId.getSequenceNumber().
compareTo(ed.getMessageId().getSequenceNumber());
}
}
/*******************************************************************************
* 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.storm.eventhubs.spout;

import com.microsoft.azure.eventhubs.EventData;

public class EventDataWrap implements Comparable<EventDataWrap> {
private final EventData eventData;
private final MessageId messageId;

public EventDataWrap(EventData eventdata, MessageId messageId) {
this.eventData = eventdata;
this.messageId = messageId;
}

public static EventDataWrap create(EventData eventData, MessageId messageId) {
return new EventDataWrap(eventData, messageId);
}

public EventData getEventData() {
return this.eventData;
}

public MessageId getMessageId() {
return this.messageId;
}

@Override
public int compareTo(EventDataWrap ed) {
return messageId.getSequenceNumber().
compareTo(ed.getMessageId().getSequenceNumber());
}
}
Loading