-
Notifications
You must be signed in to change notification settings - Fork 4.1k
STORM-2371 Implementing new eventhub driver #1951
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,10 +22,8 @@ | |
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import com.microsoft.eventhubs.client.EventHubClient; | ||
| import com.microsoft.azure.eventhubs.*; | ||
| 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; | ||
|
|
@@ -41,7 +39,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 +69,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.createFromConnectionString(boltConfig.getConnectionString()).get(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
use methods ending with sync(...) - which could return cleaner exception stacks |
||
| if (boltConfig.getPartitionMode()) | ||
| sender = ehClient.createPartitionSender(Integer.toString(context.getThisTaskIndex())).get(); | ||
| } catch (Exception ex) { | ||
| collector.reportError(ex); | ||
| throw new RuntimeException(ex); | ||
|
|
@@ -84,12 +82,39 @@ 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.send(sendEvent).get(); | ||
| else if(boltConfig.getPartitionMode() && sender==null) | ||
| throw new EventHubException("Sender is null"); | ||
| else if(!boltConfig.getPartitionMode() && ehClient!=null) | ||
| ehClient.send(sendEvent).get(); | ||
| else if(!boltConfig.getPartitionMode() && ehClient==null) | ||
| throw new EventHubException("ehclient is null"); | ||
| collector.ack(tuple); | ||
| } catch (EventHubException ex) { | ||
| collector.reportError(ex); | ||
| collector.fail(tuple); | ||
| } | ||
| catch (Exception e){ | ||
|
|
||
| } | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why catch-all ? |
||
| } | ||
|
|
||
| @Override | ||
| public void cleanup() { | ||
| try{ | ||
| if(sender!=null) { | ||
| sender.close().get(); | ||
| sender = null; | ||
| } | ||
| if(ehClient!=null){ | ||
| ehClient.close().get(); | ||
| ehClient = null; | ||
| } | ||
| }catch (Exception e){ | ||
| logger.error("Exception occured during close phase"+e.toString()); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,11 @@ | |
| *******************************************************************************/ | ||
| package org.apache.storm.eventhubs.spout; | ||
|
|
||
| import com.microsoft.azure.eventhubs.EventHubClient; | ||
| import com.microsoft.azure.eventhubs.PartitionReceiver; | ||
| import org.apache.qpid.amqp_1_0.client.Message; | ||
| import org.apache.qpid.amqp_1_0.type.Binary; | ||
| import org.apache.qpid.amqp_1_0.type.messaging.Data; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -30,7 +34,10 @@ | |
| import com.microsoft.eventhubs.client.IEventHubFilter; | ||
| import com.microsoft.eventhubs.client.ResilientEventHubReceiver; | ||
|
|
||
| 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.type.Section; | ||
|
|
@@ -39,24 +46,21 @@ | |
|
|
||
| public class EventHubReceiverImpl implements IEventHubReceiver { | ||
| private static final Logger logger = LoggerFactory.getLogger(EventHubReceiverImpl.class); | ||
| private static final Symbol OffsetKey = Symbol.valueOf(Constants.OffsetKey); | ||
| private static final Symbol SequenceNumberKey = Symbol.valueOf(Constants.SequenceNumberKey); | ||
|
|
||
| private final String connectionString; | ||
| private final String entityName; | ||
| private final String partitionId; | ||
| private final int defaultCredits; | ||
| private final String consumerGroupName; | ||
|
|
||
| private ResilientEventHubReceiver receiver; | ||
| private PartitionReceiver receiver; | ||
| private EventHubClient ehClient=null; | ||
| private ReducedMetric receiveApiLatencyMean; | ||
| private CountMetric receiveApiCallCount; | ||
| private CountMetric receiveMessageCount; | ||
|
|
||
| public EventHubReceiverImpl(EventHubSpoutConfig config, String partitionId) { | ||
| this.connectionString = config.getConnectionString(); | ||
| this.entityName = config.getEntityPath(); | ||
| this.defaultCredits = config.getReceiverCredits(); | ||
| this.partitionId = partitionId; | ||
| this.consumerGroupName = config.getConsumerGroupName(); | ||
| receiveApiLatencyMean = new ReducedMetric(new MeanReducer()); | ||
|
|
@@ -65,24 +69,38 @@ public EventHubReceiverImpl(EventHubSpoutConfig config, String partitionId) { | |
| } | ||
|
|
||
| @Override | ||
| public void open(IEventHubFilter filter) throws EventHubException { | ||
| logger.info("creating eventhub receiver: partitionId=" + partitionId + | ||
| ", filterString=" + filter.getFilterString()); | ||
| public void open(String offset) throws EventHubException { | ||
| logger.info("creating eventhub receiver: partitionId=" + partitionId + | ||
| ", offset=" + offset); | ||
| long start = System.currentTimeMillis(); | ||
| receiver = new ResilientEventHubReceiver(connectionString, entityName, | ||
| partitionId, consumerGroupName, defaultCredits, filter); | ||
| receiver.initialize(); | ||
|
|
||
| try { | ||
| ehClient = EventHubClient.createFromConnectionString(connectionString).get(); | ||
| receiver = ehClient.createEpochReceiver( | ||
| consumerGroupName, | ||
| partitionId, | ||
| offset, | ||
| false, | ||
| 1).get(); | ||
| }catch (Exception e){ | ||
| logger.info("Exception in creating EventhubClient"+e.toString()); | ||
| } | ||
| long end = System.currentTimeMillis(); | ||
| logger.info("created eventhub receiver, time taken(ms): " + (end-start)); | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| public void close(){ | ||
| if(receiver != null) { | ||
| receiver.close(); | ||
| try { | ||
| receiver.close().get(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
if receiver.close() fails - this results into connection leak; handle this case. something like: |
||
| if(ehClient!=null) | ||
| ehClient.close().get(); | ||
| }catch (Exception e){ | ||
| logger.error("Exception occured during close phase"+e.toString()); | ||
| } | ||
| logger.info("closed eventhub receiver: partitionId=" + partitionId ); | ||
| receiver = null; | ||
| ehClient = null; | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -92,50 +110,34 @@ public boolean isOpen() { | |
| } | ||
|
|
||
| @Override | ||
| public EventData receive(long timeoutInMilliseconds) { | ||
| public EventData receive() { | ||
| long start = System.currentTimeMillis(); | ||
| Message message = receiver.receive(timeoutInMilliseconds); | ||
| Iterable<com.microsoft.azure.eventhubs.EventData> receivedEvents=null; | ||
| /*Get one message at a time for backward compatibility behaviour*/ | ||
| try { | ||
| receivedEvents = receiver.receive(1).get(); | ||
| }catch (Exception e){ | ||
| logger.error("Exception occured during receive"+e.toString()); | ||
| } | ||
| long end = System.currentTimeMillis(); | ||
| long millis = (end - start); | ||
| receiveApiLatencyMean.update(millis); | ||
| receiveApiCallCount.incr(); | ||
|
|
||
| if (message == null) { | ||
| //Temporary workaround for AMQP/EH bug of failing to receive messages | ||
| /*if(timeoutInMilliseconds > 100 && millis < timeoutInMilliseconds/2) { | ||
| throw new RuntimeException( | ||
| "Restart EventHubSpout due to failure of receiving messages in " | ||
| + millis + " millisecond"); | ||
| }*/ | ||
| if (receivedEvents == null) { | ||
| return null; | ||
| } | ||
|
|
||
| receiveMessageCount.incr(); | ||
|
|
||
| MessageId messageId = createMessageId(message); | ||
| return EventData.create(message, messageId); | ||
| } | ||
|
|
||
| private MessageId createMessageId(Message message) { | ||
| String offset = null; | ||
| long sequenceNumber = 0; | ||
|
|
||
| for (Section section : message.getPayload()) { | ||
| if (section instanceof MessageAnnotations) { | ||
| MessageAnnotations annotations = (MessageAnnotations) section; | ||
| HashMap annonationMap = (HashMap) annotations.getValue(); | ||
|
|
||
| if (annonationMap.containsKey(OffsetKey)) { | ||
| offset = (String) annonationMap.get(OffsetKey); | ||
| } | ||
|
|
||
| if (annonationMap.containsKey(SequenceNumberKey)) { | ||
| sequenceNumber = (Long) annonationMap.get(SequenceNumberKey); | ||
| } | ||
| } | ||
| MessageId messageId=null; | ||
| Message message=null; | ||
| for (com.microsoft.azure.eventhubs.EventData receivedEvent : receivedEvents) { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Why ? In the receive(1) call above code - you explicitly specified receive 1 message - so remove this.. |
||
| messageId = new MessageId(partitionId, | ||
| receivedEvent.getSystemProperties().getOffset(), | ||
| receivedEvent.getSystemProperties().getSequenceNumber()); | ||
| List<Section> body = new ArrayList<Section>(); | ||
| body.add(new Data(new Binary((new String(receivedEvent.getBody(), Charset.defaultCharset())).getBytes()))); | ||
| message = new Message(body); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is not equivalent to the existing behavior. Message could have other amqp sections - ApplicationProperties and SystemProperties. |
||
| } | ||
|
|
||
| return MessageId.create(partitionId, offset, sequenceNumber); | ||
| return org.apache.storm.eventhubs.spout.EventData.create(message, messageId); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: keep them expanded - usual pattern in OS is to specifically import