SlideShare a Scribd company logo
1© Cloudera, Inc. All rights reserved.
13 April 2016
Ted Malaska| Principle Solutions Architect @ Cloudera,
Jonathan Hsieh| HBase Tech Lead @ Cloudera, Apache HBase PMC
Apache HBase + Spark:
Leveraging your Non-Relational
Datastore in Batch and
Streaming applications
2© Cloudera, Inc. All rights reserved.
About Ted and Jon
Ted Malaska
• Principal Solutions Architect
@ Cloudera
• Apache HBase SparkOnHBase
Contributor
• Contact
• ted.malaska@cloudera.com
Jon Hsieh
• Tech Lead/Eng Manager
HBase Team @ Cloudera
• Apache HBase PMC
• Apache Flume founder
• Contact
• jon@cloudera.com
• @jmhsieh
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
3© Cloudera, Inc. All rights reserved.
Outline
• Introduction
• Architecture and integration patterns
• Typing and API usage examples
• Future work and Conclusion
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
4© Cloudera, Inc. All rights reserved.
• Apache HBase is a distributed non-
relational datastore that specializes in
strongly consistent, low-latency,
random access reads, writes, and
short scans. As a storage system, it is
an obvious source for reading RDDs
and a destination for writing RDDs
• Apache Spark is a distributed in-
memory processing system that can
be used for batch and continuous,
near-real time streaming
jobs. Spark’s programming model is
built upon the RDD (resilient
distributed dataset) abstraction
Apache HBase + Apache Spark
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
5© Cloudera, Inc. All rights reserved.
Example Use cases
• Streaming Analytics into HBase to replace Lambda Architectures (with
Kafka)
• Weblogs
• ETL in Spark to bulkload into HBase
• 25-50B records per weekly batch
• Using SQL for extraction layer to query HBase entity-centric timeseries data
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
6© Cloudera, Inc. All rights reserved.
Architecture and Integration
Patterns
7© Cloudera, Inc. All rights reserved.
How does data get in and out of HBase?
HBase Client
Put, Incr, Append
HBase Client
Get, Scan
Bulk Import
HBase Client
HBase ReplicationHBase Replication
low latency
high throughput
Gets
Short scan
Full Scan, Snapshot,
MapReduce
HBase Scanner
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
8© Cloudera, Inc. All rights reserved.
HBase + MapReduce: Batch processing patterns
• Read dataset from HBase Table
• Use HBase’s MR InputFormats
• TableInputFormat
• MultiTableInputFormat
• TableSnapshotInputFormat
• Write dataset to HBase Table
• Use HBase’s MR OutputFormat
• TableOutputFormat
• MultiTableOutputFormat
• HFileOutputFormat
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Read from HBase Table
Write to HBase Table
9© Cloudera, Inc. All rights reserved.
HBase + Spark: Batch processing patterns
• Read dataset(RDD) from HBase Table
• Use HBase’s MR InputFormats
• TableInputFormat
• MultiTableInputFormat
• TableSnapshotInputFormat
• Write dataset(RDD) to HBase Table
• Use HBase’s MR OutputFormat
• TableOutputFormat
• MultiTableOutputFormat
• HFileOutputFormat
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Read HBase Table as RDD
Write RDD as HBase Table
10© Cloudera, Inc. All rights reserved.
Spark Streaming
• Take an Data source
• Partition in to mini batches RDDs
• Compute using Spark engine
• Output mini batch RDDs
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Mini batch input RDD
Data source
Mini batch output RDD
11© Cloudera, Inc. All rights reserved.
HBase + Spark Streaming – Enriching With HBase Data
• “Join” a dataset with HBase data
• Enrich Streaming data source with
HBase data
• Extract information from minibatch
• Read/write/update HBase data in
processing
• Output HBase-data enriched stream
of output RDDs
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Mini batch input RDD
Data source
HBase-enriched mini batch output RDD
12© Cloudera, Inc. All rights reserved.
How does Spark get data in and out of HBase?
HBase Client
Put, Incr, Append
HBase Client
Get, Scan
Bulk Import
HBase Client
HBase ReplicationHBase Replication
low latency
high throughput
Gets
Short scan
Full Scan, Snapshot,
MapReduce
HBase Scanner
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
13© Cloudera, Inc. All rights reserved.
How does Spark get data in and out of HBase?
HBase Client
Put, Incr, Append
HBase Client
Get, Scan
Bulk Import
HBase Client
HBase ReplicationHBase Replication
low latency
high throughput
Gets
Short scan
Full Scan, Snapshot,
MapReduce
HBase Scanner
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Batch RDD via HBase’s MR
Input/ Output Formats
Streaming using Hbase to
Enrich stream data
Streaming using HBase to
Enrich stream data
14© Cloudera, Inc. All rights reserved.
Typing and API Usage
15© Cloudera, Inc. All rights reserved.
Under the covers
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Driver
Walker Node
Configs
Executor
Static Space
Configs
HConnection
Tasks Tasks
Walker Node
Executor
Static Space
Configs
HConnection
Tasks Tasks
16© Cloudera, Inc. All rights reserved.
Key Addition: HBaseContext
• Create an HBaseContext
// an Hadoop/HBase Configuration object
val conf = HBaseConfiguration.create()
conf.addResource(new Path("/etc/hbase/conf/core-site.xml"))
conf.addResource(new Path("/etc/hbase/conf/hbase-site.xml"))
// sc is the Spark Context; hbase context corresponds to an HBase Connection
val hbaseContext = new HBaseContext(sc, conf)
// A sample RDD
val rdd = sc.parallelize(Array(
(Bytes.toBytes("1")), (Bytes.toBytes("2")),
(Bytes.toBytes("3")), (Bytes.toBytes("4")),
(Bytes.toBytes("5")), (Bytes.toBytes("6")),
(Bytes.toBytes("7"))))
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
17© Cloudera, Inc. All rights reserved.
• Foreach
• Map
• BulkLoad
• BulkLoadThinRows
• BulkGet (aka Multiget)
• BulkDelete
Operations on the HBaseContext
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
18© Cloudera, Inc. All rights reserved.
Foreach
• Read HBase data in parallel for each partition and compute
rdd.hbaseForeachPartition(hbaseContext, (it, conn) => {
// do something
val bufferedMutator = conn.getBufferedMutator(
TableName.valueOf("t1"))
it.foreach(r => {
... // HBase API put/incr/append/cas calls
}
bufferedMutator.flush()
bufferedMutator.close()
})
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
19© Cloudera, Inc. All rights reserved.
Map
• Take an HBase dataset and map it in parallel for each partition to produce a new
RDD
val getRdd = rdd.hbaseMapPartitions(hbaseContext, (it, conn) => {
val table = conn.getTable(TableName.valueOf("t1"))
var res = mutable.MutableList[String]()
it.map( r => {
... // HBase API Scan Results
}
})
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
20© Cloudera, Inc. All rights reserved.
BulkLoad
• Bulk load a data set into Hbase (for all cases, generally wide tables)
rdd.hbaseBulkLoad (tableName, t => {
Seq((new KeyFamilyQualifier(t.rowKey, t.family,
t.qualifier), t.value)).iterator
},
stagingFolder)
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
21© Cloudera, Inc. All rights reserved.
BulkLoadThinRows
• Bulk load a data set into HBase (for skinny tables, <10k cols)
hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte],
Array[Byte])])] (rdd, TableName.valueOf(tableName), t => {
val rowKey = Bytes.toBytes(t._1)
val familyQualifiersValues = new FamiliesQualifiersValues
t._2.foreach(f => {
val family:Array[Byte] = f._1
val qualifier = f._2
val value:Array[Byte] = f._3
familyQualifiersValues +=(family, qualifier, value)
})
(new ByteArrayWrapper(rowKey), familyQualifiersValues)
}, stagingFolder.getPath)
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
22© Cloudera, Inc. All rights reserved.
Scan vs Bulk Get (Parallel HBase Multigets)
Scan HBase Table Bulk Get HBase Table
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
23© Cloudera, Inc. All rights reserved.
BulkPut
• Parallelized HBase Multiput
hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte],
Array[Byte])])](rdd, tableName, (putRecord) => {
val put = new Put(putRecord._1)
putRecord._2.foreach((putValue) =>
put.add(putValue._1, putValue._2, putValue._3))
put
}
}
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
24© Cloudera, Inc. All rights reserved.
BulkDelete
• Parallelized HBase Multi-deletes
hbaseContext.bulkDelete[Array[Byte]](rdd, tableName,
putRecord => new Delete(putRecord),
4) // batch size
rdd.hbaseBulkDelete(hbaseContext, tableName,
putRecord => new Delete(putRecord),
4) // batch size
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
25© Cloudera, Inc. All rights reserved.
SparkSQL
• Using SparkSQL to query HBase Data
// Setup Schema Mapping
val dataframe = sqlContext.load("org.apache.hadoop.hbase.spark",
Map("hbase.columns.mapping" -> "KEY_FIELD STRING :key, A_FIELD STRING c:a,
B_FIELD STRING c:b,", "hbase.table" -> "t1"))
dataframe.registerTempTable("hbaseTmp")
// Query
sqlContext.sql("SELECT KEY_FIELD FROM hbaseTmp " +
"WHERE " + "(KEY_FIELD = 'get1' and B_FIELD < '3') or " +
"(KEY_FIELD <= 'get3' and B_FIELD = '8')")
.foreach(r => println(" - "+r))
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
26© Cloudera, Inc. All rights reserved.
SparkSQL + MLLib
• Process data extracted from SparkSQL
val resultDf = sqlContext.sql("SELECT gamer_id, oks, games_won, games_played
FROM gamer")
// Parse data to apply typing information
val parsedData = resultDf.map(r => {
val array = Array(r.getInt(1).toDouble, r.getInt(2).toDouble,
r.getInt(3).toDouble)
Vectors.dense(array) })
val dataCount = parsedData.count()
if (dataCount > 0) {
val clusters = KMeans.train(parsedData, 3, 5)
clusters.clusterCenters.foreach(v => println(" Vector Center:" + v))
}
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
27© Cloudera, Inc. All rights reserved.
Future work and Conclusion
28© Cloudera, Inc. All rights reserved.
Development and Distribution Status
• Today
• Batch Analysis patterns with existing MR Input/Output Formats
• Streaming Analysis Patterns
• Committed to HBase trunk branch (2.0) as part of HBase project
• Available in CDH5.7.0 with commercial support
• Used in production and pre-production today at ~10 Cloudera customers
• Recent Additions
• Kerberos and Secure HBase access
• To come: Kerberos ticket renewals for Spark Streaming
• New JSON based HBase table schema specification
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
29© Cloudera, Inc. All rights reserved.
How does Spark get data in and out of HBase?
HBase Client
Put, Incr, Append
HBase Client
Get, Scan
Bulk Import
HBase Client
HBase ReplicationHBase Replication
low latency
high throughput
Gets
Short scan
Full Scan,
MapReduce
HBase Scanner
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
Batch RDD via HBase’s MR
Input/ Output Formats
Streaming using Hbase to
Enrich stream data
Streaming using Hbase to
Enrich stream data
HBase Data as Spark
Streaming data source
30© Cloudera, Inc. All rights reserved.
Future: HBase Data as a Source
• HBase edits as a Spark streaming data
source (with Kafka?)
• Gather other data
• Do some computation
• Write the data out
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
HBase
Replication
Mini batch input RDD
Data source
31© Cloudera, Inc. All rights reserved.
Thank you!
32© Cloudera, Inc. All rights reserved.
Use Case – Streaming Counting
Hsieh and Malaska, Hadoop Summit EU
• Puts vs Increments
• Bulk Puts/Gets is good
• You can get perfect counting
4/13/2016
33© Cloudera, Inc. All rights reserved.
DStream
DStream
DStream
Spark Streaming
Single Pass
Source Receiver RDD
Source Receiver RDD
RDD
Filter Count HBase Increments
Source Receiver RDD
RDD
RDD
Single Pass
Filter Count HBase Increments
First
Batch
Second
Batch
Hsieh and Malaska, Hadoop Summit EU Dublin 2016
34© Cloudera, Inc. All rights reserved.
DStream
DStream
DStream
Single Pass
Source Receiver RDD
Source Receiver RDD
RDD
Filter Count
HBase Puts
Source Receiver
RDD
partitions
RDD
Parition
RDD
Single Pass
Filter Count
Pre-first
Batch
First
Batch
Second
Batch
Stateful RDD 1
HBase Puts
Stateful RDD 2
Stateful RDD 1
Spark Streaming
Hsieh and Malaska, Hadoop Summit EU Dublin 2016

More Related Content

What's hot

Kafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platformKafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platform
Jean-Paul Azar
 
Hadoop Meetup Jan 2019 - Overview of Ozone
Hadoop Meetup Jan 2019 - Overview of OzoneHadoop Meetup Jan 2019 - Overview of Ozone
Hadoop Meetup Jan 2019 - Overview of Ozone
Erik Krogen
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
DataWorks Summit
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
Cloudera, Inc.
 
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
Daniel Hochman
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
StreamNative
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
DataWorks Summit
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
Alluxio, Inc.
 
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DMUpgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
Yahoo!デベロッパーネットワーク
 
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation BuffersHBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
Cloudera, Inc.
 
HBase Blockcache 101
HBase Blockcache 101HBase Blockcache 101
HBase Blockcache 101
Nick Dimiduk
 
PostgreSQL HA
PostgreSQL   HAPostgreSQL   HA
PostgreSQL HA
haroonm
 
(2014년) Active Active 데이터센터
(2014년) Active Active 데이터센터(2014년) Active Active 데이터센터
(2014년) Active Active 데이터센터
Gasida Seo
 
LLAP: long-lived execution in Hive
LLAP: long-lived execution in HiveLLAP: long-lived execution in Hive
LLAP: long-lived execution in Hive
DataWorks Summit
 
Optimizing Hive Queries
Optimizing Hive QueriesOptimizing Hive Queries
Optimizing Hive Queries
DataWorks Summit
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Adam Doyle
 
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
Brian Brazil
 
Apache HBase™
Apache HBase™Apache HBase™
Apache HBase™
Prashant Gupta
 
HBase
HBaseHBase
Hadoop security
Hadoop securityHadoop security
Hadoop security
Shivaji Dutta
 

What's hot (20)

Kafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platformKafka Tutorial - basics of the Kafka streaming platform
Kafka Tutorial - basics of the Kafka streaming platform
 
Hadoop Meetup Jan 2019 - Overview of Ozone
Hadoop Meetup Jan 2019 - Overview of OzoneHadoop Meetup Jan 2019 - Overview of Ozone
Hadoop Meetup Jan 2019 - Overview of Ozone
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
Geospatial Indexing at Scale: The 15 Million QPS Redis Architecture Powering ...
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
 
Hive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep DiveHive + Tez: A Performance Deep Dive
Hive + Tez: A Performance Deep Dive
 
Apache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic DatasetsApache Iceberg - A Table Format for Hige Analytic Datasets
Apache Iceberg - A Table Format for Hige Analytic Datasets
 
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DMUpgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
Upgrading HDFS to 3.3.0 and deploying RBF in production #LINE_DM
 
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation BuffersHBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
HBase HUG Presentation: Avoiding Full GCs with MemStore-Local Allocation Buffers
 
HBase Blockcache 101
HBase Blockcache 101HBase Blockcache 101
HBase Blockcache 101
 
PostgreSQL HA
PostgreSQL   HAPostgreSQL   HA
PostgreSQL HA
 
(2014년) Active Active 데이터센터
(2014년) Active Active 데이터센터(2014년) Active Active 데이터센터
(2014년) Active Active 데이터센터
 
LLAP: long-lived execution in Hive
LLAP: long-lived execution in HiveLLAP: long-lived execution in Hive
LLAP: long-lived execution in Hive
 
Optimizing Hive Queries
Optimizing Hive QueriesOptimizing Hive Queries
Optimizing Hive Queries
 
Apache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEAApache Iceberg Presentation for the St. Louis Big Data IDEA
Apache Iceberg Presentation for the St. Louis Big Data IDEA
 
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
Monitoring Hadoop with Prometheus (Hadoop User Group Ireland, December 2015)
 
Apache HBase™
Apache HBase™Apache HBase™
Apache HBase™
 
HBase
HBaseHBase
HBase
 
Hadoop security
Hadoop securityHadoop security
Hadoop security
 

Viewers also liked

HBaseConEast2016: HBase and Spark, State of the Art
HBaseConEast2016: HBase and Spark, State of the ArtHBaseConEast2016: HBase and Spark, State of the Art
HBaseConEast2016: HBase and Spark, State of the Art
Michael Stack
 
Apache HBase Internals you hoped you Never Needed to Understand
Apache HBase Internals you hoped you Never Needed to UnderstandApache HBase Internals you hoped you Never Needed to Understand
Apache HBase Internals you hoped you Never Needed to Understand
Josh Elser
 
Apache Spark streaming and HBase
Apache Spark streaming and HBaseApache Spark streaming and HBase
Apache Spark streaming and HBase
Carol McDonald
 
HBaseとSparkでセンサーデータを有効活用 #hbasejp
HBaseとSparkでセンサーデータを有効活用 #hbasejpHBaseとSparkでセンサーデータを有効活用 #hbasejp
HBaseとSparkでセンサーデータを有効活用 #hbasejp
FwardNetwork
 
Apache HBase 入門 (第2回)
Apache HBase 入門 (第2回)Apache HBase 入門 (第2回)
Apache HBase 入門 (第2回)
tatsuya6502
 
Spark + HBase
Spark + HBase Spark + HBase
Free Code Friday - Spark Streaming with HBase
Free Code Friday - Spark Streaming with HBaseFree Code Friday - Spark Streaming with HBase
Free Code Friday - Spark Streaming with HBase
MapR Technologies
 
Apache HBase 入門 (第1回)
Apache HBase 入門 (第1回)Apache HBase 入門 (第1回)
Apache HBase 入門 (第1回)
tatsuya6502
 

Viewers also liked (8)

HBaseConEast2016: HBase and Spark, State of the Art
HBaseConEast2016: HBase and Spark, State of the ArtHBaseConEast2016: HBase and Spark, State of the Art
HBaseConEast2016: HBase and Spark, State of the Art
 
Apache HBase Internals you hoped you Never Needed to Understand
Apache HBase Internals you hoped you Never Needed to UnderstandApache HBase Internals you hoped you Never Needed to Understand
Apache HBase Internals you hoped you Never Needed to Understand
 
Apache Spark streaming and HBase
Apache Spark streaming and HBaseApache Spark streaming and HBase
Apache Spark streaming and HBase
 
HBaseとSparkでセンサーデータを有効活用 #hbasejp
HBaseとSparkでセンサーデータを有効活用 #hbasejpHBaseとSparkでセンサーデータを有効活用 #hbasejp
HBaseとSparkでセンサーデータを有効活用 #hbasejp
 
Apache HBase 入門 (第2回)
Apache HBase 入門 (第2回)Apache HBase 入門 (第2回)
Apache HBase 入門 (第2回)
 
Spark + HBase
Spark + HBase Spark + HBase
Spark + HBase
 
Free Code Friday - Spark Streaming with HBase
Free Code Friday - Spark Streaming with HBaseFree Code Friday - Spark Streaming with HBase
Free Code Friday - Spark Streaming with HBase
 
Apache HBase 入門 (第1回)
Apache HBase 入門 (第1回)Apache HBase 入門 (第1回)
Apache HBase 入門 (第1回)
 

Similar to Apache HBase + Spark: Leveraging your Non-Relational Datastore in Batch and Streaming applications

Big Data Journey
Big Data JourneyBig Data Journey
Big Data Journey
Tugdual Grall
 
Mar 2012 HUG: Hive with HBase
Mar 2012 HUG: Hive with HBaseMar 2012 HUG: Hive with HBase
Mar 2012 HUG: Hive with HBase
Yahoo Developer Network
 
HBaseCon 2013: Integration of Apache Hive and HBase
HBaseCon 2013: Integration of Apache Hive and HBaseHBaseCon 2013: Integration of Apache Hive and HBase
HBaseCon 2013: Integration of Apache Hive and HBase
Cloudera, Inc.
 
The Future of Hbase
The Future of HbaseThe Future of Hbase
The Future of Hbase
Salesforce Engineering
 
מיכאל
מיכאלמיכאל
מיכאל
sqlserver.co.il
 
Hadoop demo ppt
Hadoop demo pptHadoop demo ppt
Hadoop demo ppt
Phil Young
 
Apache Spark on Apache HBase: Current and Future
Apache Spark on Apache HBase: Current and Future Apache Spark on Apache HBase: Current and Future
Apache Spark on Apache HBase: Current and Future
HBaseCon
 
Multi-tenant, Multi-cluster and Multi-container Apache HBase Deployments
Multi-tenant, Multi-cluster and Multi-container Apache HBase DeploymentsMulti-tenant, Multi-cluster and Multi-container Apache HBase Deployments
Multi-tenant, Multi-cluster and Multi-container Apache HBase Deployments
DataWorks Summit
 
Accelerating Hadoop, Spark, and Memcached with HPC Technologies
Accelerating Hadoop, Spark, and Memcached with HPC TechnologiesAccelerating Hadoop, Spark, and Memcached with HPC Technologies
Accelerating Hadoop, Spark, and Memcached with HPC Technologies
inside-BigData.com
 
Apache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingApache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data Processing
DataWorks Summit
 
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Data Con LA
 
Integration of HIve and HBase
Integration of HIve and HBaseIntegration of HIve and HBase
Integration of HIve and HBase
Hortonworks
 
Integration of Hive and HBase
Integration of Hive and HBaseIntegration of Hive and HBase
Integration of Hive and HBase
Hortonworks
 
Big Data Hoopla Simplified - TDWI Memphis 2014
Big Data Hoopla Simplified - TDWI Memphis 2014Big Data Hoopla Simplified - TDWI Memphis 2014
Big Data Hoopla Simplified - TDWI Memphis 2014
Rajan Kanitkar
 
Big Data and Hadoop Components
Big Data and Hadoop ComponentsBig Data and Hadoop Components
Big Data and Hadoop Components
DezyreAcademy
 
Leveraging SAP HANA with Apache Hadoop and SAP Analytics
Leveraging SAP HANA with Apache Hadoop and SAP AnalyticsLeveraging SAP HANA with Apache Hadoop and SAP Analytics
Leveraging SAP HANA with Apache Hadoop and SAP Analytics
Method360
 
Srikanth hadoop 3.6yrs_hyd
Srikanth hadoop 3.6yrs_hydSrikanth hadoop 3.6yrs_hyd
Srikanth hadoop 3.6yrs_hyd
srikanth K
 
What are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
What are Hadoop Components? Hadoop Ecosystem and Architecture | EdurekaWhat are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
What are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
Edureka!
 
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
NashvilleTechCouncil
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 Release
Nick Dimiduk
 

Similar to Apache HBase + Spark: Leveraging your Non-Relational Datastore in Batch and Streaming applications (20)

Big Data Journey
Big Data JourneyBig Data Journey
Big Data Journey
 
Mar 2012 HUG: Hive with HBase
Mar 2012 HUG: Hive with HBaseMar 2012 HUG: Hive with HBase
Mar 2012 HUG: Hive with HBase
 
HBaseCon 2013: Integration of Apache Hive and HBase
HBaseCon 2013: Integration of Apache Hive and HBaseHBaseCon 2013: Integration of Apache Hive and HBase
HBaseCon 2013: Integration of Apache Hive and HBase
 
The Future of Hbase
The Future of HbaseThe Future of Hbase
The Future of Hbase
 
מיכאל
מיכאלמיכאל
מיכאל
 
Hadoop demo ppt
Hadoop demo pptHadoop demo ppt
Hadoop demo ppt
 
Apache Spark on Apache HBase: Current and Future
Apache Spark on Apache HBase: Current and Future Apache Spark on Apache HBase: Current and Future
Apache Spark on Apache HBase: Current and Future
 
Multi-tenant, Multi-cluster and Multi-container Apache HBase Deployments
Multi-tenant, Multi-cluster and Multi-container Apache HBase DeploymentsMulti-tenant, Multi-cluster and Multi-container Apache HBase Deployments
Multi-tenant, Multi-cluster and Multi-container Apache HBase Deployments
 
Accelerating Hadoop, Spark, and Memcached with HPC Technologies
Accelerating Hadoop, Spark, and Memcached with HPC TechnologiesAccelerating Hadoop, Spark, and Memcached with HPC Technologies
Accelerating Hadoop, Spark, and Memcached with HPC Technologies
 
Apache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingApache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data Processing
 
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
Big Data Day LA 2015 - What's new and next in Apache Tez by Bikas Saha of Hor...
 
Integration of HIve and HBase
Integration of HIve and HBaseIntegration of HIve and HBase
Integration of HIve and HBase
 
Integration of Hive and HBase
Integration of Hive and HBaseIntegration of Hive and HBase
Integration of Hive and HBase
 
Big Data Hoopla Simplified - TDWI Memphis 2014
Big Data Hoopla Simplified - TDWI Memphis 2014Big Data Hoopla Simplified - TDWI Memphis 2014
Big Data Hoopla Simplified - TDWI Memphis 2014
 
Big Data and Hadoop Components
Big Data and Hadoop ComponentsBig Data and Hadoop Components
Big Data and Hadoop Components
 
Leveraging SAP HANA with Apache Hadoop and SAP Analytics
Leveraging SAP HANA with Apache Hadoop and SAP AnalyticsLeveraging SAP HANA with Apache Hadoop and SAP Analytics
Leveraging SAP HANA with Apache Hadoop and SAP Analytics
 
Srikanth hadoop 3.6yrs_hyd
Srikanth hadoop 3.6yrs_hydSrikanth hadoop 3.6yrs_hyd
Srikanth hadoop 3.6yrs_hyd
 
What are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
What are Hadoop Components? Hadoop Ecosystem and Architecture | EdurekaWhat are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
What are Hadoop Components? Hadoop Ecosystem and Architecture | Edureka
 
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
The Nuts and Bolts of Hadoop and it's Ever-changing Ecosystem, Presented by J...
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 Release
 

More from DataWorks Summit/Hadoop Summit

Running Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in ProductionRunning Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in Production
DataWorks Summit/Hadoop Summit
 
State of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache ZeppelinState of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache Zeppelin
DataWorks Summit/Hadoop Summit
 
Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache Ranger
DataWorks Summit/Hadoop Summit
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science Platform
DataWorks Summit/Hadoop Summit
 
Revolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and ZeppelinRevolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and Zeppelin
DataWorks Summit/Hadoop Summit
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSense
DataWorks Summit/Hadoop Summit
 
Hadoop Crash Course
Hadoop Crash CourseHadoop Crash Course
Hadoop Crash Course
DataWorks Summit/Hadoop Summit
 
Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
DataWorks Summit/Hadoop Summit
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
DataWorks Summit/Hadoop Summit
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
DataWorks Summit/Hadoop Summit
 
Schema Registry - Set you Data Free
Schema Registry - Set you Data FreeSchema Registry - Set you Data Free
Schema Registry - Set you Data Free
DataWorks Summit/Hadoop Summit
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
DataWorks Summit/Hadoop Summit
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
DataWorks Summit/Hadoop Summit
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and ML
DataWorks Summit/Hadoop Summit
 
How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient
DataWorks Summit/Hadoop Summit
 
HBase in Practice
HBase in Practice HBase in Practice
HBase in Practice
DataWorks Summit/Hadoop Summit
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)
DataWorks Summit/Hadoop Summit
 
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS HadoopBreaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
DataWorks Summit/Hadoop Summit
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
DataWorks Summit/Hadoop Summit
 
Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop
DataWorks Summit/Hadoop Summit
 

More from DataWorks Summit/Hadoop Summit (20)

Running Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in ProductionRunning Apache Spark & Apache Zeppelin in Production
Running Apache Spark & Apache Zeppelin in Production
 
State of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache ZeppelinState of Security: Apache Spark & Apache Zeppelin
State of Security: Apache Spark & Apache Zeppelin
 
Unleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache RangerUnleashing the Power of Apache Atlas with Apache Ranger
Unleashing the Power of Apache Atlas with Apache Ranger
 
Enabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science PlatformEnabling Digital Diagnostics with a Data Science Platform
Enabling Digital Diagnostics with a Data Science Platform
 
Revolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and ZeppelinRevolutionize Text Mining with Spark and Zeppelin
Revolutionize Text Mining with Spark and Zeppelin
 
Double Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSenseDouble Your Hadoop Performance with Hortonworks SmartSense
Double Your Hadoop Performance with Hortonworks SmartSense
 
Hadoop Crash Course
Hadoop Crash CourseHadoop Crash Course
Hadoop Crash Course
 
Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Apache Spark Crash Course
Apache Spark Crash CourseApache Spark Crash Course
Apache Spark Crash Course
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
 
Schema Registry - Set you Data Free
Schema Registry - Set you Data FreeSchema Registry - Set you Data Free
Schema Registry - Set you Data Free
 
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
Building a Large-Scale, Adaptive Recommendation Engine with Apache Flink and ...
 
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
Real-Time Anomaly Detection using LSTM Auto-Encoders with Deep Learning4J on ...
 
Mool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and MLMool - Automated Log Analysis using Data Science and ML
Mool - Automated Log Analysis using Data Science and ML
 
How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient How Hadoop Makes the Natixis Pack More Efficient
How Hadoop Makes the Natixis Pack More Efficient
 
HBase in Practice
HBase in Practice HBase in Practice
HBase in Practice
 
The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)The Challenge of Driving Business Value from the Analytics of Things (AOT)
The Challenge of Driving Business Value from the Analytics of Things (AOT)
 
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS HadoopBreaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
Breaking the 1 Million OPS/SEC Barrier in HOPS Hadoop
 
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
From Regulatory Process Verification to Predictive Maintenance and Beyond wit...
 
Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop Backup and Disaster Recovery in Hadoop
Backup and Disaster Recovery in Hadoop
 

Recently uploaded

Retrieval Augmented Generation Evaluation with Ragas
Retrieval Augmented Generation Evaluation with RagasRetrieval Augmented Generation Evaluation with Ragas
Retrieval Augmented Generation Evaluation with Ragas
Zilliz
 
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
Fwdays
 
DefCamp_2016_Chemerkin_Yury_--_publish.pdf
DefCamp_2016_Chemerkin_Yury_--_publish.pdfDefCamp_2016_Chemerkin_Yury_--_publish.pdf
DefCamp_2016_Chemerkin_Yury_--_publish.pdf
Yury Chemerkin
 
FIDO Munich Seminar Workforce Authentication Case Study.pptx
FIDO Munich Seminar Workforce Authentication Case Study.pptxFIDO Munich Seminar Workforce Authentication Case Study.pptx
FIDO Munich Seminar Workforce Authentication Case Study.pptx
FIDO Alliance
 
Mule Experience Hub and Release Channel with Java 17
Mule Experience Hub and Release Channel with Java 17Mule Experience Hub and Release Channel with Java 17
Mule Experience Hub and Release Channel with Java 17
Bhajan Mehta
 
Camunda Chapter NY Meetup July 2024.pptx
Camunda Chapter NY Meetup July 2024.pptxCamunda Chapter NY Meetup July 2024.pptx
Camunda Chapter NY Meetup July 2024.pptx
ZachWylie3
 
NVIDIA at Breakthrough Discuss for Space Exploration
NVIDIA at Breakthrough Discuss for Space ExplorationNVIDIA at Breakthrough Discuss for Space Exploration
NVIDIA at Breakthrough Discuss for Space Exploration
Alison B. Lowndes
 
Perth MuleSoft Meetup July 2024
Perth MuleSoft Meetup July 2024Perth MuleSoft Meetup July 2024
Perth MuleSoft Meetup July 2024
Michael Price
 
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan..."Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
Fwdays
 
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
Snarky Security
 
FIDO Munich Seminar In-Vehicle Payment Trends.pptx
FIDO Munich Seminar In-Vehicle Payment Trends.pptxFIDO Munich Seminar In-Vehicle Payment Trends.pptx
FIDO Munich Seminar In-Vehicle Payment Trends.pptx
FIDO Alliance
 
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptxFIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
FIDO Alliance
 
Increase Quality with User Access Policies - July 2024
Increase Quality with User Access Policies - July 2024Increase Quality with User Access Policies - July 2024
Increase Quality with User Access Policies - July 2024
Peter Caitens
 
FIDO Munich Seminar FIDO Automotive Apps.pptx
FIDO Munich Seminar FIDO Automotive Apps.pptxFIDO Munich Seminar FIDO Automotive Apps.pptx
FIDO Munich Seminar FIDO Automotive Apps.pptx
FIDO Alliance
 
Zaitechno Handheld Raman Spectrometer.pdf
Zaitechno Handheld Raman Spectrometer.pdfZaitechno Handheld Raman Spectrometer.pdf
Zaitechno Handheld Raman Spectrometer.pdf
AmandaCheung15
 
The Path to General-Purpose Robots - Coatue
The Path to General-Purpose Robots - CoatueThe Path to General-Purpose Robots - Coatue
The Path to General-Purpose Robots - Coatue
Razin Mustafiz
 
The History of Embeddings & Multimodal Embeddings
The History of Embeddings & Multimodal EmbeddingsThe History of Embeddings & Multimodal Embeddings
The History of Embeddings & Multimodal Embeddings
Zilliz
 
FIDO Munich Seminar: Securing Smart Car.pptx
FIDO Munich Seminar: Securing Smart Car.pptxFIDO Munich Seminar: Securing Smart Car.pptx
FIDO Munich Seminar: Securing Smart Car.pptx
FIDO Alliance
 
Top 12 AI Technology Trends For 2024.pdf
Top 12 AI Technology Trends For 2024.pdfTop 12 AI Technology Trends For 2024.pdf
Top 12 AI Technology Trends For 2024.pdf
Marrie Morris
 
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
OnBoard
 

Recently uploaded (20)

Retrieval Augmented Generation Evaluation with Ragas
Retrieval Augmented Generation Evaluation with RagasRetrieval Augmented Generation Evaluation with Ragas
Retrieval Augmented Generation Evaluation with Ragas
 
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
"Hands-on development experience using wasm Blazor", Furdak Vladyslav.pptx
 
DefCamp_2016_Chemerkin_Yury_--_publish.pdf
DefCamp_2016_Chemerkin_Yury_--_publish.pdfDefCamp_2016_Chemerkin_Yury_--_publish.pdf
DefCamp_2016_Chemerkin_Yury_--_publish.pdf
 
FIDO Munich Seminar Workforce Authentication Case Study.pptx
FIDO Munich Seminar Workforce Authentication Case Study.pptxFIDO Munich Seminar Workforce Authentication Case Study.pptx
FIDO Munich Seminar Workforce Authentication Case Study.pptx
 
Mule Experience Hub and Release Channel with Java 17
Mule Experience Hub and Release Channel with Java 17Mule Experience Hub and Release Channel with Java 17
Mule Experience Hub and Release Channel with Java 17
 
Camunda Chapter NY Meetup July 2024.pptx
Camunda Chapter NY Meetup July 2024.pptxCamunda Chapter NY Meetup July 2024.pptx
Camunda Chapter NY Meetup July 2024.pptx
 
NVIDIA at Breakthrough Discuss for Space Exploration
NVIDIA at Breakthrough Discuss for Space ExplorationNVIDIA at Breakthrough Discuss for Space Exploration
NVIDIA at Breakthrough Discuss for Space Exploration
 
Perth MuleSoft Meetup July 2024
Perth MuleSoft Meetup July 2024Perth MuleSoft Meetup July 2024
Perth MuleSoft Meetup July 2024
 
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan..."Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
"Building Future-Ready Apps with .NET 8 and Azure Serverless Ecosystem", Stan...
 
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
Welcome to Cyberbiosecurity. Because regular cybersecurity wasn't complicated...
 
FIDO Munich Seminar In-Vehicle Payment Trends.pptx
FIDO Munich Seminar In-Vehicle Payment Trends.pptxFIDO Munich Seminar In-Vehicle Payment Trends.pptx
FIDO Munich Seminar In-Vehicle Payment Trends.pptx
 
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptxFIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
FIDO Munich Seminar Blueprint for In-Vehicle Payment Standard.pptx
 
Increase Quality with User Access Policies - July 2024
Increase Quality with User Access Policies - July 2024Increase Quality with User Access Policies - July 2024
Increase Quality with User Access Policies - July 2024
 
FIDO Munich Seminar FIDO Automotive Apps.pptx
FIDO Munich Seminar FIDO Automotive Apps.pptxFIDO Munich Seminar FIDO Automotive Apps.pptx
FIDO Munich Seminar FIDO Automotive Apps.pptx
 
Zaitechno Handheld Raman Spectrometer.pdf
Zaitechno Handheld Raman Spectrometer.pdfZaitechno Handheld Raman Spectrometer.pdf
Zaitechno Handheld Raman Spectrometer.pdf
 
The Path to General-Purpose Robots - Coatue
The Path to General-Purpose Robots - CoatueThe Path to General-Purpose Robots - Coatue
The Path to General-Purpose Robots - Coatue
 
The History of Embeddings & Multimodal Embeddings
The History of Embeddings & Multimodal EmbeddingsThe History of Embeddings & Multimodal Embeddings
The History of Embeddings & Multimodal Embeddings
 
FIDO Munich Seminar: Securing Smart Car.pptx
FIDO Munich Seminar: Securing Smart Car.pptxFIDO Munich Seminar: Securing Smart Car.pptx
FIDO Munich Seminar: Securing Smart Car.pptx
 
Top 12 AI Technology Trends For 2024.pdf
Top 12 AI Technology Trends For 2024.pdfTop 12 AI Technology Trends For 2024.pdf
Top 12 AI Technology Trends For 2024.pdf
 
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
Mastering Board Best Practices: Essential Skills for Effective Non-profit Lea...
 

Apache HBase + Spark: Leveraging your Non-Relational Datastore in Batch and Streaming applications

  • 1. 1© Cloudera, Inc. All rights reserved. 13 April 2016 Ted Malaska| Principle Solutions Architect @ Cloudera, Jonathan Hsieh| HBase Tech Lead @ Cloudera, Apache HBase PMC Apache HBase + Spark: Leveraging your Non-Relational Datastore in Batch and Streaming applications
  • 2. 2© Cloudera, Inc. All rights reserved. About Ted and Jon Ted Malaska • Principal Solutions Architect @ Cloudera • Apache HBase SparkOnHBase Contributor • Contact • ted.malaska@cloudera.com Jon Hsieh • Tech Lead/Eng Manager HBase Team @ Cloudera • Apache HBase PMC • Apache Flume founder • Contact • jon@cloudera.com • @jmhsieh Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 3. 3© Cloudera, Inc. All rights reserved. Outline • Introduction • Architecture and integration patterns • Typing and API usage examples • Future work and Conclusion Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 4. 4© Cloudera, Inc. All rights reserved. • Apache HBase is a distributed non- relational datastore that specializes in strongly consistent, low-latency, random access reads, writes, and short scans. As a storage system, it is an obvious source for reading RDDs and a destination for writing RDDs • Apache Spark is a distributed in- memory processing system that can be used for batch and continuous, near-real time streaming jobs. Spark’s programming model is built upon the RDD (resilient distributed dataset) abstraction Apache HBase + Apache Spark Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 5. 5© Cloudera, Inc. All rights reserved. Example Use cases • Streaming Analytics into HBase to replace Lambda Architectures (with Kafka) • Weblogs • ETL in Spark to bulkload into HBase • 25-50B records per weekly batch • Using SQL for extraction layer to query HBase entity-centric timeseries data Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 6. 6© Cloudera, Inc. All rights reserved. Architecture and Integration Patterns
  • 7. 7© Cloudera, Inc. All rights reserved. How does data get in and out of HBase? HBase Client Put, Incr, Append HBase Client Get, Scan Bulk Import HBase Client HBase ReplicationHBase Replication low latency high throughput Gets Short scan Full Scan, Snapshot, MapReduce HBase Scanner Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 8. 8© Cloudera, Inc. All rights reserved. HBase + MapReduce: Batch processing patterns • Read dataset from HBase Table • Use HBase’s MR InputFormats • TableInputFormat • MultiTableInputFormat • TableSnapshotInputFormat • Write dataset to HBase Table • Use HBase’s MR OutputFormat • TableOutputFormat • MultiTableOutputFormat • HFileOutputFormat Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Read from HBase Table Write to HBase Table
  • 9. 9© Cloudera, Inc. All rights reserved. HBase + Spark: Batch processing patterns • Read dataset(RDD) from HBase Table • Use HBase’s MR InputFormats • TableInputFormat • MultiTableInputFormat • TableSnapshotInputFormat • Write dataset(RDD) to HBase Table • Use HBase’s MR OutputFormat • TableOutputFormat • MultiTableOutputFormat • HFileOutputFormat Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Read HBase Table as RDD Write RDD as HBase Table
  • 10. 10© Cloudera, Inc. All rights reserved. Spark Streaming • Take an Data source • Partition in to mini batches RDDs • Compute using Spark engine • Output mini batch RDDs Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Mini batch input RDD Data source Mini batch output RDD
  • 11. 11© Cloudera, Inc. All rights reserved. HBase + Spark Streaming – Enriching With HBase Data • “Join” a dataset with HBase data • Enrich Streaming data source with HBase data • Extract information from minibatch • Read/write/update HBase data in processing • Output HBase-data enriched stream of output RDDs Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Mini batch input RDD Data source HBase-enriched mini batch output RDD
  • 12. 12© Cloudera, Inc. All rights reserved. How does Spark get data in and out of HBase? HBase Client Put, Incr, Append HBase Client Get, Scan Bulk Import HBase Client HBase ReplicationHBase Replication low latency high throughput Gets Short scan Full Scan, Snapshot, MapReduce HBase Scanner Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 13. 13© Cloudera, Inc. All rights reserved. How does Spark get data in and out of HBase? HBase Client Put, Incr, Append HBase Client Get, Scan Bulk Import HBase Client HBase ReplicationHBase Replication low latency high throughput Gets Short scan Full Scan, Snapshot, MapReduce HBase Scanner Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Batch RDD via HBase’s MR Input/ Output Formats Streaming using Hbase to Enrich stream data Streaming using HBase to Enrich stream data
  • 14. 14© Cloudera, Inc. All rights reserved. Typing and API Usage
  • 15. 15© Cloudera, Inc. All rights reserved. Under the covers Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Driver Walker Node Configs Executor Static Space Configs HConnection Tasks Tasks Walker Node Executor Static Space Configs HConnection Tasks Tasks
  • 16. 16© Cloudera, Inc. All rights reserved. Key Addition: HBaseContext • Create an HBaseContext // an Hadoop/HBase Configuration object val conf = HBaseConfiguration.create() conf.addResource(new Path("/etc/hbase/conf/core-site.xml")) conf.addResource(new Path("/etc/hbase/conf/hbase-site.xml")) // sc is the Spark Context; hbase context corresponds to an HBase Connection val hbaseContext = new HBaseContext(sc, conf) // A sample RDD val rdd = sc.parallelize(Array( (Bytes.toBytes("1")), (Bytes.toBytes("2")), (Bytes.toBytes("3")), (Bytes.toBytes("4")), (Bytes.toBytes("5")), (Bytes.toBytes("6")), (Bytes.toBytes("7")))) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 17. 17© Cloudera, Inc. All rights reserved. • Foreach • Map • BulkLoad • BulkLoadThinRows • BulkGet (aka Multiget) • BulkDelete Operations on the HBaseContext Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 18. 18© Cloudera, Inc. All rights reserved. Foreach • Read HBase data in parallel for each partition and compute rdd.hbaseForeachPartition(hbaseContext, (it, conn) => { // do something val bufferedMutator = conn.getBufferedMutator( TableName.valueOf("t1")) it.foreach(r => { ... // HBase API put/incr/append/cas calls } bufferedMutator.flush() bufferedMutator.close() }) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 19. 19© Cloudera, Inc. All rights reserved. Map • Take an HBase dataset and map it in parallel for each partition to produce a new RDD val getRdd = rdd.hbaseMapPartitions(hbaseContext, (it, conn) => { val table = conn.getTable(TableName.valueOf("t1")) var res = mutable.MutableList[String]() it.map( r => { ... // HBase API Scan Results } }) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 20. 20© Cloudera, Inc. All rights reserved. BulkLoad • Bulk load a data set into Hbase (for all cases, generally wide tables) rdd.hbaseBulkLoad (tableName, t => { Seq((new KeyFamilyQualifier(t.rowKey, t.family, t.qualifier), t.value)).iterator }, stagingFolder) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 21. 21© Cloudera, Inc. All rights reserved. BulkLoadThinRows • Bulk load a data set into HBase (for skinny tables, <10k cols) hbaseContext.bulkLoadThinRows[(String, Iterable[(Array[Byte], Array[Byte], Array[Byte])])] (rdd, TableName.valueOf(tableName), t => { val rowKey = Bytes.toBytes(t._1) val familyQualifiersValues = new FamiliesQualifiersValues t._2.foreach(f => { val family:Array[Byte] = f._1 val qualifier = f._2 val value:Array[Byte] = f._3 familyQualifiersValues +=(family, qualifier, value) }) (new ByteArrayWrapper(rowKey), familyQualifiersValues) }, stagingFolder.getPath) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 22. 22© Cloudera, Inc. All rights reserved. Scan vs Bulk Get (Parallel HBase Multigets) Scan HBase Table Bulk Get HBase Table Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 23. 23© Cloudera, Inc. All rights reserved. BulkPut • Parallelized HBase Multiput hbaseContext.bulkPut[(Array[Byte], Array[(Array[Byte], Array[Byte], Array[Byte])])](rdd, tableName, (putRecord) => { val put = new Put(putRecord._1) putRecord._2.foreach((putValue) => put.add(putValue._1, putValue._2, putValue._3)) put } } Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 24. 24© Cloudera, Inc. All rights reserved. BulkDelete • Parallelized HBase Multi-deletes hbaseContext.bulkDelete[Array[Byte]](rdd, tableName, putRecord => new Delete(putRecord), 4) // batch size rdd.hbaseBulkDelete(hbaseContext, tableName, putRecord => new Delete(putRecord), 4) // batch size Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 25. 25© Cloudera, Inc. All rights reserved. SparkSQL • Using SparkSQL to query HBase Data // Setup Schema Mapping val dataframe = sqlContext.load("org.apache.hadoop.hbase.spark", Map("hbase.columns.mapping" -> "KEY_FIELD STRING :key, A_FIELD STRING c:a, B_FIELD STRING c:b,", "hbase.table" -> "t1")) dataframe.registerTempTable("hbaseTmp") // Query sqlContext.sql("SELECT KEY_FIELD FROM hbaseTmp " + "WHERE " + "(KEY_FIELD = 'get1' and B_FIELD < '3') or " + "(KEY_FIELD <= 'get3' and B_FIELD = '8')") .foreach(r => println(" - "+r)) Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 26. 26© Cloudera, Inc. All rights reserved. SparkSQL + MLLib • Process data extracted from SparkSQL val resultDf = sqlContext.sql("SELECT gamer_id, oks, games_won, games_played FROM gamer") // Parse data to apply typing information val parsedData = resultDf.map(r => { val array = Array(r.getInt(1).toDouble, r.getInt(2).toDouble, r.getInt(3).toDouble) Vectors.dense(array) }) val dataCount = parsedData.count() if (dataCount > 0) { val clusters = KMeans.train(parsedData, 3, 5) clusters.clusterCenters.foreach(v => println(" Vector Center:" + v)) } Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 27. 27© Cloudera, Inc. All rights reserved. Future work and Conclusion
  • 28. 28© Cloudera, Inc. All rights reserved. Development and Distribution Status • Today • Batch Analysis patterns with existing MR Input/Output Formats • Streaming Analysis Patterns • Committed to HBase trunk branch (2.0) as part of HBase project • Available in CDH5.7.0 with commercial support • Used in production and pre-production today at ~10 Cloudera customers • Recent Additions • Kerberos and Secure HBase access • To come: Kerberos ticket renewals for Spark Streaming • New JSON based HBase table schema specification Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 29. 29© Cloudera, Inc. All rights reserved. How does Spark get data in and out of HBase? HBase Client Put, Incr, Append HBase Client Get, Scan Bulk Import HBase Client HBase ReplicationHBase Replication low latency high throughput Gets Short scan Full Scan, MapReduce HBase Scanner Hsieh and Malaska, Hadoop Summit EU Dublin 2016 Batch RDD via HBase’s MR Input/ Output Formats Streaming using Hbase to Enrich stream data Streaming using Hbase to Enrich stream data HBase Data as Spark Streaming data source
  • 30. 30© Cloudera, Inc. All rights reserved. Future: HBase Data as a Source • HBase edits as a Spark streaming data source (with Kafka?) • Gather other data • Do some computation • Write the data out Hsieh and Malaska, Hadoop Summit EU Dublin 2016 HBase Replication Mini batch input RDD Data source
  • 31. 31© Cloudera, Inc. All rights reserved. Thank you!
  • 32. 32© Cloudera, Inc. All rights reserved. Use Case – Streaming Counting Hsieh and Malaska, Hadoop Summit EU • Puts vs Increments • Bulk Puts/Gets is good • You can get perfect counting 4/13/2016
  • 33. 33© Cloudera, Inc. All rights reserved. DStream DStream DStream Spark Streaming Single Pass Source Receiver RDD Source Receiver RDD RDD Filter Count HBase Increments Source Receiver RDD RDD RDD Single Pass Filter Count HBase Increments First Batch Second Batch Hsieh and Malaska, Hadoop Summit EU Dublin 2016
  • 34. 34© Cloudera, Inc. All rights reserved. DStream DStream DStream Single Pass Source Receiver RDD Source Receiver RDD RDD Filter Count HBase Puts Source Receiver RDD partitions RDD Parition RDD Single Pass Filter Count Pre-first Batch First Batch Second Batch Stateful RDD 1 HBase Puts Stateful RDD 2 Stateful RDD 1 Spark Streaming Hsieh and Malaska, Hadoop Summit EU Dublin 2016

Editor's Notes

  1. Apache Spark and Apache HBase are an ideal combination for low-latency processing, storage, and serving of entity data. Combining both distributed in-memory processing and non-relational storage enables new near-real-time enrichment use cases and improves the performance of existing workflows. In this talk, we will first describe batch in-memory applications that need to process HBase tables. You'll learn about the importance of data locality between Spark and HBase table data and the impact on performance. Next, we'll look at Spark Streaming applications that leverage HBase for storing state. The ability to update streaming state by key and/or windows enables an array of applications such as near real-time fraud detection. We will conclude with a discussion on current open challenges and future work.
  2. Given that Hbase stores a large sorted map, the API looks similar to a map. You can get or put individual rows, or scan a range of rows. There is also a very efficient way of incrementing a particular cell – this can be useful for maintaining high performance counters or statistics. Lastly, it’s possible to write MapReduce jobs that analyze the data in Hbase.
  3. Given that Hbase stores a large sorted map, the API looks similar to a map. You can get or put individual rows, or scan a range of rows. There is also a very efficient way of incrementing a particular cell – this can be useful for maintaining high performance counters or statistics. Lastly, it’s possible to write MapReduce jobs that analyze the data in Hbase.
  4. Given that Hbase stores a large sorted map, the API looks similar to a map. You can get or put individual rows, or scan a range of rows. There is also a very efficient way of incrementing a particular cell – this can be useful for maintaining high performance counters or statistics. Lastly, it’s possible to write MapReduce jobs that analyze the data in Hbase.
  5. Given that Hbase stores a large sorted map, the API looks similar to a map. You can get or put individual rows, or scan a range of rows. There is also a very efficient way of incrementing a particular cell – this can be useful for maintaining high performance counters or statistics. Lastly, it’s possible to write MapReduce jobs that analyze the data in Hbase.