blob: c2472eefed9ac2013ee0b09ac739d01948a0b363 [file] [log] [blame]
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!DOCTYPE html>
<html lang="en">
<head>
<meta charset="UTF-8">
<!--[if IE]><meta http-equiv="X-UA-Compatible" content="IE=edge"><![endif]-->
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="generator" content="Asciidoctor 1.5.8">
<meta name="author" content="Apache NiFi Team">
<title>NiFi Developer&#8217;s Guide</title>
<style>
/* Asciidoctor default stylesheet | MIT License | http://asciidoctor.org */
/* Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE. */
/* Remove the comments around the @import statement below when using this as a custom stylesheet */
@import "https://fonts.googleapis.com/css?family=Open+Sans:300,300italic,400,400italic,600,600italic%7CNoto+Serif:400,400italic,700,700italic%7CDroid+Sans+Mono:400";
article,aside,details,figcaption,figure,footer,header,hgroup,main,nav,section,summary{display:block}
audio,canvas,video{display:inline-block}
audio:not([controls]){display:none;height:0}
[hidden],template{display:none}
script{display:none!important}
html{font-family:sans-serif;-ms-text-size-adjust:100%;-webkit-text-size-adjust:100%}
body{margin:0}
a{background:transparent}
a:focus{outline:thin dotted}
a:active,a:hover{outline:0}
h1{font-size:2em;margin:.67em 0}
abbr[title]{border-bottom:1px dotted}
b,strong{font-weight:bold}
dfn{font-style:italic}
hr{-moz-box-sizing:content-box;box-sizing:content-box;height:0}
mark{background:#ff0;color:#000}
code,kbd,pre,samp{font-family:monospace;font-size:1em}
pre{white-space:pre-wrap}
q{quotes:"\201C" "\201D" "\2018" "\2019"}
small{font-size:80%}
sub,sup{font-size:75%;line-height:0;position:relative;vertical-align:baseline}
sup{top:-.5em}
sub{bottom:-.25em}
img{border:0}
svg:not(:root){overflow:hidden}
figure{margin:0}
fieldset{border:1px solid silver;margin:0 2px;padding:.35em .625em .75em}
legend{border:0;padding:0}
button,input,select,textarea{font-family:inherit;font-size:100%;margin:0}
button,input{line-height:normal}
button,select{text-transform:none}
button,html input[type="button"],input[type="reset"],input[type="submit"]{-webkit-appearance:button;cursor:pointer}
button[disabled],html input[disabled]{cursor:default}
input[type="checkbox"],input[type="radio"]{box-sizing:border-box;padding:0}
input[type="search"]{-webkit-appearance:textfield;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;box-sizing:content-box}
input[type="search"]::-webkit-search-cancel-button,input[type="search"]::-webkit-search-decoration{-webkit-appearance:none}
button::-moz-focus-inner,input::-moz-focus-inner{border:0;padding:0}
textarea{overflow:auto;vertical-align:top}
table{border-collapse:collapse;border-spacing:0}
*,*:before,*:after{-moz-box-sizing:border-box;-webkit-box-sizing:border-box;box-sizing:border-box}
html,body{font-size:100%}
body{background:#fff;color:rgba(0,0,0,.8);padding:0;margin:0;font-family:"Noto Serif","DejaVu Serif",serif;font-weight:400;font-style:normal;line-height:1;position:relative;cursor:auto}
a:hover{cursor:pointer}
img,object,embed{max-width:100%;height:auto}
object,embed{height:100%}
img{-ms-interpolation-mode:bicubic}
#map_canvas img,#map_canvas embed,#map_canvas object,.map_canvas img,.map_canvas embed,.map_canvas object{max-width:none!important}
.left{float:left!important}
.right{float:right!important}
.text-left{text-align:left!important}
.text-right{text-align:right!important}
.text-center{text-align:center!important}
.text-justify{text-align:justify!important}
.hide{display:none}
.antialiased,body{-webkit-font-smoothing:antialiased}
img{display:inline-block;vertical-align:middle}
textarea{height:auto;min-height:50px}
select{width:100%}
p.lead,.paragraph.lead>p,#preamble>.sectionbody>.paragraph:first-of-type p{font-size:1.21875em;line-height:1.6}
.subheader,.admonitionblock td.content>.title,.audioblock>.title,.exampleblock>.title,.imageblock>.title,.listingblock>.title,.literalblock>.title,.stemblock>.title,.openblock>.title,.paragraph>.title,.quoteblock>.title,table.tableblock>.title,.verseblock>.title,.videoblock>.title,.dlist>.title,.olist>.title,.ulist>.title,.qlist>.title,.hdlist>.title{line-height:1.45;color:#7a2518;font-weight:400;margin-top:0;margin-bottom:.25em}
div,dl,dt,dd,ul,ol,li,h1,h2,h3,#toctitle,.sidebarblock>.content>.title,h4,h5,h6,pre,form,p,blockquote,th,td{margin:0;padding:0;direction:ltr}
a{color:#2156a5;text-decoration:underline;line-height:inherit}
a:hover,a:focus{color:#1d4b8f}
a img{border:none}
p{font-family:inherit;font-weight:400;font-size:1em;line-height:1.6;margin-bottom:1.25em;text-rendering:optimizeLegibility}
p aside{font-size:.875em;line-height:1.35;font-style:italic}
h1,h2,h3,#toctitle,.sidebarblock>.content>.title,h4,h5,h6{font-family:"Open Sans","DejaVu Sans",sans-serif;font-weight:300;font-style:normal;color:#ba3925;text-rendering:optimizeLegibility;margin-top:1em;margin-bottom:.5em;line-height:1.0125em}
h1 small,h2 small,h3 small,#toctitle small,.sidebarblock>.content>.title small,h4 small,h5 small,h6 small{font-size:60%;color:#e99b8f;line-height:0}
h1{font-size:2.125em}
h2{font-size:1.6875em}
h3,#toctitle,.sidebarblock>.content>.title{font-size:1.375em}
h4,h5{font-size:1.125em}
h6{font-size:1em}
hr{border:solid #ddddd8;border-width:1px 0 0;clear:both;margin:1.25em 0 1.1875em;height:0}
em,i{font-style:italic;line-height:inherit}
strong,b{font-weight:bold;line-height:inherit}
small{font-size:60%;line-height:inherit}
code{font-family:"Droid Sans Mono","DejaVu Sans Mono",monospace;font-weight:400;color:rgba(0,0,0,.9);padding-right: 1px;}
ul,ol,dl{font-size:1em;line-height:1.6;margin-bottom:1.25em;list-style-position:outside;font-family:inherit}
ul,ol,ul.no-bullet,ol.no-bullet{margin-left:1.5em}
ul li ul,ul li ol{margin-left:1.25em;margin-bottom:0;font-size:1em}
ul.square li ul,ul.circle li ul,ul.disc li ul{list-style:inherit}
ul.square{list-style-type:square}
ul.circle{list-style-type:circle}
ul.disc{list-style-type:disc}
ul.no-bullet{list-style:none}
ol li ul,ol li ol{margin-left:1.25em;margin-bottom:0}
dl dt{margin-bottom:.3125em;font-weight:bold}
dl dd{margin-bottom:1.25em}
abbr,acronym{text-transform:uppercase;font-size:90%;color:rgba(0,0,0,.8);border-bottom:1px dotted #ddd;cursor:help}
abbr{text-transform:none}
blockquote{margin:0 0 1.25em;padding:.5625em 1.25em 0 1.1875em;border-left:1px solid #ddd}
blockquote cite{display:block;font-size:.9375em;color:rgba(0,0,0,.6)}
blockquote cite:before{content:"\2014 \0020"}
blockquote cite a,blockquote cite a:visited{color:rgba(0,0,0,.6)}
blockquote,blockquote p{line-height:1.6;color:rgba(0,0,0,.85)}
@media only screen and (min-width:768px){h1,h2,h3,#toctitle,.sidebarblock>.content>.title,h4,h5,h6{line-height:1.2}
h1{font-size:2.75em}
h2{font-size:2.3125em}
h3,#toctitle,.sidebarblock>.content>.title{font-size:1.6875em}
h4{font-size:1.4375em}}table{background:#fff;margin-bottom:1.25em;border:solid 1px #dedede}
table thead,table tfoot{background:#f7f8f7;font-weight:bold}
table thead tr th,table thead tr td,table tfoot tr th,table tfoot tr td{padding:.5em .625em .625em;font-size:inherit;color:rgba(0,0,0,.8);text-align:left}
table tr th,table tr td{padding:.5625em .625em;font-size:inherit;color:rgba(0,0,0,.8)}
table tr.even,table tr.alt,table tr:nth-of-type(even){background:#f8f8f7}
table thead tr th,table tfoot tr th,table tbody tr td,table tr td,table tfoot tr td{display:table-cell;line-height:1.6}
h1,h2,h3,#toctitle,.sidebarblock>.content>.title,h4,h5,h6{line-height:1.2;word-spacing:-.05em}
h1 strong,h2 strong,h3 strong,#toctitle strong,.sidebarblock>.content>.title strong,h4 strong,h5 strong,h6 strong{font-weight:400}
.clearfix:before,.clearfix:after,.float-group:before,.float-group:after{content:" ";display:table}
.clearfix:after,.float-group:after{clear:both}
*:not(pre)>code{font-size:.9375em;font-style:normal!important;letter-spacing:0;word-spacing:-.15em;background-color:#f7f7f8;-webkit-border-radius:4px;border-radius:4px;line-height:1.45;text-rendering:optimizeSpeed}
pre,pre>code{line-height:1.45;color:rgba(0,0,0,.9);font-family:"Droid Sans Mono","DejaVu Sans Mono",monospace;font-weight:400;text-rendering:optimizeSpeed}
.keyseq{color:rgba(51,51,51,.8)}
kbd{display:inline-block;color:rgba(0,0,0,.8);font-size:.75em;line-height:1.4;background-color:#f7f7f7;border:1px solid #ccc;-webkit-border-radius:3px;border-radius:3px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,.2),0 0 0 .1em white inset;box-shadow:0 1px 0 rgba(0,0,0,.2),0 0 0 .1em #fff inset;margin:-.15em .15em 0 .15em;padding:.2em .6em .2em .5em;vertical-align:middle;white-space:nowrap}
.keyseq kbd:first-child{margin-left:0}
.keyseq kbd:last-child{margin-right:0}
.menuseq,.menu{color:rgba(0,0,0,.8)}
b.button:before,b.button:after{position:relative;top:-1px;font-weight:400}
b.button:before{content:"[";padding:0 3px 0 2px}
b.button:after{content:"]";padding:0 2px 0 3px}
p a>code:hover{color:rgba(0,0,0,.9)}
#header,#content,#footnotes,#footer{width:100%;margin-left:auto;margin-right:auto;margin-top:0;margin-bottom:0;max-width:62.5em;*zoom:1;position:relative;padding-left:.9375em;padding-right:.9375em}
#header:before,#header:after,#content:before,#content:after,#footnotes:before,#footnotes:after,#footer:before,#footer:after{content:" ";display:table}
#header:after,#content:after,#footnotes:after,#footer:after{clear:both}
#content{margin-top:1.25em}
#content:before{content:none}
#header>h1:first-child{color:rgba(0,0,0,.85);margin-top:2.25rem;margin-bottom:0}
#header>h1:first-child+#toc{margin-top:8px;border-top:1px solid #ddddd8}
#header>h1:only-child,body.toc2 #header>h1:nth-last-child(2){border-bottom:1px solid #ddddd8;padding-bottom:8px}
#header .details{border-bottom:1px solid #ddddd8;line-height:1.45;padding-top:.25em;padding-bottom:.25em;padding-left:.25em;color:rgba(0,0,0,.6);display:-ms-flexbox;display:-webkit-flex;display:flex;-ms-flex-flow:row wrap;-webkit-flex-flow:row wrap;flex-flow:row wrap}
#header .details span:first-child{margin-left:-.125em}
#header .details span.email a{color:rgba(0,0,0,.85)}
#header .details br{display:none}
#header .details br+span:before{content:"\00a0\2013\00a0"}
#header .details br+span.author:before{content:"\00a0\22c5\00a0";color:rgba(0,0,0,.85)}
#header .details br+span#revremark:before{content:"\00a0|\00a0"}
#header #revnumber{text-transform:capitalize}
#header #revnumber:after{content:"\00a0"}
#content>h1:first-child:not([class]){color:rgba(0,0,0,.85);border-bottom:1px solid #ddddd8;padding-bottom:8px;margin-top:0;padding-top:1rem;margin-bottom:1.25rem}
#toc{border-bottom:1px solid #efefed;padding-bottom:.5em}
#toc>ul{margin-left:.125em}
#toc ul.sectlevel0>li>a{font-style:italic}
#toc ul.sectlevel0 ul.sectlevel1{margin:.5em 0}
#toc ul{font-family:"Open Sans","DejaVu Sans",sans-serif;list-style-type:none}
#toc a{text-decoration:none}
#toc a:active{text-decoration:underline}
#toctitle{color:#7a2518;font-size:1.2em}
@media only screen and (min-width:768px){#toctitle{font-size:1.375em}
body.toc2{padding-left:15em;padding-right:0}
#toc.toc2{margin-top:0!important;background-color:#f8f8f7;position:fixed;width:15em;left:0;top:0;border-right:1px solid #efefed;border-top-width:0!important;border-bottom-width:0!important;z-index:1000;padding:1.25em 1em;height:100%;overflow:auto}
#toc.toc2 #toctitle{margin-top:0;font-size:1.2em}
#toc.toc2>ul{font-size:.9em;margin-bottom:0}
#toc.toc2 ul ul{margin-left:0;padding-left:1em}
#toc.toc2 ul.sectlevel0 ul.sectlevel1{padding-left:0;margin-top:.5em;margin-bottom:.5em}
body.toc2.toc-right{padding-left:0;padding-right:15em}
body.toc2.toc-right #toc.toc2{border-right-width:0;border-left:1px solid #efefed;left:auto;right:0}}@media only screen and (min-width:1280px){body.toc2{padding-left:20em;padding-right:0}
#toc.toc2{width:20em}
#toc.toc2 #toctitle{font-size:1.375em}
#toc.toc2>ul{font-size:.95em}
#toc.toc2 ul ul{padding-left:1.25em}
body.toc2.toc-right{padding-left:0;padding-right:20em}}#content #toc{border-style:solid;border-width:1px;border-color:#e0e0dc;margin-bottom:1.25em;padding:1.25em;background:#f8f8f7;-webkit-border-radius:4px;border-radius:4px}
#content #toc>:first-child{margin-top:0}
#content #toc>:last-child{margin-bottom:0}
#footer{max-width:100%;background-color:rgba(0,0,0,.8);padding:1.25em}
#footer-text{color:rgba(255,255,255,.8);line-height:1.44}
.sect1{padding-bottom:.625em}
@media only screen and (min-width:768px){.sect1{padding-bottom:1.25em}}.sect1+.sect1{border-top:1px solid #efefed}
#content h1>a.anchor,h2>a.anchor,h3>a.anchor,#toctitle>a.anchor,.sidebarblock>.content>.title>a.anchor,h4>a.anchor,h5>a.anchor,h6>a.anchor{position:absolute;z-index:1001;width:1.5ex;margin-left:-1.5ex;display:block;text-decoration:none!important;visibility:hidden;text-align:center;font-weight:400}
#content h1>a.anchor:before,h2>a.anchor:before,h3>a.anchor:before,#toctitle>a.anchor:before,.sidebarblock>.content>.title>a.anchor:before,h4>a.anchor:before,h5>a.anchor:before,h6>a.anchor:before{content:"\00A7";font-size:.85em;display:block;padding-top:.1em}
#content h1:hover>a.anchor,#content h1>a.anchor:hover,h2:hover>a.anchor,h2>a.anchor:hover,h3:hover>a.anchor,#toctitle:hover>a.anchor,.sidebarblock>.content>.title:hover>a.anchor,h3>a.anchor:hover,#toctitle>a.anchor:hover,.sidebarblock>.content>.title>a.anchor:hover,h4:hover>a.anchor,h4>a.anchor:hover,h5:hover>a.anchor,h5>a.anchor:hover,h6:hover>a.anchor,h6>a.anchor:hover{visibility:visible}
#content h1>a.link,h2>a.link,h3>a.link,#toctitle>a.link,.sidebarblock>.content>.title>a.link,h4>a.link,h5>a.link,h6>a.link{color:#ba3925;text-decoration:none}
#content h1>a.link:hover,h2>a.link:hover,h3>a.link:hover,#toctitle>a.link:hover,.sidebarblock>.content>.title>a.link:hover,h4>a.link:hover,h5>a.link:hover,h6>a.link:hover{color:#a53221}
.audioblock,.imageblock,.literalblock,.listingblock,.stemblock,.videoblock{margin-bottom:1.25em}
.admonitionblock td.content>.title,.audioblock>.title,.exampleblock>.title,.imageblock>.title,.listingblock>.title,.literalblock>.title,.stemblock>.title,.openblock>.title,.paragraph>.title,.quoteblock>.title,table.tableblock>.title,.verseblock>.title,.videoblock>.title,.dlist>.title,.olist>.title,.ulist>.title,.qlist>.title,.hdlist>.title{text-rendering:optimizeLegibility;text-align:left;font-family:"Noto Serif","DejaVu Serif",serif;font-size:1rem;font-style:italic}
table.tableblock>caption.title{white-space:nowrap;overflow:visible;max-width:0}
.paragraph.lead>p,#preamble>.sectionbody>.paragraph:first-of-type p{color:rgba(0,0,0,.85)}
table.tableblock #preamble>.sectionbody>.paragraph:first-of-type p{font-size:inherit}
.admonitionblock>table{border-collapse:separate;border:0;background:none;width:100%}
.admonitionblock>table td.icon{text-align:center;width:80px}
.admonitionblock>table td.icon img{max-width:none}
.admonitionblock>table td.icon .title{font-weight:bold;font-family:"Open Sans","DejaVu Sans",sans-serif;text-transform:uppercase}
.admonitionblock>table td.content{padding-left:1.125em;padding-right:1.25em;border-left:1px solid #ddddd8;color:rgba(0,0,0,.6)}
.admonitionblock>table td.content>:last-child>:last-child{margin-bottom:0}
.exampleblock>.content{border-style:solid;border-width:1px;border-color:#e6e6e6;margin-bottom:1.25em;padding:1.25em;background:#fff;-webkit-border-radius:4px;border-radius:4px}
.exampleblock>.content>:first-child{margin-top:0}
.exampleblock>.content>:last-child{margin-bottom:0}
.sidebarblock{border-style:solid;border-width:1px;border-color:#e0e0dc;margin-bottom:1.25em;padding:1.25em;background:#f8f8f7;-webkit-border-radius:4px;border-radius:4px}
.sidebarblock>:first-child{margin-top:0}
.sidebarblock>:last-child{margin-bottom:0}
.sidebarblock>.content>.title{color:#7a2518;margin-top:0;text-align:center}
.exampleblock>.content>:last-child>:last-child,.exampleblock>.content .olist>ol>li:last-child>:last-child,.exampleblock>.content .ulist>ul>li:last-child>:last-child,.exampleblock>.content .qlist>ol>li:last-child>:last-child,.sidebarblock>.content>:last-child>:last-child,.sidebarblock>.content .olist>ol>li:last-child>:last-child,.sidebarblock>.content .ulist>ul>li:last-child>:last-child,.sidebarblock>.content .qlist>ol>li:last-child>:last-child{margin-bottom:0}
.literalblock pre,.listingblock pre:not(.highlight),.listingblock pre[class="highlight"],.listingblock pre[class^="highlight "],.listingblock pre.CodeRay,.listingblock pre.prettyprint{background:#f7f7f8}
.sidebarblock .literalblock pre,.sidebarblock .listingblock pre:not(.highlight),.sidebarblock .listingblock pre[class="highlight"],.sidebarblock .listingblock pre[class^="highlight "],.sidebarblock .listingblock pre.CodeRay,.sidebarblock .listingblock pre.prettyprint{background:#f2f1f1}
.literalblock pre,.literalblock pre[class],.listingblock pre,.listingblock pre[class]{-webkit-border-radius:4px;border-radius:4px;word-wrap:break-word;padding:1em;font-size:.8125em}
.literalblock pre.nowrap,.literalblock pre[class].nowrap,.listingblock pre.nowrap,.listingblock pre[class].nowrap{overflow-x:auto;white-space:pre;word-wrap:normal}
@media only screen and (min-width:768px){.literalblock pre,.literalblock pre[class],.listingblock pre,.listingblock pre[class]{font-size:.90625em}}@media only screen and (min-width:1280px){.literalblock pre,.literalblock pre[class],.listingblock pre,.listingblock pre[class]{font-size:1em}}.literalblock.output pre{color:#f7f7f8;background-color:rgba(0,0,0,.9)}
.listingblock pre.highlightjs{padding:0}
.listingblock pre.highlightjs>code{padding:1em;-webkit-border-radius:4px;border-radius:4px}
.listingblock pre.prettyprint{border-width:0}
.listingblock>.content{position:relative}
.listingblock code[data-lang]:before{display:none;content:attr(data-lang);position:absolute;font-size:.75em;top:.425rem;right:.5rem;line-height:1;text-transform:uppercase;color:#999}
.listingblock:hover code[data-lang]:before{display:block}
.listingblock.terminal pre .command:before{content:attr(data-prompt);padding-right:.5em;color:#999}
.listingblock.terminal pre .command:not([data-prompt]):before{content:"$"}
table.pyhltable{border-collapse:separate;border:0;margin-bottom:0;background:none}
table.pyhltable td{vertical-align:top;padding-top:0;padding-bottom:0}
table.pyhltable td.code{padding-left:.75em;padding-right:0}
pre.pygments .lineno,table.pyhltable td:not(.code){color:#999;padding-left:0;padding-right:.5em;border-right:1px solid #ddddd8}
pre.pygments .lineno{display:inline-block;margin-right:.25em}
table.pyhltable .linenodiv{background:none!important;padding-right:0!important}
.quoteblock{margin:0 1em 1.25em 1.5em;display:table}
.quoteblock>.title{margin-left:-1.5em;margin-bottom:.75em}
.quoteblock blockquote,.quoteblock blockquote p{color:rgba(0,0,0,.85);font-size:1.15rem;line-height:1.75;word-spacing:.1em;letter-spacing:0;font-style:italic;text-align:justify}
.quoteblock blockquote{margin:0;padding:0;border:0}
.quoteblock blockquote:before{content:"\201c";float:left;font-size:2.75em;font-weight:bold;line-height:.6em;margin-left:-.6em;color:#7a2518;text-shadow:0 1px 2px rgba(0,0,0,.1)}
.quoteblock blockquote>.paragraph:last-child p{margin-bottom:0}
.quoteblock .attribution{margin-top:.5em;margin-right:.5ex;text-align:right}
.quoteblock .quoteblock{margin-left:0;margin-right:0;padding:.5em 0;border-left:3px solid rgba(0,0,0,.6)}
.quoteblock .quoteblock blockquote{padding:0 0 0 .75em}
.quoteblock .quoteblock blockquote:before{display:none}
.verseblock{margin:0 1em 1.25em 1em}
.verseblock pre{font-family:"Open Sans","DejaVu Sans",sans;font-size:1.15rem;color:rgba(0,0,0,.85);font-weight:300;text-rendering:optimizeLegibility}
.verseblock pre strong{font-weight:400}
.verseblock .attribution{margin-top:1.25rem;margin-left:.5ex}
.quoteblock .attribution,.verseblock .attribution{font-size:.9375em;line-height:1.45;font-style:italic}
.quoteblock .attribution br,.verseblock .attribution br{display:none}
.quoteblock .attribution cite,.verseblock .attribution cite{display:block;letter-spacing:-.05em;color:rgba(0,0,0,.6)}
.quoteblock.abstract{margin:0 0 1.25em 0;display:block}
.quoteblock.abstract blockquote,.quoteblock.abstract blockquote p{text-align:left;word-spacing:0}
.quoteblock.abstract blockquote:before,.quoteblock.abstract blockquote p:first-of-type:before{display:none}
table.tableblock{max-width:100%;border-collapse:separate}
table.tableblock td>.paragraph:last-child p>p:last-child,table.tableblock th>p:last-child,table.tableblock td>p:last-child{margin-bottom:0}
table.spread{width:100%}
table.tableblock,th.tableblock,td.tableblock{border:0 solid #dedede}
table.grid-all th.tableblock,table.grid-all td.tableblock{border-width:0 1px 1px 0}
table.grid-all tfoot>tr>th.tableblock,table.grid-all tfoot>tr>td.tableblock{border-width:1px 1px 0 0}
table.grid-cols th.tableblock,table.grid-cols td.tableblock{border-width:0 1px 0 0}
table.grid-all *>tr>.tableblock:last-child,table.grid-cols *>tr>.tableblock:last-child{border-right-width:0}
table.grid-rows th.tableblock,table.grid-rows td.tableblock{border-width:0 0 1px 0}
table.grid-all tbody>tr:last-child>th.tableblock,table.grid-all tbody>tr:last-child>td.tableblock,table.grid-all thead:last-child>tr>th.tableblock,table.grid-rows tbody>tr:last-child>th.tableblock,table.grid-rows tbody>tr:last-child>td.tableblock,table.grid-rows thead:last-child>tr>th.tableblock{border-bottom-width:0}
table.grid-rows tfoot>tr>th.tableblock,table.grid-rows tfoot>tr>td.tableblock{border-width:1px 0 0 0}
table.frame-all{border-width:1px}
table.frame-sides{border-width:0 1px}
table.frame-topbot{border-width:1px 0}
th.halign-left,td.halign-left{text-align:left}
th.halign-right,td.halign-right{text-align:right}
th.halign-center,td.halign-center{text-align:center}
th.valign-top,td.valign-top{vertical-align:top}
th.valign-bottom,td.valign-bottom{vertical-align:bottom}
th.valign-middle,td.valign-middle{vertical-align:middle}
table thead th,table tfoot th{font-weight:bold}
tbody tr th{display:table-cell;line-height:1.6;background:#f7f8f7}
tbody tr th,tbody tr th p,tfoot tr th,tfoot tr th p{color:rgba(0,0,0,.8);font-weight:bold}
p.tableblock>code:only-child{background:none;padding:0}
p.tableblock{font-size:1em}
td>div.verse{white-space:pre}
ol{margin-left:1.75em}
ul li ol{margin-left:1.5em}
dl dd{margin-left:1.125em}
dl dd:last-child,dl dd:last-child>:last-child{margin-bottom:0}
ol>li p,ul>li p,ul dd,ol dd,.olist .olist,.ulist .ulist,.ulist .olist,.olist .ulist{margin-bottom:.625em}
ul.unstyled,ol.unnumbered,ul.checklist,ul.none{list-style-type:none}
ul.unstyled,ol.unnumbered,ul.checklist{margin-left:.625em}
ul.checklist li>p:first-child>.fa-square-o:first-child,ul.checklist li>p:first-child>.fa-check-square-o:first-child{width:1em;font-size:.85em}
ul.checklist li>p:first-child>input[type="checkbox"]:first-child{width:1em;position:relative;top:1px}
ul.inline{margin:0 auto .625em auto;margin-left:-1.375em;margin-right:0;padding:0;list-style:none;overflow:hidden}
ul.inline>li{list-style:none;float:left;margin-left:1.375em;display:block}
ul.inline>li>*{display:block}
.unstyled dl dt{font-weight:400;font-style:normal}
ol.arabic{list-style-type:decimal}
ol.decimal{list-style-type:decimal-leading-zero}
ol.loweralpha{list-style-type:lower-alpha}
ol.upperalpha{list-style-type:upper-alpha}
ol.lowerroman{list-style-type:lower-roman}
ol.upperroman{list-style-type:upper-roman}
ol.lowergreek{list-style-type:lower-greek}
.hdlist>table,.colist>table{border:0;background:none}
.hdlist>table>tbody>tr,.colist>table>tbody>tr{background:none}
td.hdlist1{padding-right:.75em;font-weight:bold}
td.hdlist1,td.hdlist2{vertical-align:top}
.literalblock+.colist,.listingblock+.colist{margin-top:-.5em}
.colist>table tr>td:first-of-type{padding:0 .75em;line-height:1}
.colist>table tr>td:last-of-type{padding:.25em 0}
.thumb,.th{line-height:0;display:inline-block;border:solid 4px #fff;-webkit-box-shadow:0 0 0 1px #ddd;box-shadow:0 0 0 1px #ddd}
.imageblock.left,.imageblock[style*="float: left"]{margin:.25em .625em 1.25em 0}
.imageblock.right,.imageblock[style*="float: right"]{margin:.25em 0 1.25em .625em}
.imageblock>.title{margin-bottom:0}
.imageblock.thumb,.imageblock.th{border-width:6px}
.imageblock.thumb>.title,.imageblock.th>.title{padding:0 .125em}
.image.left,.image.right{margin-top:.25em;margin-bottom:.25em;display:inline-block;line-height:0}
.image.left{margin-right:.625em}
.image.right{margin-left:.625em}
a.image{text-decoration:none}
span.footnote,span.footnoteref{vertical-align:super;font-size:.875em}
span.footnote a,span.footnoteref a{text-decoration:none}
span.footnote a:active,span.footnoteref a:active{text-decoration:underline}
#footnotes{padding-top:.75em;padding-bottom:.75em;margin-bottom:.625em}
#footnotes hr{width:20%;min-width:6.25em;margin:-.25em 0 .75em 0;border-width:1px 0 0 0}
#footnotes .footnote{padding:0 .375em;line-height:1.3;font-size:.875em;margin-left:1.2em;text-indent:-1.2em;margin-bottom:.2em}
#footnotes .footnote a:first-of-type{font-weight:bold;text-decoration:none}
#footnotes .footnote:last-of-type{margin-bottom:0}
#content #footnotes{margin-top:-.625em;margin-bottom:0;padding:.75em 0}
.gist .file-data>table{border:0;background:#fff;width:100%;margin-bottom:0}
.gist .file-data>table td.line-data{width:99%}
div.unbreakable{page-break-inside:avoid}
.big{font-size:larger}
.small{font-size:smaller}
.underline{text-decoration:underline}
.overline{text-decoration:overline}
.line-through{text-decoration:line-through}
.aqua{color:#00bfbf}
.aqua-background{background-color:#00fafa}
.black{color:#000}
.black-background{background-color:#000}
.blue{color:#0000bf}
.blue-background{background-color:#0000fa}
.fuchsia{color:#bf00bf}
.fuchsia-background{background-color:#fa00fa}
.gray{color:#606060}
.gray-background{background-color:#7d7d7d}
.green{color:#006000}
.green-background{background-color:#007d00}
.lime{color:#00bf00}
.lime-background{background-color:#00fa00}
.maroon{color:#600000}
.maroon-background{background-color:#7d0000}
.navy{color:#000060}
.navy-background{background-color:#00007d}
.olive{color:#606000}
.olive-background{background-color:#7d7d00}
.purple{color:#600060}
.purple-background{background-color:#7d007d}
.red{color:#bf0000}
.red-background{background-color:#fa0000}
.silver{color:#909090}
.silver-background{background-color:#bcbcbc}
.teal{color:#006060}
.teal-background{background-color:#007d7d}
.white{color:#bfbfbf}
.white-background{background-color:#fafafa}
.yellow{color:#bfbf00}
.yellow-background{background-color:#fafa00}
span.icon>.fa{cursor:default}
.admonitionblock td.icon [class^="fa icon-"]{font-size:2.5em;text-shadow:1px 1px 2px rgba(0,0,0,.5);cursor:default}
.admonitionblock td.icon .icon-note:before{content:"\f05a";color:#19407c}
.admonitionblock td.icon .icon-tip:before{content:"\f0eb";text-shadow:1px 1px 2px rgba(155,155,0,.8);color:#111}
.admonitionblock td.icon .icon-warning:before{content:"\f071";color:#bf6900}
.admonitionblock td.icon .icon-caution:before{content:"\f06d";color:#bf3400}
.admonitionblock td.icon .icon-important:before{content:"\f06a";color:#bf0000}
.conum[data-value]{display:inline-block;color:#fff!important;background-color:rgba(0,0,0,.8);-webkit-border-radius:100px;border-radius:100px;text-align:center;font-size:.75em;width:1.67em;height:1.67em;line-height:1.67em;font-family:"Open Sans","DejaVu Sans",sans-serif;font-style:normal;font-weight:bold}
.conum[data-value] *{color:#fff!important}
.conum[data-value]+b{display:none}
.conum[data-value]:after{content:attr(data-value)}
pre .conum[data-value]{position:relative;top:-.125em}
b.conum *{color:inherit!important}
.conum:not([data-value]):empty{display:none}
h1,h2{letter-spacing:-.01em}
dt,th.tableblock,td.content{text-rendering:optimizeLegibility}
p,td.content{letter-spacing:-.01em}
p strong,td.content strong{letter-spacing:-.005em}
p,blockquote,dt,td.content{font-size:1.0625rem}
p{margin-bottom:1.25rem}
.sidebarblock p,.sidebarblock dt,.sidebarblock td.content,p.tableblock{font-size:1em}
.exampleblock>.content{background-color:#fffef7;border-color:#e0e0dc;-webkit-box-shadow:0 1px 4px #e0e0dc;box-shadow:0 1px 4px #e0e0dc}
.print-only{display:none!important}
@media print{@page{margin:1.25cm .75cm}
*{-webkit-box-shadow:none!important;box-shadow:none!important;text-shadow:none!important}
a{color:inherit!important;text-decoration:underline!important}
a.bare,a[href^="#"],a[href^="mailto:"]{text-decoration:none!important}
a[href^="http:"]:not(.bare):after,a[href^="https:"]:not(.bare):after{content:"(" attr(href) ")";display:inline-block;font-size:.875em;padding-left:.25em}
abbr[title]:after{content:" (" attr(title) ")"}
pre,blockquote,tr,img{page-break-inside:avoid}
thead{display:table-header-group}
img{max-width:100%!important}
p,blockquote,dt,td.content{font-size:1em;orphans:3;widows:3}
h2,h3,#toctitle,.sidebarblock>.content>.title{page-break-after:avoid}
#toc,.sidebarblock,.exampleblock>.content{background:none!important}
#toc{border-bottom:1px solid #ddddd8!important;padding-bottom:0!important}
.sect1{padding-bottom:0!important}
.sect1+.sect1{border:0!important}
#header>h1:first-child{margin-top:1.25rem}
body.book #header{text-align:center}
body.book #header>h1:first-child{border:0!important;margin:2.5em 0 1em 0}
body.book #header .details{border:0!important;display:block;padding:0!important}
body.book #header .details span:first-child{margin-left:0!important}
body.book #header .details br{display:block}
body.book #header .details br+span:before{content:none!important}
body.book #toc{border:0!important;text-align:left!important;padding:0!important;margin:0!important}
body.book #toc,body.book #preamble,body.book h1.sect0,body.book .sect1>h2{page-break-before:always}
.listingblock code[data-lang]:before{display:block}
#footer{background:none!important;padding:0 .9375em}
#footer-text{color:rgba(0,0,0,.6)!important;font-size:.9em}
.hide-on-print{display:none!important}
.print-only{display:block!important}
.hide-for-print{display:none!important}
.show-for-print{display:inherit!important}}
</style>
<link rel="stylesheet" href="https://cdnjs.cloudflare.com/ajax/libs/font-awesome/4.7.0/css/font-awesome.min.css">
</head>
<body class="article">
<div id="header">
<h1>NiFi Developer&#8217;s Guide</h1>
<div class="details">
<span id="author" class="author">Apache NiFi Team</span><br>
<span id="email" class="email"><a href="mailto:dev@nifi.apache.org">dev@nifi.apache.org</a></span><br>
</div>
<div id="toc" class="toc">
<div id="toctitle">Table of Contents</div>
<ul class="sectlevel1">
<li><a href="developer-guide.html#introduction">Introduction</a></li>
<li><a href="developer-guide.html#components">NiFi Components</a></li>
<li><a href="developer-guide.html#processor_api">Processor API</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#supporting_api">Supporting API</a>
<ul class="sectlevel3">
<li><a href="developer-guide.html#flowfile">FlowFile</a></li>
<li><a href="developer-guide.html#process_session">ProcessSession</a></li>
<li><a href="developer-guide.html#process_context">ProcessContext</a></li>
<li><a href="developer-guide.html#property_descriptor">PropertyDescriptor</a></li>
<li><a href="developer-guide.html#validator">Validator</a></li>
<li><a href="developer-guide.html#validation_context">ValidationContext</a></li>
<li><a href="developer-guide.html#property_value">PropertyValue</a></li>
<li><a href="developer-guide.html#relationship">Relationship</a></li>
<li><a href="developer-guide.html#supporting_api_state_manager">StateManager</a></li>
<li><a href="developer-guide.html#processor_initialization_context">ProcessorInitializationContext</a></li>
<li><a href="developer-guide.html#ComponentLog">ComponentLog</a></li>
</ul>
</li>
<li><a href="developer-guide.html#AbstractProcessor">AbstractProcessor API</a>
<ul class="sectlevel3">
<li><a href="developer-guide.html#processor-initialization">Processor Initialization</a></li>
<li><a href="developer-guide.html#exposing-processors-relationships">Exposing Processor&#8217;s Relationships</a></li>
<li><a href="developer-guide.html#exposing-processor-properties">Exposing Processor Properties</a></li>
<li><a href="developer-guide.html#validating-processor-properties">Validating Processor Properties</a></li>
<li><a href="developer-guide.html#responding-to-changes-in-configuration">Responding to Changes in Configuration</a></li>
<li><a href="developer-guide.html#performing-the-work">Performing the Work</a></li>
<li><a href="developer-guide.html#when-processors-are-triggered">When Processors are Triggered</a></li>
</ul>
</li>
<li><a href="developer-guide.html#component-lifecycle">Component Lifecycle</a>
<ul class="sectlevel3">
<li><a href="developer-guide.html#onadded">@OnAdded</a></li>
<li><a href="developer-guide.html#onenabled">@OnEnabled</a></li>
<li><a href="developer-guide.html#onremoved">@OnRemoved</a></li>
<li><a href="developer-guide.html#onscheduled">@OnScheduled</a></li>
<li><a href="developer-guide.html#onunscheduled">@OnUnscheduled</a></li>
<li><a href="developer-guide.html#onstopped">@OnStopped</a></li>
<li><a href="developer-guide.html#onshutdown">@OnShutdown</a></li>
</ul>
</li>
<li><a href="developer-guide.html#component-notification">Component Notification</a>
<ul class="sectlevel3">
<li><a href="developer-guide.html#onprimarynodestatechange">@OnPrimaryNodeStateChange</a></li>
</ul>
</li>
<li><a href="developer-guide.html#restricted">Restricted</a></li>
<li><a href="developer-guide.html#state_manager">State Manager</a>
<ul class="sectlevel3">
<li><a href="developer-guide.html#state_scope">Scope</a></li>
<li><a href="developer-guide.html#storing-and-retrieving-state">Storing and Retrieving State</a></li>
<li><a href="developer-guide.html#unit-tests">Unit Tests</a></li>
</ul>
</li>
<li><a href="developer-guide.html#reporting-processor-activity">Reporting Processor Activity</a></li>
</ul>
</li>
<li><a href="developer-guide.html#documenting-a-component">Documenting a Component</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#documenting-properties">Documenting Properties</a></li>
<li><a href="developer-guide.html#documenting-relationships">Documenting Relationships</a></li>
<li><a href="developer-guide.html#documenting-capability-and-keywords">Documenting Capability and Keywords</a></li>
<li><a href="developer-guide.html#documenting-flowfile-attribute-interaction">Documenting FlowFile Attribute Interaction</a></li>
<li><a href="developer-guide.html#documenting-related-components">Documenting Related Components</a></li>
<li><a href="developer-guide.html#advanced-documentation">Advanced Documentation</a></li>
</ul>
</li>
<li><a href="developer-guide.html#provenance_events">Provenance Events</a></li>
<li><a href="developer-guide.html#common-processor-patterns">Common Processor Patterns</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#ingress">Data Ingress</a></li>
<li><a href="developer-guide.html#data-egress">Data Egress</a></li>
<li><a href="developer-guide.html#route-based-on-content-one-to-one">Route Based on Content (One-to-One)</a></li>
<li><a href="developer-guide.html#route-based-on-content-one-to-many">Route Based on Content (One-to-Many)</a></li>
<li><a href="developer-guide.html#route-streams-based-on-content-one-to-many">Route Streams Based on Content (One-to-Many)</a></li>
<li><a href="developer-guide.html#route-based-on-attributes">Route Based on Attributes</a></li>
<li><a href="developer-guide.html#split-content-one-to-many">Split Content (One-to-Many)</a></li>
<li><a href="developer-guide.html#update-attributes-based-on-content">Update Attributes Based on Content</a></li>
<li><a href="developer-guide.html#enrichmodify-content">Enrich/Modify Content</a></li>
</ul>
</li>
<li><a href="developer-guide.html#error-handling">Error Handling</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#exceptions-within-the-processor">Exceptions within the Processor</a></li>
<li><a href="developer-guide.html#exceptions-within-a-callback-ioexception-runtimeexception">Exceptions within a callback: IOException, RuntimeException</a></li>
<li><a href="developer-guide.html#penalization-vs-yielding">Penalization vs. Yielding</a></li>
<li><a href="developer-guide.html#session-rollback">Session Rollback</a></li>
</ul>
</li>
<li><a href="developer-guide.html#general-design-considerations">General Design Considerations</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#consider-the-user">Consider the User</a></li>
<li><a href="developer-guide.html#cohesion-and-reusability">Cohesion and Reusability</a></li>
<li><a href="developer-guide.html#naming-conventions">Naming Conventions</a></li>
<li><a href="developer-guide.html#processor-behavior-annotations">Processor Behavior Annotations</a></li>
<li><a href="developer-guide.html#data-buffering">Data Buffering</a></li>
</ul>
</li>
<li><a href="developer-guide.html#controller-services">Controller Services</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#developing-controller-service">Developing a ControllerService</a></li>
<li><a href="developer-guide.html#interacting-with-controller-service">Interacting with a ControllerService</a></li>
</ul>
</li>
<li><a href="developer-guide.html#reporting-tasks">Reporting Tasks</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#developing-a-reporting-task">Developing a Reporting Task</a></li>
</ul>
</li>
<li><a href="developer-guide.html#parameter-providers">Parameter Providers</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#developing-a-parameter-provider">Developing a Parameter Provider</a></li>
</ul>
</li>
<li><a href="developer-guide.html#ui-extensions">UI Extensions</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#custom-processor-uis">Custom Processor UIs</a></li>
<li><a href="developer-guide.html#content-viewers">Content Viewers</a></li>
</ul>
</li>
<li><a href="developer-guide.html#command-line-tools">Command Line Tools</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#tls-toolkit">tls-toolkit</a></li>
</ul>
</li>
<li><a href="developer-guide.html#testing">Testing</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#instantiate-testrunner">Instantiate TestRunner</a></li>
<li><a href="developer-guide.html#add-controllerservices">Add ControllerServices</a></li>
<li><a href="developer-guide.html#set-property-values">Set Property Values</a></li>
<li><a href="developer-guide.html#enqueue-flowfiles">Enqueue FlowFiles</a></li>
<li><a href="developer-guide.html#run-the-processor">Run the Processor</a></li>
<li><a href="developer-guide.html#validate-output">Validate Output</a></li>
<li><a href="developer-guide.html#mocking-external-resources">Mocking External Resources</a></li>
<li><a href="developer-guide.html#additional-testing-capabilities">Additional Testing Capabilities</a></li>
</ul>
</li>
<li><a href="developer-guide.html#nars">NiFi Archives (NARs)</a></li>
<li><a href="developer-guide.html#per-instance-classloading">Per-Instance ClassLoading</a></li>
<li><a href="developer-guide.html#deprecation">Deprecating Components and Features</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#component-deprecation">Component Deprecation</a></li>
<li><a href="developer-guide.html#feature-deprecation">Feature Deprecation</a></li>
<li><a href="developer-guide.html#deprecation-logging">Deprecation Logging</a></li>
</ul>
</li>
<li><a href="developer-guide.html#build">Build Options</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#maven-profiles">Maven Profiles</a></li>
<li><a href="developer-guide.html#standard-build-instructions">Standard Build Instructions</a></li>
</ul>
</li>
<li><a href="developer-guide.html#how-to-contribute-to-apache-nifi">How to contribute to Apache NiFi</a>
<ul class="sectlevel2">
<li><a href="developer-guide.html#technologies">Technologies</a></li>
<li><a href="developer-guide.html#where-to-start">Where to Start?</a></li>
<li><a href="developer-guide.html#supplying-a-contribution">Supplying a contribution</a></li>
<li><a href="developer-guide.html#contact-us">Contact Us</a></li>
</ul>
</li>
</ul>
</div>
</div>
<div id="content">
<div class="sect1">
<h2 id="introduction"><a class="anchor" href="developer-guide.html#introduction"></a>Introduction</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The intent of this Developer Guide is to provide the reader with the information needed to understand how Apache NiFi
extensions are developed and help to explain the thought process behind developing the components. It provides an introduction to
and explanation of the API that is used to develop extensions. It does not, however, go into great detail about each
of the methods in the API, as this guide is intended to supplement the JavaDocs of the API rather than replace them.
This guide also assumes that the reader is familiar with Java 7 and Apache Maven.</p>
</div>
<div class="paragraph">
<p>This guide is written by developers for developers. It is expected that before reading this
guide, you have a basic understanding of NiFi and the concepts of dataflow. If not, please see the <a href="overview.html">NiFi Overview</a>
and the <a href="user-guide.html">NiFi User Guide</a> to familiarize yourself with the concepts of NiFi.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="components"><a class="anchor" href="developer-guide.html#components"></a>NiFi Components</h2>
<div class="sectionbody">
<div class="paragraph">
<p>NiFi provides several extension points to provide developers the
ability to add functionality to the application to meet their needs. The following list provides a
high-level description of the most common extension points:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Processor</p>
<div class="ulist">
<ul>
<li>
<p>The Processor interface is the mechanism through which NiFi exposes access to
<a href="developer-guide.html#flowfile">FlowFile</a>s, their attributes, and their content. The Processor is the basic building
block used to comprise a NiFi dataflow. This interface is used to accomplish
all of the following tasks:</p>
<div class="ulist">
<ul>
<li>
<p>Create FlowFiles</p>
</li>
<li>
<p>Read FlowFile content</p>
</li>
<li>
<p>Write FlowFile content</p>
</li>
<li>
<p>Read FlowFile attributes</p>
</li>
<li>
<p>Update FlowFile attributes</p>
</li>
<li>
<p>Ingest data</p>
</li>
<li>
<p>Egress data</p>
</li>
<li>
<p>Route data</p>
</li>
<li>
<p>Extract data</p>
</li>
<li>
<p>Modify data</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</li>
<li>
<p>ReportingTask</p>
<div class="ulist">
<ul>
<li>
<p>The ReportingTask interface is a mechanism that NiFi exposes to allow metrics,
monitoring information, and internal NiFi state to be published to external
endpoints, such as log files, e-mail, and remote web services.</p>
</li>
</ul>
</div>
</li>
<li>
<p>ParameterProvider</p>
<div class="ulist">
<ul>
<li>
<p>The ParameterProvider interface allows Parameters to be provided by external sources. Provided parameters
are still stored in Parameter Contexts, but this mechanism allows them to be provided and managed externally.</p>
</li>
</ul>
</div>
</li>
<li>
<p>ControllerService</p>
<div class="ulist">
<ul>
<li>
<p>A ControllerService provides shared state and functionality across Processors, other ControllerServices,
Parameter Providers, and ReportingTasks within a single JVM. An example use case may include loading a very
large dataset into memory. By performing this work in a ControllerService, the data
can be loaded once and be exposed to all Processors via this service, rather than requiring
many different Processors to load the dataset themselves.</p>
</li>
</ul>
</div>
</li>
<li>
<p>FlowFilePrioritizer</p>
<div class="ulist">
<ul>
<li>
<p>The FlowFilePrioritizer interface provides a mechanism by which <a href="developer-guide.html#flowfile">FlowFile</a>s
in a queue can be prioritized, or sorted, so that the FlowFiles can be processed in an order
that is most effective for a particular use case.</p>
</li>
</ul>
</div>
</li>
<li>
<p>AuthorityProvider</p>
<div class="ulist">
<ul>
<li>
<p>An AuthorityProvider is responsible for determining which privileges and roles, if any,
a given user should be granted.</p>
</li>
</ul>
</div>
</li>
</ul>
</div>
</div>
</div>
<div class="sect1">
<h2 id="processor_api"><a class="anchor" href="developer-guide.html#processor_api"></a>Processor API</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The Processor is the most widely used Component available in NiFi.
Processors are the only Component
to which access is given to create, remove, modify, or inspect
FlowFiles (data and attributes).</p>
</div>
<div class="paragraph">
<p>All Processors are loaded and instantiated using Java&#8217;s ServiceLoader
mechanism. This means that all
Processors must adhere to the following rules:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The Processor must have a default constructor.</p>
</li>
<li>
<p>The Processor&#8217;s JAR file must contain an entry in the META-INF/services directory named
<code>org.apache.nifi.processor.Processor</code>. This is a text file where each line contains the
fully-qualified class name of a Processor.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>While <code>Processor</code> is an interface that can be implemented directly, it
will be extremely rare to do so, as
the <code>org.apache.nifi.processor.AbstractProcessor</code> is the base class
for almost all Processor implementations. The <code>AbstractProcessor</code> class provides a significant
amount of functionality, which makes the task of developing a Processor much easier and more convenient.
For the scope of this document, we will focus primarily on the <code>AbstractProcessor</code> class when dealing
with the Processor API.</p>
</div>
<div class="paragraph">
<div class="title">Concurrency Note</div>
<p>NiFi is a highly concurrent framework. This means that all extensions
must be thread-safe. If unfamiliar with writing concurrent software in Java, it is highly
recommended that you familiarize yourself with the principles of Java concurrency.</p>
</div>
<div class="sect2">
<h3 id="supporting_api"><a class="anchor" href="developer-guide.html#supporting_api"></a>Supporting API</h3>
<div class="paragraph">
<p>In order to understand the Processor API, we must first understand -
at least at a high level - several supporting classes and interfaces, which are discussed below.</p>
</div>
<div class="sect3">
<h4 id="flowfile"><a class="anchor" href="developer-guide.html#flowfile"></a>FlowFile</h4>
<div class="paragraph">
<p>A FlowFile is a logical notion that correlates a piece of data with a
set of Attributes about that data.
Such attributes include a FlowFile&#8217;s unique identifier, as well as its
name, size, and any number of other
flow-specific values. While the contents and attributes of a FlowFile
can change, the FlowFile object is
immutable. Modifications to a FlowFile are made possible by the ProcessSession.</p>
</div>
<div class="paragraph">
<p>The core attributes for FlowFiles are defined in the <code>org.apache.nifi.flowfile.attributes.CoreAttributes</code> enum.
The most common attributes you&#8217;ll see are <code>filename</code>, <code>path</code> and <code>uuid</code>. The string in parentheses is the value of the
attribute within the <code>CoreAttributes</code> enum and how it appears in the UI/API.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Filename (<code>filename</code>): The filename of the FlowFile. The filename should not contain any directory structure.</p>
</li>
<li>
<p>UUID (<code>uuid</code>): A Universally Unique Identifier assigned to this FlowFile that distinguishes the FlowFile from other FlowFiles in the system.</p>
</li>
<li>
<p>Path (<code>path</code>): The FlowFile&#8217;s path indicates the relative directory to which a FlowFile belongs and does not contain the filename.</p>
</li>
<li>
<p>Absolute Path (<code>absolute.path</code>): The FlowFile&#8217;s absolute path indicates the absolute directory to which a FlowFile belongs and does not contain the filename.</p>
</li>
<li>
<p>Priority (<code>priority</code>): A numeric value indicating the FlowFile priority.</p>
</li>
<li>
<p>MIME Type (<code>mime.type</code>): The MIME Type of this FlowFile.</p>
</li>
<li>
<p>Discard Reason (<code>discard.reason</code>): Specifies the reason that a FlowFile is being discarded.</p>
</li>
<li>
<p>Alternate Identifier (<code>alternate.identifier</code>): Indicates an identifier other than the FlowFile&#8217;s UUID that is known to refer to this FlowFile.</p>
</li>
</ul>
</div>
<div class="sect4">
<h5 id="additional-common-attributes"><a class="anchor" href="developer-guide.html#additional-common-attributes"></a>Additional Common Attributes</h5>
<div class="paragraph">
<p>While these attributes are not members of the <code>CoreAttributes</code> enum, they are de facto standards across the system and found on most FlowFiles.</p>
</div>
<div class="ulist">
<ul>
<li>
<p>File Size (<code>fileSize</code>): The size of the FlowFile content in bytes.</p>
</li>
<li>
<p>Entry Date (<code>entryDate</code>): The date and time at which the FlowFile entered the system (i.e., was created). The value of this attribute is a number that represents the number of milliseconds since midnight, Jan. 1, 1970 (UTC).</p>
</li>
<li>
<p>Lineage Start Date (<code>lineageStartDate</code>): Any time that a FlowFile is cloned, merged, or split, this results in a "child" FlowFile being created. As those children are then cloned, merged, or split, a chain of ancestors is built. This value represents the date and time at which the oldest ancestor entered the system. Another way to think about this is that this attribute represents the latency of the FlowFile through the system. The value is a number that represents the number of milliseconds since midnight, Jan. 1, 1970 (UTC).</p>
</li>
</ul>
</div>
</div>
</div>
<div class="sect3">
<h4 id="process_session"><a class="anchor" href="developer-guide.html#process_session"></a>ProcessSession</h4>
<div class="paragraph">
<p>The ProcessSession, often referred to as simply a "session," provides
a mechanism by which FlowFiles can be created, destroyed, examined, cloned, and transferred to other
Processors. Additionally, a ProcessSession provides mechanism for creating modified versions of
FlowFiles, by adding or removing attributes, or by modifying the FlowFile&#8217;s content. The ProcessSession
also exposes a mechanism for emitting <a href="developer-guide.html#provenance_events">Provenance Events</a> that provide for the ability to track the
lineage and history of a FlowFile. After operations are performed on one or more FlowFiles, a
ProcessSession can be either committed or rolled back.</p>
</div>
</div>
<div class="sect3">
<h4 id="process_context"><a class="anchor" href="developer-guide.html#process_context"></a>ProcessContext</h4>
<div class="paragraph">
<p>The ProcessContext provides a bridge between a Processor and the framework. It provides information
about how the Processor is currently configured and allows the Processor to perform
Framework-specific tasks, such as yielding its resources so that the framework will schedule other
Processors to run without consuming resources unnecessarily.</p>
</div>
</div>
<div class="sect3">
<h4 id="property_descriptor"><a class="anchor" href="developer-guide.html#property_descriptor"></a>PropertyDescriptor</h4>
<div class="paragraph">
<p>PropertyDescriptor defines a property that is to be used by a
Processor, ReportingTask, ParameterProvider, or ControllerService.
The definition of a property includes its name, a description of the
property, an optional default value,
validation logic, and an indicator as to whether or not the property
is required in order for the Processor
to be valid. PropertyDescriptors are created by instantiating an
instance of the <code>PropertyDescriptor.Builder</code>
class, calling the appropriate methods to fill in the details about
the property, and finally calling
the <code>build</code> method.</p>
</div>
</div>
<div class="sect3">
<h4 id="validator"><a class="anchor" href="developer-guide.html#validator"></a>Validator</h4>
<div class="paragraph">
<p>A PropertyDescriptor MUST specify one or more Validators that can be
used to ensure that the user-entered value
for a property is valid. If a Validator indicates that a property
value is invalid, the Component will not be
able to be run or used until the property becomes valid. If a
Validator is not specified, the Component will be assumed invalid and
NiFi will report that the property is not supported.</p>
</div>
</div>
<div class="sect3">
<h4 id="validation_context"><a class="anchor" href="developer-guide.html#validation_context"></a>ValidationContext</h4>
<div class="paragraph">
<p>When validating property values, a ValidationContext can be used to
obtain ControllerServices,
create PropertyValue objects, and compile and evaluate property values
using the Expression Language.</p>
</div>
</div>
<div class="sect3">
<h4 id="property_value"><a class="anchor" href="developer-guide.html#property_value"></a>PropertyValue</h4>
<div class="paragraph">
<p>All property values returned to a Processor are returned in the form
of a PropertyValue object. This
object has convenience methods for converting the value from a String
to other forms, such as numbers
and time periods, as well as providing an API for evaluating the
Expression Language.</p>
</div>
</div>
<div class="sect3">
<h4 id="relationship"><a class="anchor" href="developer-guide.html#relationship"></a>Relationship</h4>
<div class="paragraph">
<p>Relationships define the routes to which a FlowFile may be transferred
from a Processor. Relationships
are created by instantiating an instance of the <code>Relationship.Builder</code>
class, calling the appropriate methods
to fill in the details of the Relationship, and finally calling the
<code>build</code> method.</p>
</div>
</div>
<div class="sect3">
<h4 id="supporting_api_state_manager"><a class="anchor" href="developer-guide.html#supporting_api_state_manager"></a>StateManager</h4>
<div class="paragraph">
<p>The StateManager provides Processors, Reporting Tasks, and Controller Services a mechanism
for easily storing and retrieving state. The API is similar to that of ConcurrentHashMap
but requires a Scope for each operation. The Scope indicates whether the state is to be
retrieved/stored locally or in a cluster-wide manner. For more information, see the
<a href="developer-guide.html#state_manager">State Manager</a> section.</p>
</div>
</div>
<div class="sect3">
<h4 id="processor_initialization_context"><a class="anchor" href="developer-guide.html#processor_initialization_context"></a>ProcessorInitializationContext</h4>
<div class="paragraph">
<p>After a Processor is created, its <code>initialize</code> method will be called
with an <code>InitializationContext</code> object.
This object exposes configuration to the Processor that will not
change throughout the life of the Processor,
such as the unique identifier of the Processor.</p>
</div>
</div>
<div class="sect3">
<h4 id="ComponentLog"><a class="anchor" href="developer-guide.html#ComponentLog"></a>ComponentLog</h4>
<div class="paragraph">
<p>Processors are encouraged to perform their logging via the
<code>ComponentLog</code> interface, rather than obtaining
a direct instance of a third-party logger. This is because logging via
the ComponentLog allows the framework
to render log messages that exceeds a configurable severity level to
the User Interface, allowing those who
monitor the dataflow to be notified when important events occur.
Additionally, it provides a consistent logging
format for all Processors by logging stack traces when in DEBUG mode
and providing the Processor&#8217;s unique
identifier in log messages.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="AbstractProcessor"><a class="anchor" href="developer-guide.html#AbstractProcessor"></a>AbstractProcessor API</h3>
<div class="paragraph">
<p>Since the vast majority of Processors will be created by extending the
AbstractProcessor, it is the
abstract class that we will examine in this section. The
AbstractProcessor provides several methods that
will be of interest to Processor developers.</p>
</div>
<div class="sect3">
<h4 id="processor-initialization"><a class="anchor" href="developer-guide.html#processor-initialization"></a>Processor Initialization</h4>
<div class="paragraph">
<p>When a Processor is created, before any other methods are invoked, the
<code>init</code> method of the
AbstractProcessor will be invoked. The method takes a single argument,
which is of type
<code>ProcessorInitializationContext</code>. The context object supplies the
Processor with a ComponentLog,
the Processor&#8217;s unique identifier, and a ControllerServiceLookup that
can be used to interact with the
configured ControllerServices. Each of these objects is stored by the
AbstractProcessor and may be obtained by
subclasses via the <code>getLogger</code>, <code>getIdentifier</code>, and
<code>getControllerServiceLookup</code> methods, respectively.</p>
</div>
</div>
<div class="sect3">
<h4 id="exposing-processors-relationships"><a class="anchor" href="developer-guide.html#exposing-processors-relationships"></a>Exposing Processor&#8217;s Relationships</h4>
<div class="paragraph">
<p>In order for a Processor to transfer a FlowFile to a new destination
for follow-on processing, the
Processor must first be able to expose to the Framework all of the
Relationships that it currently supports.
This allows users of the application to connect Processors to one
another by creating
Connections between Processors and assigning the appropriate
Relationships to those Connections.</p>
</div>
<div class="paragraph">
<p>A Processor exposes the valid set of Relationships by overriding the
<code>getRelationships</code> method.
This method takes no arguments and returns a <code>Set</code> of <code>Relationship</code>
objects. For most Processors, this Set
will be static, but other Processors will generate the Set
dynamically, based on user configuration.
For those Processors for which the Set is static, it is advisable to
create an immutable Set in the Processor&#8217;s
constructor or init method and return that value, rather than
dynamically generating the Set. This
pattern lends itself to cleaner code and better performance.</p>
</div>
</div>
<div class="sect3">
<h4 id="exposing-processor-properties"><a class="anchor" href="developer-guide.html#exposing-processor-properties"></a>Exposing Processor Properties</h4>
<div class="paragraph">
<p>Most Processors will require some amount of user configuration before
they are able to be used. The properties
that a Processor supports are exposed to the Framework via the
<code>getSupportedPropertyDescriptors</code> method.
This method takes no arguments and returns a <code>List</code> of
<code>PropertyDescriptor</code> objects. The order of the objects in the
List is important in that it dictates the order in which the
properties will be rendered in the User Interface.</p>
</div>
<div class="paragraph">
<p>A <code>PropertyDescriptor</code> object is constructed by creating a new
instance of the <code>PropertyDescriptor.Builder</code> object,
calling the appropriate methods on the builder, and finally calling
the <code>build</code> method.</p>
</div>
<div class="sect4">
<h5 id="dynamic-processor-properties"><a class="anchor" href="developer-guide.html#dynamic-processor-properties"></a>Dynamic Processor Properties</h5>
<div class="paragraph">
<p>In addition to standard properties, it is sometimes
desirable to allow users to configure
additional properties whose names are not predefined.
This can be achieved by overriding the
<code>getSupportedDynamicPropertyDescriptor</code> method. This method takes a
<code>String</code> as its only argument, which
indicates the name of the property. The method returns a
<code>PropertyDescriptor</code> object that can be used to validate
both the name of the property, as well as the value. Any
PropertyDescriptor that is returned from this method
should be built setting the value of <code>isDynamic</code> to <code>true</code> in the
<code>PropertyDescriptor.Builder</code> class. The default
behavior of AbstractProcessor is to not allow any dynamically created
properties.</p>
</div>
</div>
<div class="sect4">
<h5 id="sensitive-dynamic-properties"><a class="anchor" href="developer-guide.html#sensitive-dynamic-properties"></a>Sensitive Dynamic Properties</h5>
<div class="paragraph">
<p>The default implementation for dynamic properties does not treat the
property values as sensitive. This approach is sufficient when configuring
features such as FlowFile attributes or custom expressions, but it does
not provide protection for values such as passwords or keys.</p>
</div>
<div class="paragraph">
<p>NiFi 1.17.0 introduced framework support for sensitive dynamic properties
through a new behavior annotation named <code>SupportsSensitiveDynamicProperties</code>.
The annotation can be applied to a Processor, Controller Service, or
Reporting Task that supports dynamic properties through the
<code>getSupportedDynamicPropertyDescriptor</code> method. The annotation indicates
that the component allows individual dynamic properties to be marked as
sensitive for the purpose of persistence and framework processing.</p>
</div>
<div class="paragraph">
<p>The <code>getSupportedDynamicPropertyDescriptor</code> must return a <code>PropertyDescriptor</code>
with the <code>sensitive</code> field set to <code>false</code> to allow customization of sensitive
status. Setting the <code>sensitive</code> field to <code>true</code> in this method forces all
dynamic properties to be handled as sensitive. This approach allows sensitive
status to be upgraded in supported components, but not downgraded.</p>
</div>
<div class="paragraph">
<p>Secure handling of sensitive property values is the responsibility of the
annotated class. Components that support sensitive dynamic properties must
not log property values or provide property values as FlowFile attributes.</p>
</div>
</div>
</div>
<div class="sect3">
<h4 id="validating-processor-properties"><a class="anchor" href="developer-guide.html#validating-processor-properties"></a>Validating Processor Properties</h4>
<div class="paragraph">
<p>A Processor is not able to be started if its configuration is not
valid. Validation of a Processor property can
be achieved by setting a Validator on a PropertyDescriptor or by
restricting the allowable values for a
property via the PropertyDescriptor.Builder&#8217;s <code>allowableValues</code> method
or <code>identifiesControllerService</code> method.</p>
</div>
<div class="paragraph">
<p>In addition, if a property is dependent on another property (by means of the <code>PropertyDescriptor.Builder&#8217;s
`dependsOn</code> method) and the dependency is not satisfied, then the Property will be validated.</p>
</div>
<div class="paragraph">
<p>For example, consider the following two Property Descriptors:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code>PropertyDescriptor USE_FILE = new PropertyDescriptor.Buildler()
.name("Use File")
.displayName("Use File")
.required(true)
.allowableValues("true", "false")
.defaultValue("true")
.build();</code></pre>
</div>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code>PropertyDescriptor FILE = new PropertyDescriptor.Builder()
.name("File to Use")
.displayName("File to Use")
.required(true)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.dependsOn(USE_FILE, "true")
.build();</code></pre>
</div>
</div>
<div class="paragraph">
<p>In this case, if the "Use File" property is set to <code>true</code>, then the Processor will not be valid unless the "File to Use" property is
set to a valid filename. If "Use File" is set to <code>true</code> and "File to Use" does not have a value set, the Processor will be invalid
(because the "File to Use" property is required). If "Use File" is set to <code>true</code> and "File to Use" has a value set but the specified
file does not exist, the Processor will also be invalid because the "File to Use" property is invalid according to the Validator.</p>
</div>
<div class="paragraph">
<p>However, if the "Use File" property is set to <code>false</code>, then the "File to Use" property is said to have a dependency that is not satisfied.
As a result, the "File to Use" property will not be considered in the validation. Therefore, if "Use File" is set to <code>false</code> and "File to Use"
has no value said, the Processor will still be valid (even though "File to Use" is required, it is only required if "Use File" is <code>true</code>).
Likewise, if "File to Use" is set to a non-existent filename such as <code>/file/that/does/not/exist</code>, the Processor will still be valid so long as
the "Use File" property is set to <code>false</code>.</p>
</div>
<div class="paragraph">
<p>Furthermore, the "File to Use" property will not even be shown in the NiFi UI unless "Use File" is set to <code>true</code>.</p>
</div>
<div class="paragraph">
<p>There are times, though, when validating a Processor&#8217;s properties
individually is not sufficient. For this purpose,
the AbstractProcessor exposes a <code>customValidate</code> method. The method
takes a single argument of type <code>ValidationContext</code>.
The return value of this method is a <code>Collection</code> of
<code>ValidationResult</code> objects that describe any problems that were
found during validation. Only those ValidationResult objects whose
<code>isValid</code> method returns <code>false</code> should be returned.
This method will be invoked only if all properties are valid according
to their associated Validators and Allowable Values.
I.e., this method will be called only if all properties are valid
in-and-of themselves, and this method allows for
validation of a Processor&#8217;s configuration as a whole.</p>
</div>
</div>
<div class="sect3">
<h4 id="responding-to-changes-in-configuration"><a class="anchor" href="developer-guide.html#responding-to-changes-in-configuration"></a>Responding to Changes in Configuration</h4>
<div class="paragraph">
<p>It is sometimes desirable to have a Processor eagerly react when its
properties are changed. The <code>onPropertyModified</code>
method allows a Processor to do just that. When a user changes the
property values for a Processor, the
<code>onPropertyModified</code> method will be called for each modified property.
The method takes three arguments: the PropertyDescriptor that
indicates which property was modified,
the old value, and the new value. If the property had no previous
value, the second argument will be <code>null</code>. If the property
was removed, the third argument will be <code>null</code>. It is important to
note that this method will be called regardless of whether
or not the values are valid. This method will be called only when a
value is actually modified, rather than being
called when a user updates a Processor without changing its value. At
the point that this method is invoked, it is guaranteed
that the thread invoking this method is the only thread currently
executing code in the Processor, unless the Processor itself
creates its own threads.</p>
</div>
</div>
<div class="sect3">
<h4 id="performing-the-work"><a class="anchor" href="developer-guide.html#performing-the-work"></a>Performing the Work</h4>
<div class="paragraph">
<p>When a Processor has work to do, it is scheduled to do so by having
its <code>onTrigger</code> method called by the framework.
The method takes two arguments: a <code>ProcessContext</code> and a
<code>ProcessSession</code>. The first step in the <code>onTrigger</code> method
is often to obtain a FlowFile on which the work is to be performed by
calling one of the <code>get</code> methods on the ProcessSession.
For Processors that ingest data into NiFi from external sources, this
step is skipped. The Processor is then free to examine
FlowFile attributes; add, remove, or modify attributes; read or modify
FlowFile content; and transfer FlowFiles to the appropriate
Relationships.</p>
</div>
</div>
<div class="sect3">
<h4 id="when-processors-are-triggered"><a class="anchor" href="developer-guide.html#when-processors-are-triggered"></a>When Processors are Triggered</h4>
<div class="paragraph">
<p>A Processor&#8217;s <code>onTrigger</code> method will be called only when it is
scheduled to run and when work exists for the Processor.
Work is said to exist for a Processor if any of the following conditions is met:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>A Connection whose destination is the Processor has at least one
FlowFile in its queue</p>
</li>
<li>
<p>The Processors has no incoming Connections</p>
</li>
<li>
<p>The Processor is annotated with the @TriggerWhenEmpty annotation</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Several factors exist that will contribute to when a Processor&#8217;s
<code>onTrigger</code> method is invoked. First, the Processor will not
be triggered unless a user has configured the Processor to run. If a
Processor is scheduled to run, the Framework periodically
(the period is configured by users in the User Interface) checks if
there is work for the Processor to do, as described above.
If so, the Framework will check downstream destinations of the
Processor. If any of the Processor&#8217;s outbound Connections is full,
by default, the Processor will not be scheduled to run.</p>
</div>
<div class="paragraph">
<p>However, the <code>@TriggerWhenAnyDestinationAvailable</code> annotation may be
added to the Processor&#8217;s class. In this case, the requirement
is changed so that only one downstream destination must be "available"
(a destination is considered "available" if the Connection&#8217;s
queue is not full), rather than requiring that all downstream
destinations be available.</p>
</div>
<div class="paragraph">
<p>Also related to Processor scheduling is the <code>@TriggerSerially</code>
annotation. Processors that use this Annotation will never have more
than one thread running the <code>onTrigger</code> method simultaneously. It is
crucial to note, though, that the thread executing the code
may change from invocation to invocation. Therefore, care must still
be taken to ensure that the Processor is thread-safe!</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="component-lifecycle"><a class="anchor" href="developer-guide.html#component-lifecycle"></a>Component Lifecycle</h3>
<div class="paragraph">
<p>The NiFi API provides lifecycle support through use of Java
Annotations. The <code>org.apache.nifi.annotation.lifecycle</code> package
contains
several annotations for lifecycle management. The following
Annotations may be applied to Java methods in a NiFi component to
indicate to
the framework when the methods should be called. For the discussion of
Component Lifecycle, we will define a NiFi component as a
Processor, ControllerServices, ParameterProviders, or ReportingTask.</p>
</div>
<div class="sect3">
<h4 id="onadded"><a class="anchor" href="developer-guide.html#onadded"></a>@OnAdded</h4>
<div class="paragraph">
<p>The <code>@OnAdded</code> annotation causes a method to be invoked as soon as a
component is created. The
component&#8217;s <code>initialize</code> method (or <code>init</code> method, if subclasses
<code>AbstractProcessor</code>) will be invoked after the component is
constructed,
followed by methods that are annotated with <code>@OnAdded</code>. If any method
annotated with <code>@OnAdded</code> throws an Exception, an error will
be returned to the user, and that component will not be added to the
flow. Furthermore, other methods with this
Annotation will not be invoked. This method will be called only once
for the lifetime of a component.
Methods with this Annotation must take zero arguments.</p>
</div>
</div>
<div class="sect3">
<h4 id="onenabled"><a class="anchor" href="developer-guide.html#onenabled"></a>@OnEnabled</h4>
<div class="paragraph">
<p>The <code>@OnEnabled</code> annotation can be used to indicate a method should be called
whenever the Controller Service is enabled. Any method that has this annotation will be
called every time a user enables the service. Additionally, each time that NiFi
is restarted, if NiFi is configured to "auto-resume state" and the service
is enabled, the method will be invoked.</p>
</div>
<div class="paragraph">
<p>If a method with this annotation throws a Throwable, a log message and
bulletin will be issued for the component. In this event, the service will
remain in an 'ENABLING' state and will not be usable. All methods with this
annotation will then be called again after a delay. The service will not be
made available for use until all methods with this annotation have returned
without throwing anything.</p>
</div>
<div class="paragraph">
<p>Methods using this annotation must take either 0 arguments or a single argument
of type <code>org.apache.nifi.controller.ConfigurationContext</code>.</p>
</div>
<div class="paragraph">
<p>Note that this annotation will be ignored if applied to a ReportingTask,
Parameter Provider, or Processor. For a Controller Service, enabling and disabling
are considered lifecycle events, as the action makes them usable or unusable by other
components. However, for a Processor and a Reporting Task, these are not
lifecycle events but rather a mechanism to allow a component to be excluded
when starting or stopping a group of components.</p>
</div>
</div>
<div class="sect3">
<h4 id="onremoved"><a class="anchor" href="developer-guide.html#onremoved"></a>@OnRemoved</h4>
<div class="paragraph">
<p>The <code>@OnRemoved</code> annotation causes a method to be invoked before a
component is removed from the flow.
This allows resources to be cleaned up before removing a component.
Methods with this annotation must take zero arguments.
If a method with this annotation throws an Exception, the component
will still be removed.</p>
</div>
</div>
<div class="sect3">
<h4 id="onscheduled"><a class="anchor" href="developer-guide.html#onscheduled"></a>@OnScheduled</h4>
<div class="paragraph">
<p>This annotation indicates that a method should be called every time
the component is scheduled to run. Because ControllerServices
are not scheduled, using this annotation on a ControllerService does
not make sense and will not be honored. It should be
used only for Processors and Reporting Tasks. If any method with this
annotation throws an Exception, other methods with this
annotation will not be invoked, and a notification will be presented
to the user. In this case, methods annotated with
<code>@OnUnscheduled</code> are then triggered, followed by methods with the
<code>@OnStopped</code> annotation (during this state, if any of these
methods throws an Exception, those Exceptions are ignored). The
component will then yield its execution for some period of time,
referred to as the "Administrative Yield Duration," which is a value
that is configured in the <code>nifi.properties</code> file. Finally, the
process will start again, until all of the methods annotated with
<code>@OnScheduled</code> have returned without throwing any Exception.
Methods with this annotation may take zero arguments or may take a
single argument. If the single argument variation is used,
the argument must be of type <code>ProcessContext</code> if the component is a
Processor or <code>ConfigurationContext</code> if the component
is a ReportingTask.</p>
</div>
</div>
<div class="sect3">
<h4 id="onunscheduled"><a class="anchor" href="developer-guide.html#onunscheduled"></a>@OnUnscheduled</h4>
<div class="paragraph">
<p>Methods with this annotation will be called whenever a Processor or
ReportingTask is no longer scheduled to run. At that time, many threads
may still be active in the Processor&#8217;s <code>onTrigger</code> method. If such a method
throws an Exception, a log message will be generated, and the
Exception will be otherwise
ignored and other methods with this annotation will still be invoked.
Methods with this annotation may take zero arguments or may take a
single argument.
If the single argument variation is used, the argument must be of type
<code>ProcessContext</code> if the component is a Processor or
<code>ConfigurationContext</code> if the
component is a ReportingTask.</p>
</div>
</div>
<div class="sect3">
<h4 id="onstopped"><a class="anchor" href="developer-guide.html#onstopped"></a>@OnStopped</h4>
<div class="paragraph">
<p>Methods with this annotation will be called when a Processor or
ReportingTask is no longer scheduled to run
and all threads have returned from the <code>onTrigger</code> method. If such a
method throws an Exception,
a log message will be generated, and the Exception will otherwise be
ignored; other methods with
this annotation will still be invoked.
Methods with this annotation are permitted to take either 0 or 1 argument. If
an argument is used, it must be of type ConfigurationContext if the
component is a ReportingTask or of type ProcessContext if the
component is a Processor.</p>
</div>
</div>
<div class="sect3">
<h4 id="onshutdown"><a class="anchor" href="developer-guide.html#onshutdown"></a>@OnShutdown</h4>
<div class="paragraph">
<p>Any method that is annotated with the <code>@OnShutdown</code> annotation will be
called when NiFi is successfully
shut down. If such a method throws an Exception, a log message will be
generated, and the
Exception will be otherwise ignored and other methods with this
annotation will still be invoked.
Methods with this annotation must take zero arguments. Note: while
NiFi will attempt to invoke methods
with this annotation on all components that use it, this is not always
possible. For example, the process
may be killed unexpectedly, in which case it does not have a chance to
invoke these methods. Therefore,
while methods using this annotation can be used to clean up resources,
for instance, they should not be
relied upon for critical business logic.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="component-notification"><a class="anchor" href="developer-guide.html#component-notification"></a>Component Notification</h3>
<div class="paragraph">
<p>The NiFi API provides notification support through use of Java
Annotations. The <code>org.apache.nifi.annotation.notification</code> package
contains several annotations for notification management. The following
annotations may be applied to Java methods in a NiFi component to
indicate to the framework when the methods should be called. For the
discussion of Component Notification, we will define a NiFi component
as a <strong>Processor</strong>, <strong>Controller Service</strong>, or <strong>Reporting Task</strong>.</p>
</div>
<div class="sect3">
<h4 id="onprimarynodestatechange"><a class="anchor" href="developer-guide.html#onprimarynodestatechange"></a>@OnPrimaryNodeStateChange</h4>
<div class="paragraph">
<p>The <code>@OnPrimaryNodeStateChange</code> annotation causes a method to be invoked
as soon as the state of the Primary Node in a cluster has changed.
Methods with this annotation should take either no arguments or one
argument of type <code>PrimaryNodeState</code>. The <code>PrimaryNodeState</code> provides
context about what changed so that the component can take appropriate
action. The <code>PrimaryNodeState</code> enumerator has two possible values:
<code>ELECTED_PRIMARY_NODE</code> (the node receiving this
state has been elected the Primary Node of the NiFi cluster), or
<code>PRIMARY_NODE_REVOKED</code> (the node receiving this state was the Primary
Node but has now had its Primary Node role revoked).</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="restricted"><a class="anchor" href="developer-guide.html#restricted"></a>Restricted</h3>
<div class="paragraph">
<p>A Restricted component is one that can be used to execute arbitrary unsanitized code provided by the operator
through the NiFi REST API/UI or can be used to obtain or alter data on the NiFi host system using the NiFi OS
credentials. These components could be used by an otherwise authorized NiFi user to go beyond the intended use of
the application, escalate privilege, or could expose data about the internals of the NiFi process or the host
system. All of these capabilities should be considered privileged, and admins should be aware of these
capabilities and explicitly enable them for a subset of trusted users.</p>
</div>
<div class="paragraph">
<p>A Processor, Controller Service, or Reporting Task can be marked with the @Restricted annotation. This
will result in the component being treated as restricted and will require a user to be explicitly added to the
list of users who can access restricted components. Once a user is permitted to access restricted components,
they will be allowed to create and modify those components assuming all other permissions are permitted.
Without access to restricted components, a user will still be aware these types of components exist but will
be unable to create or modify them even with otherwise sufficient permissions.</p>
</div>
</div>
<div class="sect2">
<h3 id="state_manager"><a class="anchor" href="developer-guide.html#state_manager"></a>State Manager</h3>
<div class="paragraph">
<p>From the ProcessContext, ReportingContext, and ControllerServiceInitializationContext, components are
able to call the <code>getStateManager()</code> method. This State Manager is responsible for providing a simple API
for storing and retrieving state. This mechanism is intended to provide developers with the ability to
very easily store a set of key/value pairs, retrieve those values, and update them atomically. The state
can be stored local to the node or across all nodes in a cluster. It is important to note, however, that
this mechanism is intended only to provide a mechanism for storing very 'simple' state. As such, the API
simply allows a <code>Map&lt;String, String&gt;</code> to be stored and retrieved and for the entire Map to be atomically
replaced. Moreover, the only implementation that is currently supported for storing cluster-wide state is
backed by ZooKeeper. As such, the entire State Map must be less than 1 MB in size, after being serialized.
Attempting to store more than this will result in an Exception being thrown. If the interactions required
by the Processor for managing state are more complex than this (e.g., large amounts of data must be stored
and retrieved, or individual keys must be stored and fetched individually) than a different mechanism should
be used (e.g., communicating with an external database).</p>
</div>
<div class="sect3">
<h4 id="state_scope"><a class="anchor" href="developer-guide.html#state_scope"></a>Scope</h4>
<div class="paragraph">
<p>When communicating with the State Manager, all method calls require that a Scope be provided. This Scope will
either be <code>Scope.LOCAL</code> or <code>Scope.CLUSTER</code>. If NiFi is run in a cluster, this Scope provides important information
to the framework about how the operation should occur.</p>
</div>
<div class="paragraph">
<p>If state as stored using <code>Scope.CLUSTER</code>, then all nodes in the cluster will be communicating with the same
state storage mechanism. If state is stored and retrieved using <code>Scope.LOCAL</code>, then each node will see a different
representation of the state.</p>
</div>
<div class="paragraph">
<p>It is also worth noting that if NiFi is configured to run as a standalone instance, rather than running in a cluster,
a scope of <code>Scope.LOCAL</code> is always used. This is done in order to allow the developer of a NiFi component to write the code
in one consistent way, without worrying about whether or not the NiFi instance is clustered. The developer should instead assume
that the instance is clustered and write the code accordingly.</p>
</div>
</div>
<div class="sect3">
<h4 id="storing-and-retrieving-state"><a class="anchor" href="developer-guide.html#storing-and-retrieving-state"></a>Storing and Retrieving State</h4>
<div class="paragraph">
<p>State is stored using the StateManager&#8217;s <code>getState</code>, <code>setState</code>, <code>replace</code>, and <code>clear</code> methods. All of these methods
require that a Scope be provided. It should be noted that the state that is stored with the Local scope is entirely different
than state stored with a Cluster scope. If a Processor stores a value with the key of <em>My Key</em> using the <code>Scope.CLUSTER</code> scope,
and then attempts to retrieve the value using the <code>Scope.LOCAL</code> scope, the value retrieved will be <code>null</code> (unless a value was
also stored with the same key using the <code>Scope.CLUSTER</code> scope). Each Processor&#8217;s state is stored in isolation from other
Processors' state.</p>
</div>
<div class="paragraph">
<p>It follows, then, that two Processors cannot share the same state. There are, however, some circumstances in which it is very
necessary to share state between two Processors of different types, or two Processors of the same type. This can be accomplished
by using a Controller Service. By storing and retrieving state from a Controller Service, multiple Processors can use the same
Controller Service and the state can be exposed via the Controller Service&#8217;s API.</p>
</div>
</div>
<div class="sect3">
<h4 id="unit-tests"><a class="anchor" href="developer-guide.html#unit-tests"></a>Unit Tests</h4>
<div class="paragraph">
<p>NiFi&#8217;s Mock Framework provides an extensive collection of tools to perform unit testing of Processors. Processor unit tests typically
begin with the <code>TestRunner</code> class. As a result, the <code>TestRunner</code> class contains a <code>getStateManager</code> method of its own. The StateManager
that is returned, however, is of a specific type: <code>MockStateManager</code>. This implementation provides several methods in addition to those
defined by the <code>StateManager</code> interface, that help developers to more easily develop unit tests.</p>
</div>
<div class="paragraph">
<p>First, the <code>MockStateManager</code> implements the <code>StateManager</code> interface, so all of the state can be examined from within a unit test.
Additionally, the <code>MockStateManager</code> exposes a handful of <code>assert*</code> methods to perform assertions that the State is set as expected.
The <code>MockStateManager</code> also provides the ability to indicate that the unit test should immediately fail if state is updated for a particular
<code>Scope</code>.</p>
</div>
</div>
</div>
<div class="sect2">
<h3 id="reporting-processor-activity"><a class="anchor" href="developer-guide.html#reporting-processor-activity"></a>Reporting Processor Activity</h3>
<div class="paragraph">
<p>Processors are responsible for reporting their activity so that users
are able to understand what happens
to their data. Processors should log events via the ComponentLog,
which is accessible via the InitializationContext
or by calling the <code>getLogger</code> method of <code>AbstractProcessor</code>.</p>
</div>
<div class="paragraph">
<p>Additionally, Processors should use the <code>ProvenanceReporter</code>
interface, obtained via the ProcessSession&#8217;s
<code>getProvenanceReporter</code> method. The ProvenanceReporter should be used
to indicate any time that content is
received from an external source or sent to an external location. The
ProvenanceReporter also has methods for
reporting when a FlowFile is cloned, forked, or modified, and when
multiple FlowFiles are merged into a single FlowFile
as well as associating a FlowFile with some other identifier. However,
these functions are less critical to report, as
the framework is able to detect these things and emit appropriate
events on the Processor&#8217;s behalf. Yet, it is a best practice
for the Processor developer to emit these events, as it becomes
explicit in the code that these events are being emitted, and
the developer is able to provide additional details to the events,
such as the amount of time that the action took or
pertinent information about the action that was taken. If the
Processor emits an event, the framework will not emit a duplicate
event. Instead, it always assumes that the Processor developer knows
what is happening in the context of the Processor
better than the framework does. The framework may, however, emit a
different event. For example, if a Processor modifies both the
content of a FlowFile and its attributes and then emits only an
ATTRIBUTES_MODIFIED event, the framework will emit a CONTENT_MODIFIED
event. The framework will not emit an ATTRIBUTES_MODIFIED event if any
other event is emitted for that FlowFile (either by the
Processor or the framework). This is due to the fact that all
<a href="developer-guide.html#provenance_events">Provenance Events</a> know about the attributes of the FlowFile before the
event occurred as well as those attributes that occurred as a result
of the processing of that FlowFile, and as a result the
ATTRIBUTES_MODIFIED is generally considered redundant and would result
in a rendering of the FlowFile lineage being very verbose.
It is, however, acceptable for a Processor to emit this event along
with others, if the event is considered pertinent from the
perspective of the Processor.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="documenting-a-component"><a class="anchor" href="developer-guide.html#documenting-a-component"></a>Documenting a Component</h2>
<div class="sectionbody">
<div class="paragraph">
<p>NiFi attempts to make the user experience as simple and convenient as
possible by providing significant amount of documentation
to the user from within the NiFi application itself via the User
Interface. In order for this to happen, of course, Processor
developers must provide that documentation to the framework. NiFi
exposes a few different mechanisms for supplying documentation to
the framework.</p>
</div>
<div class="sect2">
<h3 id="documenting-properties"><a class="anchor" href="developer-guide.html#documenting-properties"></a>Documenting Properties</h3>
<div class="paragraph">
<p>Individual properties can be documented by calling the <code>description</code>
method of a PropertyDescriptor&#8217;s builder as such:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">public static final PropertyDescriptor MY_PROPERTY = new PropertyDescriptor.Builder()
.name("My Property")
.description("Description of the Property")
...
.build();</code></pre>
</div>
</div>
<div class="paragraph">
<p>If the property is to provide a set of allowable values, those values
are presented to the user in a drop-down field in the UI.
Each of those values can also be given a description:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">public static final AllowableValue EXTENSIVE = new AllowableValue("Extensive", "Extensive",
"Everything will be logged - use with caution!");
public static final AllowableValue VERBOSE = new AllowableValue("Verbose", "Verbose",
"Quite a bit of logging will occur");
public static final AllowableValue REGULAR = new AllowableValue("Regular", "Regular",
"Typical logging will occur");
public static final PropertyDescriptor LOG_LEVEL = new PropertyDescriptor.Builder()
.name("Amount to Log")
.description("How much the Processor should log")
.allowableValues(REGULAR, VERBOSE, EXTENSIVE)
.defaultValue(REGULAR.getValue())
...
.build();</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="documenting-relationships"><a class="anchor" href="developer-guide.html#documenting-relationships"></a>Documenting Relationships</h3>
<div class="paragraph">
<p>Processor Relationships are documented in much the same way that
properties are - by calling the <code>description</code> method of a
Relationship&#8217;s builder:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">public static final Relationship MY_RELATIONSHIP = new Relationship.Builder()
.name("My Relationship")
.description("This relationship is used only if the Processor fails to process the data.")
.build();</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="documenting-capability-and-keywords"><a class="anchor" href="developer-guide.html#documenting-capability-and-keywords"></a>Documenting Capability and Keywords</h3>
<div class="paragraph">
<p>The <code>org.apache.nifi.annotation.documentation</code> package provides Java
annotations that can be used to document components. The
CapabilityDescription
annotation can be added to a Processor, Reporting Task, or Controller
Service and is intended to provide a brief description of the
functionality
provided by the component. The Tags annotation has a <code>value</code> variable
that is defined to be an Array of Strings. As such, it is used
by providing multiple values as a comma-separated list of Strings
with curly braces. These values are then incorporated into the UI by
allowing
users to filter the components based on a tag (i.e., a keyword).
Additionally, the UI provides a tag cloud that allows users to select
the tags that
they want to filter by. The tags that are largest in the cloud are
those tags that exist the most on the components in that instance of
NiFi. An
example of using these annotations is provided below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">@Tags({"example", "documentation", "developer guide", "processor", "tags"})
@CapabilityDescription("Example Processor that provides no real functionality but is provided" +
" for an example in the Developer Guide")
public static final ExampleProcessor extends Processor {
...
}</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="documenting-flowfile-attribute-interaction"><a class="anchor" href="developer-guide.html#documenting-flowfile-attribute-interaction"></a>Documenting FlowFile Attribute Interaction</h3>
<div class="paragraph">
<p>Many times a processor will expect certain FlowFile attributes be set on in-bound FlowFiles in order
for the processor to function properly. In other cases a processor may update or
create FlowFile attributes on the out-bound FlowFile. Processor developers may document both of these
behaviors using the <code>ReadsAttribute</code> and <code>WritesAttribute</code> documentation annotations. These attributes are used to generate documentation
that gives users a better understanding of how a processor will interact with the flow.</p>
</div>
<div class="paragraph">
<p>Note: Because Java 7 does not support
repeated annotations on a type, you may need to use <code>ReadsAttributes</code> and <code>WritesAttributes</code> to indicate
that a processor reads or writes multiple FlowFile attributes. This annotation can only be applied to Processors. An example is listed below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">@WritesAttributes({ @WritesAttribute(attribute = "invokehttp.status.code", description = "The status code that is returned"),
@WritesAttribute(attribute = "invokehttp.status.message", description = "The status message that is returned"),
@WritesAttribute(attribute = "invokehttp.response.body", description = "The response body"),
@WritesAttribute(attribute = "invokehttp.request.url", description = "The request URL"),
@WritesAttribute(attribute = "invokehttp.tx.id", description = "The transaction ID that is returned after reading the response"),
@WritesAttribute(attribute = "invokehttp.remote.dn", description = "The DN of the remote server") })
public final class InvokeHTTP extends AbstractProcessor {</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="documenting-related-components"><a class="anchor" href="developer-guide.html#documenting-related-components"></a>Documenting Related Components</h3>
<div class="paragraph">
<p>Often Processors and ControllerServices are related to one another. Sometimes it is a put/get relation as in <code>PutFile</code> and <code>GetFile</code>.
Sometimes a Processor uses a ControllerService like <code>InvokeHTTP</code> and <code>StandardSSLContextService</code>. Sometimes one ControllerService uses another
like <code>DistributedMapCacheClientService</code> and <code>DistributedMapCacheServer</code>. Developers of these extension points may relate these
different components using the <code>SeeAlso</code> tag. This annotation links these components in the documentation.
<code>SeeAlso</code> can be applied to Processors, ControllerServices, ParameterProviders, and ReportingTasks. An example of how to do this is listed below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">@SeeAlso(GetFile.class)
public class PutFile extends AbstractProcessor {</code></pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="advanced-documentation"><a class="anchor" href="developer-guide.html#advanced-documentation"></a>Advanced Documentation</h3>
<div class="paragraph">
<p>When the documentation methods above are not sufficient, NiFi provides
the ability to expose more advanced documentation to the user via the
"Usage" documentation. When a user right-clicks on a Processor, NiFi
provides a "Usage" menu item in the context menu. Additionally, the
UI exposes a "Help" link in the top-right corner, from which the same
Usage information can be found.</p>
</div>
<div class="paragraph">
<p>The advanced documentation of a Processor is provided as an HTML file named <code>additionalDetails.html</code>.
This file should exist within a directory whose name is the
fully-qualified
name of the Processor, and this directory&#8217;s parent should be named
<code>docs</code> and exist in the root of the Processor&#8217;s jar.
This file will be linked from a generated HTML file that will contain
all the Capability, Keyword, PropertyDescription and Relationship information,
so it will not be necessary to duplicate that. This is a place
to provide a rich explanation of what this Processor is doing, what kind of
data it expects and produces, and what FlowFile attributes it expects and produces.
Because this documentation is in an HTML format, you may include images and tables
to best describe this component. The same methods can be used to provide advanced
documentation for Processors, ControllerServices, ParameterProviders, and ReportingTasks.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="provenance_events"><a class="anchor" href="developer-guide.html#provenance_events"></a>Provenance Events</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The different event types for provenance reporting are:</p>
</div>
<table class="tableblock frame-all grid-all stretch">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Provenance Event</th>
<th class="tableblock halign-left valign-top">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">ADDINFO</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for adding additional information such as new linkage to a new URI or UUID</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">ATTRIBUTES_MODIFIED</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a FlowFile&#8217;s attributes were modified in some way. This event is not needed when another event is reported at the same time, as the other event will already contain all FlowFile attributes</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">CLONE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a FlowFile is an exact duplicate of its parent FlowFile</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">CONTENT_MODIFIED</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a FlowFile&#8217;s content was modified in some way. When using this Event Type, it is advisable to provide details about how the content is modified</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">CREATE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a FlowFile was generated from data that was not received from a remote system or external process</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">DOWNLOAD</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that the contents of a FlowFile were downloaded by a user or external entity</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">DROP</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for the conclusion of an object&#8217;s life for some reason other than object expiration</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">EXPIRE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for the conclusion of an object&#8217;s life due to the object not being processed in a timely manner</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">FETCH</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that the contents of a FlowFile were overwritten using the contents of some external resource. This is similar to the RECEIVE event but varies in that RECEIVE events are intended to be used as the event that introduces the FlowFile into the system, whereas FETCH is used to indicate that the contents of an existing FlowFile were overwritten</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">FORK</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that one or more FlowFiles were derived from a parent FlowFile</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">JOIN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a single FlowFile is derived from joining together multiple parent FlowFiles</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">RECEIVE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for receiving data from an external process. This Event Type is expected to be the first event for a FlowFile. As such, a Processor that receives data from an external source and uses that data to replace the content of an existing FlowFile should use the FETCH event type, rather than the RECEIVE event type</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">REMOTE_INVOCATION</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a remote invocation was requested to an external endpoint (e.g. deleting a remote resource). The external endpoint may exist in a remote or a local system, but is external to NiFi</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">REPLAY</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for replaying a FlowFile. The UUID of the event indicates the UUID of the original FlowFile that is being replayed. The event contains one Parent UUID that is also the UUID of the FlowFile that is being replayed and one Child UUID that is the UUID of the a newly created FlowFile that will be re-queued for processing</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">ROUTE</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that a FlowFile was routed to a specified relationship and provides information about why the FlowFile was routed to this relationship</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">SEND</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates a provenance event for sending data to an external process</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">UNKNOWN</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Indicates that the type of provenance event is unknown because the user who is attempting to access the event is not authorized to know the type</p></td>
</tr>
</tbody>
</table>
</div>
</div>
<div class="sect1">
<h2 id="common-processor-patterns"><a class="anchor" href="developer-guide.html#common-processor-patterns"></a>Common Processor Patterns</h2>
<div class="sectionbody">
<div class="paragraph">
<p>While there are many different Processors available to NiFi users, the
vast majority of them fall into
one of several common design patterns. Below, we discuss these
patterns, when the patterns are appropriate,
reasons we follow these patterns, and things to watch out for when
applying such patterns. Note that the patterns
and recommendations discussed below are general guidelines and not
hardened rules.</p>
</div>
<div class="sect2">
<h3 id="ingress"><a class="anchor" href="developer-guide.html#ingress"></a>Data Ingress</h3>
<div class="paragraph">
<p>A Processor that ingests data into NiFi has a single Relationship
named <code>success</code>. This Processor generates
new FlowFiles via the ProcessSession <code>create</code> method and does not pull
FlowFiles from incoming Connections.
The Processor name starts with "Get" or "Listen," depending on whether
it polls an external source or exposes
some interface to which external sources can connect. The name ends
with the protocol used for communications.
Processors that follow this pattern include <code>GetFile</code>, <code>GetSFTP</code>,
<code>ListenHTTP</code>, and <code>GetHTTP</code>.</p>
</div>
<div class="paragraph">
<p>This Processor may create or initialize a Connection Pool in a method
that uses the <code>@OnScheduled</code> annotation.
However, because communications problems may prevent connections from
being established or cause connections
to be terminated, connections themselves are not created at this
point. Rather, the connections are
created or leased from the pool in the <code>onTrigger</code> method.</p>
</div>
<div class="paragraph">
<p>The <code>onTrigger</code> method of this Processor begins by leasing a
connection from the Connection Pool, if possible,
or otherwise creates a connection to the external service. When no
data is available from the
external source, the <code>yield</code> method of the ProcessContext is called by
the Processor and the method returns so
that this Processor avoids continually running and depleting resources
without benefit. Otherwise, this
Processor then creates a FlowFile via the ProcessSession&#8217;s <code>create</code>
method and assigns an appropriate
filename and path to the FlowFile (by adding the <code>filename</code> and <code>path</code>
attributes), as well as any other
attributes that may be appropriate. An OutputStream to the FlowFile&#8217;s content is
obtained via the ProcessSession&#8217;s <code>write</code> method, passing a new
OutputStreamCallback (which is usually
an anonymous inner class). From within this callback, the Processor is
able to write to the FlowFile and streams
the content from the external resource to the FlowFile&#8217;s OutputStream.
If the desire is to write the entire contents
of an InputStream to the FlowFile, the <code>importFrom</code> method of
ProcessSession may be more convenient to use than the
<code>write</code> method.</p>
</div>
<div class="paragraph">
<p>When this Processor expects to receive many small files, it may be
advisable to create several FlowFiles from a
single session before committing the session. Typically, this allows
the Framework to treat the content of the
newly created FlowFiles much more efficiently.</p>
</div>
<div class="paragraph">
<p>This Processor generates a Provenance event indicating that it has
received data and specifies from
where the data came. This Processor should log the creation of the
FlowFile so that the FlowFile&#8217;s
origin can be determined by analyzing logs, if necessary.</p>
</div>
<div class="paragraph">
<p>This Processor acknowledges receipt of the data and/or removes the
data from the external source in order
to prevent receipt of duplicate files. <strong>This is done only after the
ProcessSession by which the FlowFile was
created has been committed!</strong> Failure to adhere to this principle may
result in data loss, as restarting NiFi
before the session has been committed will result in the temporary
file being deleted. Note, however, that it
is possible using this approach to receive duplicate data because the
application could be restarted after
committing the session and before acknowledging or removing the data
from the external source. In general, though,
potential data duplication is preferred over potential data loss. The
connection is finally returned or added to
the Connection Pool, depending on whether the connection was leased
from the Connection Pool to begin with or
was created in the <code>onTrigger</code> method.</p>
</div>
<div class="paragraph">
<p>If there is a communications problem, the connection is typically
terminated and not returned (or added) to
the Connection Pool. Connections to remote systems are torn down and
the Connection Pool shutdown in a method
annotated with the <code>@OnStopped</code> annotation so that resources can be reclaimed.</p>
</div>
</div>
<div class="sect2">
<h3 id="data-egress"><a class="anchor" href="developer-guide.html#data-egress"></a>Data Egress</h3>
<div class="paragraph">
<p>A Processor that publishes data to an external source has two
Relationships: <code>success</code> and <code>failure</code>. The
Processor name starts with "Put" followed by the protocol that is used
for data transmission. Processors
that follow this pattern include <code>PutEmail</code>, <code>PutSFTP</code>, and
<code>PostHTTP</code> (note that the name does not
begin with "Put" because this would lead to confusion, since PUT and
POST have special meanings when dealing with
HTTP).</p>
</div>
<div class="paragraph">
<p>This Processor may create or initialize a Connection Pool in a method
that uses the <code>@OnScheduled</code> annotation.
However, because communications problems may prevent connections from
being established or cause connections
to be terminated, connections themselves are not created at this
point. Rather, the connections are
created or leased from the pool in the <code>onTrigger</code> method.</p>
</div>
<div class="paragraph">
<p>The <code>onTrigger</code> method first obtains a FlowFile from the
ProcessSession via the <code>get</code> method. If no FlowFile is
available, the method returns without obtaining a connection to the
remote resource.</p>
</div>
<div class="paragraph">
<p>If at least one FlowFile is available, the Processor obtains a
connection from the Connection Pool, if possible,
or otherwise creates a new connection. If the Processor is neither
able to lease a connection from the Connection Pool
nor create a new connection, the FlowFile is routed to <code>failure</code>, the
event is logged, and the method returns.</p>
</div>
<div class="paragraph">
<p>If a connection was obtained, the Processor obtains an InputStream to
the FlowFile&#8217;s content by invoking the
<code>read</code> method on the ProcessSession and passing an InputStreamCallback
(which is often an anonymous inner class)
and from within that callback transmits the contents of the FlowFile
to the destination. The event is logged
along with the amount of time taken to transfer the file and the data
rate at which the file was transferred.
A SEND event is reported to the ProvenanceReporter by obtaining the
reporter from the ProcessSession via the
<code>getProvenanceReporter</code> method and calling the <code>send</code> method on the
reporter. The connection is returned or added
to the Connection Pool, depending on whether the connection was leased
from the pool or newly created by the
<code>onTrigger</code> method.</p>
</div>
<div class="paragraph">
<p>If there is a communications problem, the connection is typically
terminated and not returned (or added) to
the Connection Pool. If there is an issue sending the data to the
remote resource, the desired approach for handling the
error depends on a few considerations. If the issue is related to a
network condition, the FlowFile is generally
routed to <code>failure</code>. The FlowFile is not penalized because there is
not necessary a problem with the data. Unlike the
case of the <a href="developer-guide.html#ingress">Data Ingress</a> Processor, we typically do not call <code>yield</code> on
the ProcessContext. This is because in the case of
ingest, the FlowFile does not exist until the Processor is able to
perform its function. However, in the case of a Put Processor,
the DataFlow Manager may choose to route <code>failure</code> to a different
Processor. This can allow for a "backup" system to be
used in the case of problems with one system or can be used for load
distribution across many systems.</p>
</div>
<div class="paragraph">
<p>If a problem occurs that is data-related, one of two approaches should
be taken. First, if the problem is likely to
sort itself out, the FlowFile is penalized and then routed to
<code>failure</code>. This is the case, for instance, with PutFTP,
when a FlowFile cannot be transferred because of a file naming
conflict. The presumption is that the file will eventually
be removed from the directory so that the new file can be transferred.
As a result, we penalize the FlowFile and route to
<code>failure</code> so that we can try again later. In the other case, if there
is an actual problem with the data (such as the data does
not conform to some required specification), a different approach may
be taken. In this case, it may be advantageous
to break apart the <code>failure</code> relationship into a <code>failure</code> and a
<code>communications failure</code> relationship. This allows the
DataFlow Manager to determine how to handle each of these cases
individually. It is important in these situations to document
well the differences between the two Relationships by clarifying it in
the "description" when creating the Relationship.</p>
</div>
<div class="paragraph">
<p>Connections to remote systems are torn down and the Connection Pool
shutdown in a method
annotated with <code>@OnStopped</code> so that resources can be reclaimed.</p>
</div>
</div>
<div class="sect2">
<h3 id="route-based-on-content-one-to-one"><a class="anchor" href="developer-guide.html#route-based-on-content-one-to-one"></a>Route Based on Content (One-to-One)</h3>
<div class="paragraph">
<p>A Processor that routes data based on its content will take one of two
forms: Route an incoming FlowFile to exactly
one destination, or route incoming data to 0 or more destinations.
Here, we will discuss the first case.</p>
</div>
<div class="paragraph">
<p>This Processor has two relationships: <code>matched</code> and <code>unmatched</code>. If a
particular data format is expected, the Processor
will also have a <code>failure</code> relationship that is used when the input is
not of the expected format. The Processor exposes
a Property that indicates the routing criteria.</p>
</div>
<div class="paragraph">
<p>If the Property that specifies routing criteria requires processing,
such as compiling a Regular Expression, this processing
is done in a method annotated with <code>@OnScheduled</code>, if possible. The
result is then stored in a member variable that is marked
as <code>volatile</code>.</p>
</div>
<div class="paragraph">
<p>The <code>onTrigger</code> method obtains a single FlowFile. The method reads the
contents of the FlowFile via the ProcessSession&#8217;s <code>read</code>
method, evaluating the Match Criteria as the data is streamed. The
Processor then determines whether the FlowFile should be
routed to <code>matched</code> or <code>unmatched</code> based on whether or not the
criteria matched, and routes the FlowFile to the appropriate
relationship.</p>
</div>
<div class="paragraph">
<p>The Processor then emits a Provenance ROUTE event indicating which
Relationship to which the Processor routed the FlowFile.</p>
</div>
<div class="paragraph">
<p>This Processor is annotated with the <code>@SideEffectFree</code> and
<code>@SupportsBatching</code> annotations from the <code>org.apache.nifi.annotation.behavior</code>
package.</p>
</div>
</div>
<div class="sect2">
<h3 id="route-based-on-content-one-to-many"><a class="anchor" href="developer-guide.html#route-based-on-content-one-to-many"></a>Route Based on Content (One-to-Many)</h3>
<div class="paragraph">
<p>If a Processor will route a single FlowFile to potentially many
relationships, this Processor will be slightly different than
the above-described Processor for Routing Data Based on Content. This
Processor typically has Relationships that are dynamically
defined by the user as well as an <code>unmatched</code> relationship.</p>
</div>
<div class="paragraph">
<p>In order for the user to be able to define additionally Properties,
the <code>getSupportedDynamicPropertyDescriptor</code> method must be
overridden. This method returns a PropertyDescriptor with the supplied
name and an applicable Validator to ensure that the
user-specified Matching Criteria is valid.</p>
</div>
<div class="paragraph">
<p>In this Processor, the Set of Relationships that is returned by the
<code>getRelationships</code> method is a member variable that is
marked <code>volatile</code>. This Set is initially constructed with a single
Relationship named <code>unmatched</code>. The <code>onPropertyModified</code> method
is overridden so that when a Property is added or removed, a new
Relationship is created with the same name. If the Processor has
Properties that are not user-defined, it is important to check if the
specified Property is user-defined. This can be achieved by
calling the <code>isDynamic</code> method of the PropertyDescriptor that is
passed to this method. If this Property is dynamic,
a new Set of Relationships is then created, and the previous set of
Relationships is copied into it. This new Set
either has the newly created Relationship added to it or removed from
it, depending on whether a new Property was added
to the Processor or a Property was removed (Property removal is
detected by check if the third argument to this function is <code>null</code>).
The member variable holding the Set of Relationships is then updated
to point to this new Set.</p>
</div>
<div class="paragraph">
<p>If the Properties that specify routing criteria require processing,
such as compiling a Regular Expression, this processing is done
in a method annotated with <code>@OnScheduled</code>, if possible. The result is
then stored in a member variable that is marked as <code>volatile</code>.
This member variable is generally of type <code>Map</code> where the key is of
type <code>Relationship</code> and the value&#8217;s type is defined by the result of
processing the property value.</p>
</div>
<div class="paragraph">
<p>The <code>onTrigger</code> method obtains a FlowFile via the <code>get</code> method of
ProcessSession. If no FlowFile is available, it returns immediately.
Otherwise, a Set of type Relationship is created. The method reads the
contents of the FlowFile via the ProcessSession&#8217;s <code>read</code> method,
evaluating each of the Match Criteria as the data is streamed. For any
criteria that matches, the relationship associated with that Match
Criteria is added to the Set of Relationships.</p>
</div>
<div class="paragraph">
<p>After reading the contents of the FlowFile, the method checks if the
Set of Relationships is empty. If so, the original FlowFile has
an attribute added to it to indicate the Relationship to which it was
routed and is routed to the <code>unmatched</code>. This is logged, a
Provenance ROUTE event is emitted, and the method returns. If the size
of the Set is equal to 1, the original FlowFile has an attribute
added to it to indicate the Relationship to which it was routed and
is routed to the Relationship specified by the entry in the Set.
This is logged, a Provenance ROUTE event is emitted for the FlowFile,
and the method returns.</p>
</div>
<div class="paragraph">
<p>In the event that the Set contains more than 1 Relationship, the
Processor creates a clone of the FlowFile for each Relationship,
except
for the first. This is done via the <code>clone</code> method of the
ProcessSession. There is no need to report a CLONE Provenance Event,
as the
framework will handle this for you. The original FlowFile and each
clone are routed to their appropriate Relationship with attribute
indicating the name of the Relationship. A Provenance ROUTE event is
emitted for each FlowFile. This is logged, and the method returns.</p>
</div>
<div class="paragraph">
<p>This Processor is annotated with the <code>@SideEffectFree</code> and
<code>@SupportsBatching</code> annotations from the
<code>org.apache.nifi.annotation.behavior</code>
package.</p>
</div>
</div>
<div class="sect2">
<h3 id="route-streams-based-on-content-one-to-many"><a class="anchor" href="developer-guide.html#route-streams-based-on-content-one-to-many"></a>Route Streams Based on Content (One-to-Many)</h3>
<div class="paragraph">
<p>The previous description of Route Based on Content (One-to-Many)
provides an abstraction
for creating a very powerful Processor. However, it assumes that each
FlowFile will be routed
in its entirety to zero or more Relationships. What if the incoming
data format is a "stream" of
many different pieces of information - and we want to send different
pieces of this stream to
different Relationships? For example, imagine that we want to have a
RouteCSV Processor such that
it is configured with multiple Regular Expressions. If a line in the
CSV file matches a Regular
Expression, that line should be included in the outbound FlowFile to
the associated relationship.
If a Regular Expression is associated with the Relationship
"has-apples" and that Regular Expression
matches 1,000 of the lines in the FlowFile, there should be one outbound
FlowFile for the "has-apples" relationship that has 1,000 lines in it.
If a different Regular Expression
is associated with the Relationship "has-oranges" and that Regular
Expression matches 50 lines in the
FlowFile, there should be one outbound FlowFile for the "has-oranges"
relationship that has 50 lines in it.
I.e., one FlowFile comes in and two FlowFiles come out. The two
FlowFiles may contain some of the same lines
of text from the original FlowFile, or they may be entirely different.
This is the type of Processor that
we will discuss in this section.</p>
</div>
<div class="paragraph">
<p>This Processor&#8217;s name starts with "Route" and ends with the name of
the data type that it routes. In our
example here, we are routing CSV data, so the Processor is named
RouteCSV. This Processor supports dynamic
properties. Each user-defined property has a name that maps to the
name of a Relationship. The value of
the Property is in the format necessary for the "Match Criteria." In
our example, the value of the property
must be a valid Regular Expression.</p>
</div>
<div class="paragraph">
<p>This Processor maintains an internal <code>ConcurrentMap</code> where the key is
a <code>Relationship</code> and the value is of
a type dependent on the format of the Match Criteria. In our example,
we would maintain a
<code>ConcurrentMap&lt;Relationship, Pattern&gt;</code>. This Processor overrides the
<code>onPropertyModified</code> method.
If the new value supplied to this method (the third argument) is null,
the Relationship whose name is
defined by the property name (the first argument) is removed from the
ConcurrentMap. Otherwise, the new value
is processed (in our example, by calling <code>Pattern.compile(newValue)</code>)
and this value is added to the ConcurrentMap
with the key again being the Relationship whose name is specified by
the property name.</p>
</div>
<div class="paragraph">
<p>This Processor will override the <code>customValidate</code> method. In this
method, it will retrieve all Properties from
the <code>ValidationContext</code> and count the number of PropertyDescriptors
that are dynamic (by calling <code>isDynamic()</code>
on the PropertyDescriptor). If the number of dynamic
PropertyDescriptors is 0, this indicates that the user
has not added any Relationships, so the Processor returns a
<code>ValidationResult</code> indicating that the Processor
is not valid because it has no Relationships added.</p>
</div>
<div class="paragraph">
<p>The Processor returns all of the Relationships specified by the user
when its <code>getRelationships</code> method is
called and will also return an <code>unmatched</code> Relationship. Because this
Processor will have to read and write to the
Content Repository (which can be relatively expensive), if this
Processor is expected to be used for very high
data volumes, it may be advantageous to add a Property that allows the
user to specify whether or not they care
about the data that does not match any of the Match Criteria.</p>
</div>
<div class="paragraph">
<p>When the <code>onTrigger</code> method is called, the Processor obtains a
FlowFile via <code>ProcessSession.get</code>. If no data
is available, the Processor returns. Otherwise, the Processor creates
a <code>Map&lt;Relationship, FlowFile&gt;</code>. We will
refer to this Map as <code>flowFileMap</code>. The Processor reads the incoming
FlowFile by calling <code>ProcessSession.read</code>
and provides an <code>InputStreamCallback</code>.
From within the Callback, the Processor reads the first piece of data
from the FlowFile. The Processor then
evaluates each of the Match Criteria against this piece of data. If a
particular criteria (in our example,
a Regular Expression) matches, the Processor obtains the FlowFile from
<code>flowFileMap</code> that belongs to the appropriate
Relationship. If no FlowFile yet exists in the Map for this
Relationship, the Processor creates a new FlowFile
by calling <code>session.create(incomingFlowFile)</code> and then adds the new
FlowFile to <code>flowFileMap</code>. The Processor then
writes this piece of data to the FlowFile by calling <code>session.append</code>
with an <code>OutputStreamCallback</code>. From within
this OutputStreamCallback, we have access to the new FlowFile&#8217;s
OutputStream, so we are able to write the data
to the new FlowFile. We then return from the OutputStreamCallback.
After iterating over each of the Match Criteria,
if none of them match, we perform the same routines as above for the
<code>unmatched</code> relationship (unless the user
configures us to not write out unmatched data). Now that we have
called <code>session.append</code>, we have a new version of
the FlowFile. As a result, we need to update our <code>flowFileMap</code> to
associate the Relationship with the new FlowFile.</p>
</div>
<div class="paragraph">
<p>If at any point, an Exception is thrown, we will need to route the
incoming FlowFile to <code>failure</code>. We will also
need to remove each of the newly created FlowFiles, as we won&#8217;t be
transferring them anywhere. We can accomplish
this by calling <code>session.remove(flowFileMap.values())</code>. At this point,
we will log the error and return.</p>
</div>
<div class="paragraph">
<p>Otherwise, if all is successful, we can now iterate through the
<code>flowFileMap</code> and transfer each FlowFile to the
corresponding Relationship. The original FlowFile is then either
removed or routed to an <code>original</code> relationship.
For each of the newly created FlowFiles, we also emit a Provenance
ROUTE event indicating which Relationship
the FlowFile went to. It is also helpful to include in the details of
the ROUTE event how many pieces of information
were included in this FlowFile. This allows DataFlow Managers to
easily see when looking at the Provenance
Lineage view how many pieces of information went to each of the
relationships for a given input FlowFile.</p>
</div>
<div class="paragraph">
<p>Additionally, some Processors may need to "group" the data that is
sent to each Relationship so that each FlowFile
that is sent to a relationship has the same value. In our example, we
may want to allow the Regular Expression
to have a Capturing Group and if two different lines in the CSV match
the Regular Expression but have different
values for the Capturing Group, we want them to be added to two
different FlowFiles. The matching value could then
be added to each FlowFile as an Attribute. This can be accomplished by
modifying the <code>flowFileMap</code> such that
it is defined as <code>Map&lt;Relationship, Map&lt;T, FlowFile&gt;&gt;</code> where <code>T</code> is
the type of the Grouping Function (in our
example, the Group would be a <code>String</code> because it is the result of
evaluating a Regular Expression&#8217;s
Capturing Group).</p>
</div>
</div>
<div class="sect2">
<h3 id="route-based-on-attributes"><a class="anchor" href="developer-guide.html#route-based-on-attributes"></a>Route Based on Attributes</h3>
<div class="paragraph">
<p>This Processor is almost identical to the Route Data Based on Content
Processors described above. It takes two different forms: One-to-One
and
One-to-Many, as do the Content-Based Routing Processors. This
Processor, however, does not make any call to ProcessSession&#8217;s <code>read</code>
method,
as it does not read FlowFile content. This Processor is typically very
fast, so the <code>@SupportsBatching</code> annotation can be very important
in this case.</p>
</div>
</div>
<div class="sect2">
<h3 id="split-content-one-to-many"><a class="anchor" href="developer-guide.html#split-content-one-to-many"></a>Split Content (One-to-Many)</h3>
<div class="paragraph">
<p>This Processor generally requires no user configuration, with the
exception of the size of each Split to create. The <code>onTrigger</code> method
obtains
a FlowFile from its input queues. A List of type FlowFile is created.
The original FlowFile is read via the ProcessSession&#8217;s <code>read</code> method,
and an InputStreamCallback is used. Within the InputStreamCallback,
the content is read until a point is reached at which the FlowFile
should be
split. If no split is needed, the Callback returns, and the original
FlowFile is routed to <code>success</code>. In this case, a Provenance ROUTE
event
is emitted. Typically, ROUTE events are not emitted when routing a
FlowFile to <code>success</code> because this generates a very verbose lineage
that
becomes difficult to navigate. However, in this case, the event is
useful because we would otherwise expect a FORK event and the absence
of
any event is likely to cause confusion. The fact that the FlowFile was
not split but was instead transferred to <code>success</code> is logged, and the
method returns.</p>
</div>
<div class="paragraph">
<p>If a point is reached at which a FlowFile needs to be split, a new
FlowFile is created via the ProcessSession&#8217;s <code>create(FlowFile)</code> method
or the
<code>clone(FlowFile, long, long)</code> method. The next section of code depends
on whether the <code>create</code> method is used or the <code>clone</code> method is used.
Both methods are described below. Which solution is appropriate must
be determined on a case-by-case basis.</p>
</div>
<div class="paragraph">
<p>The Create Method is most appropriate when the data will not be
directly copied from the original FlowFile to the new FlowFile.
For example, if only some of the data will be copied, or if the data
will be modified in some way before being copied to the new
FlowFile, this method is necessary. However, if the content of the new
FlowFile will be an exact copy of a portion of the original
FlowFile, the Clone Method is much preferred.</p>
</div>
<div class="paragraph">
<p><strong>Create Method</strong>
If using the <code>create</code> method, the method is called with the original
FlowFile as the argument so that the newly created FlowFile will
inherit
the attributes of the original FlowFile and a Provenance FORK event
will be created by the framework.</p>
</div>
<div class="paragraph">
<p>The code then enters a <code>try/finally</code> block. Within the <code>finally</code>
block, the newly created FlowFile is added to the List of FlowFiles
that have
been created. This is done within a <code>finally</code> block so that if an
Exception is thrown, the newly created FlowFile will be appropriately
cleaned up.
Within the <code>try</code> block, the callback initiates a new callback by
calling the ProcessSession&#8217;s <code>write</code> method with an
OutputStreamCallback.
The appropriate data is then copied from the InputStream of the
original FlowFile to the OutputStream for the new FlowFile.</p>
</div>
<div class="paragraph">
<p><strong>Clone Method</strong>
If the content of the newly created FlowFile is to be only a
contiguous subset of the bytes of the original FlowFile, it is
preferred
to use the <code>clone(FlowFile, long, long)</code> method instead of the
<code>create(FlowFile)</code> method of the ProcessSession. In this case, the
offset
of the original FlowFile at which the new FlowFile&#8217;s content should
begin is passed as the second argument to the <code>clone</code> method. The
length
of the new FlowFile is passed as the third argument to the <code>clone</code>
method. For example, if the original FlowFile was 10,000 bytes
and we called <code>clone(flowFile, 500, 100)</code>, the FlowFile that would be
returned to us would be identical to <code>flowFile</code> with respect to its
attributes. However, the content of the newly created FlowFile would
be 100 bytes in length and would start at offset 500 of the original
FlowFile. That is, the contents of the newly created FlowFile would be
the same as if you had copied bytes 500 through 599 of the original
FlowFile.</p>
</div>
<div class="paragraph">
<p>After the clone has been created, it is added to the List of FlowFiles.</p>
</div>
<div class="paragraph">
<p>This method is much more highly preferred than the Create method, when
applicable,
because no disk I/O is required. The framework is able to simply
create a new FlowFile
that references a subset of the original FlowFile&#8217;s content, rather
than actually copying
the data. However, this is not always possible. For example, if header
information must be copied
from the beginning of the original FlowFile and added to the beginning
of each Split,
then this method is not possible.</p>
</div>
<div class="paragraph">
<p><strong>Both Methods</strong>
Regardless of whether the Clone Method or the Create Method is used,
the following is applicable:</p>
</div>
<div class="paragraph">
<p>If at any point in the InputStreamCallback, a condition is reached in
which processing cannot continue
(for example, the input is malformed), a <code>ProcessException</code> should be
thrown. The call to the
ProcessSession&#8217;s <code>read</code> method is wrapped in a <code>try/catch</code> block
where <code>ProcessException</code> is
caught. If an Exception is caught, a log message is generated
explaining the error. The List of
newly created FlowFiles is removed via the ProcessSession&#8217;s <code>remove</code>
method. The original FlowFile
is routed to <code>failure</code>.</p>
</div>
<div class="paragraph">
<p>If no problems arise, the original FlowFile is routed to <code>original</code>
and all newly created FlowFiles
are updated to include the following attributes:</p>
</div>
<table class="tableblock frame-all grid-all stretch">
<colgroup>
<col style="width: 50%;">
<col style="width: 50%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Attribute Name</th>
<th class="tableblock halign-left valign-top">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>split.parent.uuid</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The UUID of the original FlowFile</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>split.index</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">A one-up number indicating which FlowFile in the list this is (the first FlowFile
created will have a value <code>0</code>, the second will have a value <code>1</code>, etc.)</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock"><code>split.count</code></p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The total number of split FlowFiles that were created</p></td>
</tr>
</tbody>
</table>
<div class="paragraph">
<p>The newly created FlowFiles are routed to <code>success</code>; this event is
logged; and the method returns.</p>
</div>
</div>
<div class="sect2">
<h3 id="update-attributes-based-on-content"><a class="anchor" href="developer-guide.html#update-attributes-based-on-content"></a>Update Attributes Based on Content</h3>
<div class="paragraph">
<p>This Processor is very similar to the Route Based on Content
Processors discussed above. Rather than
routing a FlowFile to <code>matched</code> or <code>unmatched</code>, the FlowFile is
generally routed to <code>success</code> or <code>failure</code>
and attributes are added to the FlowFile as appropriate. The
attributes to be added are configured in a
manner similar to that of the Route Based on Content (One-to-Many),
with the user defining their own
properties. The name of the property indicates the name of an
attribute to add. The value of the
property indicates some Matching Criteria to be applied to the data.
If the Matching Criteria matches
the data, an attribute is added with the name the same as that of the
Property. The value of the
attribute is the criteria from the content that matched.</p>
</div>
<div class="paragraph">
<p>For example, a Processor that evaluates XPath Expressions may allow
user-defined XPaths to be
entered. If the XPath matches the content of a FlowFile, that FlowFile
will have an attribute added with
the name being equal to that of the Property name and a value equal to
the textual content of the XML Element or
Attribute that matched the XPath. The <code>failure</code> relationship would
then be used if the incoming FlowFile
was not valid XML in this example. The <code>success</code> relationship would be
used regardless of whether or not
any matches were found. This can then be used to route the FlowFile
when appropriate.</p>
</div>
<div class="paragraph">
<p>This Processor emits a Provenance Event of type ATTRIBUTES_MODIFIED.</p>
</div>
</div>
<div class="sect2">
<h3 id="enrichmodify-content"><a class="anchor" href="developer-guide.html#enrichmodify-content"></a>Enrich/Modify Content</h3>
<div class="paragraph">
<p>The Enrich/Modify Content pattern is very common and very generic.
This pattern is responsible for any
general content modification. For the majority of cases, this
Processor is marked with the
<code>@SideEffectFree</code> and <code>@SupportsBatching</code> annotations. The Processor
has any number of required and optional
Properties, depending on the Processor&#8217;s function. The Processor
generally has a <code>success</code> and <code>failure</code> relationship.
The <code>failure</code> relationship is generally used when the input file is
not in the expected format.</p>
</div>
<div class="paragraph">
<p>This Processor obtains a FlowFile and updates it using the
ProcessSession&#8217;s <code>write(StreamCallback)</code> method
so that it is able to both read from the FlowFile&#8217;s content and write
to the next version of the FlowFile&#8217;s
content. If errors are encountered during the callback, the callback
will throw a <code>ProcessException</code>. The
call to the ProcessSession&#8217;s <code>write</code> method is wrapped in a
<code>try/catch</code> block that catches <code>ProcessException</code>
and routes the FlowFile to failure.</p>
</div>
<div class="paragraph">
<p>If the callback succeeds, a CONTENT_MODIFIED Provenance Event is emitted.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="error-handling"><a class="anchor" href="developer-guide.html#error-handling"></a>Error Handling</h2>
<div class="sectionbody">
<div class="paragraph">
<p>When writing a Processor, there are several different unexpected cases that can occur.
It is important that Processor developers understand the mechanics of how the NiFi framework
behaves if Processors do not handle errors themselves, and it&#8217;s important to understand
what error handling is expected of Processors. Here, we will discuss how Processors should
handle unexpected errors during the course of their work.</p>
</div>
<div class="sect2">
<h3 id="exceptions-within-the-processor"><a class="anchor" href="developer-guide.html#exceptions-within-the-processor"></a>Exceptions within the Processor</h3>
<div class="paragraph">
<p>During the execution of the <code>onTrigger</code> method of a Processor, many things can potentially go
awry. Common failure conditions include:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Incoming data is not in the expected format.</p>
</li>
<li>
<p>Network connections to external services fail.</p>
</li>
<li>
<p>Reading or writing data to a disk fails.</p>
</li>
<li>
<p>There is a bug in the Processor or a dependent library.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Any of these conditions can result in an Exception being thrown from the Processor. From the framework
perspective, there are two types of Exceptions that can escape a Processor: <code>ProcessException</code> and
all others.</p>
</div>
<div class="paragraph">
<p>If a ProcessException is thrown from the Processor, the framework will assume that this is a failure that
is a known outcome. Moreover, it is a condition where attempting to process the data again later may
be successful. As a result, the framework will roll back the session that was being processed and penalize
the FlowFiles that were being processed.</p>
</div>
<div class="paragraph">
<p>If any other Exception escapes the Processor, though, the framework will assume that it is a failure that
was not taken into account by the developer. In this case, the framework will also roll back the session
and penalize the FlowFiles. However, in this case, we can get into some very problematic cases. For example,
the Processor may be in a bad state and may continually run, depleting system resources, without providing
any useful work. This is fairly common, for instance, when a NullPointerException is thrown continually.
In order to avoid this case, if an Exception other than ProcessException is able to escape the Processor&#8217;s
<code>onTrigger</code> method, the framework will also "Administratively Yield" the Processor. This means that the
Processor will not be triggered to run again for some amount of time. The amount of time is configured
in the <code>nifi.properties</code> file but is 10 seconds by default.</p>
</div>
</div>
<div class="sect2">
<h3 id="exceptions-within-a-callback-ioexception-runtimeexception"><a class="anchor" href="developer-guide.html#exceptions-within-a-callback-ioexception-runtimeexception"></a>Exceptions within a callback: IOException, RuntimeException</h3>
<div class="paragraph">
<p>More often than not, when an Exception occurs in a Processor, it occurs from within a callback (I.e.,
<code>InputStreamCallback</code>, <code>OutputStreamCallback</code>, or <code>StreamCallback</code>). That is, during the processing of a
FlowFile&#8217;s content. Callbacks are allowed to throw either <code>RuntimeException</code> or <code>IOException</code>. In the case
of RuntimeException, this Exception will propagate back to the <code>onTrigger</code> method. In the case of an
<code>IOException</code>, the Exception will be wrapped within a ProcessException and this ProcessException will then
be thrown from the Framework.</p>
</div>
<div class="paragraph">
<p>For this reason, it is recommended that Processors that use callbacks do so within a <code>try/catch</code> block
and catch <code>ProcessException</code> as well as any other <code>RuntimeException</code> that they expect their callback to
throw. It is <strong>not</strong> recommended that Processors catch the general <code>Exception</code> or <code>Throwable</code> cases, however.
This is discouraged for two reasons.</p>
</div>
<div class="paragraph">
<p>First, if an unexpected RuntimeException is thrown, it is likely a bug
and allowing the framework to rollback the session will ensure no data loss and ensures that DataFlow Managers
are able to deal with the data as they see fit by keeping the data queued up in place.</p>
</div>
<div class="paragraph">
<p>Second, when an IOException is thrown from a callback, there really are two types of IOExceptions: those thrown
from Processor code (for example, the data is not in the expected format or a network connection fails), and
those that are thrown from the Content Repository (where the FlowFile content is stored). If the latter is the case,
the framework will catch this IOException and wrap it into a <code>FlowFileAccessException</code>, which extends <code>RuntimeException</code>.
This is done explicitly so that the Exception will escape the <code>onTrigger</code> method and the framework can handle this
condition appropriately. Catching the general Exception prevents this from happening.</p>
</div>
</div>
<div class="sect2">
<h3 id="penalization-vs-yielding"><a class="anchor" href="developer-guide.html#penalization-vs-yielding"></a>Penalization vs. Yielding</h3>
<div class="paragraph">
<p>When an issue occurs during processing, the framework exposes two methods to allow Processor developers to avoid performing
unnecessary work: "penalization" and "yielding." These two concepts can become confusing for developers new to the NiFi API.
A developer is able to penalize a FlowFile by calling the <code>penalize(FlowFile)</code> method of ProcessSession. This causes the
FlowFile itself to be inaccessible to downstream Processors for a period of time. The amount of time that the FlowFile is
inaccessible is determined by the DataFlow Manager by setting the "Penalty Duration" setting in the Processor Configuration
dialog. The default value is 30 seconds. Typically, this is done when a Processor determines that the data cannot be processed
due to environmental reasons that are expected to sort themselves out. A great example of this is the PutSFTP processor, which
will penalize a FlowFile if a file already exists on the SFTP server that has the same filename. In this case, the Processor
penalizes the FlowFile and routes it to failure. A DataFlow Manager can then route failure back to the same PutSFTP Processor.
This way, if a file exists with the same filename, the Processor will not attempt to send the file again for 30 seconds
(or whatever period the DFM has configured the Processor to use). In the meantime, it is able to continue to process other
FlowFiles.</p>
</div>
<div class="paragraph">
<p>On the other hand, yielding allows a Processor developer to indicate to the framework that it will not be able to perform
any useful function for some period of time. This commonly happens with a Processor that is communicating with a remote
resource. If the Processor cannot connect to the remote resource, or if the remote resource is expected to provide data
but reports that it has none, the Processor should call <code>yield</code> on the <code>ProcessContext</code> object and then return. By doing
this, the Processor is telling the framework that it should not waste resources triggering this Processor to run, because
there&#8217;s nothing that it can do - it&#8217;s better to use those resources to allow other Processors to run.</p>
</div>
</div>
<div class="sect2">
<h3 id="session-rollback"><a class="anchor" href="developer-guide.html#session-rollback"></a>Session Rollback</h3>
<div class="paragraph">
<p>Thus far, when we have discussed the <code>ProcessSession</code>, we have typically referred to it simply as a mechanism for accessing
FlowFiles. However, it provides another very important capability, which is transactionality. All methods that are called
on a ProcessSession happen as a transaction. When we decided to end the transaction, we can do so either by calling
<code>commit()</code> or by calling <code>rollback()</code>. Typically, this is handled by the <code>AbstractProcessor</code> class: if the <code>onTrigger</code> method
throws an Exception, the AbstractProcessor will catch the Exception, call <code>session.rollback()</code>, and then re-throw the Exception.
Otherwise, the AbstractProcessor will call <code>commit()</code> on the ProcessSession.</p>
</div>
<div class="paragraph">
<p>There are times, however, that developers will want to roll back a session explicitly. This can be accomplished at any time
by calling the <code>rollback()</code> or <code>rollback(boolean)</code> method. If using the latter, the boolean indicates whether or not those
FlowFiles that have been pulled from queues (via the ProcessSession <code>get</code> methods) should be penalized before being added
back to their queues.</p>
</div>
<div class="paragraph">
<p>When <code>rollback</code> is called, any modification that has occurred to the FlowFiles in that session are discarded, this includes
both content and attribute modifications. Additionally, all Provenance Events are rolled back (with the exception
of any SEND event that was emitted by passing a value of <code>true</code> for the <code>force</code> argument). The FlowFiles that were pulled from
the input queues are then transferred back to the input queues (and optionally penalized) so that they can be processed again.</p>
</div>
<div class="paragraph">
<p>On the other hand, when the <code>commit</code> method is called, the FlowFile&#8217;s new state is persisted in the FlowFile Repository, and
any Provenance Events that occurred are persisted in the Provenance Repository. The previous content is destroyed (unless
another FlowFile references the same piece of content), and the FlowFiles are transferred to the outbound queues so that the
next Processors can operate on the data.</p>
</div>
<div class="paragraph">
<p>It is also important to note how this behavior is affected by using the <code>org.apache.nifi.annotation.behavior.SupportsBatching</code>
annotation. If a Processor utilizes this annotation, calls to <code>ProcessSession.commit</code> may not take effect immediately. Rather,
these commits may be batched together in order to provide higher throughput. However, if at any point, the Processor rolls back
the ProcessSession, all changes since the last call to <code>commit</code> will be discarded and all "batched" commits will take effect.
These "batched" commits are not rolled back.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="general-design-considerations"><a class="anchor" href="developer-guide.html#general-design-considerations"></a>General Design Considerations</h2>
<div class="sectionbody">
<div class="paragraph">
<p>When designing a Processor, there are a few important design considering to keep in mind. This section of the Developer Guide
brings to the forefront some of the ideas that a developer should be thinking about when creating a Processor.</p>
</div>
<div class="sect2">
<h3 id="consider-the-user"><a class="anchor" href="developer-guide.html#consider-the-user"></a>Consider the User</h3>
<div class="paragraph">
<p>One of the most important concepts to keep in mind when developing a Processor (or any other component) is the user
experience that you are creating. It&#8217;s important to remember that as the developer of such a component, you may have
important knowledge about the context that others do not have. Documentation should always be supplied so that those
less familiar with the process are able to use it with ease.</p>
</div>
<div class="paragraph">
<p>When thinking about the user experience, it is also important to note that consistency is very important. It is best
to stick with the standard <a href="developer-guide.html#naming-conventions">Naming Conventions</a>. This is true for Processor names, Property names and value, Relationship
names, and any other aspect that the user will experience.</p>
</div>
<div class="paragraph">
<p>Simplicity is crucial! Avoid adding properties that you don&#8217;t expect users to understand or change. As developers, we are
told that hard-coding values is bad. But this sometimes results in developers exposing properties that, when asked for clarification,
tell users to just leave the default value. This leads to confusion and complexity.</p>
</div>
</div>
<div class="sect2">
<h3 id="cohesion-and-reusability"><a class="anchor" href="developer-guide.html#cohesion-and-reusability"></a>Cohesion and Reusability</h3>
<div class="paragraph">
<p>For the sake of making a single, cohesive unit, developers are sometimes tempted to combine several functions into a single Processor.
This is very true for the case when a Processor expects input data to be in format X so that the Processor can convert the data into
format Y and send the newly-formatted data to some external service.</p>
</div>
<div class="paragraph">
<p>Taking this approach of formatting the data for a particular endpoint and then sending the data to that endpoint within the same Processor
has several drawbacks:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>The Processor becomes very complex, as it has to perform the data translation task as well as the task of
sending the data to the remote service.</p>
</li>
<li>
<p>If the Processor is unable to communicate with the remote service, it will route the data to a <code>failure</code> Relationship. In this case,
the Processor will be responsible to perform the data translation again. And if it fails again, the translation is done yet again.</p>
</li>
<li>
<p>If we have five different Processors that translate the incoming data into this new format before sending the data, we have a great
deal of duplicated code. If the schema changes, for instance, many Processors must be updated.</p>
</li>
<li>
<p>This intermediate data is thrown away when the Processor finishes sending to the remote service. The intermediate data format
may well be useful to other Processors.</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>In order to avoid these issues, and make Processors more reusable, a Processor should always stick to the principal of "do one thing and do
it well." Such a Processor should be broken into two separate Processors: one to convert the data from Format X to Format Y, and another
Processor to send data to the remote resource.</p>
</div>
</div>
<div class="sect2">
<h3 id="naming-conventions"><a class="anchor" href="developer-guide.html#naming-conventions"></a>Naming Conventions</h3>
<div class="paragraph">
<p>In order to deliver a consistent look and feel to users, it is advisable that Processors keep with standard naming conventions. The following
is a list of standard conventions that are used:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Processors that pull data from a remote system are named Get&lt;Service&gt; or Get&lt;Protocol&gt;, depending on if they poll data from arbitrary
sources over a known Protocol (such as GetHTTP or GetFTP) or if they pull data from a known service (such as GetKafka)</p>
</li>
<li>
<p>Processors that push data to a remote system are named Put&lt;Service&gt; or Put&lt;Protocol&gt;.</p>
</li>
<li>
<p>Relationship names are lower-cased and use spaces to delineated words.</p>
</li>
<li>
<p>Property names capitalize significant words, as would be done with the title of a book.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="processor-behavior-annotations"><a class="anchor" href="developer-guide.html#processor-behavior-annotations"></a>Processor Behavior Annotations</h3>
<div class="paragraph">
<p>When creating a Processor, the developer is able to provide hints to the framework about how to utilize the Processor most
effectively. This is done by applying annotations to the Processor&#8217;s class. The annotations that can be applied to a
Processor exist in three sub-packages of <code>org.apache.nifi.annotation</code>. Those in the <code>documentation</code> sub-package are used
to provide documentation to the user. Those in the <code>lifecycle</code> sub-package instruct the framework which methods should
be called on the Processor in order to respond to the appropriate life-cycle events. Those in the <code>behavior</code> package
help the framework understand how to interact with the Processor in terms of scheduling and general behavior.</p>
</div>
<div class="paragraph">
<p>The following annotations from the <code>org.apache.nifi.annotation.behavior</code> package can be used to modify how the framework
will handle your Processor:</p>
</div>
<div class="ulist">
<ul>
<li>
<p><code>EventDriven</code>: Instructs the framework that the Processor can be scheduled using the Event-Driven scheduling
strategy. This strategy is still experimental at this point, but can result in reduced resource utilization
on dataflows that do not handle extremely high data rates.</p>
</li>
<li>
<p><code>SideEffectFree</code>: Indicates that the Processor does not have any side effects external to NiFi. As a result, the
framework is free to invoke the Processor many times with the same input without causing any unexpected
results to occur. This implies idempotent behavior. This can be used by the framework to improve efficiency by
performing actions such as transferring a ProcessSession from one Processor to another, such that if
a problem occurs many Processors' actions can be rolled back and performed again.</p>
</li>
<li>
<p><code>SupportsBatching</code>: This annotation indicates that it is okay for the framework to batch together multiple
ProcessSession commits into a single commit. If this annotation is present, the user will be able to choose
whether they prefer high throughput or lower latency in the Processor&#8217;s Scheduling tab. This annotation should
be applied to most Processors, but it comes with a caveat: if the Processor calls <code>ProcessSession.commit</code>,
there is no guarantee that the data has been safely stored in NiFi&#8217;s Content, FlowFile, and Provenance Repositories.
As a result, it is not appropriate for those Processors that receive data from an external source, commit the session,
and then delete the remote data or confirm a transaction with a remote resource.</p>
</li>
<li>
<p><code>TriggerSerially</code>: When this annotation is present, the framework will not allow the user to schedule more than one
concurrent thread to execute the <code>onTrigger</code> method at a time. Instead, the number of thread ("Concurrent Tasks")
will always be set to <code>1</code>. This does <strong>not</strong>, however, mean that the Processor does not have to be thread-safe,
as the thread that is executing <code>onTrigger</code> may change between invocations.</p>
</li>
<li>
<p><code>PrimaryNodeOnly</code>: Apache NiFi, when clustered, offers two modes of execution for Processors: "Primary Node" and
"All Nodes". Although running in all the nodes offers better parallelism, some Processors are known to cause unintended
behaviors when run in multiple nodes. For instance, some Processors list or read files from remote filesystems. If such
Processors are scheduled to run on "All Nodes", it will cause unnecessary duplication and even errors. Such Processors
should use this annotation. Applying this annotation will restrict the Processor to run only on the "Primary Node".</p>
</li>
<li>
<p><code>TriggerWhenAnyDestinationAvailable</code>: By default, NiFi will not schedule a Processor to run if any of its outbound
queues is full. This allows back-pressure to be applied all the way a chain of Processors. However, some Processors
may need to run even if one of the outbound queues is full. This annotation indicates that the Processor should run
if any Relationship is "available." A Relationship is said to be "available" if none of the connections that use
that Relationship is full. For example, the DistributeLoad Processor makes use of this annotation. If the "round robin"
scheduling strategy is used, the Processor will not run if any outbound queue is full. However, if the "next available"
scheduling strategy is used, the Processor will run if any Relationship at all is available and will route FlowFiles
only to those relationships that are available.</p>
</li>
<li>
<p><code>TriggerWhenEmpty</code>: The default behavior is to trigger a Processor to run only if its input queue has at least one
FlowFile or if the Processor has no input queues (which is typical of a "source" Processor). Applying this annotation
will cause the framework to ignore the size of the input queues and trigger the Processor regardless of whether or
not there is any data on an input queue. This is useful, for example, if the Processor needs to be triggered to run
periodically to time out a network connection.</p>
</li>
<li>
<p><code>InputRequirement</code>: By default, all Processors will allow users to create incoming connections for the Processor, but
if the user does not create an incoming connection, the Processor is still valid and can be scheduled to run. For Processors
that are expected to be used as a "Source Processor," though, this can be confusing to the user, and the user may attempt to
send FlowFiles to that Processor, only for the FlowFiles to queue up without being processed. Conversely, if the Processor
expects incoming FlowFiles but does not have an input queue, the Processor will be scheduled to run but will perform no work,
as it will receive no FlowFile, and this leads to confusion as well. As a result, we can use the <code>@InputRequirement</code> annotation
and provide it a value of <code>INPUT_REQUIRED</code>, <code>INPUT_ALLOWED</code>, or <code>INPUT_FORBIDDEN</code>. This provides information to the framework
about when the Processor should be made invalid, or whether or not the user should even be able to draw a Connection to the
Processor. For instance, if a Processor is annotated with <code>InputRequirement(Requirement.INPUT_FORBIDDEN)</code>, then the user will
not even be able to create a Connection with that Processor as the destination.</p>
</li>
</ul>
</div>
</div>
<div class="sect2">
<h3 id="data-buffering"><a class="anchor" href="developer-guide.html#data-buffering"></a>Data Buffering</h3>
<div class="paragraph">
<p>An important point to keep in mind is that NiFi provides a generic data processing capability. Data can be in any format. Processors
are generally scheduled with several threads. A common mistake that developers new to NiFi make is to buffer all the contents of a
FlowFile in memory. While there are cases when this is required, it should be avoided if at all possible, unless it is well-known
what format the data is in. For example, a Processor responsible for executing XPath against an XML document will need to load the
entire contents of the data into memory. This is generally acceptable, as XML is not expected to be extremely large. However, a Processor
that searches for a specific byte sequence may be used to search files that are hundreds of gigabytes or more. Attempting to load this
into memory can cause a lot of problems - especially if multiple threads are processing different FlowFiles simultaneously.</p>
</div>
<div class="paragraph">
<p>Instead of buffering this data into memory, it is advisable to instead evaluate the data as it is streamed from the Content Repository
(i.e., scan the content from the <code>InputStream</code> that is provided to your callback by <code>ProcessSession.read</code>). Of course, in this case,
we don&#8217;t want to read from the Content Repository for each byte, so we would use a BufferedInputStream or somehow buffer some small
amount of data, as appropriate.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="controller-services"><a class="anchor" href="developer-guide.html#controller-services"></a>Controller Services</h2>
<div class="sectionbody">
<div class="paragraph">
<p>The <code>ControllerService</code> interface allows developers to share
functionality and state across the JVM in a clean
and consistent manner. The interface resembles that of the <code>Processor</code>
interface but does not
have an <code>onTrigger</code> method because Controller Services are not
scheduled to run periodically, and
Controller Services do not have Relationships because they are not
integrated into the flow directly. Rather,
they are used by Processors, Reporting Tasks, and
other Controller Services.</p>
</div>
<div class="sect2">
<h3 id="developing-controller-service"><a class="anchor" href="developer-guide.html#developing-controller-service"></a>Developing a ControllerService</h3>
<div class="paragraph">
<p>Just like with the Processor interface, the ControllerService
interface exposes methods for configuration,
validation, and initialization. These methods are all identical to
those of the Processor interface
except that the <code>initialize</code> method is passed a
<code>ControllerServiceInitializationContext</code>, rather
than a <code>ProcessorInitializationContext</code>.</p>
</div>
<div class="paragraph">
<p>Controller Services come with an additional constraint that Processors
do not have. A Controller Service
must be comprised of an interface that extends <code>ControllerService</code>.
Implementations can then be interacted
with only through their interface. A Processor, for instance, will
never be given a concrete implementation of
a ControllerService and therefore must reference the service only via
interfaces that extends <code>ControllerService</code>.</p>
</div>
<div class="paragraph">
<p>This constraint is in place mainly because a Processor can exist in
one NiFi Archive (NAR) while the implementation
of the Controller Service that the Processor lives in can exist in a
different NAR. This is accomplished by
the framework by dynamically implementing the exposed interfaces in
such a way that the framework can
switch to the appropriate ClassLoader and invoke the desired method on
the concrete implementation. However,
in order to make this work, the Processor and the Controller Service
implementation must share the same definition
of the Controller Service interface. Therefore, both of these NARs
must depend on the NAR that houses the
Controller Service&#8217;s interface. See <a href="developer-guide.html#nars">NiFi Archives (NARs)</a> for more information.</p>
</div>
</div>
<div class="sect2">
<h3 id="interacting-with-controller-service"><a class="anchor" href="developer-guide.html#interacting-with-controller-service"></a>Interacting with a ControllerService</h3>
<div class="paragraph">
<p>ControllerServices may be obtained by a Processor, another
ControllerService, a ParameterProvider, or a ReportingTask
by means of the ControllerServiceLookup or by using the
<code>identifiesControllerService</code> method of the
PropertyDescriptor&#8217;s Builder class. The ControllerServiceLookup can be
obtained by a Processor from the
ProcessorInitializationContext that is passed to the <code>initialize</code>
method. Likewise, it is obtained by
a ControllerService from the ControllerServiceInitializationContext,
a ParameterProvider from the ParameterProviderInitializationContext,
and by a ReportingTask via the
ReportingConfiguration object passed to the <code>initialize</code> method.</p>
</div>
<div class="paragraph">
<p>For most use cases, though, using the <code>identifiesControllerService</code>
method of a PropertyDescriptor Builder
is preferred and is the least complicated method. In order to use this
method, we create a PropertyDescriptor
that references a Controller Service as such:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("SSL Context Service")
.description("Specified the SSL Context Service that can be used to create secure connections")
.required(true)
.identifiesControllerService(SSLContextService.class)
.build();</code></pre>
</div>
</div>
<div class="paragraph">
<p>Using this method, the user will be prompted to supply the SSL Context
Service that should be used. This is
done by providing the user with a drop-down menu from which they are
able to choose any of the <code>SSLContextService</code>
configurations that have been configured, regardless of the implementation.</p>
</div>
<div class="paragraph">
<p>In order to make use of this service, the Processor can use code such as:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE)
.asControllerService(SSLContextService.class);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Note here that <code>SSLContextService</code> is an interface that extends
ControllerService. The only implementation
at this time is the <code>StandardSSLContextService</code>. However, the
Processor developer need not worry about this
detail.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="reporting-tasks"><a class="anchor" href="developer-guide.html#reporting-tasks"></a>Reporting Tasks</h2>
<div class="sectionbody">
<div class="paragraph">
<p>So far, we have mentioned little about how to convey to the outside
world how NiFi and its components
are performing. Is the system able to keep up with the incoming data
rate? How much more can
the system handle? How much data is processed at the peak time of day
versus the least busy time of day?</p>
</div>
<div class="paragraph">
<p>In order to answer these questions, and many more, NiFi provides a
capability for reporting status,
statistics, metrics, and monitoring information to external services
by means of the <code>ReportingTask</code>
interface. ReportingTasks are given access to a host of information to
determine how the system is performing.</p>
</div>
<div class="sect2">
<h3 id="developing-a-reporting-task"><a class="anchor" href="developer-guide.html#developing-a-reporting-task"></a>Developing a Reporting Task</h3>
<div class="paragraph">
<p>Just like with the Processor and ControllerService interfaces, the
ReportingTask interface exposes methods for
configuration, validation, and initialization. These methods are all
identical to those of the
Processor and ControllerService interfaces except that the
<code>initialize</code> method is passed a <code>ReportingInitializationContext</code>
object, as opposed to the initialization objects received by the other
Components. The ReportingTask also has
an <code>onTrigger</code> method that is invoked by the framework to trigger the
task to perform its job.</p>
</div>
<div class="paragraph">
<p>Within the <code>onTrigger</code> method, the ReportingTask is given access to a
ReportingContext, from which configuration
and information about the NiFi instance can be obtained. The
BulletinRepository allows Bulletins to be queried
and allows the ReportingTask to submit its own Bulletins, so that
information will be rendered to users. The
ControllerServiceLookup that is accessible via the Context provides
access to ControllerServices that have been
configured. However, this method of obtaining Controller Services is
not the preferred method. Rather, the
preferred method for obtaining a Controller Service is to reference
the Controller Service in a PropertyDescriptor,
as is discussed in the <a href="developer-guide.html#interacting-with-controller-service">Interacting with a ControllerService</a> section.</p>
</div>
<div class="paragraph">
<p>The <code>EventAccess</code> object that is exposed via the ReportingContext
provides access to the <code>ProcessGroupStatus</code>,
which exposes statistics about the amount of data processed in the
past five minutes by Process Groups,
Processors, Connections, and other Components. Additionally, the
EventAccess object provides access to
the ProvenanceEventRecords
that have been stored in the <code>ProvenanceEventRepository</code>. These
Provenance Events are emitted by Processors when
data is received from external sources, emitted to external services,
removed from the system, modified,
or routed according to some decision that was made.</p>
</div>
<div class="paragraph">
<p>Each ProvenanceEvent has the ID of the FlowFile, the type of Event,
the creation time of the Event, and
all FlowFile attributes associated with the FlowFile at the time that
the FlowFile was accessed by the component
as well as the FlowFile attributes that were associated with the
FlowFile as a result of the processing that the
event describes. This provides a great deal of information to
ReportingTasks, allowing reports to be generated
in many different ways to expose metrics and monitoring capabilities
needed for any number of operational concerns.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="parameter-providers"><a class="anchor" href="developer-guide.html#parameter-providers"></a>Parameter Providers</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Although Parameter Contexts cannot be extended, the <code>ParameterProvider</code>
interface allows an extension point for providing parameters to
Parameter Contexts. Parameter Providers can fetch groups of parameters
that can be mapped to new Parameter Contexts, which can then be kept
up to date by re-fetching the parameters.</p>
</div>
<div class="paragraph">
<p>This extensibility allows parameters to be both initially provided and
managed external to NiFi.</p>
</div>
<div class="sect2">
<h3 id="developing-a-parameter-provider"><a class="anchor" href="developer-guide.html#developing-a-parameter-provider"></a>Developing a Parameter Provider</h3>
<div class="paragraph">
<p>The ParameterProvider interface exposes methods for
configuration, validation, and initialization. These methods are all
identical to those of the Processor and ControllerService interfaces except that the
<code>initialize</code> method is passed a <code>ParameterProviderInitializationContext</code>
object, as opposed to the initialization objects received by the other
Components. The ParameterProvider also has a <code>fetchParameters</code> method that is invoked
by the framework to fetch the parameters from its source.</p>
</div>
<div class="paragraph">
<p>Fetched parameters are returned in <code>ParameterGroup</code> s, each of which has
a group name and a list of parameters. Parameter Groups may be mapped by the user
to Parameter Contexts. This allows a single <code>ParameterProvider</code> instance to provide different
parameters to multiple Parameter Contexts based on their mappings.</p>
</div>
<div class="paragraph">
<p>Within the <code>fetchParameters</code> method, the ParameterProvider is given access to a
<code>ConfigurationContext</code>, from which configuration
and information about the NiFi instance can be obtained. The
ControllerServiceLookup that is accessible via the Context provides
access to ControllerServices that have been
configured. However, this method of obtaining Controller Services is
not the preferred method. Rather, the
preferred method for obtaining a Controller Service is to reference
the Controller Service in a PropertyDescriptor,
as is discussed in the <a href="developer-guide.html#interacting-with-controller-service">Interacting with a ControllerService</a> section.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="ui-extensions"><a class="anchor" href="developer-guide.html#ui-extensions"></a>UI Extensions</h2>
<div class="sectionbody">
<div class="paragraph">
<p>There are two UI extension points that are available in NiFi:</p>
</div>
<div class="ulist">
<ul>
<li>
<p>Custom Processor UIs</p>
</li>
<li>
<p>Content Viewers</p>
</li>
</ul>
</div>
<div class="paragraph">
<p>Custom UIs can be created to provide configuration options beyond the standard property/value tables available in most processor settings. Examples of processors with Custom UIs are <a href="https://github.com/apache/nifi/tree/main/nifi-nar-bundles/nifi-update-attribute-bundle" target="_blank" rel="noopener">UpdateAttribute</a> and <a href="https://github.com/apache/nifi/tree/main/nifi-nar-bundles/nifi-standard-bundle" target="_blank" rel="noopener">JoltTransformJSON</a>.</p>
</div>
<div class="paragraph">
<p>Content Viewers can be created to extend the types of data that can be viewed within NiFi. NiFi comes with NARs in the lib directory which contain content viewers for data types such as csv, xml, avro, json (standard-nar) and image types such as png, jpeg and gif (media-nar).</p>
</div>
<div class="sect2">
<h3 id="custom-processor-uis"><a class="anchor" href="developer-guide.html#custom-processor-uis"></a>Custom Processor UIs</h3>
<div class="paragraph">
<p>To add a Custom UI to a processor:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Create your UI.</p>
</li>
<li>
<p>Build and bundle your WAR in a processor NAR.</p>
</li>
<li>
<p>The WAR needs to contain a <code>nifi-processor-configuration</code> file in the META-INF directory, which associates the Custom UI with that processor.</p>
</li>
<li>
<p>Place the NAR in the lib directory and it will be discovered when NiFi starts up.</p>
</li>
<li>
<p>In the Configure Processor window for the processor, the Properties tab should now have an <code>Advanced</code> button, which will access the Custom UI.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>As an example, here is the NAR layout for UpdateAttribute:</p>
</div>
<div class="listingblock">
<div class="title">Update Attribute NAR Layout</div>
<div class="content">
<pre>nifi-update-attribute-bundle
│
├── nifi-update-attribute-model
│
├── nifi-update-attribute-nar
│
├── nifi-update-attribute-processor
│
├── nifi-update-attribute-ui
│ ├── pom.xml
│ └── src
│ └── main
│ ├── java
│ ├── resources
│ └── webapp
│ └── css
│ └── images
│ └── js
│ └── META-INF
│ │ └── nifi-processor-configuration
│ └── WEB-INF
│
└── pom.xml</pre>
</div>
</div>
<div class="paragraph">
<p>with the contents of the <code>nifi-processor-configuration</code> as follows:</p>
</div>
<div class="paragraph">
<p><code>org.apache.nifi.processors.attributes.UpdateAttribute:${project.groupId}:nifi-update-attribute-nar:${project.version}</code></p>
</div>
<div class="admonitionblock note">
<table>
<tr>
<td class="icon">
<i class="fa icon-note" title="Note"></i>
</td>
<td class="content">
Custom UIs can also be implemented for Controller Services and Reporting Tasks.
</td>
</tr>
</table>
</div>
</div>
<div class="sect2">
<h3 id="content-viewers"><a class="anchor" href="developer-guide.html#content-viewers"></a>Content Viewers</h3>
<div class="paragraph">
<p>To add a Content Viewer:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>Build and bundle your WAR in a processor NAR.</p>
</li>
<li>
<p>The WAR needs to contain a <code>nifi-content-viewer</code> file in the META-INF directory, which lists the supported content types.</p>
</li>
<li>
<p>Place the NAR in the lib directory and it will be discovered when NiFi starts up.</p>
</li>
<li>
<p>When a matching content type is encountered, the content viewer will generate the appropriate view.</p>
</li>
</ol>
</div>
<div class="paragraph">
<p>A good example to follow is the NAR layout for the Standard Content Viewer:</p>
</div>
<div class="listingblock">
<div class="title">Standard Content Viewer NAR Layout</div>
<div class="content">
<pre>nifi-standard-bundle
│
├── nifi-jolt-transform-json-ui
│
├── nifi-standard-content-viewer
│ ├── pom.xml
│ └── src
│ └── main
│ ├── java
│ ├── resources
│ └── webapp
│ └── css
│ └── META-INF
│ │ └── nifi-content-viewer
│ └── WEB-INF
│
├── nifi-standard-nar
│
├── nifi-standard-prioritizers
│
├── nifi-standard-processors
│
├── nifi-standard-reporting-tasks
│
├── nifi-standard-utils
│
└── pom.xml</pre>
</div>
</div>
<div class="paragraph">
<p>with the contents of <code>nifi-content-viewer</code> as follows:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>application/xml
application/json
text/plain
text/csv
avro/binary
application/avro-binary
application/avro+binary</pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="command-line-tools"><a class="anchor" href="developer-guide.html#command-line-tools"></a>Command Line Tools</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="tls-toolkit"><a class="anchor" href="developer-guide.html#tls-toolkit"></a>tls-toolkit</h3>
<div class="paragraph">
<p>The Client/Server mode of operation came about from the desire to automatically generate required TLS configuration artifacts without needing to perform that generation in a centralized place. This simplifies configuration in a clustered environment. Since we don’t necessarily have a central place to run the generation logic or a trusted Certificate Authority, a shared secret is used to authenticate the clients and server to each other.</p>
</div>
<div class="paragraph">
<p>The tls-toolkit prevents man in the middle attacks using HMAC verification of the public keys of the CA server and the CSR the client sends. A shared secret (the token) is used as the HMAC key.</p>
</div>
<div class="paragraph">
<p>The basic process goes as follows:</p>
</div>
<div class="olist arabic">
<ol class="arabic">
<li>
<p>The client generates a KeyPair.</p>
</li>
<li>
<p>The client generates a request json payload containing a CSR and an HMAC with the token as the key and the CSR’s public key fingerprint as the data.</p>
</li>
<li>
<p>The client connects to the CA Hostname at the https port specified and validates that the CN of the CA’s certificate matches the hostname (NOTE: because we don’t trust the CA at this point, this adds NO security, it is just a way to error out early if possible).</p>
</li>
<li>
<p>The server validates the HMAC from the client payload using the token as the key and the CSR’s public key fingerprint as the data. This proves that the client knows the shared secret and that it wanted a CSR with that public key to be signed. (NOTE: a man in the middle could forward this on but wouldn’t be able to change the CSR without invalidating the HMAC, defeating the purpose).</p>
</li>
<li>
<p>The server signs the CSR and sends back a response json payload containing the certificate and an HMAC with the token as the key and a fingerprint of its public key as the data.</p>
</li>
<li>
<p>The client validates the response HMAC using the token as the key and a fingerprint of the certificate public key supplied by the TLS session. This validates that a CA that knows the shared secret is the one we are talking to over TLS.</p>
</li>
<li>
<p>The client verifies that the CA certificate from the TLS session signed the certificate in the payload.</p>
</li>
<li>
<p>The client adds the generated KeyPair to its keystore with the certificate chain and adds the CA certificate from the TLS connection to its truststore.</p>
</li>
<li>
<p>The client writes out the configuration json containing keystore, truststore passwords and other details about the exchange.</p>
</li>
</ol>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="testing"><a class="anchor" href="developer-guide.html#testing"></a>Testing</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Testing the components that will be used within a larger framework can often be very cumbersome
and tricky. With NiFi, we strive to make testing components as easy as possible. In order to do this,
we have created a <code>nifi-mock</code> module that can be used in conjunction with JUnit to provide extensive
testing of components.</p>
</div>
<div class="paragraph">
<p>The Mock Framework is mostly aimed at testing Processors, as these are by far the most commonly
developed extension point. However, the framework does provide the ability to test Controller Services
as well.</p>
</div>
<div class="paragraph">
<p>Components have typically been tested by creating functional tests to verify component behavior. This is
done because often a Processor will consist of a handful of helper methods but the logic will largely be
encompassed within the <code>onTrigger</code> method. The <code>TestRunner</code> interface allows us to test Processors
and Controller Services by converting more "primitive" objects such as files and byte arrays into
FlowFiles and handles creating the ProcessSessions and ProcessContexts needed for a Processor to do its job,
as well as invoking the necessary lifecycle methods in order to ensure that the Processor behaves the
same way in the unit tests as it does in production.</p>
</div>
<div class="sect2">
<h3 id="instantiate-testrunner"><a class="anchor" href="developer-guide.html#instantiate-testrunner"></a>Instantiate TestRunner</h3>
<div class="paragraph">
<p>Most unit tests for a Processor or a Controller Service start by creating an instance of the <code>TestRunner</code>
class. In order to add the necessary classes to your Processor,
you can use the Maven dependency:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code>&lt;dependency&gt;
&lt;groupId&gt;org.apache.nifi&lt;/groupId&gt;
&lt;artifactId&gt;nifi-mock&lt;/artifactId&gt;
&lt;version&gt;${nifi version}&lt;/version&gt;
&lt;/dependency&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>We create a new <code>TestRunner</code> by calling one of the static <code>newTestRunner</code> methods of the <code>TestRunners</code> class
(located in the <code>org.apache.nifi.util</code> package). These methods take an argument for the Processor under test (can
either be the class of the Processor to test or can be an instance of a Processor), and allow the setting of
the processor name as well.</p>
</div>
</div>
<div class="sect2">
<h3 id="add-controllerservices"><a class="anchor" href="developer-guide.html#add-controllerservices"></a>Add ControllerServices</h3>
<div class="paragraph">
<p>After creating a new Test Runner, we can add any Controller Services to the Test Runner that our Processor
will need in order to perform its job. We do this by calling the <code>addControllerService</code> method and supply
both an identifier for the Controller Service and an instance of the Controller Service.</p>
</div>
<div class="paragraph">
<p>If the Controller Service needs to be configured, its properties can be set by
calling the <code>setProperty(ControllerService, PropertyDescriptor, String)</code>, <code>setProperty(ControllerService, String, String)</code>,
or <code>setProperty(ControllerService, PropertyDescriptor, AllowableValue)</code> method. Each of these methods returns a
<code>ValidationResult</code>. This object can then be inspected to ensure that the property is valid by calling <code>isValid</code>.
Annotation data can be set by calling the <code>setAnnotationData(ControllerService, String)</code> method.</p>
</div>
<div class="paragraph">
<p>We can now ensure that the Controller Service is valid by calling <code>assertValid(ControllerService)</code> - or ensure
that the configured values are not valid, if testing the Controller Service itself, by calling
<code>assertNotValid(ControllerService)</code>.</p>
</div>
<div class="paragraph">
<p>Once a Controller Service has been added to the Test Runner and configured, it can now be enabled by calling the
<code>enableControllerService(ControllerService)</code> method. If the Controller Service is not valid, this method
will throw an IllegalStateException. Otherwise, the service is now ready to use.</p>
</div>
</div>
<div class="sect2">
<h3 id="set-property-values"><a class="anchor" href="developer-guide.html#set-property-values"></a>Set Property Values</h3>
<div class="paragraph">
<p>After configuring any necessary Controller Services, we need to configure our Processor. We can do this by
calling the same methods as we do for Controller Services, without specifying any Controller Service. I.e.,
we can call <code>setProperty(PropertyDescriptor, String)</code>, and so on. Each of the <code>setProperty</code> methods again
returns a <code>ValidationResult</code> property that can be used to ensure that the property value is valid.</p>
</div>
<div class="paragraph">
<p>Similarly, we can also call <code>assertValid()</code> and <code>assertNotValid()</code> to ensure that the configuration of the
Processor is valid or not, according to our expectations.</p>
</div>
</div>
<div class="sect2">
<h3 id="enqueue-flowfiles"><a class="anchor" href="developer-guide.html#enqueue-flowfiles"></a>Enqueue FlowFiles</h3>
<div class="paragraph">
<p>Before triggering a Processor to run, it is usually necessary to enqueue FlowFiles for the Processor to process.
This can be achieved by using the <code>enqueue</code> methods of the <code>TestRunner</code> class. The <code>enqueue</code> method has several
different overrides, and allows data to be added in the form of a <code>byte[]</code>, <code>InputStream</code>, or <code>Path</code>. Each of these
methods also supports a variation that allows a <code>Map&lt;String, String&gt;</code> to be added to support FlowFile attributes.</p>
</div>
<div class="paragraph">
<p>Additionally, there is an <code>enqueue</code> method that takes a var-args of FlowFile objects. This can be useful, for example,
to obtain the output of a Processor and then feed this to the input of the Processor.</p>
</div>
</div>
<div class="sect2">
<h3 id="run-the-processor"><a class="anchor" href="developer-guide.html#run-the-processor"></a>Run the Processor</h3>
<div class="paragraph">
<p>After configuring the Controller Services and enqueuing the necessary FlowFiles, the Processor can be triggered
to run by calling the <code>run</code> method of <code>TestRunner</code>. If this method is called without any arguments, it will
invoke any method in the Processor with an <code>@OnScheduled</code> annotation, call the Processor&#8217;s <code>onTrigger</code> method once,
and then run the <code>@OnUnscheduled</code> and finally <code>@OnStopped</code> methods.</p>
</div>
<div class="paragraph">
<p>If it is desirable to run several iterations of the <code>onTrigger</code> method before the other <code>@OnUnscheduled</code> and
<code>@OnStopped</code> life-cycle events are triggered, the <code>run(int)</code> method can be used to specify how many iterations
of <code>onTrigger</code> should be called.</p>
</div>
<div class="paragraph">
<p>There are times when we want to trigger the Processor to run but not trigger the <code>@OnUnscheduled</code> and <code>@OnStopped</code>
life-cycle events. This is useful, for instance, to inspect the Processor&#8217;s state before these events occur. This
can be achieved using the <code>run(int, boolean)</code> and passing <code>false</code> as the second argument. After doing this, though,
calling the <code>@OnScheduled</code> life-cycle methods could cause an issue. As a result, we can now run <code>onTrigger</code> again
without causing these events to occur by using the <code>run(int,boolean,boolean)</code> version of the <code>run</code> method and
passing <code>false</code> as the third argument.</p>
</div>
<div class="paragraph">
<p>If it is useful to test behavior that occurs with multiple threads, this can also be achieved by calling the
<code>setThreadCount</code> method of <code>TestRunner</code>. The default is 1 thread. If using multiple threads, it is important
to remember that the <code>run</code> call of <code>TestRunner</code> specifies how many times the Processor should be triggered, not
the number of times that the Processor should be triggered per thread. So, if the thread count is set to 2 but
<code>run(1)</code> is called, only a single thread will be used.</p>
</div>
</div>
<div class="sect2">
<h3 id="validate-output"><a class="anchor" href="developer-guide.html#validate-output"></a>Validate Output</h3>
<div class="paragraph">
<p>After a Processor has finished running, a unit test will generally want to validate that the FlowFiles went where
they were expected to go. This can be achieved using the <code>TestRunners</code> <code>assertAllFlowFilesTransferred</code> and
<code>assertTransferCount</code> methods. The former method takes as arguments a Relationship and an integer to dictate how many
FlowFiles should have been transferred to that Relationship. The method will fail the unit test unless this number of
FlowFiles were transferred to the given Relationship <strong>or</strong> if any FlowFile was transferred to any other Relationship.
The <code>assertTransferCount</code> method validates only that the FlowFile count was the expected number for the given Relationship.</p>
</div>
<div class="paragraph">
<p>After validating the counts, we can then obtain the actual output FlowFiles via the <code>getFlowFilesForRelationship</code> method.
This method returns a <code>List&lt;MockFlowFile&gt;</code>. It&#8217;s important to note that the type of the List is <code>MockFlowFile</code>, rather
than the <code>FlowFile</code> interface. This is done because <code>MockFlowFile</code> comes with many methods for validating the contents.</p>
</div>
<div class="paragraph">
<p>For example, <code>MockFlowFile</code> has methods for asserting that FlowFile Attributes exist (<code>assertAttributeExists</code>), asserting
that other attributes are not present (<code>assertAttributeNotExists</code>), or that Attributes have the correct value
(<code>assertAttributeEquals</code>, <code>assertAttributeNotEquals</code>). Similar methods exist for verifying the contents of the FlowFile.
The contents of a FlowFile can be compared to a <code>byte[]</code>, and <code>InputStream</code>, a file, or a String. If the data is expected
to be textual, the String version is preferred, as it provides a more intuitive error message if the output is not
as expected.</p>
</div>
</div>
<div class="sect2">
<h3 id="mocking-external-resources"><a class="anchor" href="developer-guide.html#mocking-external-resources"></a>Mocking External Resources</h3>
<div class="paragraph">
<p>One of the biggest problems when testing a NiFi processor that connects to a remote resource is that we don&#8217;t want to
actually connect to some remote resource from a unit test. We can stand up a simple server ourselves in the unit test
and configure the Processor to communicate with it, but then we have to understand and implement the server-specific
specification and may not be able to properly send back error messages, etc. that we would like for testing.</p>
</div>
<div class="paragraph">
<p>Generally, the approach taken here is to have a method in the Processor that is responsible for obtaining a connection
or a client to a remote resource. We generally mark this method as protected. In the unit test, instead of creating
the <code>TestRunner</code> by calling <code>TestRunners.newTestRunner(Class)</code> and providing the Processor class, we instead create
a subclass of the Processor in our unit test and use this:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">@Test
public void testConnectionFailure() {
final TestRunner runner = TestRunners.newTestRunner(new MyProcessor() {
protected Client getClient() {
// Return a mocked out client here.
return new Client() {
public void connect() throws IOException {
throw new IOException();
}
// ...
// other client methods
// ...
};
}
});
// rest of unit test.
}</code></pre>
</div>
</div>
<div class="paragraph">
<p>This allows us to implement a Client that mocks out all of the network communications and returns the different
error results that we want to test, as well as ensure that our logic is correct for handling successful calls
to the client.</p>
</div>
</div>
<div class="sect2">
<h3 id="additional-testing-capabilities"><a class="anchor" href="developer-guide.html#additional-testing-capabilities"></a>Additional Testing Capabilities</h3>
<div class="paragraph">
<p>In addition to the above-mentioned capabilities provided by the
testing framework, the TestRunner provides several
convenience methods for verifying the behavior of a Processor. Methods
are provided for ensuring that the Processor&#8217;s
Input Queue has been emptied. Unit Tests are able to obtain the
ProcessContext, ProcessSessionFactory, ProvenanceReporter,
and other framework-specific entities that will be used by the
TestRunner. The <code>shutdown</code> method provides the ability to
test Processor methods that are annotated to be run only on shutdown
of NiFi. Annotation Data can be set for Processors
that make use of Custom User Interfaces. Finally, the number of
threads that should be used to run the Processor can
be set via the <code>setThreadCount(int)</code> method.</p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="nars"><a class="anchor" href="developer-guide.html#nars"></a>NiFi Archives (NARs)</h2>
<div class="sectionbody">
<div class="paragraph">
<p>When software from many different organizations is all hosted within
the same environment, Java ClassLoaders quickly
become a concern. If multiple components have a dependency on the same
library but each depends on a different
version, many problems arise, typically resulting in unexpected
behavior or <code>NoClassDefFoundError</code> errors occurring.
In order to prevent these issues from becoming problematic, NiFi
introduces the notion of a NiFi Archive, or NAR.</p>
</div>
<div class="paragraph">
<p>A NAR allows several components and their dependencies to be packaged
together into a single package.
The NAR package is then provided ClassLoader isolation from other NAR
packages. Developers should always deploy
their NiFi components as NAR packages.</p>
</div>
<div class="paragraph">
<p>To achieve this, a developer creates a new Maven Artifact, which we
refer to as the NAR artifact. The packaging is
set to <code>nar</code>. The <code>dependencies</code> section of the POM is then created so
that the NAR has a dependency on all NiFi Components that are to be included within the NAR.</p>
</div>
<div class="paragraph">
<p>In order to use a packaging of <code>nar</code>, we must use the <code>nifi-nar-maven-plugin</code> module.
This is included by adding the following snippet to the NAR&#8217;s pom.xml:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-xml" data-lang="xml">&lt;build&gt;
&lt;plugins&gt;
&lt;plugin&gt;
&lt;groupId&gt;org.apache.nifi&lt;/groupId&gt;
&lt;artifactId&gt;nifi-nar-maven-plugin&lt;/artifactId&gt;
&lt;version&gt;1.3.4&lt;/version&gt;
&lt;extensions&gt;true&lt;/extensions&gt;
&lt;/plugin&gt;
&lt;/plugins&gt;
&lt;/build&gt;</code></pre>
</div>
</div>
<div class="paragraph">
<p>In the Apache NiFi codebase, this exists in the NiFi root POM from which all other NiFi artifacts
(with the exception of the nifi-nar-maven-plugin itself) inherit, so that we do not need to include
this in any of our other POM files.</p>
</div>
<div class="paragraph">
<p>The NAR is able to have one dependency that is of type <code>nar</code>. If more
than one dependency is specified that is of type
<code>nar</code>, then the nifi-nar-maven-plugin will error. If NAR A adds a
dependency on NAR B, this will <strong>not</strong> result in
NAR B packaging all of the components of NAR A. Rather, this will add
a <code>Nar-Dependency-Id</code> element to the <code>MANIFEST.MF</code>
file of NAR A. This will result in setting the ClassLoader of NAR B as
the Parent ClassLoader of NAR A. In this case,
we refer to NAR B as the <em>Parent</em> of NAR A.</p>
</div>
<div class="paragraph">
<p>This linkage of Parent ClassLoaders is the mechanism that NiFi uses in
order to enable Controller Services to be shared
across all NARs. As mentioned in the <a href="developer-guide.html#developing-controller-service">Developing a ControllerService</a>
section, A Controller Service must be separated
into an interface that extends <code>ControllerService</code> and an
implementation that implements that interface. Controller Services
can be referenced from any Processor, regardless of which NAR it is
in, as long as both the Controller Service Implementation
and the Processor share the same definition of the Controller Service interface.</p>
</div>
<div class="paragraph">
<p>In order to share this same definition, both the Processor&#8217;s NAR and
the Controller Service Implementation&#8217;s NAR must have
as a Parent the Controller Service definition&#8217;s NAR. An example
hierarchy may look like this:</p>
</div>
<div class="listingblock">
<div class="title">Controller Service NAR Layout</div>
<div class="content">
<pre>root
├── my-controller-service-api
│ ├── pom.xml
│ └── src
│ └── main
│ └── java
│ └── org
│ └── my
│ └── services
│ └── MyService.java
│
├── my-controller-service-api-nar
│ └── pom.xml <i class="conum" data-value="1"></i><b>(1)</b>
│
│
│
├── my-controller-service-impl
│ ├── pom.xml <i class="conum" data-value="2"></i><b>(2)</b>
│ └── src
│ ├── main
│ │ ├── java
│ │ │ └── org
│ │ │ └── my
│ │ │ └── services
│ │ │ └── MyServiceImpl.java
│ │ └── resources
│ │ └── META-INF
│ │ └── services
│ │ └── org.apache.nifi.controller.ControllerService
│ └── test
│ └── java
│ └── org
│ └── my
│ └── services
│ └── TestMyServiceImpl.java
│
│
├── my-controller-service-nar
│ └── pom.xml <i class="conum" data-value="3"></i><b>(3)</b>
│
│
└── other-processor-nar
└── pom.xml <i class="conum" data-value="3"></i><b>(3)</b></pre>
</div>
</div>
<div class="colist arabic">
<table>
<tr>
<td><i class="conum" data-value="1"></i><b>1</b></td>
<td>This POM file has a type of <code>nar</code>. It has a dependency on
<code>nifi-standard-services-api-nar</code>.</td>
</tr>
<tr>
<td><i class="conum" data-value="2"></i><b>2</b></td>
<td>This POM file is of type <code>jar</code>. It has a dependency on
<code>my-controller-service-api</code>. It does <strong>not</strong> have a dependency on any
<code>nar</code> artifacts.</td>
</tr>
<tr>
<td><i class="conum" data-value="3"></i><b>3</b></td>
<td>This POM file has a type of <code>nar</code>. It has a dependency on
<code>my-controller-service-api-nar</code>.</td>
</tr>
</table>
</div>
<div class="paragraph">
<p>While these may seem very complex at first, after creating such a
hierarchy once or twice, it becomes
far less complicated. Note here that the
<code>my-controller-service-api-nar</code> has a dependency on
<code>nifi-standard-services-api-nar</code>.
This is done so that any NAR that has a dependency on
<code>my-controller-service-api-nar</code> will also be able to access
all of the Controller Services that are provided by the
<code>nifi-standard-services-api-nar</code>, such as the <code>SSLContextService</code>.
In this same vein, it is not necessary to create a different
"service-api" NAR for each service. Instead, it often
makes sense to have a single "service-api" NAR that encapsulates the
APIs for many different Controller Services,
as is done by the <code>nifi-standard-services-api-nar</code>. Generally, the API
will not include extensive dependencies, and as a result,
ClassLoader isolation may be less important, so lumping together many
API artifacts into the same NAR is often acceptable.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="per-instance-classloading"><a class="anchor" href="developer-guide.html#per-instance-classloading"></a>Per-Instance ClassLoading</h2>
<div class="sectionbody">
<div class="paragraph">
<p>A component developer may wish to add additional resources to the component’s classpath at runtime.
For example, you may want to provide the location of a JDBC driver to a processor that interacts with a
relational database, thus allowing the processor to work with any driver rather than trying to bundle a
driver into the NAR.</p>
</div>
<div class="paragraph">
<p>This may be accomplished by declaring one or more PropertyDescriptor instances with
<code>dynamicallyModifiesClasspath</code> set to true. For example:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">PropertyDescriptor EXTRA_RESOURCE = new PropertyDescriptor.Builder()
.name("Extra Resources")
.description("The path to one or more resources to add to the classpath.")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.dynamicallyModifiesClasspath(true)
.build();</code></pre>
</div>
</div>
<div class="paragraph">
<p>When these properties are set on a component, the framework identifies all properties where
<code>dynamicallyModifiesClasspath</code> is set to true. For each of these properties, the framework
attempts to resolve filesystem resources from the value of the property. The value may be a
comma-separated list of one or more directories or files, where any paths that do not exist are
skipped. If the resource represents a directory, the directory is listed, and all of the files
in that directory are added to the classpath individually.
These directories also will be scanned for native libraries. If a library is found in one of these
directories, an OS-handled temporary copy is created and cached before loading it to maintain consistency
and classloader isolation.</p>
</div>
<div class="paragraph">
<p>Each property may impose further restrictions on the format of the value through the validators.
For example, using <code>StandardValidators.FILE_EXISTS_VALIDATOR</code> restricts the property to accepting a
single file. Using <code>StandardValidators.NON_EMPTY_VALIDATOR</code> allows any combination of comma-separated
files or directories.</p>
</div>
<div class="paragraph">
<p>Resources are added to the instance ClassLoader by adding them to an inner ClassLoader that is always
checked first. Anytime the value of these properties change, the inner ClassLoader is closed and
re-created with the new resources.</p>
</div>
<div class="paragraph">
<p>NiFi provides the <code>@RequiresInstanceClassLoading</code> annotation to further expand and isolate the libraries
available on a component’s classpath. You can annotate a component with <code>@RequiresInstanceClassLoading</code>
to indicate that the instance ClassLoader for the component requires a copy of all the resources in the
component&#8217;s NAR ClassLoader. When <code>@RequiresInstanceClassLoading</code> is not present, the
instance ClassLoader simply has its parent ClassLoader set to the NAR ClassLoader, rather than
copying resources.</p>
</div>
<div class="paragraph">
<p>The <code>@RequiresInstanceClassLoading</code> annotation also provides an optional flag <code>cloneAncestorResources</code>. If
set to true, the instance ClassLoader will include ancestor resources up to the first ClassLoader containing a
controller service API referenced by the component, or up to the Jetty NAR. If set to false, or not specified,
only the resources from the component&#8217;s NAR will be included.</p>
</div>
<div class="paragraph">
<p>Because <code>@RequiresInstanceClassLoading</code> copies resources from the NAR ClassLoader for each instance of the
component, use this capability judiciously. If ten instances of one component are created, all classes
from the component&#8217;s NAR ClassLoader are loaded into memory ten times. This could eventually increase the
memory footprint significantly when enough instances of the component are created.</p>
</div>
<div class="paragraph">
<p>Additionally, there are some restrictions when using <code>@RequiresInstanceClassLoading</code> when using Controller
Services. Processors, Reporting Tasks, and Controller Services can reference a Controller Service API
in one of its Property Descriptors. An issue may arise when the Controller Service API is bundled in the same
NAR with a component that references it or with the Controller Service implementation. If either of these cases
are encountered and the extension requires instance classloading, the extension will be skipped and an appropriate
ERROR will be logged. To address this issue, the Controller Service API should be bundled in a parent NAR. The
service implementation and extensions that reference that service should depend on the Controller Service API NAR.
Please refer to the Controller Service NAR Layout in the <a href="developer-guide.html#nars">NiFi Archives (NARs)</a> section. Anytime a Controller Service API is
bundled with an extension that requires it, even if <code>@RequiresInstanceClassLoading</code> isn&#8217;t used, a WARNING will be
logged to help avoid this bad practice.</p>
</div>
</div>
</div>
<div class="sect1">
<h2 id="deprecation"><a class="anchor" href="developer-guide.html#deprecation"></a>Deprecating Components and Features</h2>
<div class="sectionbody">
<div class="paragraph">
<p>Deprecating features is an important part of the software development lifecycle, providing an upgrade path for
developers and users to follow.</p>
</div>
<div class="paragraph">
<p>Apache NiFi follows the <a href="https://semver.org" target="_blank" rel="noopener">Semantic Versioning Specification 2.0.0</a> for features identified as
part of the public version contract according to
<a href="https://cwiki.apache.org/confluence/display/NIFI/Version+Scheme+and+API+Compatibility">Version Scheme</a>
documentation.</p>
</div>
<div class="paragraph">
<p>Components and features that fit under the public version contract require deprecation marking prior to removal. This
approach allows developers to implement changes as part of minor version upgrades, in preparation for future removal
of features in a subsequent major version.</p>
</div>
<div class="sect2">
<h3 id="component-deprecation"><a class="anchor" href="developer-guide.html#component-deprecation"></a>Component Deprecation</h3>
<div class="paragraph">
<p>Sometimes it may be desirable to deprecate a component. Whenever this occurs the developer may use the
<code>@DeprecationNotice</code> annotation to indicate that a component has been deprecated, allowing the developer
to describe a <code>reason</code> for the deprecation and suggest alternative components. An example of how to do this can
be found below:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">@DeprecationNotice(
reason = "Legacy Protocol has been superseded",
alternatives = {ListenSyslog.class},
classNames = {"org.apache.nifi.processors.standard.ListenRELP"}
)
public class ListenLegacyProtocol extends AbstractProcessor {}</code></pre>
</div>
</div>
<div class="paragraph">
<p>The <code>alternatives</code> property can be used to define an array of recommended replacement Components, while <code>classNames</code>
can be used to represent similar content through an array of class name strings.</p>
</div>
<div class="paragraph">
<p>Adding the <code>@DeprecationNotice</code> annotation renders a warning message in generated documentation and also logs the
following warning when the Flow Configuration includes the component:</p>
</div>
<div class="listingblock">
<div class="content">
<pre>Added Deprecated Component ListenLegacyProtocol[id=929a52c7-1e3e-423e-b303-6ca2ef657617] See alternatives [ListenSyslog,ListenRELP]</pre>
</div>
</div>
</div>
<div class="sect2">
<h3 id="feature-deprecation"><a class="anchor" href="developer-guide.html#feature-deprecation"></a>Feature Deprecation</h3>
<div class="paragraph">
<p>Deprecating features includes changing component configuration strategies, introducing new repository classes, and
refactoring a Controller Service interface. Removing component properties can create invalid Flow Configurations after
upgrading, and removing public methods from a Controller Service interface can break components compiled against
previous versions. For these reasons, introducing new properties and methods must include a deprecation strategy that
supports compatibility when upgrading from one minor version to another.</p>
</div>
<div class="paragraph">
<p>Annotating methods and properties with the Java <code>@Deprecated</code> annotation provides a warning to software developers, but
does not provide any information to users. Adding selective deprecation logging provides a method for users to determine
whether the current Flow Configuration uses deprecated properties or features.</p>
</div>
</div>
<div class="sect2">
<h3 id="deprecation-logging"><a class="anchor" href="developer-guide.html#deprecation-logging"></a>Deprecation Logging</h3>
<div class="paragraph">
<p>The <code>nifi-deprecation-log</code> module provides an abstraction for logging deprecation warnings. The <code>DeprecationLogger</code>
interface has a <code>warn()</code> method that accepts a message string with a variable number of arguments. The standard
implementation delegates to SLF4J, supporting standard features for providing and resolving placeholder arguments. The
standard <code>DeprecationLogger</code> implementation also logs a <code>DeprecationException</code> with the referenced class, supporting
tracing to the usage location.</p>
</div>
<div class="paragraph">
<p>Instantiating a <code>DeprecationLogger</code> follows the same conventions as instantiating an SLF4J <code>Logger</code>:</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">private static final DeprecationLogger deprecationLogger = DeprecationLoggerFactory.getLogger(Service.class);</code></pre>
</div>
</div>
<div class="paragraph">
<p>Logging warnings for deprecated extension component properties and methods should include the simple class name and the
component identifier for tracing. Warning messages should also reference recommended alternatives.</p>
</div>
<div class="listingblock">
<div class="content">
<pre class="highlight"><code class="language-java" data-lang="java">deprecationLogger.warn(
"{}[id={}] createServiceContext() should be replaced with createContext()",
getClass().getSimpleName(),
getIdentifier()
);</code></pre>
</div>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="build"><a class="anchor" href="developer-guide.html#build"></a>Build Options</h2>
<div class="sectionbody">
<div class="sect2">
<h3 id="maven-profiles"><a class="anchor" href="developer-guide.html#maven-profiles"></a>Maven Profiles</h3>
<table class="tableblock frame-all grid-all stretch">
<colgroup>
<col style="width: 33.3333%;">
<col style="width: 33.3333%;">
<col style="width: 33.3334%;">
</colgroup>
<thead>
<tr>
<th class="tableblock halign-left valign-top">Package</th>
<th class="tableblock halign-left valign-top">Maven Profile</th>
<th class="tableblock halign-left valign-top">Description</th>
</tr>
</thead>
<tbody>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Apache Accumulo Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-accumulo</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for <a href="https://accumulo.apache.org">Apache Accumulo</a>.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Apache Atlas Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-atlas</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for the Apache Atlas data governance tool. The functionality from this bundle is based around reporting tasks that integrate with <a href="https://atlas.apache.org">Apache Atlas</a>.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Apache Hive 1.1 Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-hive1_1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for Apache Hive 1.1.X.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Apache Hive 3 Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-hive3</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for Apache Hive 3.X</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Apache Ranger Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-ranger</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for <a href="https://ranger.apache.org">Apache Ranger</a>.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">ASN1 Support</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-asn1</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for ASN1</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Contribution Check</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">contrib-check</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Runs various quality checks that are required to be accepted before a contribution can be accepted into the core NiFi code base.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Graph Database Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-graph</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for various common graph database scenarios. Support is currently for <a href="https://neo4j.com/developer/cypher">Cypher</a> and <a href="https://tinkerpop.apache.org/gremlin.html">Gremlin</a>-compatible databases such as Neo4J and JanusGraph. Includes controller services that provide driver functionality and a suite of processors for ingestion and querying.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">GRPC Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-grpc</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock"><strong>This profile is active in official builds and should be active</strong> Provides support for the GRPC protocol.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Media Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-media</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">The media bundle provides functionality based on <a href="https://tika.apache.org">Apache Tika</a> for extracting content and metadata from various types of binary formats supported by Apache Tika (ex. PDF, Microsoft Office).</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Rules Engine Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-rules</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for creating scripted rules engines that can be integrated into existing flows. These rules engines can provide flexibility to people who need more complex flow logic or are more comfortable with flow decision-making using custom code.</p></td>
</tr>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">Snowflake Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-snowflake</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds support for integration with the <a href="https://www.snowflake.com">Snowflake platform</a>.</p></td>
</tr>
</tbody>
<tfoot>
<tr>
<td class="tableblock halign-left valign-top"><p class="tableblock">SQL Reporting Bundle</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">include-sql-reporting</p></td>
<td class="tableblock halign-left valign-top"><p class="tableblock">Adds reporting tasks that are designed to use SQL to update a RDBMS with metrics and other related data from Apache NiFi.</p></td>
</tr>
</tfoot>
</table>
</div>
<div class="sect2">
<h3 id="standard-build-instructions"><a class="anchor" href="developer-guide.html#standard-build-instructions"></a>Standard Build Instructions</h3>
<div class="paragraph">
<p>The following command is used to generate a standard binary distribution of Apache NiFi:</p>
</div>
<div class="paragraph">
<p><code>mvn clean install -Pinclude-grpc,contrib-check</code></p>
</div>
</div>
</div>
</div>
<div class="sect1">
<h2 id="how-to-contribute-to-apache-nifi"><a class="anchor" href="developer-guide.html#how-to-contribute-to-apache-nifi"></a>How to contribute to Apache NiFi</h2>
<div class="sectionbody">
<div class="paragraph">
<p>We are always excited to have contributions from the community - especially from new contributors!
We are interested in accepting contributions of code, as well as documentation and even artwork that
can be applied as icons or styling to the application.</p>
</div>
<div class="sect2">
<h3 id="technologies"><a class="anchor" href="developer-guide.html#technologies"></a>Technologies</h3>
<div class="paragraph">
<p>The back end of Apache NiFi is written in Java. The web tier makes use of JAX-RS and JavaScript is extensively
used to provide a user interface. We depend on several third-party JavaScript libraries, including D3 and JQuery,
among others. We make use of Apache Maven for our builds and Git for our version control system.</p>
</div>
<div class="paragraph">
<p>Documentation is created in <a href="http://asciidoctor.org" target="_blank" rel="noopener">AsciiDoc</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="where-to-start"><a class="anchor" href="developer-guide.html#where-to-start"></a>Where to Start?</h3>
<div class="paragraph">
<p><a href="http://issues.apache.org/jira/browse/NIFI" target="_blank" rel="noopener">NiFi&#8217;s JIRA page</a> can be used to find tickets that are tagged as "beginner",
or you can dig into any of the tickets for creating Processors. Processors should be self-contained and not rely on other
outside components (except for Controller Services), so they make for excellent starting points for new NiFi developers to
get started. This exposes the developer to the NiFi API and is the most extensible part of the dataflow system.</p>
</div>
<div class="paragraph">
<p>System-level and overview documentation is located in '&lt;code checkout location&gt;/nifi/nifi-docs/src/main/asciidoc'.
Tools available to facilitate documentation generation are available at <a href="http://asciidoctor.org/docs/editing-asciidoc-with-live-preview/" target="_blank" rel="noopener">Editing AsciiDoc with Live Preview</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="supplying-a-contribution"><a class="anchor" href="developer-guide.html#supplying-a-contribution"></a>Supplying a contribution</h3>
<div class="paragraph">
<p>Contributions can be provided either by creating a patch:</p>
</div>
<div class="paragraph">
<p><code>git format-patch</code></p>
</div>
<div class="paragraph">
<p>and attaching that patch to a ticket, or by generating a Pull Request.</p>
</div>
<div class="paragraph">
<p>For further details regarding contributions, refer to the relevant sections of the <a href="https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide#ContributorGuide-providingCodeOrDocumentationContributionProvidingcodeordocumentationcontributions" target="_blank" rel="noopener">contributor guide</a>.</p>
</div>
</div>
<div class="sect2">
<h3 id="contact-us"><a class="anchor" href="developer-guide.html#contact-us"></a>Contact Us</h3>
<div class="paragraph">
<p>The developer mailing list (<a href="mailto:dev@nifi.apache.org">dev@nifi.apache.org</a>) is monitored pretty closely, and we tend to respond pretty
quickly. If you have a question, don&#8217;t hesitate to shoot us an e-mail - we&#8217;re here to help! Unfortunately, though, e-mails
can get lost in the shuffle, so if you do send an e-mail and don&#8217;t get a response within a day or two, it&#8217;s our fault - don&#8217;t
worry about bothering us. Just ping the mailing list again.</p>
</div>
</div>
</div>
</div>
</div>
<div id="footer">
<div id="footer-text">
Last updated 2022-12-05 18:19:00 -0600
</div>
</div>
</body>
</html>