| <!DOCTYPE html> |
| <!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]--> |
| <!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]--> |
| <!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| <meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1"> |
| <meta name="viewport" content="width=device-width,initial-scale=1,maximum-scale=1"/> |
| <title>Storm Compatibility - Gearpump 0.8.0 Documentation</title> |
| |
| |
| |
| |
| <link rel="stylesheet" href="css/bootstrap-3.3.5.min.css"> |
| <style> |
| body { |
| padding-top: 60px; |
| padding-bottom: 40px; |
| } |
| </style> |
| <link rel="stylesheet" href="css/main.css"> |
| <link rel="stylesheet" href="css/pygments-default.css"> |
| <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script> |
| </head> |
| <body> |
| <!--[if lt IE 7]> |
| <p class="chromeframe">You are using an outdated browser. <a href="http://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p> |
| <![endif]--> |
| |
| <div class="navbar navbar-inverse navbar-fixed-top" id="topbar"> |
| <div class="container"> |
| <div class="navbar-header"> |
| <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#navbar" aria-expanded="false" aria-controls="navbar"> |
| <span class="sr-only">Toggle navigation</span> |
| <span class="icon-bar"></span> |
| <span class="icon-bar"></span> |
| <span class="icon-bar"></span> |
| </button> |
| <a class="navbar-brand" href="http://gearpump.io">Gearpump |
| <span class="label label-primary" style="font-size: .6em">0.8.0</span> |
| </a> |
| </div> |
| <div id="navbar" class="collapse navbar-collapse"> |
| <ul class="nav navbar-nav"> |
| <li><a href="index.html">Overview</a></li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Introduction<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="submit-your-1st-application.html">Submit Your 1st Application</a></li> |
| <li><a href="commandline.html">Client Command Line</a></li> |
| <li class="divider"></li> |
| <li><a href="basic-concepts.html">Basic Concepts</a></li> |
| <li><a href="features.html">Technical Highlights</a></li> |
| <li><a href="message-delivery.html">Reliable Message Delivery</a></li> |
| <li><a href="performance-report.html">Performance</a></li> |
| <li><a href="gearpump-internals.html">Gearpump Internals</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li class="dropdown-header">Deployment</li> |
| <li><a href="deployment-local.html">Local Mode</a><li> |
| <li><a href="deployment-standalone.html">Standalone Mode</a></li> |
| <li><a href="deployment-yarn.html">YARN Mode</a></li> |
| <li><a href="deployment-docker.html">Docker Mode</a><li> |
| <li class="divider"></li> |
| <li><a href="deployment-ui-authentication.html">UI Authentication</a></li> |
| <li><a href="deployment-ha.html">High Availability</a></li> |
| <li><a href="deployment-msg-delivery.html">Reliable Message Delivery</a></li> |
| <li><a href="deployment-configuration.html">Configuration</a></li> |
| <li><a href="deployment-resource-isolation.html">Resource Isolation</a></li> |
| <li class="divider"></li> |
| <li><a href="deployment-security.html">YARN Security Guide</a></li> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">Programming Guide<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="dev-write-1st-app.html">Write Your 1st App</a></li> |
| <li><a href="dev-custom-serializer.html">Customized Message Passing</a></li> |
| <li class="divider"></li> |
| <li><a href="api/scala/index.html">Scala API</a></li> |
| <li><a href="api/java/index.html">Java API</a></li> |
| <li><a href="dev-rest-api.html">RESTful API</a></li> |
| <li class="divider"></li> |
| <li><a href="dev-connectors.html">Gearpump Connectors</a></li> |
| <li class="divider"></li> |
| <li><a href="dev-storm.html">Storm Compatibility</a></li> |
| <!-- |
| <li><a href="dev-samoa.html">Samoa Compatibility</a></li> |
| <li class="divider"></li> |
| <li><a href="dev-iot.html">Gearpump with IoT</a></li> |
| --> |
| </ul> |
| </li> |
| |
| <li class="dropdown"> |
| <a href="#" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a> |
| <ul class="dropdown-menu"> |
| <li><a href="how-to-contribute.html">How to Contribute</a></li> |
| <li><a href="coding-style.html">Coding Style</a></li> |
| <li class="divider"></li> |
| <li><a href="faq.html">FAQ</a><li> |
| <li><a href="about.html">About</a></li> |
| </ul> |
| </li> |
| </ul> |
| </div> |
| </div> |
| </div> |
| |
| <div class="container" id="content"> |
| |
| <h1 class="title">Storm Compatibility</h1> |
| |
| |
| <p>Gearpump provides <strong>binary compatibility</strong> for Apache Storm applications. That is to say, users could easily grab an existing Storm jar and run it |
| on Gearpump. This documentation illustrates Gearpump’s comapatibility with Storm.</p> |
| |
| <h2 id="what-storm-features-are-supported-on-gearpump">What Storm features are supported on Gearpump</h2> |
| |
| <h3 id="storm-09x">Storm 0.9.x</h3> |
| |
| <table> |
| <thead> |
| <tr> |
| <th>Feature</th> |
| <th>Support</th> |
| </tr> |
| </thead> |
| <tbody> |
| <tr> |
| <td>basic topology</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>DRPC</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>multi-lang</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-kafka</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>Trident</td> |
| <td>no</td> |
| </tr> |
| </tbody> |
| </table> |
| |
| <h3 id="storm-010x">Storm 0.10.x</h3> |
| |
| <table> |
| <thead> |
| <tr> |
| <th>Feature</th> |
| <th>Support</th> |
| </tr> |
| </thead> |
| <tbody> |
| <tr> |
| <td>basic topology</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>DRPC</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>multi-lang</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-kafka</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-hdfs</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-hbase</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-hive</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-jdbc</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-redis</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>flux</td> |
| <td>yes</td> |
| </tr> |
| <tr> |
| <td>storm-eventhubs</td> |
| <td>not verfied</td> |
| </tr> |
| <tr> |
| <td>Trident</td> |
| <td>no</td> |
| </tr> |
| </tbody> |
| </table> |
| |
| <h3 id="at-least-once-support">At Least Once support</h3> |
| |
| <p>With Ackers enabled, there are two kinds of At Least Once support in both Storm 0.9.x and Storm 0.10.x.</p> |
| |
| <ol> |
| <li>spout will replay messages on message loss as long as spout is alive</li> |
| <li>If <code>KafkaSpout</code> is used, messages could be replayed from Kafka even if the spout crashes.</li> |
| </ol> |
| |
| <p>Gearpump supports the second for both Storm versions.</p> |
| |
| <h3 id="security-support">Security support</h3> |
| |
| <p>Storm 0.10.x adds security support for following connectors</p> |
| |
| <ul> |
| <li><a href="https://github.com/apache/storm/blob/0.10.x-branch/external/storm-hdfs/README.md">storm-hdfs</a></li> |
| <li><a href="https://github.com/apache/storm/blob/0.10.x-branch/external/storm-hive/README.md">storm-hive</a></li> |
| <li><a href="https://github.com/apache/storm/blob/0.10.x-branch/external/storm-hbase/README.md">storm-hbase</a></li> |
| </ul> |
| |
| <p>That means users could access kerberos enabled HDFS, Hive and HBase with these connectors. Generally, Storm provides two approaches (please refer to above links for more information)</p> |
| |
| <ol> |
| <li>configure nimbus to automatically get delegation tokens on behalf of the topology submitter user</li> |
| <li>kerberos keytabs are already distributed on worker hosts; users configure keytab path and principal</li> |
| </ol> |
| |
| <p>Gearpump supports the second approach and users needs to add classpath of HDFS/Hive/HBase to <code>gearpump.executor.extraClasspath</code> in <code>gear.conf</code> on each node. For example,</p> |
| |
| <div class="highlight"><pre><code> ################### |
| ### Executor argument configuration |
| ### Executor JVM can contains multiple tasks |
| ################### |
| executor { |
| vmargs = "-server -Xms512M -Xmx1024M -Xss1M -XX:+HeapDumpOnOutOfMemoryError -XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=80 -XX:+UseParNewGC -XX:NewRatio=3 -Djava.rmi.server.hostname=localhost" |
| extraClasspath = "/etc/hadoop/conf" |
| } |
| </code></pre></div> |
| |
| <h2 id="how-to-run-a-storm-application-on-gearpump">How to run a Storm application on Gearpump</h2> |
| |
| <p>This section shows how to run an existing Storm jar in a local Gearpump cluster.</p> |
| |
| <ol> |
| <li> |
| <p>launch a local cluster</p> |
| |
| <div class="highlight"><pre><code>bin/local |
| </code></pre></div> |
| </li> |
| <li> |
| <p>start a Gearpump Nimbus server</p> |
| |
| <p>Users need server’s address(<code>nimbus.host</code> and <code>nimbus.thrift.port</code>) to submit topologies later. The address is written to a yaml config file set with <code>-output</code> option. |
| Users can provide an existing config file where only the address will be overwritten. If not provided, a new file <code>app.yaml</code> is created with the config.</p> |
| |
| <div class="highlight"><pre><code>bin/storm nimbus -output [conf <custom yaml config>] |
| </code></pre></div> |
| </li> |
| <li> |
| <p>submit Storm applications</p> |
| |
| <p>Users can either submit Storm applications through command line or UI.</p> |
| |
| <p>a. submit Storm applications through command line</p> |
| |
| <p><code> |
| bin/storm app -verbose -config app.yaml -jar storm-starter-${STORM_VERSION}.jar storm.starter.ExclamationTopology exclamation |
| </code></p> |
| |
| <p>Users are able to configure their applications through following options</p> |
| |
| <div class="highlight"><pre><code>* `jar` - set the path of a Storm application jar |
| * `config` - submit the custom configuration file generated when launching Nimbus |
| </code></pre></div> |
| |
| <p>b. submit Storm application through UI</p> |
| |
| <ol> |
| <li>Click on the “Create” button on the applications page on UI.</li> |
| <li>Click on the “Submit Storm Application” item in the pull down menu.</li> |
| <li>In the popup console, upload the Storm application jar and the configuration file generated when launching Nimbus, |
| and fill in <code>storm.starter.ExclamationTopology exclamation</code> as arguments.</li> |
| <li>Click on the “Submit” button</li> |
| </ol> |
| |
| <p>Either way, check the dashboard and you should see data flowing through your topology.</p> |
| </li> |
| </ol> |
| |
| <h2 id="how-is-it-different-from-running-on-storm">How is it different from running on Storm</h2> |
| |
| <h3 id="topology-submission">Topology submission</h3> |
| |
| <p>When a client submits a Storm topology, Gearpump launches locally a simplified version of Storm’s Nimbus server <code>GearpumpNimbus</code>. <code>GearpumpNimbus</code> then translates topology to a directed acyclic graph (DAG) of Gearpump, which is submitted to Gearpump master and deployed as a Gearpump application.</p> |
| |
| <p><img src="img/storm_gearpump_cluster.png" alt="storm_gearpump_cluster" /></p> |
| |
| <p><code>GearpumpNimbus</code> supports the following methods</p> |
| |
| <ul> |
| <li><code>submitTopology</code> / <code>submitTopologyWithOpts</code></li> |
| <li><code>killTopology</code> / <code>killTopologyWithOpts</code></li> |
| <li><code>getTopology</code> / <code>getUserTopology</code></li> |
| <li><code>getClusterInfo</code></li> |
| </ul> |
| |
| <h3 id="topology-translation">Topology translation</h3> |
| |
| <p>Here’s an example of <code>WordCountTopology</code> with acker bolts (ackers) being translated into a Gearpump DAG.</p> |
| |
| <p><img src="img/storm_gearpump_dag.png" alt="storm_gearpump_dag" /></p> |
| |
| <p>Gearpump creates a <code>StormProducer</code> for each Storm spout and a <code>StormProcessor</code> for each Storm bolt (except for ackers) with the same parallelism, and wires them together using the same grouping strategy (partitioning in Gearpump) as in Storm.</p> |
| |
| <p>At runtime, spouts and bolts are running inside <code>StormProducer</code> tasks and <code>StormProcessor</code> tasks respectively. Messages emitted by spout are passed to <code>StormProducer</code>, transferred to <code>StormProcessor</code> and passed down to bolt. Messages are serialized / deserialized with Storm serializers.</p> |
| |
| <p>Storm ackers are dropped since Gearpump has a different mechanism of message tracking and flow control.</p> |
| |
| <h3 id="task-execution">Task execution</h3> |
| |
| <p>Each Storm task is executed by a dedicated thread while all Gearpump tasks of an executor share a thread pool. Generally, we can achieve better performance with a shared thread pool. It’s possible, however, some tasks block and take up all the threads. In that case, we can |
| fall back to the Storm way by setting <code>gearpump.task-dispatcher</code> to <code>"gaerpump.single-thread-dispatcher"</code> in <code>gear.conf</code>.</p> |
| |
| <h3 id="message-tracking">Message tracking</h3> |
| |
| <p>Storm tracks the lineage of each message with ackers to guarantee at-least-once message delivery. Failed messages are re-sent from spout.</p> |
| |
| <p>Gearpump <a href="gearpump-internals.html#how-do-we-detect-message-loss">tracks messages between a sender and receiver in an efficient way</a>. Message loss causes the whole application to replay from the <a href="gearpump-internals.html#application-clock-and-global-clock-service">minimum timestamp of all pending messages in the system</a>.</p> |
| |
| <h3 id="flow-control">Flow control</h3> |
| |
| <p>Storm throttles flow rate at spout, which stops sending messages if the number of unacked messages exceeds <code>topology.max.spout.pending</code>.</p> |
| |
| <p>Gearpump has flow control between tasks such that <a href="gearpump-internals.html#how-do-we-do-flow-control">sender cannot flood receiver</a>, which is backpressured till the source.</p> |
| |
| <h3 id="configurations">Configurations</h3> |
| |
| <p>All Storm configurations are respected with the following priority order</p> |
| |
| <div class="highlight"><pre><code>defaults.yaml < custom file config < application config < component config |
| </code></pre></div> |
| |
| <p>where</p> |
| |
| <ul> |
| <li>application config is submit from Storm application along with the topology</li> |
| <li>component config is set in spout / bolt with <code>getComponentConfiguration</code></li> |
| <li>custom file config is specified with the <code>-config</code> option when submitting Storm application from command line or uploaded from UI</li> |
| </ul> |
| |
| <h2 id="streamcql-support">StreamCQL Support</h2> |
| |
| <p><a href="https://github.com/HuaweiBigData/StreamCQL">StreamCQL</a> is a Continuous Query Language on RealTime Computation System open sourced by Huawei. |
| Since StreamCQL already supports Storm, it’s straightforward to run StreamCQL over Gearpump.</p> |
| |
| <ol> |
| <li> |
| <p>Install StreamCQL as in the official <a href="https://github.com/HuaweiBigData/StreamCQL#install-streamcql">README</a></p> |
| </li> |
| <li> |
| <p>Launch Gearpump Nimbus Server as before</p> |
| </li> |
| <li> |
| <p>Go to the installed stream-cql-binary, and change following settings in <code>conf/streaming-site.xml</code> with the output Nimbus configs in Step 2.</p> |
| |
| <div class="highlight"><pre><code class="language-xml"> <span class="nt"><property></span> |
| <span class="nt"><name></span>streaming.storm.nimbus.host<span class="nt"></name></span> |
| <span class="nt"><value></span>${nimbus.host}<span class="nt"></value></span> |
| <span class="nt"></property></span> |
| <span class="nt"><property></span> |
| <span class="nt"><name></span>streaming.storm.nimbus.port<span class="nt"></name></span> |
| <span class="nt"><value></span>${nimbus.thrift.port}<span class="nt"></value></span> |
| <span class="nt"></property></span></code></pre></div> |
| </li> |
| <li> |
| <p>Open CQL client shell and execute a simple cql example</p> |
| |
| <div class="highlight"><pre><code>bin/cql |
| </code></pre></div> |
| |
| <div class="highlight"><pre><code class="language-sql"><span class="n">Streaming</span><span class="o">></span> <span class="k">CREATE</span> <span class="k">INPUT</span> <span class="n">STREAM</span> <span class="n">s</span> |
| <span class="p">(</span><span class="n">id</span> <span class="nb">INT</span><span class="p">,</span> <span class="n">name</span> <span class="n">STRING</span><span class="p">,</span> <span class="k">type</span> <span class="nb">INT</span><span class="p">)</span> |
| <span class="k">SOURCE</span> <span class="n">randomgen</span> |
| <span class="n">PROPERTIES</span> <span class="p">(</span> <span class="n">timeUnit</span> <span class="o">=</span> <span class="ss">"SECONDS"</span><span class="p">,</span> <span class="n">period</span> <span class="o">=</span> <span class="ss">"1"</span><span class="p">,</span> |
| <span class="n">eventNumPerperiod</span> <span class="o">=</span> <span class="ss">"1"</span><span class="p">,</span> <span class="n">isSchedule</span> <span class="o">=</span> <span class="ss">"true"</span> <span class="p">);</span> |
| |
| <span class="k">CREATE</span> <span class="k">OUTPUT</span> <span class="n">STREAM</span> <span class="n">rs</span> |
| <span class="p">(</span><span class="k">type</span> <span class="nb">INT</span><span class="p">,</span> <span class="n">cc</span> <span class="nb">INT</span><span class="p">)</span> |
| <span class="n">SINK</span> <span class="n">consoleOutput</span><span class="p">;</span> |
| |
| <span class="k">INSERT</span> <span class="k">INTO</span> <span class="n">STREAM</span> <span class="n">rs</span> <span class="k">SELECT</span> <span class="k">type</span><span class="p">,</span> <span class="k">COUNT</span><span class="p">(</span><span class="n">id</span><span class="p">)</span> <span class="k">as</span> <span class="n">cc</span> |
| <span class="k">FROM</span> <span class="n">s</span><span class="p">[</span><span class="n">RANGE</span> <span class="mi">20</span> <span class="n">SECONDS</span> <span class="n">BATCH</span><span class="p">]</span> |
| <span class="k">WHERE</span> <span class="n">id</span> <span class="o">></span> <span class="mi">5</span> <span class="k">GROUP</span> <span class="k">BY</span> <span class="k">type</span><span class="p">;</span> |
| |
| <span class="n">SUBMIT</span> <span class="n">APPLICATION</span> <span class="n">example</span><span class="p">;</span></code></pre></div> |
| </li> |
| <li> |
| <p>Check the dashboard and you should see data flowing through a topology of 3 components</p> |
| </li> |
| </ol> |
| |
| |
| |
| </div> <!-- /container --> |
| |
| <script src="js/vendor/jquery-2.1.4.min.js"></script> |
| <script src="js/vendor/bootstrap-3.3.5.min.js"></script> |
| <script src="js/vendor/anchor-1.1.1.min.js"></script> |
| <script src="js/main.js"></script> |
| |
| <!-- MathJax Section --> |
| <script type="text/x-mathjax-config"> |
| MathJax.Hub.Config({ |
| TeX: { equationNumbers: { autoNumber: "AMS" } } |
| }); |
| </script> |
| <script> |
| // Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS. |
| // We could use "//cdn.mathjax...", but that won't support "file://". |
| (function(d, script) { |
| script = d.createElement('script'); |
| script.type = 'text/javascript'; |
| script.async = true; |
| script.onload = function(){ |
| MathJax.Hub.Config({ |
| tex2jax: { |
| inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ], |
| displayMath: [ ["$$","$$"], ["\\[", "\\]"] ], |
| processEscapes: true, |
| skipTags: ['script', 'noscript', 'style', 'textarea', 'pre'] |
| } |
| }); |
| }; |
| script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + |
| 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; |
| d.getElementsByTagName('head')[0].appendChild(script); |
| }(document)); |
| </script> |
| </body> |
| </html> |