blob: f95c1eed172c57bd5189b807b073a033024cb4b0 [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge">
<meta name="viewport" content="width=device-width, initial-scale=1">
<link rel="shortcut icon" href="/favicon.ico" type="image/x-icon">
<link rel="icon" href="/favicon.ico" type="image/x-icon">
<title>Storm JDBC Integration</title>
<!-- Bootstrap core CSS -->
<link href="/assets/css/bootstrap.min.css" rel="stylesheet">
<!-- Bootstrap theme -->
<link href="/assets/css/bootstrap-theme.min.css" rel="stylesheet">
<!-- Custom styles for this template -->
<link rel="stylesheet" href="http://fortawesome.github.io/Font-Awesome/assets/font-awesome/css/font-awesome.css">
<link href="/css/style.css" rel="stylesheet">
<link href="/assets/css/owl.theme.css" rel="stylesheet">
<link href="/assets/css/owl.carousel.css" rel="stylesheet">
<script type="text/javascript" src="/assets/js/jquery.min.js"></script>
<script type="text/javascript" src="/assets/js/bootstrap.min.js"></script>
<script type="text/javascript" src="/assets/js/owl.carousel.min.js"></script>
<script type="text/javascript" src="/assets/js/storm.js"></script>
<!-- Just for debugging purposes. Don't actually copy these 2 lines! -->
<!--[if lt IE 9]><script src="../../assets/js/ie8-responsive-file-warning.js"></script><![endif]-->
<!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
<!--[if lt IE 9]>
<script src="https://oss.maxcdn.com/html5shiv/3.7.2/html5shiv.min.js"></script>
<script src="https://oss.maxcdn.com/respond/1.4.2/respond.min.js"></script>
<![endif]-->
</head>
<body>
<header>
<div class="container-fluid">
<div class="row">
<div class="col-md-10">
<a href="/index.html"><img src="/images/logo.png" class="logo" /></a>
</div>
<div class="col-md-2">
<a href="/downloads.html" class="btn-std btn-block btn-download">Download</a>
</div>
</div>
</div>
</header>
<!--Header End-->
<!--Navigation Begin-->
<div class="navbar" role="banner">
<div class="container-fluid">
<div class="navbar-header">
<button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bs-navbar-collapse">
<span class="icon-bar"></span>
<span class="icon-bar"></span>
<span class="icon-bar"></span>
</button>
</div>
<nav class="collapse navbar-collapse bs-navbar-collapse" role="navigation">
<ul class="nav navbar-nav">
<li><a href="/index.html" id="home">Home</a></li>
<li><a href="/getting-help.html" id="getting-help">Getting Help</a></li>
<li><a href="/about/integrates.html" id="project-info">Project Information</a></li>
<li><a href="/documentation.html" id="documentation">Documentation</a></li>
<li><a href="/talksAndVideos.html">Talks and Slideshows</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="contribute">Community <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/contribute/Contributing-to-Storm.html">Contributing</a></li>
<li><a href="/contribute/People.html">People</a></li>
<li><a href="/contribute/BYLAWS.html">ByLaws</a></li>
</ul>
</li>
<li><a href="/2015/11/05/storm096-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">Storm JDBC Integration</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<p>Storm/Trident integration for JDBC. This package includes the core bolts and trident states that allows a storm topology
to either insert storm tuples in a database table or to execute select queries against a database and enrich tuples
in a storm topology.</p>
<p><strong>Note</strong>: Throughout the examples below, we make use of com.google.common.collect.Lists and com.google.common.collect.Maps.</p>
<h2 id="inserting-into-a-database">Inserting into a database.</h2>
<p>The bolt and trident state included in this package for inserting data into a database tables are tied to a single table.</p>
<h3 id="connectionprovider">ConnectionProvider</h3>
<p>An interface that should be implemented by different connection pooling mechanism <code>org.apache.storm.jdbc.common.ConnectionProvider</code></p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">interface</span> <span class="nc">ConnectionProvider</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="cm">/**
* method must be idempotent.
*/</span>
<span class="kt">void</span> <span class="n">prepare</span><span class="o">();</span>
<span class="cm">/**
*
* @return a DB connection over which the queries can be executed.
*/</span>
<span class="n">Connection</span> <span class="n">getConnection</span><span class="o">();</span>
<span class="cm">/**
* called once when the system is shutting down, should be idempotent.
*/</span>
<span class="kt">void</span> <span class="n">cleanup</span><span class="o">();</span>
<span class="o">}</span>
</code></pre></div>
<p>Out of the box we support <code>org.apache.storm.jdbc.common.HikariCPConnectionProvider</code> which is an implementation that uses HikariCP.</p>
<h3 id="jdbcmapper">JdbcMapper</h3>
<p>The main API for inserting data in a table using JDBC is the <code>org.apache.storm.jdbc.mapper.JdbcMapper</code> interface:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">interface</span> <span class="nc">JdbcMapper</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="n">List</span><span class="o">&lt;</span><span class="n">Column</span><span class="o">&gt;</span> <span class="n">getColumns</span><span class="o">(</span><span class="n">ITuple</span> <span class="n">tuple</span><span class="o">);</span>
<span class="o">}</span>
</code></pre></div>
<p>The <code>getColumns()</code> method defines how a storm tuple maps to a list of columns representing a row in a database.
<strong>The order of the returned list is important. The place holders in the supplied queries are resolved in the same order as returned list.</strong>
For example if the user supplied insert query is <code>insert into user(user_id, user_name, create_date) values (?,?, now())</code> the 1st item
of the returned list of <code>getColumns</code> method will map to the 1st place holder and the 2nd to the 2nd and so on. We do not parse
the supplied queries to try and resolve place holder by column names. Not making any assumptions about the query syntax allows this connector
to be used by some non-standard sql frameworks like Pheonix which only supports upsert into.</p>
<h3 id="jdbcinsertbolt">JdbcInsertBolt</h3>
<p>To use the <code>JdbcInsertBolt</code>, you construct an instance of it by specifying a <code>ConnectionProvider</code> implementation
and a <code>JdbcMapper</code> implementation that converts storm tuple to DB row. In addition, you must either supply
a table name using <code>withTableName</code> method or an insert query using <code>withInsertQuery</code>.
If you specify a insert query you should ensure that your <code>JdbcMapper</code> implementation will return a list of columns in the same order as in your insert query.
You can optionally specify a query timeout seconds param that specifies max seconds an insert query can take.
The default is set to value of topology.message.timeout.secs and a value of -1 will indicate not to set any query timeout.
You should set the query timeout value to be &lt;= topology.message.timeout.secs.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">Map</span> <span class="n">hikariConfigMap</span> <span class="o">=</span> <span class="n">Maps</span><span class="o">.</span><span class="na">newHashMap</span><span class="o">();</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSourceClassName"</span><span class="o">,</span><span class="s">"com.mysql.jdbc.jdbc2.optional.MysqlDataSource"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.url"</span><span class="o">,</span> <span class="s">"jdbc:mysql://localhost/test"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.user"</span><span class="o">,</span><span class="s">"root"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.password"</span><span class="o">,</span><span class="s">"password"</span><span class="o">);</span>
<span class="n">ConnectionProvider</span> <span class="n">connectionProvider</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HikariCPConnectionProvider</span><span class="o">(</span><span class="n">hikariConfigMap</span><span class="o">);</span>
<span class="n">String</span> <span class="n">tableName</span> <span class="o">=</span> <span class="s">"user_details"</span><span class="o">;</span>
<span class="n">JdbcMapper</span> <span class="n">simpleJdbcMapper</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SimpleJdbcMapper</span><span class="o">(</span><span class="n">tableName</span><span class="o">,</span> <span class="n">connectionProvider</span><span class="o">);</span>
<span class="n">JdbcInsertBolt</span> <span class="n">userPersistanceBolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcInsertBolt</span><span class="o">(</span><span class="n">connectionProvider</span><span class="o">,</span> <span class="n">simpleJdbcMapper</span><span class="o">)</span>
<span class="o">.</span><span class="na">withTableName</span><span class="o">(</span><span class="s">"user"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withQueryTimeoutSecs</span><span class="o">(</span><span class="mi">30</span><span class="o">);</span>
<span class="n">Or</span>
<span class="n">JdbcInsertBolt</span> <span class="n">userPersistanceBolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcInsertBolt</span><span class="o">(</span><span class="n">connectionProvider</span><span class="o">,</span> <span class="n">simpleJdbcMapper</span><span class="o">)</span>
<span class="o">.</span><span class="na">withInsertQuery</span><span class="o">(</span><span class="s">"insert into user values (?,?)"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withQueryTimeoutSecs</span><span class="o">(</span><span class="mi">30</span><span class="o">);</span>
</code></pre></div>
<h3 id="simplejdbcmapper">SimpleJdbcMapper</h3>
<p><code>storm-jdbc</code> includes a general purpose <code>JdbcMapper</code> implementation called <code>SimpleJdbcMapper</code> that can map Storm
tuple to a Database row. <code>SimpleJdbcMapper</code> assumes that the storm tuple has fields with same name as the column name in
the database table that you intend to write to.</p>
<p>To use <code>SimpleJdbcMapper</code>, you simply tell it the tableName that you want to write to and provide a connectionProvider instance.</p>
<p>The following code creates a <code>SimpleJdbcMapper</code> instance that:</p>
<ol>
<li>Will allow the mapper to transform a storm tuple to a list of columns mapping to a row in table test.user_details.</li>
<li>Will use the provided HikariCP configuration to establish a connection pool with specified Database configuration and
automatically figure out the column names and corresponding data types of the table that you intend to write to.
Please see <a href="https://github.com/brettwooldridge/HikariCP#configuration-knobs-baby">https://github.com/brettwooldridge/HikariCP#configuration-knobs-baby</a> to learn more about hikari configuration properties.</li>
</ol>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">Map</span> <span class="n">hikariConfigMap</span> <span class="o">=</span> <span class="n">Maps</span><span class="o">.</span><span class="na">newHashMap</span><span class="o">();</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSourceClassName"</span><span class="o">,</span><span class="s">"com.mysql.jdbc.jdbc2.optional.MysqlDataSource"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.url"</span><span class="o">,</span> <span class="s">"jdbc:mysql://localhost/test"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.user"</span><span class="o">,</span><span class="s">"root"</span><span class="o">);</span>
<span class="n">hikariConfigMap</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"dataSource.password"</span><span class="o">,</span><span class="s">"password"</span><span class="o">);</span>
<span class="n">ConnectionProvider</span> <span class="n">connectionProvider</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HikariCPConnectionProvider</span><span class="o">(</span><span class="n">hikariConfigMap</span><span class="o">);</span>
<span class="n">String</span> <span class="n">tableName</span> <span class="o">=</span> <span class="s">"user_details"</span><span class="o">;</span>
<span class="n">JdbcMapper</span> <span class="n">simpleJdbcMapper</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SimpleJdbcMapper</span><span class="o">(</span><span class="n">tableName</span><span class="o">,</span> <span class="n">connectionProvider</span><span class="o">);</span>
</code></pre></div>
<p>The mapper initialized in the example above assumes a storm tuple has value for all the columns of the table you intend to insert data into and its <code>getColumn</code>
method will return the columns in the order in which Jdbc connection instance&#39;s <code>connection.getMetaData().getColumns();</code> method returns them.</p>
<p><strong>If you specified your own insert query to <code>JdbcInsertBolt</code> you must initialize <code>SimpleJdbcMapper</code> with explicit columnschema such that the schema has columns in the same order as your insert queries.</strong>
For example if your insert query is <code>Insert into user (user_id, user_name) values (?,?)</code> then your <code>SimpleJdbcMapper</code> should be initialized with the following statements:
<code>java
List&lt;Column&gt; columnSchema = Lists.newArrayList(
new Column(&quot;user_id&quot;, java.sql.Types.INTEGER),
new Column(&quot;user_name&quot;, java.sql.Types.VARCHAR));
JdbcMapper simpleJdbcMapper = new SimpleJdbcMapper(columnSchema);
</code></p>
<p>If your storm tuple only has fields for a subset of columns i.e. if some of the columns in your table have default values and you want to only insert values for columns with no default values you can enforce the behavior by initializing the
<code>SimpleJdbcMapper</code> with explicit columnschema. For example, if you have a user_details table <code>create table if not exists user_details (user_id integer, user_name varchar(100), dept_name varchar(100), create_time TIMESTAMP DEFAULT CURRENT_TIMESTAMP);</code>
In this table the create_time column has a default value. To ensure only the columns with no default values are inserted
you can initialize the <code>jdbcMapper</code> as below:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">List</span><span class="o">&lt;</span><span class="n">Column</span><span class="o">&gt;</span> <span class="n">columnSchema</span> <span class="o">=</span> <span class="n">Lists</span><span class="o">.</span><span class="na">newArrayList</span><span class="o">(</span>
<span class="k">new</span> <span class="n">Column</span><span class="o">(</span><span class="s">"user_id"</span><span class="o">,</span> <span class="n">java</span><span class="o">.</span><span class="na">sql</span><span class="o">.</span><span class="na">Types</span><span class="o">.</span><span class="na">INTEGER</span><span class="o">),</span>
<span class="k">new</span> <span class="n">Column</span><span class="o">(</span><span class="s">"user_name"</span><span class="o">,</span> <span class="n">java</span><span class="o">.</span><span class="na">sql</span><span class="o">.</span><span class="na">Types</span><span class="o">.</span><span class="na">VARCHAR</span><span class="o">),</span>
<span class="k">new</span> <span class="n">Column</span><span class="o">(</span><span class="s">"dept_name"</span><span class="o">,</span> <span class="n">java</span><span class="o">.</span><span class="na">sql</span><span class="o">.</span><span class="na">Types</span><span class="o">.</span><span class="na">VARCHAR</span><span class="o">));</span>
<span class="n">JdbcMapper</span> <span class="n">simpleJdbcMapper</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SimpleJdbcMapper</span><span class="o">(</span><span class="n">columnSchema</span><span class="o">);</span>
</code></pre></div>
<h3 id="jdbctridentstate">JdbcTridentState</h3>
<p>We also support a trident persistent state that can be used with trident topologies. To create a jdbc persistent trident
state you need to initialize it with the table name or an insert query, the JdbcMapper instance and connection provider instance.
See the example below:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">JdbcState</span><span class="o">.</span><span class="na">Options</span> <span class="n">options</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcState</span><span class="o">.</span><span class="na">Options</span><span class="o">()</span>
<span class="o">.</span><span class="na">withConnectionProvider</span><span class="o">(</span><span class="n">connectionProvider</span><span class="o">)</span>
<span class="o">.</span><span class="na">withMapper</span><span class="o">(</span><span class="n">jdbcMapper</span><span class="o">)</span>
<span class="o">.</span><span class="na">withTableName</span><span class="o">(</span><span class="s">"user_details"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withQueryTimeoutSecs</span><span class="o">(</span><span class="mi">30</span><span class="o">);</span>
<span class="n">JdbcStateFactory</span> <span class="n">jdbcStateFactory</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcStateFactory</span><span class="o">(</span><span class="n">options</span><span class="o">);</span>
</code></pre></div>
<p>similar to <code>JdbcInsertBolt</code> you can specify a custom insert query using <code>withInsertQuery</code> instead of specifying a table name.</p>
<h2 id="lookup-from-database">Lookup from Database</h2>
<p>We support <code>select</code> queries from databases to allow enrichment of storm tuples in a topology. The main API for
executing select queries against a database using JDBC is the <code>org.apache.storm.jdbc.mapper.JdbcLookupMapper</code> interface:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="kt">void</span> <span class="nf">declareOutputFields</span><span class="p">(</span><span class="n">OutputFieldsDeclarer</span> <span class="n">declarer</span><span class="o">);</span>
<span class="n">List</span><span class="o">&lt;</span><span class="n">Column</span><span class="o">&gt;</span> <span class="n">getColumns</span><span class="o">(</span><span class="n">ITuple</span> <span class="n">tuple</span><span class="o">);</span>
<span class="n">List</span><span class="o">&lt;</span><span class="n">Values</span><span class="o">&gt;</span> <span class="n">toTuple</span><span class="o">(</span><span class="n">ITuple</span> <span class="n">input</span><span class="o">,</span> <span class="n">List</span><span class="o">&lt;</span><span class="n">Column</span><span class="o">&gt;</span> <span class="n">columns</span><span class="o">);</span>
</code></pre></div>
<p>The <code>declareOutputFields</code> method is used to indicate what fields will be emitted as part of output tuple of processing a storm
tuple. </p>
<p>The <code>getColumns</code> method specifies the place holder columns in a select query and their SQL type and the value to use.
For example in the user_details table mentioned above if you were executing a query <code>select user_name from user_details where
user_id = ? and create_time &gt; ?</code> the <code>getColumns</code> method would take a storm input tuple and return a List containing two items.
The first instance of <code>Column</code> type&#39;s <code>getValue()</code> method will be used as the value of <code>user_id</code> to lookup for and the
second instance of <code>Column</code> type&#39;s <code>getValue()</code> method will be used as the value of <code>create_time</code>.
<strong>Note: the order in the returned list determines the place holder&#39;s value. In other words the first item in the list maps
to first <code>?</code> in select query, the second item to second <code>?</code> in query and so on.</strong> </p>
<p>The <code>toTuple</code> method takes in the input tuple and a list of columns representing a DB row as a result of the select query
and returns a list of values to be emitted.
<strong>Please note that it returns a list of <code>Values</code> and not just a single instance of <code>Values</code>.</strong>
This allows a for a single DB row to be mapped to multiple output storm tuples.</p>
<h3 id="simplejdbclookupmapper">SimpleJdbcLookupMapper</h3>
<p><code>storm-jdbc</code> includes a general purpose <code>JdbcLookupMapper</code> implementation called <code>SimpleJdbcLookupMapper</code>. </p>
<p>To use <code>SimpleJdbcMapper</code>, you have to initialize it with the fields that will be outputted by your bolt and the list of
columns that are used in your select query as place holder. The following example shows initialization of a <code>SimpleJdbcLookupMapper</code>
that declares <code>user_id,user_name,create_date</code> as output fields and <code>user_id</code> as the place holder column in select query.
SimpleJdbcMapper assumes the field name in your tuple is equal to the place holder column name, i.e. in our example
<code>SimpleJdbcMapper</code> will look for a field <code>use_id</code> in the input tuple and use its value as the place holder&#39;s value in the
select query. For constructing output tuples, it looks for fields specified in <code>outputFields</code> in the input tuple first,
and if it is not found in input tuple then it looks at select query&#39;s output row for a column with same name as field name.
So in the example below if the input tuple had fields <code>user_id, create_date</code> and the select query was
<code>select user_name from user_details where user_id = ?</code>, For each input tuple <code>SimpleJdbcLookupMapper.getColumns(tuple)</code>
will return the value of <code>tuple.getValueByField(&quot;user_id&quot;)</code> which will be used as the value in <code>?</code> of select query.
For each output row from DB, <code>SimpleJdbcLookupMapper.toTuple()</code> will use the <code>user_id, create_date</code> from the input tuple as
is adding only <code>user_name</code> from the resulting row and returning these 3 fields as a single output tuple.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">Fields</span> <span class="n">outputFields</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"user_id"</span><span class="o">,</span> <span class="s">"user_name"</span><span class="o">,</span> <span class="s">"create_date"</span><span class="o">);</span>
<span class="n">List</span><span class="o">&lt;</span><span class="n">Column</span><span class="o">&gt;</span> <span class="n">queryParamColumns</span> <span class="o">=</span> <span class="n">Lists</span><span class="o">.</span><span class="na">newArrayList</span><span class="o">(</span><span class="k">new</span> <span class="n">Column</span><span class="o">(</span><span class="s">"user_id"</span><span class="o">,</span> <span class="n">Types</span><span class="o">.</span><span class="na">INTEGER</span><span class="o">));</span>
<span class="k">this</span><span class="o">.</span><span class="na">jdbcLookupMapper</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SimpleJdbcLookupMapper</span><span class="o">(</span><span class="n">outputFields</span><span class="o">,</span> <span class="n">queryParamColumns</span><span class="o">);</span>
</code></pre></div>
<h3 id="jdbclookupbolt">JdbcLookupBolt</h3>
<p>To use the <code>JdbcLookupBolt</code>, construct an instance of it using a <code>ConnectionProvider</code> instance, <code>JdbcLookupMapper</code> instance and the select query to execute.
You can optionally specify a query timeout seconds param that specifies max seconds the select query can take.
The default is set to value of topology.message.timeout.secs. You should set this value to be &lt;= topology.message.timeout.secs.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">String</span> <span class="n">selectSql</span> <span class="o">=</span> <span class="s">"select user_name from user_details where user_id = ?"</span><span class="o">;</span>
<span class="n">SimpleJdbcLookupMapper</span> <span class="n">lookupMapper</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SimpleJdbcLookupMapper</span><span class="o">(</span><span class="n">outputFields</span><span class="o">,</span> <span class="n">queryParamColumns</span><span class="o">)</span>
<span class="n">JdbcLookupBolt</span> <span class="n">userNameLookupBolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcLookupBolt</span><span class="o">(</span><span class="n">connectionProvider</span><span class="o">,</span> <span class="n">selectSql</span><span class="o">,</span> <span class="n">lookupMapper</span><span class="o">)</span>
<span class="o">.</span><span class="na">withQueryTimeoutSecs</span><span class="o">(</span><span class="mi">30</span><span class="o">);</span>
</code></pre></div>
<h3 id="jdbctridentstate-for-lookup">JdbcTridentState for lookup</h3>
<p>We also support a trident query state that can be used with trident topologies. </p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">JdbcState</span><span class="o">.</span><span class="na">Options</span> <span class="n">options</span> <span class="o">=</span> <span class="k">new</span> <span class="n">JdbcState</span><span class="o">.</span><span class="na">Options</span><span class="o">()</span>
<span class="o">.</span><span class="na">withConnectionProvider</span><span class="o">(</span><span class="n">connectionProvider</span><span class="o">)</span>
<span class="o">.</span><span class="na">withJdbcLookupMapper</span><span class="o">(</span><span class="k">new</span> <span class="n">SimpleJdbcLookupMapper</span><span class="o">(</span><span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"user_name"</span><span class="o">),</span> <span class="n">Lists</span><span class="o">.</span><span class="na">newArrayList</span><span class="o">(</span><span class="k">new</span> <span class="n">Column</span><span class="o">(</span><span class="s">"user_id"</span><span class="o">,</span> <span class="n">Types</span><span class="o">.</span><span class="na">INTEGER</span><span class="o">))))</span>
<span class="o">.</span><span class="na">withSelectQuery</span><span class="o">(</span><span class="s">"select user_name from user_details where user_id = ?"</span><span class="o">);</span>
<span class="o">.</span><span class="na">withQueryTimeoutSecs</span><span class="o">(</span><span class="mi">30</span><span class="o">);</span>
</code></pre></div>
<h2 id="example">Example:</h2>
<p>A runnable example can be found in the <code>src/test/java/topology</code> directory.</p>
<h3 id="setup">Setup</h3>
<ul>
<li>Ensure you have included JDBC implementation dependency for your chosen database as part of your build configuration.</li>
<li>The test topologies executes the following queries so your intended DB must support these queries for test topologies
to work.
<code>SQL
create table if not exists user (user_id integer, user_name varchar(100), dept_name varchar(100), create_date date);
create table if not exists department (dept_id integer, dept_name varchar(100));
create table if not exists user_department (user_id integer, dept_id integer);
insert into department values (1, &#39;R&amp;D&#39;);
insert into department values (2, &#39;Finance&#39;);
insert into department values (3, &#39;HR&#39;);
insert into department values (4, &#39;Sales&#39;);
insert into user_department values (1, 1);
insert into user_department values (2, 2);
insert into user_department values (3, 3);
insert into user_department values (4, 4);
select dept_name from department, user_department where department.dept_id = user_department.dept_id and user_department.user_id = ?;
</code>
### Execution
Run the <code>org.apache.storm.jdbc.topology.UserPersistanceTopology</code> class using storm jar command. The class expects 5 args
storm jar org.apache.storm.jdbc.topology.UserPersistanceTopology <dataSourceClassName> <dataSource.url> <user> <password> [topology name]</li>
</ul>
<p>To make it work with Mysql, you can add the following to the pom.xml</p>
<div class="highlight"><pre><code class="language-" data-lang="">&lt;dependency&gt;
&lt;groupId&gt;mysql&lt;/groupId&gt;
&lt;artifactId&gt;mysql-connector-java&lt;/artifactId&gt;
&lt;version&gt;5.1.31&lt;/version&gt;
&lt;/dependency&gt;
</code></pre></div>
<p>You can generate a single jar with dependencies using mvn assembly plugin. To use the plugin add the following to your pom.xml and execute
<code>mvn clean compile assembly:single</code></p>
<div class="highlight"><pre><code class="language-" data-lang="">&lt;plugin&gt;
&lt;artifactId&gt;maven-assembly-plugin&lt;/artifactId&gt;
&lt;configuration&gt;
&lt;archive&gt;
&lt;manifest&gt;
&lt;mainClass&gt;fully.qualified.MainClass&lt;/mainClass&gt;
&lt;/manifest&gt;
&lt;/archive&gt;
&lt;descriptorRefs&gt;
&lt;descriptorRef&gt;jar-with-dependencies&lt;/descriptorRef&gt;
&lt;/descriptorRefs&gt;
&lt;/configuration&gt;
&lt;/plugin&gt;
</code></pre></div>
<p>Mysql Example:
<code>
storm jar ~/repo/incubator-storm/external/storm-jdbc/target/storm-jdbc-0.10.0-SNAPSHOT-jar-with-dependencies.jar org.apache.storm.jdbc.topology.UserPersistanceTopology com.mysql.jdbc.jdbc2.optional.MysqlDataSource jdbc:mysql://localhost/test root password UserPersistenceTopology
</code></p>
<p>You can execute a select query against the user table which should show newly inserted rows:</p>
<div class="highlight"><pre><code class="language-" data-lang="">select * from user;
</code></pre></div>
<p>For trident you can view <code>org.apache.storm.jdbc.topology.UserPersistanceTridentTopology</code>.</p>
</div>
</div>
</div>
<footer>
<div class="container-fluid">
<div class="row">
<div class="col-md-3">
<div class="footer-widget">
<h5>Meetups</h5>
<ul class="latest-news">
<li><a href="http://www.meetup.com/Apache-Storm-Apache-Kafka/">Apache Storm & Apache Kafka</a> <span class="small">(Sunnyvale, CA)</span></li>
<li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Apache Storm & Kafka Users</a> <span class="small">(Seattle, WA)</span></li>
<li><a href="http://www.meetup.com/New-York-City-Storm-User-Group/">NYC Storm User Group</a> <span class="small">(New York, NY)</span></li>
<li><a href="http://www.meetup.com/Bay-Area-Stream-Processing">Bay Area Stream Processing</a> <span class="small">(Emeryville, CA)</span></li>
<li><a href="http://www.meetup.com/Boston-Storm-Users/">Boston Realtime Data</a> <span class="small">(Boston, MA)</span></li>
<li><a href="http://www.meetup.com/storm-london">London Storm User Group</a> <span class="small">(London, UK)</span></li>
<!-- <li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Seatle, WA</a> <span class="small">(27 Jun 2015)</span></li> -->
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>About Storm</h5>
<p>Storm integrates with any queueing system and any database system. Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating Storm with database systems is easy.</p>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>First Look</h5>
<ul class="footer-list">
<li><a href="/documentation/Rationale.html">Rationale</a></li>
<li><a href="/tutorial.html">Tutorial</a></li>
<li><a href="/documentation/Setting-up-development-environment.html">Setting up development environment</a></li>
<li><a href="/documentation/Creating-a-new-Storm-project.html">Creating a new Storm project</a></li>
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>Documentation</h5>
<ul class="footer-list">
<li><a href="/doc-index.html">Index</a></li>
<li><a href="/documentation.html">Manual</a></li>
<li><a href="https://storm.apache.org/javadoc/apidocs/index.html">Javadoc</a></li>
<li><a href="/documentation/FAQ.html">FAQ</a></li>
</ul>
</div>
</div>
</div>
<hr/>
<div class="row">
<div class="col-md-12">
<p align="center">Copyright © 2015 <a href="http://www.apache.org">Apache Software Foundation</a>. All Rights Reserved.
<br>Apache Storm, Apache, the Apache feather logo, and the Apache Storm project logos are trademarks of The Apache Software Foundation.
<br>All other marks mentioned may be trademarks or registered trademarks of their respective owners.</p>
</div>
</div>
</div>
</footer>
<!--Footer End-->
<!-- Scroll to top -->
<span class="totop"><a href="#"><i class="fa fa-angle-up"></i></a></span>
</body>
</html>