blob: 07af70c1887b83bd45ef6469a6a8d9b290f9bfcd [file] [log] [blame]
<!doctype html>
<html lang="en" dir="ltr" class="docs-wrapper plugin-docs plugin-id-default docs-version-current docs-doc-page docs-doc-id-ingestion/kafka-ingestion" data-has-hydrated="false">
<head>
<meta charset="UTF-8">
<meta name="generator" content="Docusaurus v3.7.0">
<title data-rh="true">Apache Kafka ingestion | Apache® Druid</title><meta data-rh="true" name="viewport" content="width=device-width,initial-scale=1"><meta data-rh="true" name="twitter:card" content="summary_large_image"><meta data-rh="true" property="og:image" content="https://druid.apache.org/img/druid_nav.png"><meta data-rh="true" name="twitter:image" content="https://druid.apache.org/img/druid_nav.png"><meta data-rh="true" property="og:url" content="https://druid.apache.org/docs/latest/ingestion/kafka-ingestion"><meta data-rh="true" property="og:locale" content="en"><meta data-rh="true" name="docusaurus_locale" content="en"><meta data-rh="true" name="docsearch:language" content="en"><meta data-rh="true" name="docusaurus_version" content="current"><meta data-rh="true" name="docusaurus_tag" content="docs-default-current"><meta data-rh="true" name="docsearch:version" content="current"><meta data-rh="true" name="docsearch:docusaurus_tag" content="docs-default-current"><meta data-rh="true" property="og:title" content="Apache Kafka ingestion | Apache® Druid"><meta data-rh="true" name="description" content="Overview of the Kafka indexing service for Druid. Includes example supervisor specs to help you get started."><meta data-rh="true" property="og:description" content="Overview of the Kafka indexing service for Druid. Includes example supervisor specs to help you get started."><link data-rh="true" rel="icon" href="/img/favicon.png"><link data-rh="true" rel="canonical" href="https://druid.apache.org/docs/latest/ingestion/kafka-ingestion"><link data-rh="true" rel="alternate" href="https://druid.apache.org/docs/latest/ingestion/kafka-ingestion" hreflang="en"><link data-rh="true" rel="alternate" href="https://druid.apache.org/docs/latest/ingestion/kafka-ingestion" hreflang="x-default"><link rel="stylesheet" href="/css/all.css">
<script src="/js/clipboard.min.js"></script><link rel="stylesheet" href="/assets/css/styles.f3140859.css">
<script src="/assets/js/runtime~main.5b007a36.js" defer="defer"></script>
<script src="/assets/js/main.7b126984.js" defer="defer"></script>
</head>
<body class="navigation-with-keyboard">
<script>!function(){function t(t){document.documentElement.setAttribute("data-theme",t)}var e=function(){try{return new URLSearchParams(window.location.search).get("docusaurus-theme")}catch(t){}}()||function(){try{return window.localStorage.getItem("theme")}catch(t){}}();t(null!==e?e:"light")}(),function(){try{const n=new URLSearchParams(window.location.search).entries();for(var[t,e]of n)if(t.startsWith("docusaurus-data-")){var a=t.replace("docusaurus-data-","data-");document.documentElement.setAttribute(a,e)}}catch(t){}}()</script><div id="__docusaurus"><div role="region" aria-label="Skip to main content"><a class="skipToContent_fXgn" href="#__docusaurus_skipToContent_fallback">Skip to main content</a></div><nav aria-label="Main" class="navbar navbar--fixed-top navbar--dark"><div class="navbar__inner"><div class="navbar__items"><button aria-label="Toggle navigation bar" aria-expanded="false" class="navbar__toggle clean-btn" type="button"><svg width="30" height="30" viewBox="0 0 30 30" aria-hidden="true"><path stroke="currentColor" stroke-linecap="round" stroke-miterlimit="10" stroke-width="2" d="M4 7h22M4 15h22M4 23h22"></path></svg></button><a class="navbar__brand" href="/"><div class="navbar__logo"><img src="/img/druid_nav.png" alt="Apache® Druid" class="themedComponent_mlkZ themedComponent--light_NVdE"><img src="/img/druid_nav.png" alt="Apache® Druid" class="themedComponent_mlkZ themedComponent--dark_xIcU"></div></a></div><div class="navbar__items navbar__items--right"><a class="navbar__item navbar__link" href="/technology">Technology</a><a class="navbar__item navbar__link" href="/use-cases">Use Cases</a><a class="navbar__item navbar__link" href="/druid-powered">Powered By</a><a class="navbar__item navbar__link" href="/docs/latest/design/">Docs</a><a class="navbar__item navbar__link" href="/community/">Community</a><div class="navbar__item dropdown dropdown--hoverable dropdown--right"><a href="#" aria-haspopup="true" aria-expanded="false" role="button" class="navbar__link">Apache®</a><ul class="dropdown__menu"><li><a href="https://www.apache.org/" target="_blank" rel="noopener noreferrer" class="dropdown__link">Foundation<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li><a href="https://apachecon.com/?ref=druid.apache.org" target="_blank" rel="noopener noreferrer" class="dropdown__link">Events<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li><a href="https://www.apache.org/licenses/" target="_blank" rel="noopener noreferrer" class="dropdown__link">License<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li><a href="https://www.apache.org/foundation/thanks.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Thanks<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li><a href="https://www.apache.org/security/" target="_blank" rel="noopener noreferrer" class="dropdown__link">Security<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li><li><a href="https://www.apache.org/foundation/sponsorship.html" target="_blank" rel="noopener noreferrer" class="dropdown__link">Sponsorship<svg width="12" height="12" aria-hidden="true" viewBox="0 0 24 24" class="iconExternalLink_nPIU"><path fill="currentColor" d="M21 13v10h-21v-19h12v2h-10v15h17v-8h2zm3-12h-10.988l4.035 4-6.977 7.07 2.828 2.828 6.977-7.07 4.125 4.172v-11z"></path></svg></a></li></ul></div><a class="navbar__item navbar__link" href="/downloads/">Download</a><div class="navbarSearchContainer_Bca1"><div class="navbar__search"><span aria-label="expand searchbar" role="button" class="search-icon" tabindex="0"></span><input id="search_input_react" type="search" placeholder="Loading..." aria-label="Search" class="navbar__search-input search-bar" disabled=""></div></div></div></div><div role="presentation" class="navbar-sidebar__backdrop"></div></nav><div id="__docusaurus_skipToContent_fallback" class="main-wrapper mainWrapper_z2l0"><div class="docsWrapper_hBAB"><button aria-label="Scroll back to top" class="clean-btn theme-back-to-top-button backToTopButton_sjWU" type="button"></button><div class="docRoot_UBD9"><aside class="theme-doc-sidebar-container docSidebarContainer_YfHR"><div class="sidebarViewport_aRkj"><div class="sidebar_njMd"><nav aria-label="Docs sidebar" class="menu thin-scrollbar menu_SIkG"><ul class="theme-doc-sidebar-menu menu__list"><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/latest/design/">Introduction to Apache Druid</a></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist" href="/docs/latest/tutorials/">Getting started</a><button aria-label="Expand sidebar category &#x27;Getting started&#x27;" aria-expanded="false" type="button" class="clean-btn menu__caret"></button></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" href="/docs/latest/design/architecture">Design</a></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--active" href="/docs/latest/ingestion/">Ingestion</a><button aria-label="Collapse sidebar category &#x27;Ingestion&#x27;" aria-expanded="true" type="button" class="clean-btn menu__caret"></button></div><ul style="display:block;overflow:visible;height:auto" class="menu__list"><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-2 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" tabindex="0" href="/docs/latest/ingestion/data-formats">Ingestion concepts</a></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-2 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" tabindex="0" href="/docs/latest/multi-stage-query/">SQL-based batch</a></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-2 menu__list-item"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret menu__link--active" role="button" aria-expanded="true" tabindex="0" href="/docs/latest/ingestion/streaming">Streaming</a></div><ul style="display:block;overflow:visible;height:auto" class="menu__list"><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-3 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/streaming">Streaming ingestion</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-3 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/supervisor">Supervisor</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-3 menu__list-item"><a class="menu__link menu__link--active" aria-current="page" tabindex="0" href="/docs/latest/ingestion/kafka-ingestion">Apache Kafka ingestion</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-3 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/kinesis-ingestion">Amazon Kinesis ingestion</a></li></ul></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-2 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" tabindex="0" href="/docs/latest/ingestion/native-batch">Classic batch</a></div></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-2 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/ingestion-spec">Ingestion spec reference</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-2 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/schema-design">Schema design tips</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-2 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/concurrent-append-replace">Concurrent append and replace</a></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-2 menu__list-item"><a class="menu__link" tabindex="0" href="/docs/latest/ingestion/faq">Troubleshooting FAQ</a></li></ul></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist" href="/docs/latest/data-management/">Data management</a><button aria-label="Expand sidebar category &#x27;Data management&#x27;" aria-expanded="false" type="button" class="clean-btn menu__caret"></button></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" href="/docs/latest/querying/sql">Querying</a></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist" href="/docs/latest/api-reference/">API reference</a><button aria-label="Expand sidebar category &#x27;API reference&#x27;" aria-expanded="false" type="button" class="clean-btn menu__caret"></button></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist" href="/docs/latest/configuration/">Configuration</a><button aria-label="Expand sidebar category &#x27;Configuration&#x27;" aria-expanded="false" type="button" class="clean-btn menu__caret"></button></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" href="/docs/latest/api-reference/automatic-compaction-api">Operations</a></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist" href="/docs/latest/development/overview">Development</a><button aria-label="Expand sidebar category &#x27;Development&#x27;" aria-expanded="false" type="button" class="clean-btn menu__caret"></button></div></li><li class="theme-doc-sidebar-item-category theme-doc-sidebar-item-category-level-1 menu__list-item menu__list-item--collapsed"><div class="menu__list-item-collapsible"><a class="menu__link menu__link--sublist menu__link--sublist-caret" role="button" aria-expanded="false" href="/docs/latest/release-info/release-notes">Release info</a></div></li><li class="theme-doc-sidebar-item-link theme-doc-sidebar-item-link-level-1 menu__list-item"><a class="menu__link" href="/docs/latest/misc/papers-and-talks">Papers</a></li></ul></nav></div></div></aside><main class="docMainContainer_TBSr"><div class="container padding-top--md padding-bottom--lg"><div class="row"><div class="col docItemCol_VOVn"><div class="docItemContainer_Djhp"><article><nav class="theme-doc-breadcrumbs breadcrumbsContainer_Z_bl" aria-label="Breadcrumbs"><ul class="breadcrumbs" itemscope="" itemtype="https://schema.org/BreadcrumbList"><li class="breadcrumbs__item"><a aria-label="Home page" class="breadcrumbs__link" href="/"><svg viewBox="0 0 24 24" class="breadcrumbHomeIcon_YNFT"><path d="M10 19v-5h4v5c0 .55.45 1 1 1h3c.55 0 1-.45 1-1v-7h1.7c.46 0 .68-.57.33-.87L12.67 3.6c-.38-.34-.96-.34-1.34 0l-8.36 7.53c-.34.3-.13.87.33.87H5v7c0 .55.45 1 1 1h3c.55 0 1-.45 1-1z" fill="currentColor"></path></svg></a></li><li itemscope="" itemprop="itemListElement" itemtype="https://schema.org/ListItem" class="breadcrumbs__item"><a class="breadcrumbs__link" itemprop="item" href="/docs/latest/ingestion/"><span itemprop="name">Ingestion</span></a><meta itemprop="position" content="1"></li><li class="breadcrumbs__item"><span class="breadcrumbs__link">Streaming</span><meta itemprop="position" content="2"></li><li itemscope="" itemprop="itemListElement" itemtype="https://schema.org/ListItem" class="breadcrumbs__item breadcrumbs__item--active"><span class="breadcrumbs__link" itemprop="name">Apache Kafka ingestion</span><meta itemprop="position" content="3"></li></ul></nav><div class="tocCollapsible_ETCw theme-doc-toc-mobile tocMobile_ITEo"><button type="button" class="clean-btn tocCollapsibleButton_TO0P">On this page</button></div><div class="theme-doc-markdown markdown"><header><h1>Apache Kafka ingestion</h1></header><div class="theme-admonition theme-admonition-info admonition_xJq3 alert alert--info"><div class="admonitionHeading_Gvgb"><span class="admonitionIcon_Rf37"><svg viewBox="0 0 14 16"><path fill-rule="evenodd" d="M7 2.3c3.14 0 5.7 2.56 5.7 5.7s-2.56 5.7-5.7 5.7A5.71 5.71 0 0 1 1.3 8c0-3.14 2.56-5.7 5.7-5.7zM7 1C3.14 1 0 4.14 0 8s3.14 7 7 7 7-3.14 7-7-3.14-7-7-7zm1 3H6v5h2V4zm0 6H6v2h2v-2z"></path></svg></span>info</div><div class="admonitionContent_BuS1"><p>To use the Kafka indexing service, you must be on Apache Kafka version 0.11.x or higher.
If you are using an older version, refer to the <a href="https://kafka.apache.org/documentation/#upgrade" target="_blank" rel="noopener noreferrer">Apache Kafka upgrade guide</a>.</p></div></div>
<p>When you enable the Kafka indexing service, you can configure supervisors on the Overlord to manage the creation and lifetime of Kafka indexing tasks.
Kafka indexing tasks read events using Kafka partition and offset mechanism to guarantee exactly-once ingestion. The supervisor oversees the state of the indexing tasks to coordinate handoffs, manage failures, and ensure that scalability and replication requirements are maintained.</p>
<p>This topic contains configuration information for the Kafka indexing service supervisor for Apache Druid.</p>
<h2 class="anchor anchorWithStickyNavbar_LWe7" id="setup">Setup<a href="#setup" class="hash-link" aria-label="Direct link to Setup" title="Direct link to Setup"></a></h2>
<p>To use the Kafka indexing service, you must first load the <code>druid-kafka-indexing-service</code> extension on both the Overlord and the Middle Manager. See <a href="/docs/latest/configuration/extensions">Loading extensions</a> for more information.</p>
<h2 class="anchor anchorWithStickyNavbar_LWe7" id="supervisor-spec-configuration">Supervisor spec configuration<a href="#supervisor-spec-configuration" class="hash-link" aria-label="Direct link to Supervisor spec configuration" title="Direct link to Supervisor spec configuration"></a></h2>
<p>This section outlines the configuration properties that are specific to the Apache Kafka streaming ingestion method. For configuration properties shared across all streaming ingestion methods supported by Druid, see <a href="/docs/latest/ingestion/supervisor#supervisor-spec">Supervisor spec</a>.</p>
<p>The following example shows a supervisor spec for the Kafka indexing service:</p>
<details class="details_lb9f alert alert--info details_b_Ee" data-collapsed="true"><summary>Click to view the example</summary><div><div class="collapsibleContent_i85q"><div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;spec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dataSchema&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dataSource&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;metrics-kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;timestampSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;column&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;timestamp&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;format&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;auto&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dimensionsSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dimensions&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dimensionExclusions&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;timestamp&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;metricsSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;name&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;count&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;count&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;name&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value_sum&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;fieldName&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;doubleSum&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;name&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value_min&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;fieldName&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;doubleMin&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;name&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value_max&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;fieldName&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;value&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;doubleMax&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;granularitySpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;uniform&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;segmentGranularity&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;HOUR&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;queryGranularity&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;NONE&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ioConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;topic&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;metrics&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;inputFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;json&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;consumerProperties&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;bootstrap.servers&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;localhost:9092&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskCount&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;replicas&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskDuration&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;PT1H&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;tuningConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;maxRowsPerSegment&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">5000000</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div></div></div></details>
<h3 class="anchor anchorWithStickyNavbar_LWe7" id="io-configuration">I/O configuration<a href="#io-configuration" class="hash-link" aria-label="Direct link to I/O configuration" title="Direct link to I/O configuration"></a></h3>
<p>The following table outlines the <code>ioConfig</code> configuration properties specific to Kafka.
For configuration properties shared across all streaming ingestion methods, refer to <a href="/docs/latest/ingestion/supervisor#io-configuration">Supervisor I/O configuration</a>.</p>
<table><thead><tr><th>Property</th><th>Type</th><th>Description</th><th>Required</th><th>Default</th></tr></thead><tbody><tr><td><code>topic</code></td><td>String</td><td>The Kafka topic to read from. To ingest data from multiple topic, use <code>topicPattern</code>.</td><td>Yes if <code>topicPattern</code> isn&#x27;t set.</td><td></td></tr><tr><td><code>topicPattern</code></td><td>String</td><td>Multiple Kafka topics to read from, passed as a regex pattern. See <a href="#ingest-from-multiple-topics">Ingest from multiple topics</a> for more information.</td><td>Yes if <code>topic</code> isn&#x27;t set.</td><td></td></tr><tr><td><code>consumerProperties</code></td><td>String, Object</td><td>A map of properties to pass to the Kafka consumer. See <a href="#consumer-properties">Consumer properties</a> for details.</td><td>Yes. At the minimum, you must set the <code>bootstrap.servers</code> property to establish the initial connection to the Kafka cluster.</td><td></td></tr><tr><td><code>pollTimeout</code></td><td>Long</td><td>The length of time to wait for the Kafka consumer to poll records, in milliseconds.</td><td>No</td><td>100</td></tr><tr><td><code>useEarliestOffset</code></td><td>Boolean</td><td>If a supervisor is managing a datasource for the first time, it obtains a set of starting offsets from Kafka. This flag determines whether the supervisor retrieves the earliest or latest offsets in Kafka. Under normal circumstances, subsequent tasks start from where the previous segments ended so this flag is only used on the first run.</td><td>No</td><td><code>false</code></td></tr><tr><td><code>idleConfig</code></td><td>Object</td><td>Defines how and when the Kafka supervisor can become idle. See <a href="#idle-configuration">Idle configuration</a> for more details.</td><td>No</td><td>null</td></tr></tbody></table>
<h4 class="anchor anchorWithStickyNavbar_LWe7" id="ingest-from-multiple-topics">Ingest from multiple topics<a href="#ingest-from-multiple-topics" class="hash-link" aria-label="Direct link to Ingest from multiple topics" title="Direct link to Ingest from multiple topics"></a></h4>
<div class="theme-admonition theme-admonition-info admonition_xJq3 alert alert--info"><div class="admonitionHeading_Gvgb"><span class="admonitionIcon_Rf37"><svg viewBox="0 0 14 16"><path fill-rule="evenodd" d="M7 2.3c3.14 0 5.7 2.56 5.7 5.7s-2.56 5.7-5.7 5.7A5.71 5.71 0 0 1 1.3 8c0-3.14 2.56-5.7 5.7-5.7zM7 1C3.14 1 0 4.14 0 8s3.14 7 7 7 7-3.14 7-7-3.14-7-7-7zm1 3H6v5h2V4zm0 6H6v2h2v-2z"></path></svg></span>info</div><div class="admonitionContent_BuS1"><p>If you enable multi-topic ingestion for a datasource, downgrading to a version older than
28.0.0 will cause the ingestion for that datasource to fail.</p></div></div>
<p>You can ingest data from one or multiple topics.
When ingesting data from multiple topics, Druid assigns partitions based on the hashcode of the topic name and the ID of the partition within that topic. The partition assignment might not be uniform across all the tasks. Druid assumes that partitions across individual topics have similar load. If you want to ingest from both high and low load topics in the same supervisor, it is recommended that you have a higher number of partitions for a high load topic and a lower number of partitions for a low load topic.</p>
<p>To ingest data from multiple topics, use the <code>topicPattern</code> property instead of <code>topic</code>.
You pass multiple topics as a regex pattern. For example, to ingest data from clicks and impressions, set <code>topicPattern</code> to <code>clicks|impressions</code>.
Similarly, you can use <code>metrics-.*</code> as the value for <code>topicPattern</code> if you want to ingest from all the topics that start with <code>metrics-</code>. If you add a new topic that matches the regex to the cluster, Druid automatically starts ingesting from the new topic. Topic names that match partially, such as <code>my-metrics-12</code>, are not included for ingestion.</p>
<h4 class="anchor anchorWithStickyNavbar_LWe7" id="consumer-properties">Consumer properties<a href="#consumer-properties" class="hash-link" aria-label="Direct link to Consumer properties" title="Direct link to Consumer properties"></a></h4>
<p>Consumer properties control how a supervisor reads and processes event messages from a Kafka stream. For more information about consumer configuration and advanced use cases, refer to the <a href="https://kafka.apache.org/documentation/#consumerconfigs" target="_blank" rel="noopener noreferrer">Kafka documentation</a>.</p>
<p>You must include <code>bootstrap.servers</code> in consumer properties with a list of Kafka brokers in the format <code>&lt;BROKER_1&gt;:&lt;PORT_1&gt;,&lt;BROKER_2&gt;:&lt;PORT_2&gt;,...</code>.
In some cases, you may need to retrieve consumer properties at runtime. For example, when <code>bootstrap.servers</code> is unknown or not static.</p>
<p>The <code>isolation.level</code> property in <code>consumerProperties</code> determines how Druid reads messages written transactionally.
If you use older versions of Kafka servers without transaction support or you don&#x27;t want Druid to consume only committed transactions, set <code>isolation.level</code> to <code>read_uncommitted</code>.
With <code>read_uncommitted</code>, which is the default setting, Druid reads all messages, including aborted transactional messages.
Make sure offsets are sequential, since there is no offset gap check in Druid.
For Druid to consume only committed transactional messages, set <code>isolation.level</code> to <code>read_committed</code>.</p>
<p>If your Kafka cluster enables consumer group ACLs, you can set <code>group.id</code> in <code>consumerProperties</code> to override the default auto generated group ID.</p>
<p>To enable SSL connections, you must provide passwords for <code>keystore</code>, <code>truststore</code>, and <code>key</code> confidentially. You can specify these settings in the <code>jaas.conf</code> login configuration file or in <code>consumerProperties</code> with <code>sasl.jaas.config</code>.
To protect sensitive information, use the <a href="/docs/latest/operations/dynamic-config-provider#environment-variable-dynamic-config-provider">environment variable dynamic config provider</a> to store credentials in system environment variables instead of plain text.
Although you can also use the <a href="/docs/latest/operations/password-provider">password provider</a> interface to specify SSL configuration for Kafka ingestion, consider using the dynamic config provider as this feature is deprecated.</p>
<p>For example, when using SASL and SSL with Kafka, set the following environment variables for the Druid user on machines running the Overlord and Peon services. Replace the values to match your environment configurations.</p>
<div class="codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-text codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token plain">export KAFKA_JAAS_CONFIG=&quot;org.apache.kafka.common.security.plain.PlainLoginModule required username=&#x27;accesskey&#x27; password=&#x27;secret key&#x27;;&quot;</span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain">export SSL_KEY_PASSWORD=mysecretkeypassword</span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain">export SSL_KEYSTORE_PASSWORD=mysecretkeystorepassword</span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain">export SSL_TRUSTSTORE_PASSWORD=mysecrettruststorepassword</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<p>When you define the consumer properties in the supervisor spec, use the dynamic config provider to refer to the environment variables:</p>
<div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token property">&quot;consumerProperties&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;bootstrap.servers&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;localhost:9092&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;security.protocol&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;SASL_SSL&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"> </span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;sasl.mechanism&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;PLAIN&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"> </span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ssl.keystore.location&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;/opt/kafka/config/kafka01.keystore.jks&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ssl.truststore.location&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;/opt/kafka/config/kafka.truststore.jks&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;druid.dynamic.config.provider&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;environment&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;variables&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;sasl.jaas.config&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;KAFKA_JAAS_CONFIG&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ssl.key.password&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;SSL_KEY_PASSWORD&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ssl.keystore.password&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;SSL_KEYSTORE_PASSWORD&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ssl.truststore.password&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;SSL_TRUSTSTORE_PASSWORD&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<p>When connecting to Kafka, Druid replaces the environment variables with their corresponding values.</p>
<h4 class="anchor anchorWithStickyNavbar_LWe7" id="idle-configuration">Idle configuration<a href="#idle-configuration" class="hash-link" aria-label="Direct link to Idle configuration" title="Direct link to Idle configuration"></a></h4>
<div class="theme-admonition theme-admonition-info admonition_xJq3 alert alert--info"><div class="admonitionHeading_Gvgb"><span class="admonitionIcon_Rf37"><svg viewBox="0 0 14 16"><path fill-rule="evenodd" d="M7 2.3c3.14 0 5.7 2.56 5.7 5.7s-2.56 5.7-5.7 5.7A5.71 5.71 0 0 1 1.3 8c0-3.14 2.56-5.7 5.7-5.7zM7 1C3.14 1 0 4.14 0 8s3.14 7 7 7 7-3.14 7-7-3.14-7-7-7zm1 3H6v5h2V4zm0 6H6v2h2v-2z"></path></svg></span>info</div><div class="admonitionContent_BuS1"><p>Idle state transitioning is currently designated as experimental.</p></div></div>
<p>When the supervisor enters the idle state, no new tasks are launched subsequent to the completion of the currently executing tasks. This strategy may lead to reduced costs for cluster operators while using topics that get sporadic data.</p>
<p>The following table outlines the configuration options for <code>idleConfig</code>:</p>
<table><thead><tr><th>Property</th><th>Description</th><th>Required</th></tr></thead><tbody><tr><td><code>enabled</code></td><td>If <code>true</code>, the supervisor becomes idle if there is no data on input stream or topic for some time.</td><td>No</td></tr><tr><td><code>inactiveAfterMillis</code></td><td>The supervisor becomes idle if all existing data has been read from input topic and no new data has been published for <code>inactiveAfterMillis</code> milliseconds.</td><td>No</td></tr></tbody></table>
<p>The following example shows a supervisor spec with idle configuration enabled:</p>
<details class="details_lb9f alert alert--info details_b_Ee" data-collapsed="true"><summary>Click to view the example</summary><div><div class="collapsibleContent_i85q"><div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;spec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dataSchema&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain">...</span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ioConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;topic&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;metrics&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;inputFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;json&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;consumerProperties&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;bootstrap.servers&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;localhost:9092&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;autoScalerConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;enableTaskAutoScaler&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">true</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskCountMax&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">6</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskCountMin&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">2</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;minTriggerScaleActionFrequencyMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">600000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;autoScalerStrategy&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;lagBased&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;lagCollectionIntervalMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">30000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;lagCollectionRangeMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">600000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleOutThreshold&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">6000000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;triggerScaleOutFractionThreshold&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">0.3</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleInThreshold&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1000000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;triggerScaleInFractionThreshold&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">0.9</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleActionStartDelayMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">300000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleActionPeriodMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">60000</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleInStep&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;scaleOutStep&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">2</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskCount&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;replicas&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;taskDuration&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;PT1H&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;idleConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;enabled&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">true</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;inactiveAfterMillis&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">600000</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;tuningConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain">...</span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div></div></div></details>
<h4 class="anchor anchorWithStickyNavbar_LWe7" id="data-format">Data format<a href="#data-format" class="hash-link" aria-label="Direct link to Data format" title="Direct link to Data format"></a></h4>
<p>The Kafka indexing service supports both <a href="/docs/latest/ingestion/data-formats#input-format"><code>inputFormat</code></a> and <a href="/docs/latest/ingestion/data-formats#parser"><code>parser</code></a> to specify the data format. Use the <code>inputFormat</code> to specify the data format for the Kafka indexing service unless you need a format only supported by the legacy <code>parser</code>. For more information, see <a href="/docs/latest/ingestion/data-formats">Source input formats</a>.</p>
<p>The Kinesis indexing service supports the following values for <code>inputFormat</code>:</p>
<ul>
<li><code>csv</code></li>
<li><code>tvs</code></li>
<li><code>json</code></li>
<li><code>kafka</code></li>
<li><code>avro_stream</code></li>
<li><code>protobuf</code></li>
</ul>
<p>You can use <code>parser</code> to read <a href="/docs/latest/development/extensions-contrib/thrift"><code>thrift</code></a> formats.</p>
<h5 class="anchor anchorWithStickyNavbar_LWe7" id="kafka-input-format-supervisor-spec-example">Kafka input format supervisor spec example<a href="#kafka-input-format-supervisor-spec-example" class="hash-link" aria-label="Direct link to Kafka input format supervisor spec example" title="Direct link to Kafka input format supervisor spec example"></a></h5>
<p>The <code>kafka</code> input format lets you parse the Kafka metadata fields in addition to the Kafka payload value contents.</p>
<p>The <code>kafka</code> input format wraps around the payload parsing input format and augments the data it outputs with the Kafka event timestamp, the Kafka topic name, the Kafka event headers, and the key field that itself can be parsed using any available input format.</p>
<p>For example, consider the following structure for a Kafka message that represents a wiki edit in a development environment:</p>
<ul>
<li><strong>Kafka timestamp</strong>: <code>1680795276351</code></li>
<li><strong>Kafka topic</strong>: <code>wiki-edits</code></li>
<li><strong>Kafka headers</strong>:<!-- -->
<ul>
<li><code>env=development</code></li>
<li><code>zone=z1</code></li>
</ul>
</li>
<li><strong>Kafka key</strong>: <code>wiki-edit</code></li>
<li><strong>Kafka payload value</strong>: <code>{&quot;channel&quot;:&quot;#sv.wikipedia&quot;,&quot;timestamp&quot;:&quot;2016-06-27T00:00:11.080Z&quot;,&quot;page&quot;:&quot;Salo Toraut&quot;,&quot;delta&quot;:31,&quot;namespace&quot;:&quot;Main&quot;}</code></li>
</ul>
<p>Using <code>{ &quot;type&quot;: &quot;json&quot; }</code> as the input format only parses the payload value.
To parse the Kafka metadata in addition to the payload, use the <code>kafka</code> input format.</p>
<p>You configure it as follows:</p>
<ul>
<li><code>valueFormat</code>: Define how to parse the payload value. Set this to the payload parsing input format (<code>{ &quot;type&quot;: &quot;json&quot; }</code>).</li>
<li><code>timestampColumnName</code>: Supply a custom name for the Kafka timestamp in the Druid schema to avoid conflicts with columns from the payload. The default is <code>kafka.timestamp</code>.</li>
<li><code>topicColumnName</code>: Supply a custom name for the Kafka topic in the Druid schema to avoid conflicts with columns from the payload. The default is <code>kafka.topic</code>. This field is useful when ingesting data from multiple topics into the same datasource.</li>
<li><code>headerFormat</code>: The default value <code>string</code> decodes strings in UTF-8 encoding from the Kafka header.<br>
<!-- -->Other supported encoding formats include the following:<!-- -->
<ul>
<li><code>ISO-8859-1</code>: ISO Latin Alphabet No. 1, that is, ISO-LATIN-1.</li>
<li><code>US-ASCII</code>: Seven-bit ASCII. Also known as ISO646-US. The Basic Latin block of the Unicode character set.</li>
<li><code>UTF-16</code>: Sixteen-bit UCS Transformation Format, byte order identified by an optional byte-order mark.</li>
<li><code>UTF-16BE</code>: Sixteen-bit UCS Transformation Format, big-endian byte order.</li>
<li><code>UTF-16LE</code>: Sixteen-bit UCS Transformation Format, little-endian byte order.</li>
</ul>
</li>
<li><code>headerColumnPrefix</code>: Supply a prefix to the Kafka headers to avoid any conflicts with columns from the payload. The default is <code>kafka.header.</code>.
Considering the header from the example, Druid maps the headers to the following columns: <code>kafka.header.env</code>, <code>kafka.header.zone</code>.</li>
<li><code>keyFormat</code>: Supply an input format to parse the key. Only the first value is used.
If, as in the example, your key values are simple strings, then you can use the <code>tsv</code> format to parse them.<!-- -->
<div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;tsv&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;findColumnsFromHeader&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">false</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;columns&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token string" style="color:rgb(195, 232, 141)">&quot;x&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"> </span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<!-- -->Note that for <code>tsv</code>,<code>csv</code>, and <code>regex</code> formats, you need to provide a <code>columns</code> array to make a valid input format. Only the first one is used, and its name will be ignored in favor of <code>keyColumnName</code>.</li>
<li><code>keyColumnName</code>: Supply the name for the Kafka key column to avoid conflicts with columns from the payload. The default is <code>kafka.key</code>.</li>
</ul>
<p>The following input format uses default values for <code>timestampColumnName</code>, <code>topicColumnName</code>, <code>headerColumnPrefix</code>, and <code>keyColumnName</code>:</p>
<div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;valueFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;json&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;headerFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;string&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;keyFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;tsv&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;findColumnsFromHeader&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">false</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;columns&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token string" style="color:rgb(195, 232, 141)">&quot;x&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<p>It parses the example message as follows:</p>
<div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;channel&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;#sv.wikipedia&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;timestamp&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;2016-06-27T00:00:11.080Z&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;page&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;Salo Toraut&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;delta&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">31</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;namespace&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;Main&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;kafka.timestamp&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token number" style="color:rgb(247, 140, 108)">1680795276351</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;kafka.topic&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;wiki-edits&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;kafka.header.env&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;development&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;kafka.header.zone&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;z1&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;kafka.key&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;wiki-edit&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<p>Finally, add these Kafka metadata columns to the <code>dimensionsSpec</code> or set your <code>dimensionsSpec</code> to auto-detect columns.</p>
<p>The following supervisor spec demonstrates how to ingest the Kafka header, key, timestamp, and topic into Druid dimensions:</p>
<details class="details_lb9f alert alert--info details_b_Ee" data-collapsed="true"><summary>Click to view the example</summary><div><div class="collapsibleContent_i85q"><div class="language-json codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-json codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;spec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;ioConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;consumerProperties&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;bootstrap.servers&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;localhost:9092&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;topic&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;wiki-edits&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;inputFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;valueFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;json&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;headerFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;string&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;keyFormat&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;tsv&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;findColumnsFromHeader&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">false</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;columns&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">[</span><span class="token string" style="color:rgb(195, 232, 141)">&quot;x&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">]</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;useEarliestOffset&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">true</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dataSchema&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dataSource&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;wikiticker&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;timestampSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;column&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;timestamp&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;format&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;posix&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;dimensionsSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token property">&quot;dimensionsSpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;useSchemaDiscovery&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">true</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;includeAllDimensions&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">true</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;granularitySpec&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;queryGranularity&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;none&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;rollup&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token boolean" style="color:rgb(255, 88, 116)">false</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;segmentGranularity&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;day&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;tuningConfig&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">{</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token property">&quot;type&quot;</span><span class="token operator" style="color:rgb(137, 221, 255)">:</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token punctuation" style="color:rgb(199, 146, 234)">}</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div></div></div></details>
<p>After Druid ingests the data, you can query the Kafka metadata columns as follows:</p>
<div class="language-sql codeBlockContainer_Ckt0 theme-code-block" style="--prism-color:#bfc7d5;--prism-background-color:#292d3e"><div class="codeBlockContent_biex"><pre tabindex="0" class="prism-code language-sql codeBlock_bY9V thin-scrollbar" style="color:#bfc7d5;background-color:#292d3e"><code class="codeBlockLines_e6Vv"><span class="token-line" style="color:#bfc7d5"><span class="token keyword" style="font-style:italic">SELECT</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka.header.env&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka.key&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka.timestamp&quot;</span><span class="token punctuation" style="color:rgb(199, 146, 234)">,</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;kafka.topic&quot;</span><span class="token plain"></span><br></span><span class="token-line" style="color:#bfc7d5"><span class="token plain"></span><span class="token keyword" style="font-style:italic">FROM</span><span class="token plain"> </span><span class="token string" style="color:rgb(195, 232, 141)">&quot;wikiticker&quot;</span><br></span></code></pre><div class="buttonGroup__atx"><button type="button" aria-label="Copy code to clipboard" title="Copy" class="clean-btn"><span class="copyButtonIcons_eSgA" aria-hidden="true"><svg viewBox="0 0 24 24" class="copyButtonIcon_y97N"><path fill="currentColor" d="M19,21H8V7H19M19,5H8A2,2 0 0,0 6,7V21A2,2 0 0,0 8,23H19A2,2 0 0,0 21,21V7A2,2 0 0,0 19,5M16,1H4A2,2 0 0,0 2,3V17H4V3H16V1Z"></path></svg><svg viewBox="0 0 24 24" class="copyButtonSuccessIcon_LjdS"><path fill="currentColor" d="M21,7L9,19L3.5,13.5L4.91,12.09L9,16.17L19.59,5.59L21,7Z"></path></svg></span></button></div></div></div>
<p>This query returns:</p>
<table><thead><tr><th><code>kafka.header.env</code></th><th><code>kafka.key</code></th><th><code>kafka.timestamp</code></th><th><code>kafka.topic</code></th></tr></thead><tbody><tr><td><code>development</code></td><td><code>wiki-edit</code></td><td><code>1680795276351</code></td><td><code>wiki-edits</code></td></tr></tbody></table>
<h3 class="anchor anchorWithStickyNavbar_LWe7" id="tuning-configuration">Tuning configuration<a href="#tuning-configuration" class="hash-link" aria-label="Direct link to Tuning configuration" title="Direct link to Tuning configuration"></a></h3>
<p>The following table outlines the <code>tuningConfig</code> configuration properties specific to Kafka.
For configuration properties shared across all streaming ingestion methods, refer to <a href="/docs/latest/ingestion/supervisor#tuning-configuration">Supervisor tuning configuration</a>.</p>
<table><thead><tr><th>Property</th><th>Type</th><th>Description</th><th>Required</th><th>Default</th></tr></thead><tbody><tr><td><code>numPersistThreads</code></td><td>Integer</td><td>The number of threads to use to create and persist incremental segments on the disk. Higher ingestion data throughput results in a larger number of incremental segments, causing significant CPU time to be spent on the creation of the incremental segments on the disk. For datasources with number of columns running into hundreds or thousands, creation of the incremental segments may take up significant time, in the order of multiple seconds. In both of these scenarios, ingestion can stall or pause frequently, causing it to fall behind. You can use additional threads to parallelize the segment creation without blocking ingestion as long as there are sufficient CPU resources available.</td><td>No</td><td>1</td></tr></tbody></table>
<h2 class="anchor anchorWithStickyNavbar_LWe7" id="deployment-notes-on-kafka-partitions-and-druid-segments">Deployment notes on Kafka partitions and Druid segments<a href="#deployment-notes-on-kafka-partitions-and-druid-segments" class="hash-link" aria-label="Direct link to Deployment notes on Kafka partitions and Druid segments" title="Direct link to Deployment notes on Kafka partitions and Druid segments"></a></h2>
<p>Druid assigns Kafka partitions to each Kafka indexing task. A task writes the events it consumes from Kafka into a single segment for the segment granularity interval until it reaches one of the following limits: <code>maxRowsPerSegment</code>, <code>maxTotalRows</code>, or <code>intermediateHandoffPeriod</code>. At this point, the task creates a new partition for this segment granularity to contain subsequent events.</p>
<p>The Kafka indexing task also does incremental hand-offs. Therefore, segments become available as they are ready and you don&#x27;t have to wait for all segments until the end of the task duration. When the task reaches one of <code>maxRowsPerSegment</code>, <code>maxTotalRows</code>, or <code>intermediateHandoffPeriod</code>, it hands off all the segments and creates a new set of segments for further events. This allows the task to run for longer durations without accumulating old segments locally on Middle Manager services.</p>
<p>The Kafka indexing service may still produce some small segments. For example, consider the following scenario:</p>
<ul>
<li>Task duration is 4 hours.</li>
<li>Segment granularity is set to an HOUR.</li>
<li>The supervisor was started at 9:10.
After 4 hours at 13:10, Druid starts a new set of tasks. The events for the interval 13:00 - 14:00 may be split across existing tasks and the new set of tasks which could result in small segments. To merge them together into new segments of an ideal size (in the range of ~500-700 MB per segment), you can schedule re-indexing tasks, optionally with a different segment granularity.</li>
</ul>
<p>For information on how to optimize the segment size, see <a href="/docs/latest/operations/segment-optimization">Segment size optimization</a>.</p>
<h2 class="anchor anchorWithStickyNavbar_LWe7" id="learn-more">Learn more<a href="#learn-more" class="hash-link" aria-label="Direct link to Learn more" title="Direct link to Learn more"></a></h2>
<p>See the following topics for more information:</p>
<ul>
<li><a href="/docs/latest/api-reference/supervisor-api">Supervisor API</a> for how to manage and monitor supervisors using the API.</li>
<li><a href="/docs/latest/ingestion/supervisor">Supervisor</a> for supervisor status and capacity planning.</li>
<li><a href="/docs/latest/tutorials/tutorial-kafka">Loading from Apache Kafka</a> for a tutorial on streaming data from Apache Kafka.</li>
<li><a href="/docs/latest/ingestion/data-formats#kafka">Kafka input format</a> to learn about the <code>kafka</code> input format.</li>
</ul></div></article><nav class="pagination-nav docusaurus-mt-lg" aria-label="Docs pages"><a class="pagination-nav__link pagination-nav__link--prev" href="/docs/latest/ingestion/supervisor"><div class="pagination-nav__sublabel">Previous</div><div class="pagination-nav__label">Supervisor</div></a><a class="pagination-nav__link pagination-nav__link--next" href="/docs/latest/ingestion/kinesis-ingestion"><div class="pagination-nav__sublabel">Next</div><div class="pagination-nav__label">Amazon Kinesis ingestion</div></a></nav></div></div><div class="col col--3"><div class="tableOfContents_bqdL thin-scrollbar theme-doc-toc-desktop"><ul class="table-of-contents table-of-contents__left-border"><li><a href="#setup" class="table-of-contents__link toc-highlight">Setup</a></li><li><a href="#supervisor-spec-configuration" class="table-of-contents__link toc-highlight">Supervisor spec configuration</a><ul><li><a href="#io-configuration" class="table-of-contents__link toc-highlight">I/O configuration</a></li><li><a href="#tuning-configuration" class="table-of-contents__link toc-highlight">Tuning configuration</a></li></ul></li><li><a href="#deployment-notes-on-kafka-partitions-and-druid-segments" class="table-of-contents__link toc-highlight">Deployment notes on Kafka partitions and Druid segments</a></li><li><a href="#learn-more" class="table-of-contents__link toc-highlight">Learn more</a></li></ul></div></div></div></div></main></div></div></div><footer class="footer"><div class="container container-fluid"><div class="footer__bottom text--center"><div class="margin-bottom--sm"><img src="/img/favicon.png" class="footer__logo themedComponent_mlkZ themedComponent--light_NVdE"><img src="/img/favicon.png" class="footer__logo themedComponent_mlkZ themedComponent--dark_xIcU"></div><div class="footer__copyright">Copyright © 2025 Apache Software Foundation. Except where otherwise noted, licensed under CC BY-SA 4.0. Apache Druid, Druid, and the Druid logo are either registered trademarks or trademarks of The Apache Software Foundation in the United States and other countries.</div></div></div></footer></div>
</body>
</html>