blob: cfb9b268b344f64303f19cc0c1285a2ccec65cb3 [file] [log] [blame]
<!DOCTYPE html><html lang="en"><head><meta charSet="utf-8"/><meta http-equiv="X-UA-Compatible" content="IE=edge"/><title>Pulsar Functions State Storage (Developer Preview) · Apache Pulsar</title><meta name="viewport" content="width=device-width, initial-scale=1.0"/><meta name="generator" content="Docusaurus"/><meta name="description" content="Since Pulsar 2.1.0 release, Pulsar integrates with Apache BookKeeper [table service](https://docs.google.com/document/d/155xAwWv5IdOitHh1NVMEwCMGgB28M3FyMiQSxEpjE-Y/edit#heading=h.56rbh52koe3f)"/><meta name="docsearch:version" content="2.7.1"/><meta name="docsearch:language" content="en"/><meta property="og:title" content="Pulsar Functions State Storage (Developer Preview) · Apache Pulsar"/><meta property="og:type" content="website"/><meta property="og:url" content="https://pulsar.apache.org/"/><meta property="og:description" content="Since Pulsar 2.1.0 release, Pulsar integrates with Apache BookKeeper [table service](https://docs.google.com/document/d/155xAwWv5IdOitHh1NVMEwCMGgB28M3FyMiQSxEpjE-Y/edit#heading=h.56rbh52koe3f)"/><meta name="twitter:card" content="summary"/><meta name="twitter:image" content="https://pulsar.apache.org/img/pulsar.svg"/><link rel="shortcut icon" href="/img/pulsar.ico"/><link rel="stylesheet" href="//cdnjs.cloudflare.com/ajax/libs/highlight.js/9.12.0/styles/atom-one-dark.min.css"/><link rel="alternate" type="application/atom+xml" href="https://pulsar.apache.org/blog/atom.xml" title="Apache Pulsar Blog ATOM Feed"/><link rel="alternate" type="application/rss+xml" href="https://pulsar.apache.org/blog/feed.xml" title="Apache Pulsar Blog RSS Feed"/><link rel="stylesheet" href="/css/code-blocks-buttons.css"/><script type="text/javascript" src="https://buttons.github.io/buttons.js"></script><script type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/clipboard.js/2.0.0/clipboard.min.js"></script><script type="text/javascript" src="/js/custom.js"></script><script src="/js/scrollSpy.js"></script><link rel="stylesheet" href="/css/main.css"/><script src="/js/codetabs.js"></script></head><body class="sideNavVisible separateOnPageNav"><div class="fixedHeaderContainer"><div class="headerWrapper wrapper"><header><a href="/en"><img class="logo" src="/img/pulsar.svg" alt="Apache Pulsar"/></a><a href="/en/versions"><h3>2.7.1</h3></a><div class="navigationWrapper navigationSlider"><nav class="slidingNav"><ul class="nav-site nav-site-internal"><li class=""><a href="/docs/en/2.7.1/getting-started-standalone" target="_self">Docs</a></li><li class=""><a href="/en/download" target="_self">Download</a></li><li class=""><a href="/docs/en/2.7.1/client-libraries" target="_self">Clients</a></li><li class=""><a href="#restapis" target="_self">REST APIs</a></li><li class=""><a href="#cli" target="_self">Cli</a></li><li class=""><a href="/blog/" target="_self">Blog</a></li><li class=""><a href="#community" target="_self">Community</a></li><li class=""><a href="#apache" target="_self">Apache</a></li><li class=""><a href="https://pulsar-next.staged.apache.org/" target="_self">New Website (Beta)</a></li><span><li><a id="languages-menu" href="#"><img class="languages-icon" src="/img/language.svg" alt="Languages icon"/>English</a><div id="languages-dropdown" class="hide"><ul id="languages-dropdown-items"><li><a href="/docs/ja/2.7.1/functions-state">日本語</a></li><li><a href="/docs/fr/2.7.1/functions-state">Français</a></li><li><a href="/docs/ko/2.7.1/functions-state">한국어</a></li><li><a href="/docs/zh-CN/2.7.1/functions-state">中文</a></li><li><a href="/docs/zh-TW/2.7.1/functions-state">繁體中文</a></li><li><a href="https://crowdin.com/project/apache-pulsar" target="_blank" rel="noreferrer noopener">Help Translate</a></li></ul></div></li><script>
const languagesMenuItem = document.getElementById("languages-menu");
const languagesDropDown = document.getElementById("languages-dropdown");
languagesMenuItem.addEventListener("click", function(event) {
event.preventDefault();
if (languagesDropDown.className == "hide") {
languagesDropDown.className = "visible";
} else {
languagesDropDown.className = "hide";
}
});
</script></span></ul></nav></div></header></div></div><div class="navPusher"><div class="docMainWrapper wrapper"><div class="container mainContainer docsContainer"><div class="wrapper"><div class="post"><header class="postHeader"><a class="edit-page-link button" href="https://github.com/apache/pulsar/edit/master/site2/docs/functions-state.md" target="_blank" rel="noreferrer noopener">Edit</a><h1 id="__docusaurus" class="postHeaderTitle">Pulsar Functions State Storage (Developer Preview)</h1></header><article><div><span><p>Since Pulsar 2.1.0 release, Pulsar integrates with Apache BookKeeper <a href="https://docs.google.com/document/d/155xAwWv5IdOitHh1NVMEwCMGgB28M3FyMiQSxEpjE-Y/edit#heading=h.56rbh52koe3f">table service</a>
for storing the <code>State</code> for functions. For example, A <code>WordCount</code> function can store its <code>counters</code> state into BookKeeper's table service via Pulsar Functions <a href="#api">State API</a>.</p>
<h2><a class="anchor" aria-hidden="true" id="api"></a><a href="#api" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>API</h2>
<h3><a class="anchor" aria-hidden="true" id="java-api"></a><a href="#java-api" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Java API</h3>
<p>Currently Pulsar Functions expose following APIs for mutating and accessing State. These APIs are available in the <a href="/docs/en/2.7.1/functions-api#context">Context</a> object when
you are using <a href="/docs/en/2.7.1/functions-api#java-sdk-functions">Java SDK</a> functions.</p>
<h4><a class="anchor" aria-hidden="true" id="incrcounter"></a><a href="#incrcounter" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>incrCounter</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Increment the builtin distributed counter referred by key
* <span class="hljs-doctag">@param</span> key The name of the key
* <span class="hljs-doctag">@param</span> amount The amount to be incremented
*/</span>
<span class="hljs-function"><span class="hljs-keyword">void</span> <span class="hljs-title">incrCounter</span><span class="hljs-params">(String key, <span class="hljs-keyword">long</span> amount)</span></span>;
</code></pre>
<p>The application can use <code>incrCounter</code> to change the counter of a given <code>key</code> by the given <code>amount</code>.</p>
<h4><a class="anchor" aria-hidden="true" id="incrcounterasync"></a><a href="#incrcounterasync" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>incrCounterAsync</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Increment the builtin distributed counter referred by key
* but dont wait for the completion of the increment operation
*
* <span class="hljs-doctag">@param</span> key The name of the key
* <span class="hljs-doctag">@param</span> amount The amount to be incremented
*/</span>
<span class="hljs-function">CompletableFuture&lt;Void&gt; <span class="hljs-title">incrCounterAsync</span><span class="hljs-params">(String key, <span class="hljs-keyword">long</span> amount)</span></span>;
</code></pre>
<p>The application can use <code>incrCounterAsync</code> to asynchronously change the counter of a given <code>key</code> by the given <code>amount</code>.</p>
<h4><a class="anchor" aria-hidden="true" id="getcounter"></a><a href="#getcounter" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>getCounter</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Retrieve the counter value for the key.
*
* <span class="hljs-doctag">@param</span> key name of the key
* <span class="hljs-doctag">@return</span> the amount of the counter value for this key
*/</span>
<span class="hljs-function"><span class="hljs-keyword">long</span> <span class="hljs-title">getCounter</span><span class="hljs-params">(String key)</span></span>;
</code></pre>
<p>The application can use <code>getCounter</code> to retrieve the counter of a given <code>key</code> mutated by <code>incrCounter</code>.</p>
<p>Besides the <code>counter</code> API, Pulsar also exposes a general key/value API for functions to store
general key/value state.</p>
<h4><a class="anchor" aria-hidden="true" id="getcounterasync"></a><a href="#getcounterasync" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>getCounterAsync</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Retrieve the counter value for the key, but don't wait
* for the operation to be completed
*
* <span class="hljs-doctag">@param</span> key name of the key
* <span class="hljs-doctag">@return</span> the amount of the counter value for this key
*/</span>
<span class="hljs-function">CompletableFuture&lt;Long&gt; <span class="hljs-title">getCounterAsync</span><span class="hljs-params">(String key)</span></span>;
</code></pre>
<p>The application can use <code>getCounterAsync</code> to asynchronously retrieve the counter of a given <code>key</code> mutated by <code>incrCounterAsync</code>.</p>
<h4><a class="anchor" aria-hidden="true" id="putstate"></a><a href="#putstate" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>putState</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Update the state value for the key.
*
* <span class="hljs-doctag">@param</span> key name of the key
* <span class="hljs-doctag">@param</span> value state value of the key
*/</span>
<span class="hljs-function"><span class="hljs-keyword">void</span> <span class="hljs-title">putState</span><span class="hljs-params">(String key, ByteBuffer value)</span></span>;
</code></pre>
<h4><a class="anchor" aria-hidden="true" id="putstateasync"></a><a href="#putstateasync" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>putStateAsync</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Update the state value for the key, but don't wait for the operation to be completed
*
* <span class="hljs-doctag">@param</span> key name of the key
* <span class="hljs-doctag">@param</span> value state value of the key
*/</span>
<span class="hljs-function">CompletableFuture&lt;Void&gt; <span class="hljs-title">putStateAsync</span><span class="hljs-params">(String key, ByteBuffer value)</span></span>;
</code></pre>
<p>The application can use <code>putStateAsync</code> to asynchronously update the state of a given <code>key</code>.</p>
<h4><a class="anchor" aria-hidden="true" id="getstate"></a><a href="#getstate" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>getState</h4>
<pre><code class="hljs"> /**
* Retrieve the <span class="hljs-keyword">state</span> value <span class="hljs-keyword">for</span> the key.
*
* @param key name of the key
* @return the <span class="hljs-keyword">state</span> value <span class="hljs-keyword">for</span> the key.
*/
ByteBuffer getState(String key);
</code></pre>
<h4><a class="anchor" aria-hidden="true" id="getstateasync"></a><a href="#getstateasync" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>getStateAsync</h4>
<pre><code class="hljs css language-java"> <span class="hljs-comment">/**
* Retrieve the state value for the key, but don't wait for the operation to be completed
*
* <span class="hljs-doctag">@param</span> key name of the key
* <span class="hljs-doctag">@return</span> the state value for the key.
*/</span>
<span class="hljs-function">CompletableFuture&lt;ByteBuffer&gt; <span class="hljs-title">getStateAsync</span><span class="hljs-params">(String key)</span></span>;
</code></pre>
<p>The application can use <code>getStateAsync</code> to asynchronously retrieve the state of a given <code>key</code>.</p>
<h3><a class="anchor" aria-hidden="true" id="python-api"></a><a href="#python-api" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Python API</h3>
<p>State currently is not supported at <a href="/docs/en/2.7.1/functions-api#python-sdk-functions">Python SDK</a>.</p>
<h2><a class="anchor" aria-hidden="true" id="query-state"></a><a href="#query-state" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Query State</h2>
<p>A Pulsar Function can use the <a href="#api">State API</a> for storing state into Pulsar's state storage
and retrieving state back from Pulsar's state storage. Additionally Pulsar also provides
CLI commands for querying its state.</p>
<pre><code class="hljs css language-shell"><span class="hljs-meta">$</span><span class="bash"> bin/pulsar-admin <span class="hljs-built_in">functions</span> querystate \</span>
--tenant &lt;tenant&gt; \
--namespace &lt;namespace&gt; \
--name &lt;function-name&gt; \
--state-storage-url &lt;bookkeeper-service-url&gt; \
--key &lt;state-key&gt; \
[---watch]
</code></pre>
<p>If <code>--watch</code> is specified, the CLI will watch the value of the provided <code>state-key</code>.</p>
<h2><a class="anchor" aria-hidden="true" id="example"></a><a href="#example" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Example</h2>
<h3><a class="anchor" aria-hidden="true" id="java-example"></a><a href="#java-example" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Java Example</h3>
<p><a href="https://github.com/apache/pulsar/tree/master//pulsar-functions/java-examples/src/main/java/org/apache/pulsar/functions/api/examples/WordCountFunction.java"><code>WordCountFunction</code></a>
is a very good example
demonstrating on how Application can easily store <code>state</code> in Pulsar Functions.</p>
<pre><code class="hljs css language-java"><span class="hljs-keyword">public</span> <span class="hljs-class"><span class="hljs-keyword">class</span> <span class="hljs-title">WordCountFunction</span> <span class="hljs-keyword">implements</span> <span class="hljs-title">Function</span>&lt;<span class="hljs-title">String</span>, <span class="hljs-title">Void</span>&gt; </span>{
<span class="hljs-meta">@Override</span>
<span class="hljs-function"><span class="hljs-keyword">public</span> Void <span class="hljs-title">process</span><span class="hljs-params">(String input, Context context)</span> <span class="hljs-keyword">throws</span> Exception </span>{
Arrays.asList(input.split(<span class="hljs-string">"\\."</span>)).forEach(word -&gt; context.incrCounter(word, <span class="hljs-number">1</span>));
<span class="hljs-keyword">return</span> <span class="hljs-keyword">null</span>;
}
}
</code></pre>
<p>The logic of this <code>WordCount</code> function is pretty simple and straightforward:</p>
<ol>
<li>The function first splits the received <code>String</code> into multiple words using regex <code>\\.</code>.</li>
<li>For each <code>word</code>, the function increments the corresponding <code>counter</code> by 1 (via <code>incrCounter(key, amount)</code>).</li>
</ol>
<h3><a class="anchor" aria-hidden="true" id="python-example"></a><a href="#python-example" aria-hidden="true" class="hash-link"><svg class="hash-link-icon" aria-hidden="true" height="16" version="1.1" viewBox="0 0 16 16" width="16"><path fill-rule="evenodd" d="M4 9h1v1H4c-1.5 0-3-1.69-3-3.5S2.55 3 4 3h4c1.45 0 3 1.69 3 3.5 0 1.41-.91 2.72-2 3.25V8.59c.58-.45 1-1.27 1-2.09C10 5.22 8.98 4 8 4H4c-.98 0-2 1.22-2 2.5S3 9 4 9zm9-3h-1v1h1c1 0 2 1.22 2 2.5S13.98 12 13 12H9c-.98 0-2-1.22-2-2.5 0-.83.42-1.64 1-2.09V6.25c-1.09.53-2 1.84-2 3.25C6 11.31 7.55 13 9 13h4c1.45 0 3-1.69 3-3.5S14.5 6 13 6z"></path></svg></a>Python Example</h3>
<p>State currently is not supported at <a href="/docs/en/2.7.1/functions-api#python-sdk-functions">Python SDK</a>.</p>
</span></div></article></div><div class="docs-prevnext"></div></div></div><nav class="onPageNav"><ul class="toc-headings"><li><a href="#api">API</a><ul class="toc-headings"><li><a href="#java-api">Java API</a></li><li><a href="#python-api">Python API</a></li></ul></li><li><a href="#query-state">Query State</a></li><li><a href="#example">Example</a><ul class="toc-headings"><li><a href="#java-example">Java Example</a></li><li><a href="#python-example">Python Example</a></li></ul></li></ul></nav></div><footer class="nav-footer" id="footer"><section class="copyright">Copyright © 2022 The Apache Software Foundation. All Rights Reserved. Apache, Apache Pulsar and the Apache feather logo are trademarks of The Apache Software Foundation.</section><span><script>
const community = document.querySelector("a[href='#community']").parentNode;
const communityMenu =
'<li>' +
'<a id="community-menu" href="#">Community <span style="font-size: 0.75em">&nbsp;▼</span></a>' +
'<div id="community-dropdown" class="hide">' +
'<ul id="community-dropdown-items">' +
'<li><a href="/en/contact">Contact</a></li>' +
'<li><a href="/en/contributing">Contributing</a></li>' +
'<li><a href="/en/coding-guide">Coding guide</a></li>' +
'<li><a href="/en/events">Events</a></li>' +
'<li><a href="https://twitter.com/Apache_Pulsar" target="_blank">Twitter &#x2750</a></li>' +
'<li><a href="https://github.com/apache/pulsar/wiki" target="_blank">Wiki &#x2750</a></li>' +
'<li><a href="https://github.com/apache/pulsar/issues" target="_blank">Issue tracking &#x2750</a></li>' +
'<li><a href="https://pulsar-summit.org/" target="_blank">Pulsar Summit &#x2750</a></li>' +
'<li>&nbsp;</li>' +
'<li><a href="/en/resources">Resources</a></li>' +
'<li><a href="/en/team">Team</a></li>' +
'<li><a href="/en/powered-by">Powered By</a></li>' +
'</ul>' +
'</div>' +
'</li>';
community.innerHTML = communityMenu;
const communityMenuItem = document.getElementById("community-menu");
const communityDropDown = document.getElementById("community-dropdown");
communityMenuItem.addEventListener("click", function(event) {
event.preventDefault();
if (communityDropDown.className == 'hide') {
communityDropDown.className = 'visible';
} else {
communityDropDown.className = 'hide';
}
});
</script></span></footer></div><script>window.twttr=(function(d,s, id){var js,fjs=d.getElementsByTagName(s)[0],t=window.twttr||{};if(d.getElementById(id))return t;js=d.createElement(s);js.id=id;js.src='https://platform.twitter.com/widgets.js';fjs.parentNode.insertBefore(js, fjs);t._e = [];t.ready = function(f) {t._e.push(f);};return t;}(document, 'script', 'twitter-wjs'));</script></body></html>