blob: 2155e2eb558ba6503f7b2300998ca0ac67012f68 [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 HDFS 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-5">
<a href="/index.html"><img src="/images/logo.png" class="logo" /></a>
</div>
<div class="col-md-5">
<h1>Version: 2.1.0</h1>
</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 class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="documentation">Documentation <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/releases/2.3.0/index.html">2.3.0</a></li>
<li><a href="/releases/2.2.0/index.html">2.2.0</a></li>
<li><a href="/releases/2.1.0/index.html">2.1.0</a></li>
<li><a href="/releases/2.0.0/index.html">2.0.0</a></li>
<li><a href="/releases/1.2.3/index.html">1.2.3</a></li>
</ul>
</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="/2021/09/27/storm230-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">Storm HDFS Integration</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<div class="documentation-content"><p>Storm components for interacting with HDFS file systems</p>
<h1 id="hdfs-bolt">HDFS Bolt</h1>
<h2 id="usage">Usage</h2>
<p>The following example will write pipe(&quot;|&quot;)-delimited files to the HDFS path hdfs://localhost:54310/foo. After every
1,000 tuples it will sync filesystem, making that data visible to other HDFS clients. It will rotate files when they
reach 5 megabytes in size.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// use "|" instead of "," for field delimiter</span>
<span class="n">RecordFormat</span> <span class="n">format</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DelimitedRecordFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFieldDelimiter</span><span class="o">(</span><span class="s">"|"</span><span class="o">);</span>
<span class="c1">// sync the filesystem after every 1k tuples</span>
<span class="n">SyncPolicy</span> <span class="n">syncPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">CountSyncPolicy</span><span class="o">(</span><span class="mi">1000</span><span class="o">);</span>
<span class="c1">// rotate files when they reach 5MB</span>
<span class="n">FileRotationPolicy</span> <span class="n">rotationPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">FileSizeRotationPolicy</span><span class="o">(</span><span class="mf">5.0f</span><span class="o">,</span> <span class="n">Units</span><span class="o">.</span><span class="na">MB</span><span class="o">);</span>
<span class="n">FileNameFormat</span> <span class="n">fileNameFormat</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultFileNameFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withPath</span><span class="o">(</span><span class="s">"/foo/"</span><span class="o">);</span>
<span class="n">HdfsBolt</span> <span class="n">bolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HdfsBolt</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRecordFormat</span><span class="o">(</span><span class="n">format</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSyncPolicy</span><span class="o">(</span><span class="n">syncPolicy</span><span class="o">);</span>
</code></pre></div>
<h3 id="packaging-a-topology">Packaging a Topology</h3>
<p>When packaging your topology, it&#39;s important that you use the <a href="">maven-shade-plugin</a> as opposed to the
<a href="">maven-assembly-plugin</a>.</p>
<p>The shade plugin provides facilities for merging JAR manifest entries, which the hadoop client leverages for URL scheme
resolution.</p>
<p>If you experience errors such as the following:</p>
<div class="highlight"><pre><code class="language-" data-lang="">java.lang.RuntimeException: Error preparing HdfsBolt: No FileSystem for scheme: hdfs
</code></pre></div>
<p>it&#39;s an indication that your topology jar file isn&#39;t packaged properly.</p>
<p>If you are using maven to create your topology jar, you should use the following <code>maven-shade-plugin</code> configuration to
create your topology jar:</p>
<div class="highlight"><pre><code class="language-xml" data-lang="xml"><span class="nt">&lt;plugin&gt;</span>
<span class="nt">&lt;groupId&gt;</span>org.apache.maven.plugins<span class="nt">&lt;/groupId&gt;</span>
<span class="nt">&lt;artifactId&gt;</span>maven-shade-plugin<span class="nt">&lt;/artifactId&gt;</span>
<span class="nt">&lt;version&gt;</span>1.4<span class="nt">&lt;/version&gt;</span>
<span class="nt">&lt;configuration&gt;</span>
<span class="nt">&lt;createDependencyReducedPom&gt;</span>true<span class="nt">&lt;/createDependencyReducedPom&gt;</span>
<span class="nt">&lt;/configuration&gt;</span>
<span class="nt">&lt;executions&gt;</span>
<span class="nt">&lt;execution&gt;</span>
<span class="nt">&lt;phase&gt;</span>package<span class="nt">&lt;/phase&gt;</span>
<span class="nt">&lt;goals&gt;</span>
<span class="nt">&lt;goal&gt;</span>shade<span class="nt">&lt;/goal&gt;</span>
<span class="nt">&lt;/goals&gt;</span>
<span class="nt">&lt;configuration&gt;</span>
<span class="nt">&lt;transformers&gt;</span>
<span class="nt">&lt;transformer</span>
<span class="na">implementation=</span><span class="s">"org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"</span><span class="nt">/&gt;</span>
<span class="nt">&lt;transformer</span>
<span class="na">implementation=</span><span class="s">"org.apache.maven.plugins.shade.resource.ManifestResourceTransformer"</span><span class="nt">&gt;</span>
<span class="nt">&lt;mainClass&gt;&lt;/mainClass&gt;</span>
<span class="nt">&lt;/transformer&gt;</span>
<span class="nt">&lt;/transformers&gt;</span>
<span class="nt">&lt;/configuration&gt;</span>
<span class="nt">&lt;/execution&gt;</span>
<span class="nt">&lt;/executions&gt;</span>
<span class="nt">&lt;/plugin&gt;</span>
</code></pre></div>
<h3 id="specifying-a-hadoop-version">Specifying a Hadoop Version</h3>
<p>By default, storm-hdfs uses the following Hadoop dependencies:</p>
<div class="highlight"><pre><code class="language-xml" data-lang="xml"><span class="nt">&lt;dependency&gt;</span>
<span class="nt">&lt;groupId&gt;</span>org.apache.hadoop<span class="nt">&lt;/groupId&gt;</span>
<span class="nt">&lt;artifactId&gt;</span>hadoop-client<span class="nt">&lt;/artifactId&gt;</span>
<span class="nt">&lt;version&gt;</span>2.6.1<span class="nt">&lt;/version&gt;</span>
<span class="nt">&lt;exclusions&gt;</span>
<span class="nt">&lt;exclusion&gt;</span>
<span class="nt">&lt;groupId&gt;</span>org.slf4j<span class="nt">&lt;/groupId&gt;</span>
<span class="nt">&lt;artifactId&gt;</span>slf4j-log4j12<span class="nt">&lt;/artifactId&gt;</span>
<span class="nt">&lt;/exclusion&gt;</span>
<span class="nt">&lt;/exclusions&gt;</span>
<span class="nt">&lt;/dependency&gt;</span>
<span class="nt">&lt;dependency&gt;</span>
<span class="nt">&lt;groupId&gt;</span>org.apache.hadoop<span class="nt">&lt;/groupId&gt;</span>
<span class="nt">&lt;artifactId&gt;</span>hadoop-hdfs<span class="nt">&lt;/artifactId&gt;</span>
<span class="nt">&lt;version&gt;</span>2.6.1<span class="nt">&lt;/version&gt;</span>
<span class="nt">&lt;exclusions&gt;</span>
<span class="nt">&lt;exclusion&gt;</span>
<span class="nt">&lt;groupId&gt;</span>org.slf4j<span class="nt">&lt;/groupId&gt;</span>
<span class="nt">&lt;artifactId&gt;</span>slf4j-log4j12<span class="nt">&lt;/artifactId&gt;</span>
<span class="nt">&lt;/exclusion&gt;</span>
<span class="nt">&lt;/exclusions&gt;</span>
<span class="nt">&lt;/dependency&gt;</span>
</code></pre></div>
<p>If you are using a different version of Hadoop, you should exclude the Hadoop libraries from the storm-hdfs dependency
and add the dependencies for your preferred version in your pom.</p>
<p>Hadoop client version incompatibilites can manifest as errors like:</p>
<div class="highlight"><pre><code class="language-" data-lang="">com.google.protobuf.InvalidProtocolBufferException: Protocol message contained an invalid tag (zero)
</code></pre></div>
<h2 id="hdfs-bolt-customization">HDFS Bolt Customization</h2>
<h3 id="record-formats">Record Formats</h3>
<p>Record format can be controlled by providing an implementation of the <code>org.apache.storm.hdfs.format.RecordFormat</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">RecordFormat</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="kt">byte</span><span class="o">[]</span> <span class="nf">format</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">);</span>
<span class="o">}</span>
</code></pre></div>
<p>The provided <code>org.apache.storm.hdfs.format.DelimitedRecordFormat</code> is capable of producing formats such as CSV and
tab-delimited files.</p>
<h3 id="file-naming">File Naming</h3>
<p>File naming can be controlled by providing an implementation of the <code>org.apache.storm.hdfs.format.FileNameFormat</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">FileNameFormat</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="kt">void</span> <span class="nf">prepare</span><span class="o">(</span><span class="n">Map</span> <span class="n">conf</span><span class="o">,</span> <span class="n">TopologyContext</span> <span class="n">topologyContext</span><span class="o">);</span>
<span class="n">String</span> <span class="nf">getName</span><span class="o">(</span><span class="kt">long</span> <span class="n">rotation</span><span class="o">,</span> <span class="kt">long</span> <span class="n">timeStamp</span><span class="o">);</span>
<span class="n">String</span> <span class="nf">getPath</span><span class="o">();</span>
<span class="o">}</span>
</code></pre></div>
<p>The provided <code>org.apache.storm.hdfs.format.DefaultFileNameFormat</code> will create file names with the following format:</p>
<div class="highlight"><pre><code class="language-" data-lang=""> {prefix}{componentId}-{taskId}-{rotationNum}-{timestamp}{extension}
</code></pre></div>
<p>For example:</p>
<div class="highlight"><pre><code class="language-" data-lang=""> MyBolt-5-7-1390579837830.txt
</code></pre></div>
<p>By default, prefix is empty and extenstion is &quot;.txt&quot;.</p>
<p><strong>New FileNameFormat:</strong></p>
<p>The new provided <code>org.apache.storm.hdfs.format.SimpleFileNameFormat</code> and <code>org.apache.storm.hdfs.trident.format.SimpleFileNameFormat</code> are more flexible, and the <code>withName</code> method support parameters as following:</p>
<ul>
<li>$TIME - current time. use <code>withTimeFormat</code> to format.</li>
<li>$NUM - rotation number</li>
<li>$HOST - local host name</li>
<li>$PARTITION - partition index (<code>org.apache.storm.hdfs.trident.format.SimpleFileNameFormat</code> only)</li>
<li>$COMPONENT - component id (<code>org.apache.storm.hdfs.format.SimpleFileNameFormat</code> only)</li>
<li>$TASK - task id (<code>org.apache.storm.hdfs.format.SimpleFileNameFormat</code> only)</li>
</ul>
<p>eg: <code>seq.$TIME.$HOST.$COMPONENT.$NUM.dat</code></p>
<p>The default file <code>name</code> is <code>$TIME.$NUM.txt</code>, and the default <code>timeFormat</code> is <code>yyyyMMddHHmmss</code>.</p>
<h3 id="sync-policies">Sync Policies</h3>
<p>Sync policies allow you to control when buffered data is flushed to the underlying filesystem (thus making it available
to clients reading the data) by implementing the <code>org.apache.storm.hdfs.sync.SyncPolicy</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">SyncPolicy</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="kt">boolean</span> <span class="nf">mark</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">,</span> <span class="kt">long</span> <span class="n">offset</span><span class="o">);</span>
<span class="kt">void</span> <span class="nf">reset</span><span class="o">();</span>
<span class="o">}</span>
</code></pre></div>
<p>The <code>HdfsBolt</code> will call the <code>mark()</code> method for every tuple it processes. Returning <code>true</code> will trigger the <code>HdfsBolt</code>
to perform a sync/flush, after which it will call the <code>reset()</code> method.</p>
<p>The <code>org.apache.storm.hdfs.sync.CountSyncPolicy</code> class simply triggers a sync after the specified number of tuples have
been processed.</p>
<h3 id="file-rotation-policies">File Rotation Policies</h3>
<p>Similar to sync policies, file rotation policies allow you to control when data files are rotated by providing a
<code>org.apache.storm.hdfs.rotation.FileRotation</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">FileRotationPolicy</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="kt">boolean</span> <span class="nf">mark</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">,</span> <span class="kt">long</span> <span class="n">offset</span><span class="o">);</span>
<span class="kt">void</span> <span class="nf">reset</span><span class="o">();</span>
<span class="n">FileRotationPolicy</span> <span class="nf">copy</span><span class="o">();</span>
<span class="o">}</span>
</code></pre></div>
<p>The <code>org.apache.storm.hdfs.rotation.FileSizeRotationPolicy</code> implementation allows you to trigger file rotation when
data files reach a specific file size:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="n">FileRotationPolicy</span> <span class="n">rotationPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">FileSizeRotationPolicy</span><span class="o">(</span><span class="mf">5.0f</span><span class="o">,</span> <span class="n">Units</span><span class="o">.</span><span class="na">MB</span><span class="o">);</span>
</code></pre></div>
<h3 id="file-rotation-actions">File Rotation Actions</h3>
<p>Both the HDFS bolt and Trident State implementation allow you to register any number of <code>RotationAction</code>s.
What <code>RotationAction</code>s do is provide a hook to allow you to perform some action right after a file is rotated. For
example, moving a file to a different location or renaming it.</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">RotationAction</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">FileSystem</span> <span class="n">fileSystem</span><span class="o">,</span> <span class="n">Path</span> <span class="n">filePath</span><span class="o">)</span> <span class="kd">throws</span> <span class="n">IOException</span><span class="o">;</span>
<span class="o">}</span>
</code></pre></div>
<p>Storm-HDFS includes a simple action that will move a file after rotation:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="kd">public</span> <span class="kd">class</span> <span class="nc">MoveFileAction</span> <span class="kd">implements</span> <span class="n">RotationAction</span> <span class="o">{</span>
<span class="kd">private</span> <span class="kd">static</span> <span class="kd">final</span> <span class="n">Logger</span> <span class="n">LOG</span> <span class="o">=</span> <span class="n">LoggerFactory</span><span class="o">.</span><span class="na">getLogger</span><span class="o">(</span><span class="n">MoveFileAction</span><span class="o">.</span><span class="na">class</span><span class="o">);</span>
<span class="kd">private</span> <span class="n">String</span> <span class="n">destination</span><span class="o">;</span>
<span class="kd">public</span> <span class="n">MoveFileAction</span> <span class="nf">withDestination</span><span class="o">(</span><span class="n">String</span> <span class="n">destDir</span><span class="o">){</span>
<span class="n">destination</span> <span class="o">=</span> <span class="n">destDir</span><span class="o">;</span>
<span class="k">return</span> <span class="k">this</span><span class="o">;</span>
<span class="o">}</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">execute</span><span class="o">(</span><span class="n">FileSystem</span> <span class="n">fileSystem</span><span class="o">,</span> <span class="n">Path</span> <span class="n">filePath</span><span class="o">)</span> <span class="kd">throws</span> <span class="n">IOException</span> <span class="o">{</span>
<span class="n">Path</span> <span class="n">destPath</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Path</span><span class="o">(</span><span class="n">destination</span><span class="o">,</span> <span class="n">filePath</span><span class="o">.</span><span class="na">getName</span><span class="o">());</span>
<span class="n">LOG</span><span class="o">.</span><span class="na">info</span><span class="o">(</span><span class="s">"Moving file {} to {}"</span><span class="o">,</span> <span class="n">filePath</span><span class="o">,</span> <span class="n">destPath</span><span class="o">);</span>
<span class="kt">boolean</span> <span class="n">success</span> <span class="o">=</span> <span class="n">fileSystem</span><span class="o">.</span><span class="na">rename</span><span class="o">(</span><span class="n">filePath</span><span class="o">,</span> <span class="n">destPath</span><span class="o">);</span>
<span class="k">return</span><span class="o">;</span>
<span class="o">}</span>
<span class="o">}</span>
</code></pre></div>
<p>If you are using Trident and sequence files you can do something like this:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="n">HdfsState</span><span class="o">.</span><span class="na">Options</span> <span class="n">seqOpts</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HdfsState</span><span class="o">.</span><span class="na">SequenceFileOptions</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSequenceFormat</span><span class="o">(</span><span class="k">new</span> <span class="n">DefaultSequenceFormat</span><span class="o">(</span><span class="s">"key"</span><span class="o">,</span> <span class="s">"data"</span><span class="o">))</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span>
<span class="o">.</span><span class="na">addRotationAction</span><span class="o">(</span><span class="k">new</span> <span class="n">MoveFileAction</span><span class="o">().</span><span class="na">withDestination</span><span class="o">(</span><span class="s">"/dest2/"</span><span class="o">));</span>
</code></pre></div>
<h3 id="data-partitioning">Data Partitioning</h3>
<p>Data can be partitioned to different HDFS directories based on characteristics of the tuple being processed or purely
external factors, such as system time. To partition your your data, write a class that implements the <code>Partitioner</code>
interface and pass it to the withPartitioner() method of your bolt. The getPartitionPath() method returns a partition
path for a given tuple.</p>
<p>Here&#39;s an example of a Partitioner that operates on a specific field of data:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java">
<span class="n">Partitioner</span> <span class="n">partitoner</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Partitioner</span><span class="o">()</span> <span class="o">{</span>
<span class="nd">@Override</span>
<span class="kd">public</span> <span class="n">String</span> <span class="nf">getPartitionPath</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">)</span> <span class="o">{</span>
<span class="k">return</span> <span class="n">Path</span><span class="o">.</span><span class="na">SEPARATOR</span> <span class="o">+</span> <span class="n">tuple</span><span class="o">.</span><span class="na">getStringByField</span><span class="o">(</span><span class="s">"city"</span><span class="o">);</span>
<span class="o">}</span>
<span class="o">};</span>
</code></pre></div>
<h2 id="hdfs-bolt-support-for-hdfs-sequence-files">HDFS Bolt Support for HDFS Sequence Files</h2>
<p>The <code>org.apache.storm.hdfs.bolt.SequenceFileBolt</code> class allows you to write storm data to HDFS sequence files:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="c1">// sync the filesystem after every 1k tuples</span>
<span class="n">SyncPolicy</span> <span class="n">syncPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">CountSyncPolicy</span><span class="o">(</span><span class="mi">1000</span><span class="o">);</span>
<span class="c1">// rotate files when they reach 5MB</span>
<span class="n">FileRotationPolicy</span> <span class="n">rotationPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">FileSizeRotationPolicy</span><span class="o">(</span><span class="mf">5.0f</span><span class="o">,</span> <span class="n">Units</span><span class="o">.</span><span class="na">MB</span><span class="o">);</span>
<span class="n">FileNameFormat</span> <span class="n">fileNameFormat</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultFileNameFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withExtension</span><span class="o">(</span><span class="s">".seq"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withPath</span><span class="o">(</span><span class="s">"/data/"</span><span class="o">);</span>
<span class="c1">// create sequence format instance.</span>
<span class="n">DefaultSequenceFormat</span> <span class="n">format</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultSequenceFormat</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"sentence"</span><span class="o">);</span>
<span class="n">SequenceFileBolt</span> <span class="n">bolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">SequenceFileBolt</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSequenceFormat</span><span class="o">(</span><span class="n">format</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSyncPolicy</span><span class="o">(</span><span class="n">syncPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withCompressionType</span><span class="o">(</span><span class="n">SequenceFile</span><span class="o">.</span><span class="na">CompressionType</span><span class="o">.</span><span class="na">RECORD</span><span class="o">)</span>
<span class="o">.</span><span class="na">withCompressionCodec</span><span class="o">(</span><span class="s">"deflate"</span><span class="o">);</span>
</code></pre></div>
<p>The <code>SequenceFileBolt</code> requires that you provide a <code>org.apache.storm.hdfs.bolt.format.SequenceFormat</code> that maps tuples to
key/value pairs:</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">SequenceFormat</span> <span class="kd">extends</span> <span class="n">Serializable</span> <span class="o">{</span>
<span class="n">Class</span> <span class="nf">keyClass</span><span class="o">();</span>
<span class="n">Class</span> <span class="nf">valueClass</span><span class="o">();</span>
<span class="n">Writable</span> <span class="nf">key</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">);</span>
<span class="n">Writable</span> <span class="nf">value</span><span class="o">(</span><span class="n">Tuple</span> <span class="n">tuple</span><span class="o">);</span>
<span class="o">}</span>
</code></pre></div>
<h2 id="hdfs-bolt-support-for-avro-files">HDFS Bolt Support for Avro Files</h2>
<p>The <code>org.apache.storm.hdfs.bolt.AvroGenericRecordBolt</code> class allows you to write Avro objects directly to HDFS:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="c1">// sync the filesystem after every 1k tuples</span>
<span class="n">SyncPolicy</span> <span class="n">syncPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">CountSyncPolicy</span><span class="o">(</span><span class="mi">1000</span><span class="o">);</span>
<span class="c1">// rotate files when they reach 5MB</span>
<span class="n">FileRotationPolicy</span> <span class="n">rotationPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">FileSizeRotationPolicy</span><span class="o">(</span><span class="mf">5.0f</span><span class="o">,</span> <span class="n">Units</span><span class="o">.</span><span class="na">MB</span><span class="o">);</span>
<span class="n">FileNameFormat</span> <span class="n">fileNameFormat</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultFileNameFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withExtension</span><span class="o">(</span><span class="s">".avro"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withPath</span><span class="o">(</span><span class="s">"/data/"</span><span class="o">);</span>
<span class="c1">// create sequence format instance.</span>
<span class="n">DefaultSequenceFormat</span> <span class="n">format</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultSequenceFormat</span><span class="o">(</span><span class="s">"timestamp"</span><span class="o">,</span> <span class="s">"sentence"</span><span class="o">);</span>
<span class="n">AvroGenericRecordBolt</span> <span class="n">bolt</span> <span class="o">=</span> <span class="k">new</span> <span class="n">AvroGenericRecordBolt</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSyncPolicy</span><span class="o">(</span><span class="n">syncPolicy</span><span class="o">);</span>
</code></pre></div>
<p>The avro bolt will write records to separate files based on the schema of the record being processed. In other words,
if the bolt receives records with two different schemas, it will write to two separate files. Each file will be rotatated
in accordance with the specified rotation policy. If a large number of Avro schemas are expected, then the bolt should
be configured with a maximum number of open files at least equal to the number of schemas expected to prevent excessive
file open/close/create operations.</p>
<p>To use this bolt you <strong>must</strong> register the appropriate Kryo serializers with your topology configuration. A convenience
method is provided for this:</p>
<p><code>AvroUtils.addAvroKryoSerializations(conf);</code></p>
<p>By default Storm will use the <code>GenericAvroSerializer</code> to handle serialization. This will work, but there are much
faster options available if you can pre-define the schemas you will be using or utilize an external schema registry. An
implementation using the Confluent Schema Registry is provided, but others can be implemented and provided to Storm.
Please see the javadoc for classes in org.apache.storm.hdfs.avro for information about using the built-in options or
creating your own.</p>
<h2 id="hdfs-bolt-support-for-trident-api">HDFS Bolt support for Trident API</h2>
<p>storm-hdfs also includes a Trident <code>state</code> implementation for writing data to HDFS, with an API that closely mirrors
that of the bolts.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="n">Fields</span> <span class="n">hdfsFields</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Fields</span><span class="o">(</span><span class="s">"field1"</span><span class="o">,</span> <span class="s">"field2"</span><span class="o">);</span>
<span class="n">FileNameFormat</span> <span class="n">fileNameFormat</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DefaultFileNameFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withPath</span><span class="o">(</span><span class="s">"/trident"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withPrefix</span><span class="o">(</span><span class="s">"trident"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withExtension</span><span class="o">(</span><span class="s">".txt"</span><span class="o">);</span>
<span class="n">RecordFormat</span> <span class="n">recordFormat</span> <span class="o">=</span> <span class="k">new</span> <span class="n">DelimitedRecordFormat</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFields</span><span class="o">(</span><span class="n">hdfsFields</span><span class="o">);</span>
<span class="n">FileRotationPolicy</span> <span class="n">rotationPolicy</span> <span class="o">=</span> <span class="k">new</span> <span class="n">FileSizeRotationPolicy</span><span class="o">(</span><span class="mf">5.0f</span><span class="o">,</span> <span class="n">FileSizeRotationPolicy</span><span class="o">.</span><span class="na">Units</span><span class="o">.</span><span class="na">MB</span><span class="o">);</span>
<span class="n">HdfsState</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">HdfsState</span><span class="o">.</span><span class="na">HdfsFileOptions</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRecordFormat</span><span class="o">(</span><span class="n">recordFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">);</span>
<span class="n">StateFactory</span> <span class="n">factory</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HdfsStateFactory</span><span class="o">().</span><span class="na">withOptions</span><span class="o">(</span><span class="n">options</span><span class="o">);</span>
<span class="n">TridentState</span> <span class="n">state</span> <span class="o">=</span> <span class="n">stream</span>
<span class="o">.</span><span class="na">partitionPersist</span><span class="o">(</span><span class="n">factory</span><span class="o">,</span> <span class="n">hdfsFields</span><span class="o">,</span> <span class="k">new</span> <span class="n">HdfsUpdater</span><span class="o">(),</span> <span class="k">new</span> <span class="n">Fields</span><span class="o">());</span>
</code></pre></div>
<p>To use the sequence file <code>State</code> implementation, use the <code>HdfsState.SequenceFileOptions</code>:</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"> <span class="n">HdfsState</span><span class="o">.</span><span class="na">Options</span> <span class="n">seqOpts</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HdfsState</span><span class="o">.</span><span class="na">SequenceFileOptions</span><span class="o">()</span>
<span class="o">.</span><span class="na">withFileNameFormat</span><span class="o">(</span><span class="n">fileNameFormat</span><span class="o">)</span>
<span class="o">.</span><span class="na">withSequenceFormat</span><span class="o">(</span><span class="k">new</span> <span class="n">DefaultSequenceFormat</span><span class="o">(</span><span class="s">"key"</span><span class="o">,</span> <span class="s">"data"</span><span class="o">))</span>
<span class="o">.</span><span class="na">withRotationPolicy</span><span class="o">(</span><span class="n">rotationPolicy</span><span class="o">)</span>
<span class="o">.</span><span class="na">withFsUrl</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span>
<span class="o">.</span><span class="na">addRotationAction</span><span class="o">(</span><span class="k">new</span> <span class="n">MoveFileAction</span><span class="o">().</span><span class="na">toDestination</span><span class="o">(</span><span class="s">"/dest2/"</span><span class="o">));</span>
</code></pre></div>
<h3 id="note">Note</h3>
<p>Whenever a batch is replayed by storm (due to failures), the trident state implementation automatically removes
duplicates from the current data file by copying the data up to the last transaction to another file. Since this
operation involves a lot of data copy, ensure that the data files are rotated at reasonable sizes with <code>FileSizeRotationPolicy</code>
and at reasonable intervals with <code>TimedRotationPolicy</code> so that the recovery can complete within topology.message.timeout.secs.</p>
<p>Also note with <code>TimedRotationPolicy</code> the files are never rotated in the middle of a batch even if the timer ticks,
but only when a batch completes so that complete batches can be efficiently recovered in case of failures.</p>
<h2 id="working-with-secure-hdfs">Working with Secure HDFS</h2>
<p>If your topology is going to interact with secure HDFS, your bolts/states needs to be authenticated by NameNode. We
currently have 2 options to support this:</p>
<h3 id="using-hdfs-delegation-tokens">Using HDFS delegation tokens</h3>
<p>Your administrator can configure nimbus to automatically get delegation tokens on behalf of the topology submitter user. The nimbus should be started with following configurations:</p>
<div class="highlight"><pre><code class="language-" data-lang="">nimbus.autocredential.plugins.classes : ["org.apache.storm.hdfs.security.AutoHDFS"]
nimbus.credential.renewers.classes : ["org.apache.storm.hdfs.security.AutoHDFS"]
hdfs.keytab.file: "/path/to/keytab/on/nimbus" (This is the keytab of hdfs super user that can impersonate other users.)
hdfs.kerberos.principal: "superuser@EXAMPLE.com"
nimbus.credential.renewers.freq.secs : 82800 (23 hours, hdfs tokens needs to be renewed every 24 hours so this value should be less then 24 hours.)
topology.hdfs.uri:"hdfs://host:port" (This is an optional config, by default we will use value of "fs.defaultFS" property specified in hadoop's core-site.xml)
</code></pre></div>
<p>Your topology configuration should have:</p>
<div class="highlight"><pre><code class="language-" data-lang="">topology.auto-credentials :["org.apache.storm.hdfs.common.security.AutoHDFS"]
</code></pre></div>
<p>If nimbus did not have the above configuration you need to add and then restart it. Ensure the hadoop configuration
files (core-site.xml and hdfs-site.xml) and the storm-hdfs jar with all the dependencies is present in nimbus&#39;s classpath.</p>
<p>As an alternative to adding the configuration files (core-site.xml and hdfs-site.xml) to the classpath, you could specify the configurations
as a part of the topology configuration. E.g. in you custom storm.yaml (or -c option while submitting the topology),</p>
<div class="highlight"><pre><code class="language-" data-lang="">hdfsCredentialsConfigKeys : ["cluster1", "cluster2"] (the hdfs clusters you want to fetch the tokens from)
"cluster1": {"config1": "value1", "config2": "value2", ... } (A map of config key-values specific to cluster1)
"cluster2": {"config1": "value1", "hdfs.keytab.file": "/path/to/keytab/for/cluster2/on/nimubs", "hdfs.kerberos.principal": "cluster2user@EXAMPLE.com"} (here along with other configs, we have custom keytab and principal for "cluster2" which will override the keytab/principal specified at topology level)
</code></pre></div>
<p>Instead of specifying key values you may also directly specify the resource files for e.g.,</p>
<div class="highlight"><pre><code class="language-" data-lang="">"cluster1": {"resources": ["/path/to/core-site1.xml", "/path/to/hdfs-site1.xml"]}
"cluster2": {"resources": ["/path/to/core-site2.xml", "/path/to/hdfs-site2.xml"]}
</code></pre></div>
<p>Storm will download the tokens separately for each of the clusters and populate it into the subject and also renew the tokens periodically. This way it would be possible to run multiple bolts connecting to separate HDFS cluster within the same topology.</p>
<p>Nimbus will use the keytab and principal specified in the config to authenticate with Namenode. From then on for every
topology submission, nimbus will impersonate the topology submitter user and acquire delegation tokens on behalf of the
topology submitter user. If topology was started with topology.auto-credentials set to AutoHDFS, nimbus will push the
delegation tokens to all the workers for your topology and the hdfs bolt/state will authenticate with namenode using
these tokens.</p>
<p>As nimbus is impersonating topology submitter user, you need to ensure the user specified in hdfs.kerberos.principal
has permissions to acquire tokens on behalf of other users. To achieve this you need to follow configuration directions
listed on this link
<a href="http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Superusers.html">http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/Superusers.html</a></p>
<p>You can read about setting up secure HDFS here: <a href="http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/SecureMode.html">http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/SecureMode.html</a>.</p>
<h3 id="using-keytabs-on-all-worker-hosts">Using keytabs on all worker hosts</h3>
<p>If you have distributed the keytab files for hdfs user on all potential worker hosts then you can use this method. You should specify a
hdfs config key using the method HdfsBolt/State.withconfigKey(&quot;somekey&quot;) and the value map of this key should have following 2 properties:</p>
<p>hdfs.keytab.file: &quot;/path/to/keytab/&quot;
hdfs.kerberos.principal: &quot;<a href="mailto:user@EXAMPLE.com">user@EXAMPLE.com</a>&quot;</p>
<p>On worker hosts the bolt/trident-state code will use the keytab file with principal provided in the config to authenticate with
Namenode. This method is little dangerous as you need to ensure all workers have the keytab file at the same location and you need
to remember this as you bring up new hosts in the cluster.</p>
<hr>
<h1 id="hdfs-spout">HDFS Spout</h1>
<p>Hdfs spout is intended to allow feeding data into Storm from a HDFS directory.
It will actively monitor the directory to consume any new files that appear in the directory.
HDFS spout does not support Trident currently.</p>
<p><strong>Impt</strong>: Hdfs spout assumes that the files being made visible to it in the monitored directory
are NOT actively being written to. Only after a file is completely written should it be made
visible to the spout. This can be achieved by either writing the files out to another directory
and once completely written, move it to the monitored directory. Alternatively the file
can be created with a &#39;.ignore&#39; suffix in the monitored directory and after data is completely
written, rename it without the suffix. File names with a &#39;.ignore&#39; suffix are ignored
by the spout.</p>
<p>When the spout is actively consuming a file, it renames the file with a &#39;.inprogress&#39; suffix.
After consuming all the contents in the file, the file will be moved to a configurable <em>done</em>
directory and the &#39;.inprogress&#39; suffix will be dropped.</p>
<p><strong>Concurrency</strong> If multiple spout instances are used in the topology, each instance will consume
a different file. Synchronization among spout instances is done using lock files created in a
(by default) &#39;.lock&#39; subdirectory under the monitored directory. A file with the same name
as the file being consumed (without the in progress suffix) is created in the lock directory.
Once the file is completely consumed, the corresponding lock file is deleted.</p>
<p><strong>Recovery from failure</strong>
Periodically, the spout also records progress information wrt to how much of the file has been
consumed in the lock file. In case of an crash of the spout instance (or force kill of topology)
another spout can take over the file and resume from the location recorded in the lock file.</p>
<p>Certain error conditions (such spout crashing) can leave behind lock files without deleting them.
Such a stale lock file also indicates that the corresponding input file has also not been completely
processed. When detected, ownership of such stale lock files will be transferred to another spout.<br>
The configuration &#39;hdfsspout.lock.timeout.sec&#39; is used to specify the duration of inactivity after
which lock files should be considered stale. For lock file ownership transfer to succeed, the HDFS
lease on the file (from prev lock owner) should have expired. Spouts scan for stale lock files
before selecting the next file for consumption.</p>
<p><strong>Lock on <em>.lock</em> Directory</strong>
Hdfs spout instances create a <em>DIRLOCK</em> file in the .lock directory to co-ordinate certain accesses to
the .lock dir itself. A spout will try to create it when it needs access to the .lock directory and
then delete it when done. In error conditions such as a topology crash, force kill or untimely death
of a spout, this file may not get deleted. Future running instances of the spout will eventually recover
this once the DIRLOCK file becomes stale due to inactivity for hdfsspout.lock.timeout.sec seconds.</p>
<h2 id="usage">Usage</h2>
<p>The following example creates an HDFS spout that reads text files from HDFS path hdfs://localhost:54310/source.</p>
<div class="highlight"><pre><code class="language-java" data-lang="java"><span class="c1">// Instantiate spout to read text files</span>
<span class="n">HdfsSpout</span> <span class="n">textReaderSpout</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HdfsSpout</span><span class="o">().</span><span class="na">setReaderType</span><span class="o">(</span><span class="s">"text"</span><span class="o">)</span>
<span class="o">.</span><span class="na">withOutputFields</span><span class="o">(</span><span class="n">TextFileReader</span><span class="o">.</span><span class="na">defaultFields</span><span class="o">)</span>
<span class="o">.</span><span class="na">setHdfsUri</span><span class="o">(</span><span class="s">"hdfs://localhost:54310"</span><span class="o">)</span> <span class="c1">// required</span>
<span class="o">.</span><span class="na">setSourceDir</span><span class="o">(</span><span class="s">"/data/in"</span><span class="o">)</span> <span class="c1">// required </span>
<span class="o">.</span><span class="na">setArchiveDir</span><span class="o">(</span><span class="s">"/data/done"</span><span class="o">)</span> <span class="c1">// required</span>
<span class="o">.</span><span class="na">setBadFilesDir</span><span class="o">(</span><span class="s">"/data/badfiles"</span><span class="o">);</span> <span class="c1">// required </span>
<span class="c1">// If using Kerberos</span>
<span class="n">HashMap</span> <span class="n">hdfsSettings</span> <span class="o">=</span> <span class="k">new</span> <span class="n">HashMap</span><span class="o">();</span>
<span class="n">hdfsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"hdfs.keytab.file"</span><span class="o">,</span> <span class="s">"/path/to/keytab"</span><span class="o">);</span>
<span class="n">hdfsSettings</span><span class="o">.</span><span class="na">put</span><span class="o">(</span><span class="s">"hdfs.kerberos.principal"</span><span class="o">,</span><span class="s">"user@EXAMPLE.com"</span><span class="o">);</span>
<span class="n">textReaderSpout</span><span class="o">.</span><span class="na">setHdfsClientSettings</span><span class="o">(</span><span class="n">hdfsSettings</span><span class="o">);</span>
<span class="c1">// Create topology</span>
<span class="n">TopologyBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">TopologyBuilder</span><span class="o">();</span>
<span class="n">builder</span><span class="o">.</span><span class="na">setSpout</span><span class="o">(</span><span class="s">"hdfsspout"</span><span class="o">,</span> <span class="n">textReaderSpout</span><span class="o">,</span> <span class="n">SPOUT_NUM</span><span class="o">);</span>
<span class="c1">// Setup bolts and wire up topology</span>
<span class="o">..</span><span class="na">snip</span><span class="o">..</span>
<span class="c1">// Submit topology with config</span>
<span class="n">Config</span> <span class="n">conf</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Config</span><span class="o">();</span>
<span class="n">StormSubmitter</span><span class="o">.</span><span class="na">submitTopologyWithProgressBar</span><span class="o">(</span><span class="s">"topologyName"</span><span class="o">,</span> <span class="n">conf</span><span class="o">,</span> <span class="n">builder</span><span class="o">.</span><span class="na">createTopology</span><span class="o">());</span>
</code></pre></div>
<p>A sample topology HdfsSpoutTopology is provided in storm-starter module.</p>
<h2 id="configuration-settings">Configuration Settings</h2>
<p>Below is a list of HdfsSpout member functions used for configuration. The equivalent config is also possible via Config object passed in during submitting topology.
However, the later mechanism is deprecated as it does not allow multiple Hdfs spouts with differing settings. : </p>
<p>Only methods mentioned in <strong>bold</strong> are required.</p>
<table><thead>
<tr>
<th>Method</th>
<th>Alternative config name (deprecated)</th>
<th>Default</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td><strong>.setReaderType()</strong></td>
<td>~~hdfsspout.reader.type~~</td>
<td></td>
<td>Determines which file reader to use. Set to &#39;seq&#39; for reading sequence files or &#39;text&#39; for text files. Set to a fully qualified class name if using a custom file reader class (that implements interface org.apache.storm.hdfs.spout.FileReader)</td>
</tr>
<tr>
<td><strong>.withOutputFields()</strong></td>
<td></td>
<td></td>
<td>Sets the names for the output fields for the spout. The number of fields depends upon the reader being used. For convenience, built-in reader types expose a static member called <code>defaultFields</code> that can be used for setting this.</td>
</tr>
<tr>
<td><strong>.setHdfsUri()</strong></td>
<td>~~hdfsspout.hdfs~~</td>
<td></td>
<td>HDFS URI for the hdfs Name node. Example: hdfs://namenodehost:8020</td>
</tr>
<tr>
<td><strong>.setSourceDir()</strong></td>
<td>~~hdfsspout.source.dir~~</td>
<td></td>
<td>HDFS directory from where to read files. E.g. /data/inputdir</td>
</tr>
<tr>
<td><strong>.setArchiveDir()</strong></td>
<td>~~hdfsspout.archive.dir~~</td>
<td></td>
<td>After a file is processed completely it will be moved to this HDFS directory. If this directory does not exist it will be created. E.g. /data/done</td>
</tr>
<tr>
<td><strong>.setBadFilesDir()</strong></td>
<td>~~hdfsspout.badfiles.dir~~</td>
<td></td>
<td>if there is an error parsing a file&#39;s contents, the file is moved to this location. If this directory does not exist it will be created. E.g. /data/badfiles</td>
</tr>
<tr>
<td>.setLockDir()</td>
<td>~~hdfsspout.lock.dir~~</td>
<td>&#39;.lock&#39; subdirectory under hdfsspout.source.dir</td>
<td>Dir in which lock files will be created. Concurrent HDFS spout instances synchronize using <em>lock</em> files. Before processing a file the spout instance creates a lock file in this directory with same name as input file and deletes this lock file after processing the file. Spouts also periodically makes a note of their progress (wrt reading the input file) in the lock file so that another spout instance can resume progress on the same file if the spout dies for any reason.</td>
</tr>
<tr>
<td>.setIgnoreSuffix()</td>
<td>~~hdfsspout.ignore.suffix~~</td>
<td>.ignore</td>
<td>File names with this suffix in the in the hdfsspout.source.dir location will not be processed</td>
</tr>
<tr>
<td>.setCommitFrequencyCount()</td>
<td>~~hdfsspout.commit.count~~</td>
<td>20000</td>
<td>Record progress in the lock file after these many records are processed. If set to 0, this criterion will not be used.</td>
</tr>
<tr>
<td>.setCommitFrequencySec()</td>
<td>~~hdfsspout.commit.sec~~</td>
<td>10</td>
<td>Record progress in the lock file after these many seconds have elapsed. Must be greater than 0</td>
</tr>
<tr>
<td>.setMaxOutstanding()</td>
<td>~~hdfsspout.max.outstanding~~</td>
<td>10000</td>
<td>Limits the number of unACKed tuples by pausing tuple generation (if ACKers are used in the topology)</td>
</tr>
<tr>
<td>.setLockTimeoutSec()</td>
<td>~~hdfsspout.lock.timeout.sec~~</td>
<td>5 minutes</td>
<td>Duration of inactivity after which a lock file is considered to be abandoned and ready for another spout to take ownership</td>
</tr>
<tr>
<td>.setClocksInSync()</td>
<td>~~hdfsspout.clocks.insync~~</td>
<td>true</td>
<td>Indicates whether clocks on the storm machines are in sync (using services like NTP). Used for detecting stale locks.</td>
</tr>
<tr>
<td>.withConfigKey()</td>
<td></td>
<td></td>
<td>Optional setting. Overrides the default key name (&#39;hdfs.config&#39;, see below) used for specifying HDFS client configs.</td>
</tr>
<tr>
<td>.setHdfsClientSettings()</td>
<td>~~hdfs.config~~ (unless changed via withConfigKey)</td>
<td></td>
<td>Set it to a Map of Key/value pairs indicating the HDFS settings to be used. For example, keytab and principal could be set using this. See section <strong>Using keytabs on all worker hosts</strong> under HDFS bolt below.</td>
</tr>
<tr>
<td>.withOutputStream()</td>
<td></td>
<td></td>
<td>Name of output stream. If set, the tuples will be emited to the specified stream. Else tuples will be emited to the default output stream</td>
</tr>
</tbody></table>
<hr>
</div>
</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 Apache Storm</h5>
<p>Apache Storm integrates with any queueing system and any database system. Apache Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating Apache 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="/releases/current/Rationale.html">Rationale</a></li>
<li><a href="/releases/current/Tutorial.html">Tutorial</a></li>
<li><a href="/releases/current/Setting-up-development-environment.html">Setting up development environment</a></li>
<li><a href="/releases/current/Creating-a-new-Storm-project.html">Creating a new Apache 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="/releases/current/index.html">Index</a></li>
<li><a href="/releases/current/javadocs/index.html">Javadoc</a></li>
<li><a href="/releases/current/FAQ.html">FAQ</a></li>
</ul>
</div>
</div>
</div>
<hr/>
<div class="row">
<div class="col-md-12">
<p align="center">Copyright © 2019 <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>