blob: 149b7abc12aae998c2b07e4a258b9722837995e2 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>Storm Compatibility - Apache Gearpump(incubating)</title>
<link rel="shortcut icon" href="../../img/favicon.ico">
<link href='https://fonts.googleapis.com/css?family=Lato:400,700|Roboto+Slab:400,700|Inconsolata:400,700' rel='stylesheet' type='text/css'>
<link rel="stylesheet" href="../../css/theme.css" type="text/css" />
<link rel="stylesheet" href="../../css/theme_extra.css" type="text/css" />
<link rel="stylesheet" href="../../css/highlight.css">
<script>
// Current page data
var mkdocs_page_name = "Storm Compatibility";
</script>
<script src="../../js/jquery-2.1.1.min.js"></script>
<script src="../../js/modernizr-2.8.3.min.js"></script>
<script type="text/javascript" src="../../js/highlight.pack.js"></script>
<script src="../../js/theme.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side stickynav">
<div class="wy-side-nav-search">
<a href="../../index.html" class="icon icon-home"> Apache Gearpump(incubating)</a>
<div role="search">
<form id ="rtd-search-form" class="wy-form" action="../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul class="current">
<li>
<li class="toctree-l1 ">
<a class="" href="../../index.html">Overview</a>
</li>
<li>
<li>
<ul class="subnav">
<li><span>Introduction</span></li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/submit-your-1st-application/index.html">Submit Your 1st Application</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/commandline/index.html">Client Command Line</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/basic-concepts/index.html">Basic Concepts</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/features/index.html">Technical Highlights</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/message-delivery/index.html">Reliable Message Delivery</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/performance-report/index.html">Performance</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../introduction/gearpump-internals/index.html">Gearpump Internals</a>
</li>
</ul>
<li>
<li>
<ul class="subnav">
<li><span>Deployment</span></li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-local/index.html">Local Mode</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-standalone/index.html">Standalone Mode</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-yarn/index.html">YARN Mode</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-docker/index.html">Docker Mode</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-ui-authentication/index.html">UI Authentication</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-ha/index.html">High Availability</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-msg-delivery/index.html">Reliable Message Delivery</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-configuration/index.html">Configuration</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-resource-isolation/index.html">Resource Isolation</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/deployment-security/index.html">YARN Security Guide</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/get-gearpump-distribution/index.html">How to Get Your Gearpump Distribution</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../deployment/hardware-requirement/index.html">Hardware Requirement</a>
</li>
</ul>
<li>
<li>
<ul class="subnav">
<li><span>Programming Guide</span></li>
<li class="toctree-l1 ">
<a class="" href="../dev-write-1st-app/index.html">Write Your 1st App</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../dev-custom-serializer/index.html">Customized Message Passing</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../dev-connectors/index.html">Gearpump Connectors</a>
</li>
<li class="toctree-l1 current">
<a class="current" href="index.html">Storm Compatibility</a>
<ul>
<li class="toctree-l3"><a href="#what-storm-features-are-supported-on-gearpump">What Storm features are supported on Gearpump</a></li>
<li><a class="toctree-l4" href="#storm-09x">Storm 0.9.x</a></li>
<li><a class="toctree-l4" href="#storm-010x">Storm 0.10.x</a></li>
<li><a class="toctree-l4" href="#at-least-once-support">At Least Once support</a></li>
<li><a class="toctree-l4" href="#security-support">Security support</a></li>
<li class="toctree-l3"><a href="#how-to-run-a-storm-application-on-gearpump">How to run a Storm application on Gearpump</a></li>
<li class="toctree-l3"><a href="#how-is-it-different-from-running-on-storm">How is it different from running on Storm</a></li>
<li><a class="toctree-l4" href="#topology-submission">Topology submission</a></li>
<li><a class="toctree-l4" href="#topology-translation">Topology translation</a></li>
<li><a class="toctree-l4" href="#task-execution">Task execution</a></li>
<li><a class="toctree-l4" href="#message-tracking">Message tracking</a></li>
<li><a class="toctree-l4" href="#flow-control">Flow control</a></li>
<li><a class="toctree-l4" href="#configurations">Configurations</a></li>
<li class="toctree-l3"><a href="#streamcql-support">StreamCQL Support</a></li>
</ul>
</li>
<li class="toctree-l1 ">
<a class="" href="../dev-ide-setup/index.html">IDE Setup</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../dev-non-streaming-example/index.html">Non Streaming Examples</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../dev-rest-api/index.html">REST API</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../api/scala/index.html">Scala API</a>
</li>
<li class="toctree-l1 ">
<a class="" href="../../api/java/index.html">Java API</a>
</li>
</ul>
<li>
</ul>
</div>
&nbsp;
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../index.html">Apache Gearpump(incubating)</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../index.html">Docs</a> &raquo;</li>
<li>Programming Guide &raquo;</li>
<li>Storm Compatibility</li>
<li class="wy-breadcrumbs-aside">
<a href="https://github.com/apache/incubator-gearpump" class="icon icon-github"> Edit on GitHub</a>
</li>
</ul>
<hr/>
</div>
<div role="main">
<div class="section">
<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 compatibility 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 verified</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>
<pre class="codehilite"><code class="language-json">###################
### Executor argument configuration
### Executor JVM can contains multiple tasks
###################
executor {
vmargs = &quot;-server -Xms512M -Xmx1024M -Xss1M -XX:+HeapDumpOnOutOfMemoryError -XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=80 -XX:+UseParNewGC -XX:NewRatio=3 -Djava.rmi.server.hostname=localhost&quot;
extraClasspath = &quot;/etc/hadoop/conf&quot;
}</code></pre>
<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>
<pre class="codehilite"><code class="language-bash">bin/local</code></pre>
</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>
<pre class="codehilite"><code class="language-bash">bin/storm nimbus -output [conf &lt;custom yaml config&gt;]</code></pre>
</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>
<pre class="codehilite"><code class="language-bash">bin/storm app -verbose -config app.yaml -jar storm-starter-${STORM_VERSION}.jar storm.starter.ExclamationTopology exclamation</code></pre>
<p>Users are able to configure their applications through following options</p>
<ul>
<li><code>jar</code> - set the path of a Storm application jar</li>
<li><code>config</code> - submit the custom configuration file generated when launching Nimbus</li>
</ul>
<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>
</li>
</ol>
<p>Either way, check the dashboard and you should see data flowing through your topology. </p>
<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 alt="storm_gearpump_cluster" src="../../img/storm_gearpump_cluster.png" /></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 alt="storm_gearpump_dag" src="../../img/storm_gearpump_dag.png" /></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 / de-serialized 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>"gearpump.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="../../introduction/gearpump-internals#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="../../introduction/gearpump-internals#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="../../introduction/gearpump-internals#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>
<pre class="codehilite"><code class="language-bash">defaults.yaml &lt; custom file config &lt; application config &lt; component config</code></pre>
<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>
<pre class="codehilite"><code class="language-xml">&lt;property&gt;
&lt;name&gt;streaming.storm.nimbus.host&lt;/name&gt;
&lt;value&gt;${nimbus.host}&lt;/value&gt;
&lt;/property&gt;
&lt;property&gt;
&lt;name&gt;streaming.storm.nimbus.port&lt;/name&gt;
&lt;value&gt;${nimbus.thrift.port}&lt;/value&gt;
&lt;/property&gt;</code></pre>
</li>
<li>
<p>Open CQL client shell with <code>bin/cql</code> and execute a simple cql example </p>
<pre class="codehilite"><code class="language-sql">Streaming&gt; CREATE INPUT STREAM s
(id INT, name STRING, type INT)
SOURCE randomgen
PROPERTIES ( timeUnit = &quot;SECONDS&quot;, period = &quot;1&quot;,
eventNumPerperiod = &quot;1&quot;, isSchedule = &quot;true&quot; );
CREATE OUTPUT STREAM rs
(type INT, cc INT)
SINK consoleOutput;
INSERT INTO STREAM rs SELECT type, COUNT(id) as cc
FROM s[RANGE 20 SECONDS BATCH]
WHERE id &gt; 5 GROUP BY type;
SUBMIT APPLICATION example;</code></pre>
</li>
<li>
<p>Check the dashboard and you should see data flowing through a topology of 3 components.</p>
</li>
</ol>
</div>
</div>
<footer>
<div class="rst-footer-buttons" role="navigation" aria-label="footer navigation">
<a href="../dev-ide-setup/index.html" class="btn btn-neutral float-right" title="IDE Setup"/>Next <span class="icon icon-circle-arrow-right"></span></a>
<a href="../dev-connectors/index.html" class="btn btn-neutral" title="Gearpump Connectors"><span class="icon icon-circle-arrow-left"></span> Previous</a>
</div>
<hr/>
<div role="contentinfo">
<!-- Copyright etc -->
</div>
Built with <a href="http://www.mkdocs.org">MkDocs</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<div class="rst-versions" role="note" style="cursor: pointer">
<span class="rst-current-version" data-toggle="rst-current-version">
<a class="icon icon-github" style="float: left; color: #fcfcfc"> GitHub</a>
<span><a href="../dev-connectors/index.html" style="color: #fcfcfc;">&laquo; Previous</a></span>
<span style="margin-left: 15px"><a href="../dev-ide-setup/index.html" style="color: #fcfcfc">Next &raquo;</a></span>
</span>
</div>
</body>
</html>