blob: b5f01fb76570ad0e020b586d7ea2d5af48a0b1d1 [file] [log] [blame]
<!DOCTYPE html>
<html lang="en">
<head>
<meta charset="UTF-8" />
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="description" content="Apache Druid">
<meta name="keywords" content="druid,kafka,database,analytics,streaming,real-time,real time,apache,open source">
<meta name="author" content="Apache Software Foundation">
<title>Druid | Avro</title>
<link rel="alternate" type="application/atom+xml" href="/feed">
<link rel="shortcut icon" href="/img/favicon.png">
<link rel="stylesheet" href="https://use.fontawesome.com/releases/v5.7.2/css/all.css" integrity="sha384-fnmOCqbTlWIlj8LyTjo7mOUStjsKC4pOpQbqyi7RrhN7udi9RwhKkMHpvLbHG9Sr" crossorigin="anonymous">
<link href='//fonts.googleapis.com/css?family=Open+Sans+Condensed:300,700,300italic|Open+Sans:300italic,400italic,600italic,400,300,600,700' rel='stylesheet' type='text/css'>
<link rel="stylesheet" href="/css/bootstrap-pure.css?v=1.1">
<link rel="stylesheet" href="/css/base.css?v=1.1">
<link rel="stylesheet" href="/css/header.css?v=1.1">
<link rel="stylesheet" href="/css/footer.css?v=1.1">
<link rel="stylesheet" href="/css/syntax.css?v=1.1">
<link rel="stylesheet" href="/css/docs.css?v=1.1">
<script>
(function() {
var cx = '000162378814775985090:molvbm0vggm';
var gcse = document.createElement('script');
gcse.type = 'text/javascript';
gcse.async = true;
gcse.src = (document.location.protocol == 'https:' ? 'https:' : 'http:') +
'//cse.google.com/cse.js?cx=' + cx;
var s = document.getElementsByTagName('script')[0];
s.parentNode.insertBefore(gcse, s);
})();
</script>
</head>
<body>
<!-- Start page_header include -->
<script src="//ajax.googleapis.com/ajax/libs/jquery/2.2.4/jquery.min.js"></script>
<div class="top-navigator">
<div class="container">
<div class="left-cont">
<a class="logo" href="/"><span class="druid-logo"></span></a>
</div>
<div class="right-cont">
<ul class="links">
<li class=""><a href="/technology">Technology</a></li>
<li class=""><a href="/use-cases">Use Cases</a></li>
<li class=""><a href="/druid-powered">Powered By</a></li>
<li class=""><a href="/docs/latest/design/">Docs</a></li>
<li class=""><a href="/community/">Community</a></li>
<li class="header-dropdown">
<a>Apache</a>
<div class="header-dropdown-menu">
<a href="https://www.apache.org/" target="_blank">Foundation</a>
<a href="https://www.apache.org/events/current-event" target="_blank">Events</a>
<a href="https://www.apache.org/licenses/" target="_blank">License</a>
<a href="https://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a>
<a href="https://www.apache.org/security/" target="_blank">Security</a>
<a href="https://www.apache.org/foundation/sponsorship.html" target="_blank">Sponsorship</a>
</div>
</li>
<li class=" button-link"><a href="/downloads.html">Download</a></li>
</ul>
</div>
</div>
<div class="action-button menu-icon">
<span class="fa fa-bars"></span> MENU
</div>
<div class="action-button menu-icon-close">
<span class="fa fa-times"></span> MENU
</div>
</div>
<script type="text/javascript">
var $menu = $('.right-cont');
var $menuIcon = $('.menu-icon');
var $menuIconClose = $('.menu-icon-close');
function showMenu() {
$menu.fadeIn(100);
$menuIcon.fadeOut(100);
$menuIconClose.fadeIn(100);
}
$menuIcon.click(showMenu);
function hideMenu() {
$menu.fadeOut(100);
$menuIconClose.fadeOut(100);
$menuIcon.fadeIn(100);
}
$menuIconClose.click(hideMenu);
$(window).resize(function() {
if ($(window).width() >= 840) {
$menu.fadeIn(100);
$menuIcon.fadeOut(100);
$menuIconClose.fadeOut(100);
}
else {
$menu.fadeOut(100);
$menuIcon.fadeIn(100);
$menuIconClose.fadeOut(100);
}
});
</script>
<!-- Stop page_header include -->
<div class="container doc-container">
<p> Looking for the <a href="/docs/0.23.0/">latest stable documentation</a>?</p>
<div class="row">
<div class="col-md-9 doc-content">
<p>
<a class="btn btn-default btn-xs visible-xs-inline-block visible-sm-inline-block" href="#toc">Table of Contents</a>
</p>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. The ASF licenses this file
~ to you under the Apache License, Version 2.0 (the
~ "License"); you may not use this file except in compliance
~ with the License. You may obtain a copy of the License at
~
~ http://www.apache.org/licenses/LICENSE-2.0
~
~ Unless required by applicable law or agreed to in writing,
~ software distributed under the License is distributed on an
~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
~ KIND, either express or implied. See the License for the
~ specific language governing permissions and limitations
~ under the License.
-->
<h1 id="avro">Avro</h1>
<p>This Apache Druid (incubating) extension enables Druid to ingest and understand the Apache Avro data format. Make sure to <a href="../../operations/including-extensions.html">include</a> <code>druid-avro-extensions</code> as an extension.</p>
<h3 id="avro-stream-parser">Avro Stream Parser</h3>
<p>This is for streaming/realtime ingestion.</p>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>avro_stream</code>.</td>
<td>no</td>
</tr>
<tr>
<td>avroBytesDecoder</td>
<td>JSON Object</td>
<td>Specifies how to decode bytes to Avro record.</td>
<td>yes</td>
</tr>
<tr>
<td>parseSpec</td>
<td>JSON Object</td>
<td>Specifies the timestamp and dimensions of the data. Should be an &quot;avro&quot; parseSpec.</td>
<td>yes</td>
</tr>
</tbody></table>
<p>An Avro parseSpec can contain a <a href="../../ingestion/flatten-json.html">flattenSpec</a> using either the &quot;root&quot; or &quot;path&quot;
field types, which can be used to read nested Avro records. The &quot;jq&quot; field type is not currently supported for Avro.</p>
<p>For example, using Avro stream parser with schema repo Avro bytes decoder:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span></span><span class="nt">&quot;parser&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;avro_stream&quot;</span><span class="p">,</span>
<span class="nt">&quot;avroBytesDecoder&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;schema_repo&quot;</span><span class="p">,</span>
<span class="nt">&quot;subjectAndIdConverter&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;avro_1124&quot;</span><span class="p">,</span>
<span class="nt">&quot;topic&quot;</span> <span class="p">:</span> <span class="s2">&quot;${YOUR_TOPIC}&quot;</span>
<span class="p">},</span>
<span class="nt">&quot;schemaRepository&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;avro_1124_rest_client&quot;</span><span class="p">,</span>
<span class="nt">&quot;url&quot;</span> <span class="p">:</span> <span class="s2">&quot;${YOUR_SCHEMA_REPO_END_POINT}&quot;</span><span class="p">,</span>
<span class="p">}</span>
<span class="p">},</span>
<span class="nt">&quot;parseSpec&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;format&quot;</span><span class="p">:</span> <span class="s2">&quot;avro&quot;</span><span class="p">,</span>
<span class="nt">&quot;timestampSpec&quot;</span><span class="p">:</span> <span class="err">&lt;s</span><span class="kc">tan</span><span class="err">dard</span> <span class="kc">t</span><span class="err">imes</span><span class="kc">ta</span><span class="err">mpSpec&gt;</span><span class="p">,</span>
<span class="nt">&quot;dimensionsSpec&quot;</span><span class="p">:</span> <span class="err">&lt;s</span><span class="kc">tan</span><span class="err">dard</span> <span class="err">dime</span><span class="kc">ns</span><span class="err">io</span><span class="kc">ns</span><span class="err">Spec&gt;</span><span class="p">,</span>
<span class="nt">&quot;flattenSpec&quot;</span><span class="p">:</span> <span class="err">&lt;op</span><span class="kc">t</span><span class="err">io</span><span class="kc">nal</span><span class="err">&gt;</span>
<span class="p">}</span>
<span class="p">}</span>
</code></pre></div>
<h4 id="avro-bytes-decoder">Avro Bytes Decoder</h4>
<p>If <code>type</code> is not included, the avroBytesDecoder defaults to <code>schema_repo</code>.</p>
<h5 id="inline-schema-based-avro-bytes-decoder">Inline Schema Based Avro Bytes Decoder</h5>
<div class="note info">
The "schema_inline" decoder reads Avro records using a fixed schema and does not support schema migration. If you
may need to migrate schemas in the future, consider one of the other decoders, all of which use a message header that
allows the parser to identify the proper Avro schema for reading records.
</div>
<p>This decoder can be used if all the input events can be read using the same schema. In that case schema can be specified in the input task json itself as described below.</p>
<div class="highlight"><pre><code class="language-text" data-lang="text"><span></span>...
&quot;avroBytesDecoder&quot;: {
&quot;type&quot;: &quot;schema_inline&quot;,
&quot;schema&quot;: {
//your schema goes here, for example
&quot;namespace&quot;: &quot;org.apache.druid.data&quot;,
&quot;name&quot;: &quot;User&quot;,
&quot;type&quot;: &quot;record&quot;,
&quot;fields&quot;: [
{ &quot;name&quot;: &quot;FullName&quot;, &quot;type&quot;: &quot;string&quot; },
{ &quot;name&quot;: &quot;Country&quot;, &quot;type&quot;: &quot;string&quot; }
]
}
}
...
</code></pre></div>
<h5 id="multiple-inline-schemas-based-avro-bytes-decoder">Multiple Inline Schemas Based Avro Bytes Decoder</h5>
<p>This decoder can be used if different input events can have different read schema. In that case schema can be specified in the input task json itself as described below.</p>
<div class="highlight"><pre><code class="language-text" data-lang="text"><span></span>...
&quot;avroBytesDecoder&quot;: {
&quot;type&quot;: &quot;multiple_schemas_inline&quot;,
&quot;schemas&quot;: {
//your id -&gt; schema map goes here, for example
&quot;1&quot;: {
&quot;namespace&quot;: &quot;org.apache.druid.data&quot;,
&quot;name&quot;: &quot;User&quot;,
&quot;type&quot;: &quot;record&quot;,
&quot;fields&quot;: [
{ &quot;name&quot;: &quot;FullName&quot;, &quot;type&quot;: &quot;string&quot; },
{ &quot;name&quot;: &quot;Country&quot;, &quot;type&quot;: &quot;string&quot; }
]
},
&quot;2&quot;: {
&quot;namespace&quot;: &quot;org.apache.druid.otherdata&quot;,
&quot;name&quot;: &quot;UserIdentity&quot;,
&quot;type&quot;: &quot;record&quot;,
&quot;fields&quot;: [
{ &quot;name&quot;: &quot;Name&quot;, &quot;type&quot;: &quot;string&quot; },
{ &quot;name&quot;: &quot;Location&quot;, &quot;type&quot;: &quot;string&quot; }
]
},
...
...
}
}
...
</code></pre></div>
<p>Note that it is essentially a map of integer schema ID to avro schema object. This parser assumes that record has following format.
first 1 byte is version and must always be 1.
next 4 bytes are integer schema ID serialized using big-endian byte order.
remaining bytes contain serialized avro message.</p>
<h5 id="schemarepo-based-avro-bytes-decoder">SchemaRepo Based Avro Bytes Decoder</h5>
<p>This Avro bytes decoder first extract <code>subject</code> and <code>id</code> from input message bytes, then use them to lookup the Avro schema with which to decode Avro record from bytes. Details can be found in <a href="https://github.com/schema-repo/schema-repo">schema repo</a> and <a href="https://issues.apache.org/jira/browse/AVRO-1124">AVRO-1124</a>. You will need an http service like schema repo to hold the avro schema. Towards schema registration on the message producer side, you can refer to <code>org.apache.druid.data.input.AvroStreamInputRowParserTest#testParse()</code>.</p>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>schema_repo</code>.</td>
<td>no</td>
</tr>
<tr>
<td>subjectAndIdConverter</td>
<td>JSON Object</td>
<td>Specifies the how to extract subject and id from message bytes.</td>
<td>yes</td>
</tr>
<tr>
<td>schemaRepository</td>
<td>JSON Object</td>
<td>Specifies the how to lookup Avro schema from subject and id.</td>
<td>yes</td>
</tr>
</tbody></table>
<h5 id="avro-1124-subject-and-id-converter">Avro-1124 Subject And Id Converter</h5>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>avro_1124</code>.</td>
<td>no</td>
</tr>
<tr>
<td>topic</td>
<td>String</td>
<td>Specifies the topic of your kafka stream.</td>
<td>yes</td>
</tr>
</tbody></table>
<h5 id="avro-1124-schema-repository">Avro-1124 Schema Repository</h5>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>avro_1124_rest_client</code>.</td>
<td>no</td>
</tr>
<tr>
<td>url</td>
<td>String</td>
<td>Specifies the endpoint url of your Avro-1124 schema repository.</td>
<td>yes</td>
</tr>
</tbody></table>
<h5 id="confluents-schema-registry">Confluent&#39;s Schema Registry</h5>
<p>This Avro bytes decoder first extract unique <code>id</code> from input message bytes, then use them it lookup in the Schema Registry for the related schema, with which to decode Avro record from bytes.
Details can be found in Schema Registry <a href="http://docs.confluent.io/current/schema-registry/docs/">documentation</a> and <a href="https://github.com/confluentinc/schema-registry">repository</a>.</p>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>schema_registry</code>.</td>
<td>no</td>
</tr>
<tr>
<td>url</td>
<td>String</td>
<td>Specifies the url endpoint of the Schema Registry.</td>
<td>yes</td>
</tr>
<tr>
<td>capacity</td>
<td>Integer</td>
<td>Specifies the max size of the cache (default == Integer.MAX_VALUE).</td>
<td>no</td>
</tr>
</tbody></table>
<h3 id="avro-hadoop-parser">Avro Hadoop Parser</h3>
<p>This is for batch ingestion using the HadoopDruidIndexer. The <code>inputFormat</code> of <code>inputSpec</code> in <code>ioConfig</code> must be set to <code>&quot;org.apache.druid.data.input.avro.AvroValueInputFormat&quot;</code>. You may want to set Avro reader&#39;s schema in <code>jobProperties</code> in <code>tuningConfig</code>, eg: <code>&quot;avro.schema.input.value.path&quot;: &quot;/path/to/your/schema.avsc&quot;</code> or <code>&quot;avro.schema.input.value&quot;: &quot;your_schema_JSON_object&quot;</code>, if reader&#39;s schema is not set, the schema in Avro object container file will be used, see <a href="http://avro.apache.org/docs/1.7.7/spec.html#Schema+Resolution">Avro specification</a>. Make sure to include &quot;org.apache.druid.extensions:druid-avro-extensions&quot; as an extension.</p>
<table><thead>
<tr>
<th>Field</th>
<th>Type</th>
<th>Description</th>
<th>Required</th>
</tr>
</thead><tbody>
<tr>
<td>type</td>
<td>String</td>
<td>This should say <code>avro_hadoop</code>.</td>
<td>no</td>
</tr>
<tr>
<td>parseSpec</td>
<td>JSON Object</td>
<td>Specifies the timestamp and dimensions of the data. Should be an &quot;avro&quot; parseSpec.</td>
<td>yes</td>
</tr>
<tr>
<td>fromPigAvroStorage</td>
<td>Boolean</td>
<td>Specifies whether the data file is stored using AvroStorage.</td>
<td>no(default == false)</td>
</tr>
</tbody></table>
<p>An Avro parseSpec can contain a <a href="../../ingestion/flatten-json.html">flattenSpec</a> using either the &quot;root&quot; or &quot;path&quot;
field types, which can be used to read nested Avro records. The &quot;jq&quot; field type is not currently supported for Avro.</p>
<p>For example, using Avro Hadoop parser with custom reader&#39;s schema file:</p>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span></span><span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;index_hadoop&quot;</span><span class="p">,</span>
<span class="nt">&quot;spec&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;dataSchema&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;dataSource&quot;</span> <span class="p">:</span> <span class="s2">&quot;&quot;</span><span class="p">,</span>
<span class="nt">&quot;parser&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;avro_hadoop&quot;</span><span class="p">,</span>
<span class="nt">&quot;parseSpec&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;format&quot;</span><span class="p">:</span> <span class="s2">&quot;avro&quot;</span><span class="p">,</span>
<span class="nt">&quot;timestampSpec&quot;</span><span class="p">:</span> <span class="err">&lt;s</span><span class="kc">tan</span><span class="err">dard</span> <span class="kc">t</span><span class="err">imes</span><span class="kc">ta</span><span class="err">mpSpec&gt;</span><span class="p">,</span>
<span class="nt">&quot;dimensionsSpec&quot;</span><span class="p">:</span> <span class="err">&lt;s</span><span class="kc">tan</span><span class="err">dard</span> <span class="err">dime</span><span class="kc">ns</span><span class="err">io</span><span class="kc">ns</span><span class="err">Spec&gt;</span><span class="p">,</span>
<span class="nt">&quot;flattenSpec&quot;</span><span class="p">:</span> <span class="err">&lt;op</span><span class="kc">t</span><span class="err">io</span><span class="kc">nal</span><span class="err">&gt;</span>
<span class="p">}</span>
<span class="p">}</span>
<span class="p">},</span>
<span class="nt">&quot;ioConfig&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;hadoop&quot;</span><span class="p">,</span>
<span class="nt">&quot;inputSpec&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;type&quot;</span> <span class="p">:</span> <span class="s2">&quot;static&quot;</span><span class="p">,</span>
<span class="nt">&quot;inputFormat&quot;</span><span class="p">:</span> <span class="s2">&quot;org.apache.druid.data.input.avro.AvroValueInputFormat&quot;</span><span class="p">,</span>
<span class="nt">&quot;paths&quot;</span> <span class="p">:</span> <span class="s2">&quot;&quot;</span>
<span class="p">}</span>
<span class="p">},</span>
<span class="nt">&quot;tuningConfig&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;jobProperties&quot;</span> <span class="p">:</span> <span class="p">{</span>
<span class="nt">&quot;avro.schema.input.value.path&quot;</span> <span class="p">:</span> <span class="s2">&quot;/path/to/my/schema.avsc&quot;</span>
<span class="p">}</span>
<span class="p">}</span>
<span class="p">}</span>
<span class="p">}</span>
</code></pre></div>
</div>
<div class="col-md-3">
<div class="searchbox">
<gcse:searchbox-only></gcse:searchbox-only>
</div>
<div id="toc" class="nav toc hidden-print">
</div>
</div>
</div>
</div>
<!-- Start page_footer include -->
<footer class="druid-footer">
<div class="container">
<div class="text-center">
<p>
<a href="/technology">Technology</a>&ensp;·&ensp;
<a href="/use-cases">Use Cases</a>&ensp;·&ensp;
<a href="/druid-powered">Powered by Druid</a>&ensp;·&ensp;
<a href="/docs/latest/">Docs</a>&ensp;·&ensp;
<a href="/community/">Community</a>&ensp;·&ensp;
<a href="/downloads.html">Download</a>&ensp;·&ensp;
<a href="/faq">FAQ</a>
</p>
</div>
<div class="text-center">
<a title="Join the user group" href="https://groups.google.com/forum/#!forum/druid-user" target="_blank"><span class="fa fa-comments"></span></a>&ensp;·&ensp;
<a title="Follow Druid" href="https://twitter.com/druidio" target="_blank"><span class="fab fa-twitter"></span></a>&ensp;·&ensp;
<a title="GitHub" href="https://github.com/apache/druid" target="_blank"><span class="fab fa-github"></span></a>
</div>
<div class="text-center license">
Copyright © 2020 <a href="https://www.apache.org/" target="_blank">Apache Software Foundation</a>.<br>
Except where otherwise noted, licensed under <a rel="license" href="http://creativecommons.org/licenses/by-sa/4.0/">CC BY-SA 4.0</a>.<br>
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>
</footer>
<script async src="https://www.googletagmanager.com/gtag/js?id=UA-131010415-1"></script>
<script>
window.dataLayer = window.dataLayer || [];
function gtag(){dataLayer.push(arguments);}
gtag('js', new Date());
gtag('config', 'UA-131010415-1');
</script>
<script>
function trackDownload(type, url) {
ga('send', 'event', 'download', type, url);
}
</script>
<script src="//code.jquery.com/jquery.min.js"></script>
<script src="//maxcdn.bootstrapcdn.com/bootstrap/3.2.0/js/bootstrap.min.js"></script>
<script src="/assets/js/druid.js"></script>
<!-- stop page_footer include -->
<script>
$(function() {
$(".toc").load("/docs/0.15.0-incubating/toc.html");
// There is no way to tell when .gsc-input will be async loaded into the page so just try to set a placeholder until it works
var tries = 0;
var timer = setInterval(function() {
tries++;
if (tries > 300) clearInterval(timer);
var searchInput = $('input.gsc-input');
if (searchInput.length) {
searchInput.attr('placeholder', 'Search');
clearInterval(timer);
}
}, 100);
});
</script>
</body>
</html>