blob: 2d47f54e0876930bdf7af7771082609a3fea09cf [file] [log] [blame]
<!DOCTYPE html>
<!--
| Generated by Apache Maven Doxia at 2017-01-25
| Rendered using Apache Maven Fluido Skin 1.3.0
-->
<html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en">
<head>
<meta charset="UTF-8" />
<meta name="viewport" content="width=device-width, initial-scale=1.0" />
<meta name="Date-Revision-yyyymmdd" content="20170125" />
<meta http-equiv="Content-Language" content="en" />
<title>AsterixDB &#x2013; Support for Data Ingestion in AsterixDB</title>
<link rel="stylesheet" href="../css/apache-maven-fluido-1.3.0.min.css" />
<link rel="stylesheet" href="../css/site.css" />
<link rel="stylesheet" href="../css/print.css" media="print" />
<script type="text/javascript" src="../js/apache-maven-fluido-1.3.0.min.js"></script>
<script>(function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
(i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
})(window,document,'script','//www.google-analytics.com/analytics.js','ga');
ga('create', 'UA-41536543-1', 'uci.edu');
ga('send', 'pageview');</script>
</head>
<body class="topBarDisabled">
<div class="container-fluid">
<div id="banner">
<div class="pull-left">
<a href=".././" id="bannerLeft">
<img src="../images/asterixlogo.png" alt="AsterixDB"/>
</a>
</div>
<div class="pull-right"> </div>
<div class="clear"><hr/></div>
</div>
<div id="breadcrumbs">
<ul class="breadcrumb">
<li id="publishDate">Last Published: 2017-01-25</li>
<li id="projectVersion" class="pull-right">Version: 0.9.0</li>
<li class="divider pull-right">|</li>
<li class="pull-right"> <a href="../index.html" title="Documentation Home">
Documentation Home</a>
</li>
</ul>
</div>
<div class="row-fluid">
<div id="leftColumn" class="span3">
<div class="well sidebar-nav">
<ul class="nav nav-list">
<li class="nav-header">Get Started - Installation</li>
<li>
<a href="../ncservice.html" title="Option 1: using NCService">
<i class="none"></i>
Option 1: using NCService</a>
</li>
<li>
<a href="../install.html" title="Option 2: using Managix">
<i class="none"></i>
Option 2: using Managix</a>
</li>
<li>
<a href="../yarn.html" title="Option 3: using YARN">
<i class="none"></i>
Option 3: using YARN</a>
</li>
<li class="nav-header">AsterixDB Primer</li>
<li>
<a href="../sqlpp/primer-sqlpp.html" title="Option 1: using SQL++">
<i class="none"></i>
Option 1: using SQL++</a>
</li>
<li>
<a href="../aql/primer.html" title="Option 2: using AQL">
<i class="none"></i>
Option 2: using AQL</a>
</li>
<li class="nav-header">Data Model</li>
<li>
<a href="../datamodel.html" title="The Asterix Data Model">
<i class="none"></i>
The Asterix Data Model</a>
</li>
<li class="nav-header">Queries - SQL++</li>
<li>
<a href="../sqlpp/manual.html" title="The SQL++ Query Language">
<i class="none"></i>
The SQL++ Query Language</a>
</li>
<li>
<a href="../sqlpp/builtins.html" title="Builtin Functions">
<i class="none"></i>
Builtin Functions</a>
</li>
<li class="nav-header">Queries - AQL</li>
<li>
<a href="../aql/manual.html" title="The Asterix Query Language (AQL)">
<i class="none"></i>
The Asterix Query Language (AQL)</a>
</li>
<li>
<a href="../aql/builtins.html" title="Builtin Functions">
<i class="none"></i>
Builtin Functions</a>
</li>
<li class="nav-header">Advanced Features</li>
<li>
<a href="../aql/similarity.html" title="Support of Similarity Queries">
<i class="none"></i>
Support of Similarity Queries</a>
</li>
<li>
<a href="../aql/fulltext.html" title="Support of Full-text Queries">
<i class="none"></i>
Support of Full-text Queries</a>
</li>
<li>
<a href="../aql/externaldata.html" title="Accessing External Data">
<i class="none"></i>
Accessing External Data</a>
</li>
<li class="active">
<a href="#"><i class="none"></i>Support for Data Ingestion</a>
</li>
<li>
<a href="../udf.html" title="User Defined Functions">
<i class="none"></i>
User Defined Functions</a>
</li>
<li>
<a href="../aql/filters.html" title="Filter-Based LSM Index Acceleration">
<i class="none"></i>
Filter-Based LSM Index Acceleration</a>
</li>
<li class="nav-header">API/SDK</li>
<li>
<a href="../api.html" title="HTTP API">
<i class="none"></i>
HTTP API</a>
</li>
</ul>
<hr class="divider" />
<div id="poweredBy">
<div class="clear"></div>
<div class="clear"></div>
<div class="clear"></div>
<a href=".././" title="AsterixDB" class="builtBy">
<img class="builtBy" alt="AsterixDB" src="../images/asterixlogo.png" />
</a>
</div>
</div>
</div>
<div id="bodyColumn" class="span9" >
<!-- ! 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>Support for Data Ingestion in AsterixDB</h1>
<div class="section">
<h2><a name="Table_of_Contents"></a><a name="atoc" id="#toc">Table of Contents</a></h2>
<ul>
<li><a href="#Introduction">Introduction</a></li>
<li><a href="#FeedAdaptors">Feed Adaptors</a></li>
<li><a href="#FeedPolicies">Feed Policies</a></li>
</ul></div>
<div class="section">
<h2><a name="Introduction">Introduction</a></h2>
<p>In this document, we describe the support for data ingestion in AsterixDB. Data feeds are a new mechanism for having continuous data arrive into a BDMS from external sources and incrementally populate a persisted dataset and associated indexes. We add a new BDMS architectural component, called a data feed, that makes a Big Data system the caretaker for functionality that used to live outside, and we show how it improves users&#x2019; lives and system performance.</p></div>
<div class="section">
<h2><a name="Feed_Adaptors"></a><a name="FeedAdaptors">Feed Adaptors</a></h2>
<p>The functionality of establishing a connection with a data source and receiving, parsing and translating its data into ADM objects (for storage inside AsterixDB) is contained in a feed adaptor. A feed adaptor is an implementation of an interface and its details are specific to a given data source. An adaptor may optionally be given parameters to configure its runtime behavior. Depending upon the data transfer protocol/APIs offered by the data source, a feed adaptor may operate in a push or a pull mode. Push mode involves just one initial request by the adaptor to the data source for setting up the connection. Once a connection is authorized, the data source &#x201c;pushes&#x201d; data to the adaptor without any subsequent requests by the adaptor. In contrast, when operating in a pull mode, the adaptor makes a separate request each time to receive data. AsterixDB currently provides built-in adaptors for several popular data sources such as Twitter, CNN, and RSS feeds. AsterixDB additionally provides a generic socket-based adaptor that can be used to ingest data that is directed at a prescribed socket.</p>
<p>In this tutorial, we shall describe building two example data ingestion pipelines that cover the popular scenario of ingesting data from (a) Twitter and (b) RSS Feed source.</p>
<div class="section">
<div class="section">
<h4><a name="Ingesting_Twitter_Stream"></a>Ingesting Twitter Stream</h4>
<p>We shall use the built-in push-based Twitter adaptor. As a pre-requisite, we must define a Tweet using the AsterixDB Data Model (ADM) and the AsterixDB Query Language (AQL). Given below are the type definition in AQL that create a Tweet datatype which is representative of a real tweet as obtained from Twitter.</p>
<div class="source">
<div class="source">
<pre> create dataverse feeds;
use dataverse feeds;
create type TwitterUser as closed {
screen_name: string,
lang: string,
friends_count: int32,
statuses_count: int32
};
create type Tweet as open {
id: int64,
user: TwitterUser
}
create dataset Tweets (Tweet)
primary key id;
</pre></div></div>
<p>We also create a dataset that we shall use to persist the tweets in AsterixDB. Next we make use of the <tt>create feed</tt> AQL statement to define our example data feed.</p>
<div class="section">
<h5><a name="Using_the_push_twitter_feed_adapter"></a>Using the &#x201c;push_twitter&#x201d; feed adapter</h5>
<p>The &#x201c;push_twitter&#x201d; adaptor requires setting up an application account with Twitter. To retrieve tweets, Twitter requires registering an application with Twitter. Registration involves providing a name and a brief description for the application. Each application has an associated OAuth authentication credential that includes OAuth keys and tokens. Accessing the Twitter API requires providing the following. 1. Consumer Key (API Key) 2. Consumer Secret (API Secret) 3. Access Token 4. Access Token Secret</p>
<p>The &#x201c;push_twitter&#x201d; adaptor takes as configuration the above mentioned parameters. End users are required to obtain the above authentication credentials prior to using the &#x201c;push_twitter&#x201d; adaptor. For further information on obtaining OAuth keys and tokens and registering an application with Twitter, please visit <a class="externalLink" href="http://apps.twitter.com">http://apps.twitter.com</a></p>
<p>Given below is an example AQL statement that creates a feed called &#x201c;TwitterFeed&#x201d; by using the &#x201c;push_twitter&#x201d; adaptor.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
create feed TwitterFeed if not exists using &quot;push_twitter&quot;
((&quot;type-name&quot;=&quot;Tweet&quot;),
(&quot;format&quot;=&quot;twitter-status&quot;),
(&quot;consumer.key&quot;=&quot;************&quot;),
(&quot;consumer.secret&quot;=&quot;**************&quot;),
(&quot;access.token&quot;=&quot;**********&quot;),
(&quot;access.token.secret&quot;=&quot;*************&quot;));
</pre></div></div>
<p>It is required that the above authentication parameters are provided valid values. Note that the <tt>create feed</tt> statement does not initiate the flow of data from Twitter into our AsterixDB instance. Instead, the <tt>create feed</tt> statement only results in registering the feed with AsterixDB. The flow of data along a feed is initiated when it is connected to a target dataset using the connect feed statement (which we shall revisit later).</p></div></div>
<div class="section">
<h4><a name="Lifecycle_of_a_Feed"></a>Lifecycle of a Feed</h4>
<p>A feed is a logical artifact that is brought to life (i.e., its data flow is initiated) only when it is connected to a dataset using the <tt>connect
feed</tt> AQL statement. Subsequent to a <tt>connect feed</tt> statement, the feed is said to be in the connected state. Multiple feeds can simultaneously be connected to a dataset such that the contents of the dataset represent the union of the connected feeds. In a supported but unlikely scenario, one feed may also be simultaneously connected to different target datasets. Note that connecting a secondary feed does not require the parent feed (or any ancestor feed) to be in the connected state; the order in which feeds are connected to their respective datasets is not important. Furthermore, additional (secondary) feeds can be added to an existing hierarchy and connected to a dataset at any time without impeding/interrupting the flow of data along a connected ancestor feed.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
connect feed TwitterFeed to dataset Tweets;
</pre></div></div>
<p>The <tt>connect feed</tt> statement above directs AsterixDB to persist the <tt>TwitterFeed</tt> feed in the <tt>Tweets</tt> dataset. If it is required (by the high-level application) to also retain the raw tweets obtained from Twitter, the end user may additionally choose to connect TwitterFeed to a different dataset.</p>
<p>Let the feed run for a minute, then run the following query to see the latest tweets that are stored into the data set.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
for $i in dataset Tweets limit 10 return $i;
</pre></div></div>
<p>The flow of data from a feed into a dataset can be terminated explicitly by use of the <tt>disconnect feed</tt> statement. Disconnecting a feed from a particular dataset does not interrupt the flow of data from the feed to any other dataset(s), nor does it impact other connected feeds in the lineage.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
disconnect feed TwitterFeed from dataset Tweets;
</pre></div></div></div>
<div class="section">
<h4><a name="Ingesting_an_RSS_Feed"></a>Ingesting an RSS Feed</h4>
<p>RSS (Rich Site Summary), originally RDF Site Summary and often called Really Simple Syndication, uses a family of standard web feed formats to publish frequently updated information: blog entries, news headlines, audio, video. An RSS document (called &#x201c;feed&#x201d;, &#x201c;web feed&#x201d;, or &#x201c;channel&#x201d;) includes full or summarized text, and metadata, like publishing date and author&#x2019;s name. RSS feeds enable publishers to syndicate data automatically.</p>
<div class="section">
<h5><a name="Using_the_rss_feed_feed_adapter"></a>Using the &#x201c;rss_feed&#x201d; feed adapter</h5>
<p>AsterixDB provides a built-in feed adaptor that allows retrieving data given a collection of RSS end point URLs. As observed in the case of ingesting tweets, it is required to model an RSS data item using AQL.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
create type Rss if not exists as open {
id: string,
title: string,
description: string,
link: string
};
create dataset RssDataset (Rss)
primary key id;
</pre></div></div>
<p>Next, we define an RSS feed using our built-in adaptor &#x201c;rss_feed&#x201d;.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
create feed my_feed using
rss_feed (
(&quot;type-name&quot;=&quot;Rss&quot;),
(&quot;format&quot;=&quot;rss&quot;),
(&quot;url&quot;=&quot;http://rss.cnn.com/rss/edition.rss&quot;)
);
</pre></div></div>
<p>In the above definition, the configuration parameter &#x201c;url&#x201d; can be a comma-separated list that reflects a collection of RSS URLs, where each URL corresponds to an RSS endpoint or a RSS feed. The &#x201c;rss_adaptor&#x201d; retrieves data from each of the specified RSS URLs (comma separated values) in parallel.</p>
<p>The following statements connect the feed into the <tt>RssDataset</tt>:</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
connect feed my_feed to dataset RssDataset;
</pre></div></div>
<p>The following statements show the latest data from the data set, and disconnect the feed from the data set.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
for $i in dataset RssDataset limit 10 return $i;
disconnect feed my_feed from dataset RssDataset;
</pre></div></div>
<p>AsterixDB also allows multiple feeds to be connected to form a cascade network to process data.</p></div></div></div></div>
<div class="section">
<h2><a name="Policies_for_Feed_Ingestion"></a><a name="FeedPolicies">Policies for Feed Ingestion</a></h2>
<p>Multiple feeds may be concurrently operational on an AsterixDB cluster, each competing for resources (CPU cycles, network bandwidth, disk IO) to maintain pace with their respective data sources. As a data management system, AsterixDB is able to manage a set of concurrent feeds and make dynamic decisions related to the allocation of resources, resolving resource bottlenecks and the handling of failures. Each feed has its own set of constraints, influenced largely by the nature of its data source and the applications that intend to consume and process the ingested data. Consider an application that intends to discover the trending topics on Twitter by analyzing tweets that are being processed. Losing a few tweets may be acceptable. In contrast, when ingesting from a data source that provides a click-stream of ad clicks, losing data would translate to a loss of revenue for an application that tracks revenue by charging advertisers per click.</p>
<p>AsterixDB allows a data feed to have an associated ingestion policy that is expressed as a collection of parameters and associated values. An ingestion policy dictates the runtime behavior of the feed in response to resource bottlenecks and failures. AsterixDB provides a list of policy parameters that help customize the system&#x2019;s runtime behavior when handling excess objects. AsterixDB provides a set of built-in policies, each constructed by setting appropriate value(s) for the policy parameter(s) from the table below.</p>
<div class="section">
<div class="section">
<h4><a name="Policy_Parameters"></a>Policy Parameters</h4>
<ul>
<li>
<p><i>excess.records.spill</i>: Set to true if objects that cannot be processed by an operator for lack of resources (referred to as excess objects hereafter) should be persisted to the local disk for deferred processing. (Default: false)</p></li>
<li>
<p><i>excess.records.discard</i>: Set to true if excess objects should be discarded. (Default: false)</p></li>
<li>
<p><i>excess.records.throttle</i>: Set to true if rate of arrival of objects is required to be reduced in an adaptive manner to prevent having any excess objects (Default: false)</p></li>
<li>
<p><i>excess.records.elastic</i>: Set to true if the system should attempt to resolve resource bottlenecks by re-structuring and/or rescheduling the feed ingestion pipeline. (Default: false)</p></li>
<li>
<p><i>recover.soft.failure</i>: Set to true if the feed must attempt to survive any runtime exception. A false value permits an early termination of a feed in such an event. (Default: true)</p></li>
<li>
<p><i>recover.soft.failure</i>: Set to true if the feed must attempt to survive a hardware failures (loss of AsterixDB node(s)). A false value permits the early termination of a feed in the event of a hardware failure (Default: false)</p></li>
</ul>
<p>Note that the end user may choose to form a custom policy. For example, it is possible in AsterixDB to create a custom policy that spills excess objects to disk and subsequently resorts to throttling if the spillage crosses a configured threshold. In all cases, the desired ingestion policy is specified as part of the <tt>connect feed</tt> statement or else the &#x201c;Basic&#x201d; policy will be chosen as the default. It is worth noting that a feed can be connected to a dataset at any time, which is independent from other related feeds in the hierarchy.</p>
<div class="source">
<div class="source">
<pre> use dataverse feeds;
connect feed TwitterFeed to dataset Tweets
using policy Basic ;
</pre></div></div></div></div></div>
</div>
</div>
</div>
<hr/>
<footer>
<div class="container-fluid">
<div class="row span12">Copyright &copy; 2017
<a href="https://www.apache.org/">The Apache Software Foundation</a>.
All Rights Reserved.
</div>
<?xml version="1.0" encoding="UTF-8"?>
<div class="row-fluid">Apache AsterixDB, AsterixDB, Apache, the Apache
feather logo, and the Apache AsterixDB project logo are either
registered trademarks or trademarks of The Apache Software
Foundation in the United States and other countries.
All other marks mentioned may be trademarks or registered
trademarks of their respective owners.</div>
</div>
</footer>
</body>
</html>