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
Eventhub2 #1998
Eventhub2 #1998
Changes from all commits
098272d
2a603b3
54cd6ff
cd733e1
e85e37d
4b034fe
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 |
---|---|---|
|
@@ -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) | ||
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. can you please add braces around if conditions. This goes to all of the code. |
||
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)->{ | ||
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. FYI, 1.x-branch still is on JDK 1.7 make sure you open another PR for 1.x-branch with JDK7 changes. |
||
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; | ||
} | ||
} | ||
|
||
|
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()); | ||
} | ||
} |
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.
Don't need to assign null