-
Notifications
You must be signed in to change notification settings - Fork 4.1k
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
STORM-583: Initiali check-in for storm-eventhubs. #336
Closed
Closed
Changes from 1 commit
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
3dca6b8
Initiali check-in for storm-eventhubs.
shanyu c977bc9
EventHubSpout add configuration for max pending messages.
shanyu 36be267
EventHubSpout add configuration for enqueuetime based receiver
shanyu effeeb6
Fix pom.xml for storm-eventhubs project
shanyu b0d7661
EventHubSender should emit message in binary for interop with other l…
shanyu File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
storm-eventhubs | ||
===================== | ||
|
||
Storm spout and bolt implementation for Microsoft Azure Eventhubs | ||
|
||
### build ### | ||
mvn clean package | ||
|
||
### run sample topology ### | ||
To run the sample topology, you need to modify the config.properties file with | ||
the eventhubs configurations. Here is an example: | ||
|
||
eventhubspout.username = [username: policy name in EventHubs Portal] | ||
eventhubspout.password = [password: shared access key in EventHubs Portal] | ||
eventhubspout.namespace = [namespace] | ||
eventhubspout.entitypath = [entitypath] | ||
eventhubspout.partitions.count = [partitioncount] | ||
|
||
# if not provided, will use storm's zookeeper settings | ||
# zookeeper.connectionstring=zookeeper0:2181,zookeeper1:2181,zookeeper2:2181 | ||
|
||
eventhubspout.checkpoint.interval = 10 | ||
eventhub.receiver.credits = 1024 | ||
|
||
Then you can use storm.cmd to submit the sample topology: | ||
storm jar {jarfile} com.microsoft.eventhubs.samples.EventCount {topologyname} {spoutconffile} | ||
where the {jarfile} should be: eventhubs-storm-spout-{version}-jar-with-dependencies.jar | ||
|
||
### Run EventHubSendClient ### | ||
We have included a simple EventHubs send client for testing purpose. You can run the client like this: | ||
java -cp .\target\eventhubs-storm-spout-{version}-jar-with-dependencies.jar com.microsoft.eventhubs.client.EventHubSendClient | ||
[username] [password] [entityPath] [partitionId] [messageSize] [messageCount] | ||
If you want to send messages to all partitions, use "-1" as partitionId. | ||
|
||
### Windows Azure Eventhubs ### | ||
http://azure.microsoft.com/en-us/services/event-hubs/ | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<parent> | ||
<artifactId>storm</artifactId> | ||
<groupId>org.apache.storm</groupId> | ||
<version>0.10.0-SNAPSHOT</version> | ||
<relativePath>../../pom.xml</relativePath> | ||
</parent> | ||
|
||
<artifactId>storm-eventhubs</artifactId> | ||
<version>0.10.0-SNAPSHOT</version> | ||
<packaging>jar</packaging> | ||
<name>storm-eventhubs</name> | ||
<description>EventHubs Storm Spout</description> | ||
|
||
<properties> | ||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> | ||
<qpid.version>0.28</qpid.version> | ||
</properties> | ||
<build> | ||
<plugins> | ||
<plugin> | ||
<groupId>org.apache.maven.plugins</groupId> | ||
<artifactId>maven-assembly-plugin</artifactId> | ||
<version>2.4.1</version> | ||
<executions> | ||
<execution> | ||
<goals> | ||
<goal>attached</goal> | ||
</goals> | ||
<phase>package</phase> | ||
<configuration> | ||
<descriptorRefs> | ||
<descriptorRef>jar-with-dependencies</descriptorRef> | ||
</descriptorRefs> | ||
<archive> | ||
<manifest> | ||
<mainClass>org.apache.storm.eventhubs.samples.EventCount</mainClass> | ||
</manifest> | ||
</archive> | ||
</configuration> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
<plugin> | ||
<artifactId>maven-antrun-plugin</artifactId> | ||
<executions> | ||
<execution> | ||
<phase>package</phase> | ||
<configuration> | ||
<tasks> | ||
<copy file="src/main/resources/config.properties" tofile="target/eventhubs-config.properties"/> | ||
</tasks> | ||
</configuration> | ||
<goals> | ||
<goal>run</goal> | ||
</goals> | ||
</execution> | ||
</executions> | ||
</plugin> | ||
</plugins> | ||
</build> | ||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.qpid</groupId> | ||
<artifactId>qpid-client</artifactId> | ||
<version>${qpid.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.qpid</groupId> | ||
<artifactId>qpid-amqp-1-0-client-jms</artifactId> | ||
<version>${qpid.version}</version> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.storm</groupId> | ||
<artifactId>storm-core</artifactId> | ||
<version>${project.version}</version> | ||
<!-- keep storm out of the jar-with-dependencies --> | ||
<type>jar</type> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.curator</groupId> | ||
<artifactId>curator-framework</artifactId> | ||
<version>${curator.version}</version> | ||
<exclusions> | ||
<exclusion> | ||
<groupId>log4j</groupId> | ||
<artifactId>log4j</artifactId> | ||
</exclusion> | ||
<exclusion> | ||
<groupId>org.slf4j</groupId> | ||
<artifactId>slf4j-log4j12</artifactId> | ||
</exclusion> | ||
</exclusions> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>junit</groupId> | ||
<artifactId>junit</artifactId> | ||
<version>4.11</version> | ||
<scope>test</scope> | ||
</dependency> | ||
</dependencies> | ||
</project> |
81 changes: 81 additions & 0 deletions
81
external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/bolt/EventHubBolt.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
/******************************************************************************* | ||
* 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.bolt; | ||
|
||
import java.util.Map; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import org.apache.storm.eventhubs.client.EventHubClient; | ||
import org.apache.storm.eventhubs.client.EventHubException; | ||
import org.apache.storm.eventhubs.client.EventHubSender; | ||
|
||
import backtype.storm.task.TopologyContext; | ||
import backtype.storm.topology.BasicOutputCollector; | ||
import backtype.storm.topology.OutputFieldsDeclarer; | ||
import backtype.storm.topology.base.BaseBasicBolt; | ||
import backtype.storm.tuple.Tuple; | ||
|
||
/** | ||
* A bolt that writes message to EventHub. | ||
* We assume the incoming tuple has only one field which is a string. | ||
*/ | ||
public class EventHubBolt extends BaseBasicBolt { | ||
private static final long serialVersionUID = 1L; | ||
private static final Logger logger = LoggerFactory | ||
.getLogger(EventHubBolt.class); | ||
|
||
private EventHubSender sender; | ||
private String connectionString; | ||
private String entityPath; | ||
|
||
public EventHubBolt(String connectionString, String entityPath) { | ||
this.connectionString = connectionString; | ||
this.entityPath = entityPath; | ||
} | ||
|
||
@Override | ||
public void prepare(Map config, TopologyContext context) { | ||
try { | ||
EventHubClient eventHubClient = EventHubClient.create(connectionString, entityPath); | ||
sender = eventHubClient.createPartitionSender(null); | ||
} | ||
catch(Exception ex) { | ||
logger.error(ex.getMessage()); | ||
throw new RuntimeException(ex); | ||
} | ||
|
||
} | ||
|
||
@Override | ||
public void execute(Tuple tuple, BasicOutputCollector collector) { | ||
try { | ||
sender.send((String)tuple.getValue(0)); | ||
} | ||
catch(EventHubException ex) { | ||
logger.error(ex.getMessage()); | ||
} | ||
} | ||
|
||
@Override | ||
public void declareOutputFields(OutputFieldsDeclarer declarer) { | ||
|
||
} | ||
|
||
} |
116 changes: 116 additions & 0 deletions
116
...rm-eventhubs/src/main/java/org/apache/storm/eventhubs/client/ConnectionStringBuilder.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,116 @@ | ||
/******************************************************************************* | ||
* 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.client; | ||
|
||
import java.io.IOException; | ||
import java.net.MalformedURLException; | ||
import java.net.URL; | ||
import java.net.URLConnection; | ||
import java.net.URLDecoder; | ||
import java.net.URLStreamHandler; | ||
|
||
public class ConnectionStringBuilder { | ||
|
||
private final String connectionString; | ||
|
||
private String host; | ||
private int port; | ||
private String userName; | ||
private String password; | ||
private boolean ssl; | ||
|
||
// amqps://[username]:[password]@[namespace].servicebus.windows.net/ | ||
public ConnectionStringBuilder(String connectionString) throws EventHubException { | ||
this.connectionString = connectionString; | ||
this.initialize(); | ||
} | ||
|
||
public String getHost() { | ||
return this.host; | ||
} | ||
|
||
public void setHost(String value) { | ||
this.host = value; | ||
} | ||
|
||
public int getPort() { | ||
return this.port; | ||
} | ||
|
||
public void setPort(int value) { | ||
this.port = value; | ||
} | ||
|
||
public String getUserName() { | ||
return this.userName; | ||
} | ||
|
||
public void setUserName(String value) { | ||
this.userName = value; | ||
} | ||
|
||
public String getPassword() { | ||
return this.password; | ||
} | ||
|
||
public void setPassword(String value) { | ||
this.password = value; | ||
} | ||
|
||
public boolean getSsl() { | ||
return this.ssl; | ||
} | ||
|
||
public void setSsl(boolean value) { | ||
this.ssl = value; | ||
} | ||
|
||
private void initialize() throws EventHubException { | ||
|
||
URL url; | ||
try { | ||
url = new URL(null, this.connectionString, new NullURLStreamHandler()); | ||
} catch (MalformedURLException e) { | ||
throw new EventHubException("connectionString is not valid.", e); | ||
} | ||
|
||
String protocol = url.getProtocol(); | ||
this.ssl = protocol.equalsIgnoreCase(Constants.SslScheme); | ||
this.host = url.getHost(); | ||
this.port = url.getPort(); | ||
|
||
if (this.port == -1) { | ||
this.port = this.ssl ? Constants.DefaultSslPort : Constants.DefaultPort; | ||
} | ||
|
||
String userInfo = url.getUserInfo(); | ||
if (userInfo != null) { | ||
String[] credentials = userInfo.split(":", 2); | ||
this.userName = URLDecoder.decode(credentials[0]); | ||
this.password = URLDecoder.decode(credentials[1]); | ||
} | ||
} | ||
|
||
class NullURLStreamHandler extends URLStreamHandler { | ||
|
||
@Override | ||
protected URLConnection openConnection(URL u) throws IOException { | ||
throw new UnsupportedOperationException("Not supported yet."); | ||
} | ||
} | ||
} |
31 changes: 31 additions & 0 deletions
31
external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/client/Constants.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/******************************************************************************* | ||
* 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.client; | ||
|
||
public class Constants { | ||
|
||
public static final String DefaultStartingOffset = "-1"; | ||
public static final String SelectorFilterName = "apache.org:selector-filter:string"; | ||
public static final String OffsetFilterFormatString = "amqp.annotation.x-opt-offset > '%s'"; | ||
public static final String ConsumerAddressFormatString = "%s/ConsumerGroups/%s/Partitions/%s"; | ||
public static final String DestinationAddressFormatString = "%s/Partitions/%s"; | ||
|
||
public static final String SslScheme = "amqps"; | ||
public static final int DefaultPort = 5672; | ||
public static final int DefaultSslPort = 5671; | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Need apache license header.
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.
Thanks @ptgoetz !This is fixed.