HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer over HBase

HBaseCon
The Evolution of a Relational Database Layer over HBase
@ApachePhoenix
http://phoenix.apache.org/
James Taylor (@JamesPlusPlus)
Maryann Xue (@MaryannXue)
V5
About James
• Architect at Salesforce.com
– Part of the Big Data group
• Lead of Apache Phoenix project
• PMC member of Apache Calcite
• Engineer and Product Manager at BEA Systems
– XQuery-based federated query engine
– SQL-based complex event processing engine
• Various startups prior to that
Agenda
• What is Apache Phoenix?
• State of the Union
• A Deeper Look
– Joins and Subquery Support
• What’s New?
• What’s Next?
• Q&A
What is Apache Phoenix?
• A relational database layer for Apache HBase
– Query engine
• Transforms SQL queries into native HBase API calls
• Pushes as much work as possible onto the cluster for parallel
execution
– Metadata repository
• Typed access to data stored in HBase tables
– A JDBC driver
• A top level Apache Software Foundation project
– Originally developed at Salesforce
– Now a top-level project at the ASF (Happy Birthday!)
– A growing community with momentum
Where Does Phoenix Fit In?
Sqoop
RDBDataCollector
Flume
LogDataCollector
Zookeeper
Coordination
YARN (MRv2)
Cluster Resource
Manager /
MapReduce
HDFS 2.0
Hadoop Distributed File System
GraphX
Graph analysis
framework
Phoenix
Query execution engine
HBase
Distributed Database
The Java Virtual Machine
Hadoop
Common JNI
Spark
Iterative In-Memory
Computation
MLLib
Data mining
Pig
Data Manipulation
Hive
Structured Query
Phoenix
JDBC client
State of the Union
• Broad enough SQL support to run TPC queries
– Joins, Sub-queries, Derived tables, etc.
• Three different secondary indexing strategies
– Immutable for write-once/append only data
– Global for read-heavy mutable data
– Local for write-heavy mutable or immutable data
• Statistics driven parallel execution
• Tracing and metrics for Monitoring & Management
About Maryann
• Software Engineer at Intel
• PMC Member of Apache Phoenix project
– Joins, Subqueries, Phoenix+Calcite Integration, etc.
• IDH and Intel XML Libraries
– The HBase part of Intel’s Distribution of Hadoop
– XSLT compiler of Intel XML Libraries
Join and Subquery Support
• Grammar: inner join; left/right/full outer join; cross join
• Additional: semi join; anti join
• Algorithms: hash-join; sort-merge-join
• Optimizations:
– Predicate push-down
– FK-to-PK join optimization
– Global index with missing data columns
– Correlated subquery rewrite
TPC Example 1
Small-Quantity-Order Revenue Query (Q17)
select sum(l_extendedprice) / 7.0 as avg_yearly
from lineitem, part
where p_partkey = l_partkey
and p_brand = '[B]'
and p_container = '[C]'
and l_quantity < (
select 0.2 * avg(l_quantity)
from lineitem
where l_partkey = p_partkey
);
CLIENT 4-WAY FULL SCAN OVER lineitem
PARALLEL INNER JOIN TABLE 0
CLIENT 1-WAY FULL SCAN OVER part
SERVER FILTER BY p_partkey = ‘[B]’ AND p_container = ‘[C]’
PARALLEL INNER JOIN TABLE 1
CLIENT 4-WAY FULL SCAN OVER lineitem
SERVER AGGREGATE INTO DISTINCT ROWS BY l_partkey
AFTER-JOIN SERVER FILTER BY l_quantity < $0
TPC Example 2
Order Priority Checking Query (Q4)
select o_orderpriority, count(*) as order_count
from orders
where o_orderdate >= date '[D]'
and o_orderdate < date '[D]' + interval '3' month
and exists (
select * from lineitem
where l_orderkey = o_orderkey and l_commitdate < l_receiptdate
)
group by o_orderpriority
order by o_orderpriority;
CLIENT 4-WAY FULL SCAN OVER orders
SERVER FILTER o_orderdate >= ‘[D]’ AND o_orderdate < ‘[D]’ + 3(d)
SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY o_orderpriority
CLIENT MERGE SORT
SKIP-SCAN JOIN TABLE 0
CLIENT 4-WAY FULL SCAN OVER lineitem
SERVER FILTER BY l_commitdate < l_receiptdate
DYNAMIC SERVER FILTER BY o_orderkey IN l_orderkey
Join support - what can't we do?
• Nested Loop Join
• Statistics Guided Join Algorithm
– Smartly choose the smaller table for the build side
– Smartly switch between hash-join and sort-merge-join
– Smartly turn on/off FK-to-PK join optimization
What’s New?
• HBase 1.0 Support
• Functional Indexes
Functional Indexes
• Creating an index on an expression as opposed to just a
column value. For example, the following will be a full
table scan:
SELECT AVG(response_time) FROM SERVER_METRICS
WHERE DAYOFMONTH(create_time) = 1
• Adding the following functional index will turn it
into a range scan:
CREATE INDEX day_of_month_idx
ON SERVER_METRICS (DAYOFMONTH(create_time))
INCLUDE (response_time)
What’s New?
• HBase 1.0 Support
• Functional Indexes
• User Defined Functions
User Defined Functions
• Extension points to Phoenix for domain-specific
functions. For example, a geo-location application might
load a set of UDFs like this:
CREATE FUNCTION WOEID_DISTANCE(INTEGER,INTEGER)
RETURNS INTEGER AS ‘org.apache.geo.woeidDistance’
USING JAR ‘/lib/geo/geoloc.jar’
• Querying, functional indexing, etc. then possible:
SELECT * FROM woeid a JOIN woeid b ON a.country = b.country
WHERE woeid_distance(a.ID,b.ID) < 5
What’s New?
• HBase 1.0 Support
• Functional Indexes
• User Defined Functions
• Query Server with Thin Driver
Query Server + Thin Driver
• Offloads query planning and execution to different server(s)
• Minimizes client dependencies
– Enabler for ODBC driver (not available yet, though)
• Connect like this instead:
Connection conn = DriverManager.getConnection(
“jdbc:phoenix:thin:url=http://localhost:8765”);
• Still evolving, so no backward compatibility guarantees yet
• For more information, see
http://phoenix.apache.org/server.html
What’s New?
• HBase 1.0 Support
• Functional Indexes
• User Defined Functions
• Query Server with Thin Driver
• Union All support
• Testing at scale with Pherf
• MR index build
• Spark integration
• Date built-in functions – WEEK, DAYOFMONTH, etc.
• Transactions (WIP - will be in next release)
Transactions
• Snapshot isolation model
– Using Tephra (http://tephra.io/)
– Supports REPEABLE_READ isolation level
– Allows reading your own uncommitted data
• Optional
– Enabled on a table by table basis
– No performance penalty when not used
• Work in progress, but close to release
– Come by the Phoenix booth for a demo
– Try our txn branch
– Will be available in next release
Optimistic Concurrency Control
• Avoids cost of locking rows and tables
• No deadlocks or lock escalations
• Cost of conflict detection and possible rollback is higher
• Good if conflicts are rare: short transaction, disjoint
partitioning of work
• Conflict detection not always necessary: write-once/append-
only data
Tephra Architecture
ZooKeeper
Tx Manager
(standby)
HBase
Master 1
Master 2
RS 1
RS 2 RS 4
RS 3
Client 1
Client 2
Client N
Tx Manager
(active)
time out
try abort
failed
roll back
in HBase
write
to
HBase
do work
Client Tx Manager
none
complete V
abortsucceeded
in progress
start tx
start start tx
commit
try commit check conflicts
invalid X
invalidate
failed
Transaction Lifecycle
Tephra Architecture
• TransactionAware client
• Coordinates transaction lifecycle with manager
• Communicates directly with HBase for reads and writes
• Transaction Manager
• Assigns transaction IDs
• Maintains state on in-progress, committed and invalid transactions
• Transaction Processor coprocessor
• Applies server-side filtering for reads
• Cleans up data from failed transactions, and no longer visible
versions
What’s New?
• HBase 1.0 Support
• Functional Indexes
• User Defined Functions
• Query Server with Thin Driver
• Union All support
• Testing at scale with Pherf
• MR index build
• Spark integration
• Date built-in functions – WEEK, DAYOFMONTH, etc.
• Transactions (WIP - will be in next release)
What’s Next?
• Is Phoenix done?
• What about the Big Picture?
– How can Phoenix be leveraged in the larger ecosystem?
– Hive, Pig, Spark, MR integration with Phoenix exists
today, but not a great story
What’s Next?
You are here
Introducing Apache Calcite
• Query parser, compiler, and planner framework
– SQL-92 compliant (ever argue SQL with Julian? :-) )
– Enables Phoenix to get missing SQL support
• Pluggable cost-based optimizer framework
– Sane way to model push down through rules
• Interop with other Calcite adaptors
– Not for free, but it becomes feasible
– Already used by Drill, Hive, Kylin, Samza
– Supports any JDBC source (i.e. RDBMS - remember them :-) )
– One cost-model to rule them all
How does Phoenix plug in?
Calcite Parser & Validator
Calcite Query Optimizer
Phoenix Query Plan Generator
Phoenix Runtime
Phoenix Tables over HBase
JDBC Client
SQL + Phoenix
specific
grammar Built-in rules
+ Phoenix
specific rules
Optimization Rules
• AggregateRemoveRule
• FilterAggregateTransposeRule
• FilterJoinRule
• FilterMergeRule
• JoinCommuteRule
• PhoenixFilterScanMergeRule
• PhoenixJoinSingleAggregateMergeRule
• …
Query Example
(filter push-down and smart join algorithm)
LogicalFilter
filter: $0 = ‘x’
LogicalJoin
type: inner
cond: $3 = $7
LogicalProject
projects: $0, $5
LogicalTableScan
table: A
LogicalTableScan
table: B
PhoenixTableScan
table: ‘a’
filter: $0 = ‘x’
PhoenixServerJoin
type: inner
cond: $3 = $1
PhoenixServerProject
projects: $2, $0
Optimizer
(with
RelOptRules &
ConvertRules)
PhoenixTableScan
table: ‘b’
PhoenixServerProject
projects: $0, $2
PhoenixServerProject
projects: $0, $3
Query Example
(filter push-down and smart join algorithm)
ScanPlan
table: ‘a’
skip-scan: pk0 = ‘x’
projects: pk0, c3
HashJoinPlan
types {inner}
join-keys: {$1}
projects: $2, $0
Build
hash-key: $1
Phoenix
Implementor
PhoenixTableScan
table: ‘a’
filter: $0 = ‘x’
PhoenixServerJoin
type: inner
cond: $3 = $1
PhoenixServerProject
projects: $2, $0
PhoenixTableScan
table: ‘b’
PhoenixServerProject
projects: $0, $2
PhoenixServerProject
projects: $0, $3
ScanPlan
table: ‘b’
projects: col0, col2
Probe
Interoperibility Example
• Joining data from Phoenix and mySQL
EnumerableJoin
PhoenixTableScan JdbcTableScan
Phoenix Tables over HBase mySQL Database
PhoenixToEnumerable
Converter
JdbcToEnumerable
Converter
Query Example 1
WITH m AS
(SELECT *
FROM dept_manager dm
WHERE from_date =
(SELECT max(from_date)
FROM dept_manager dm2
WHERE dm.dept_no = dm2.dept_no))
SELECT m.dept_no, d.dept_name, e.first_name, e.last_name
FROM employees e
JOIN m ON e.emp_no = m.emp_no
JOIN departments d ON d.dept_no = m.dept_no
ORDER BY d.dept_no;
Query Example 2
SELECT dept_no, title, count(*)
FROM titles t
JOIN dept_emp de ON t.emp_no = de.emp_no
WHERE dept_no <= 'd006'
GROUP BY rollup(dept_no, title)
ORDER BY dept_no, title;
Thank you!
Questions?
* who uses Phoenix
1 of 35

Recommended

HBaseCon 2012 | Mignify: A Big Data Refinery Built on HBase - Internet Memory... by
HBaseCon 2012 | Mignify: A Big Data Refinery Built on HBase - Internet Memory...HBaseCon 2012 | Mignify: A Big Data Refinery Built on HBase - Internet Memory...
HBaseCon 2012 | Mignify: A Big Data Refinery Built on HBase - Internet Memory...Cloudera, Inc.
4.1K views32 slides
HBaseCon 2015- HBase @ Flipboard by
HBaseCon 2015- HBase @ FlipboardHBaseCon 2015- HBase @ Flipboard
HBaseCon 2015- HBase @ FlipboardMatthew Blair
7.6K views34 slides
HBaseCon 2013: Compaction Improvements in Apache HBase by
HBaseCon 2013: Compaction Improvements in Apache HBaseHBaseCon 2013: Compaction Improvements in Apache HBase
HBaseCon 2013: Compaction Improvements in Apache HBaseCloudera, Inc.
19.1K views33 slides
HBase Read High Availability Using Timeline-Consistent Region Replicas by
HBase Read High Availability Using Timeline-Consistent Region ReplicasHBase Read High Availability Using Timeline-Consistent Region Replicas
HBase Read High Availability Using Timeline-Consistent Region ReplicasHBaseCon
4.1K views38 slides
Dancing with the elephant h base1_final by
Dancing with the elephant   h base1_finalDancing with the elephant   h base1_final
Dancing with the elephant h base1_finalasterix_smartplatf
1.9K views24 slides
HBaseCon 2012 | Building a Large Search Platform on a Shoestring Budget by
HBaseCon 2012 | Building a Large Search Platform on a Shoestring BudgetHBaseCon 2012 | Building a Large Search Platform on a Shoestring Budget
HBaseCon 2012 | Building a Large Search Platform on a Shoestring BudgetCloudera, Inc.
3.1K views26 slides

More Related Content

What's hot

HBase in Practice by
HBase in Practice HBase in Practice
HBase in Practice DataWorks Summit/Hadoop Summit
5.4K views46 slides
Apache Big Data EU 2015 - HBase by
Apache Big Data EU 2015 - HBaseApache Big Data EU 2015 - HBase
Apache Big Data EU 2015 - HBaseNick Dimiduk
3.3K views40 slides
A Survey of HBase Application Archetypes by
A Survey of HBase Application ArchetypesA Survey of HBase Application Archetypes
A Survey of HBase Application ArchetypesHBaseCon
20K views60 slides
HBaseCon 2013: Full-Text Indexing for Apache HBase by
HBaseCon 2013: Full-Text Indexing for Apache HBaseHBaseCon 2013: Full-Text Indexing for Apache HBase
HBaseCon 2013: Full-Text Indexing for Apache HBaseCloudera, Inc.
7.3K views13 slides
Mapreduce over snapshots by
Mapreduce over snapshotsMapreduce over snapshots
Mapreduce over snapshotsenissoz
6.4K views16 slides
HBase Read High Availability Using Timeline Consistent Region Replicas by
HBase  Read High Availability Using Timeline Consistent Region ReplicasHBase  Read High Availability Using Timeline Consistent Region Replicas
HBase Read High Availability Using Timeline Consistent Region Replicasenissoz
8.7K views38 slides

What's hot(20)

Apache Big Data EU 2015 - HBase by Nick Dimiduk
Apache Big Data EU 2015 - HBaseApache Big Data EU 2015 - HBase
Apache Big Data EU 2015 - HBase
Nick Dimiduk3.3K views
A Survey of HBase Application Archetypes by HBaseCon
A Survey of HBase Application ArchetypesA Survey of HBase Application Archetypes
A Survey of HBase Application Archetypes
HBaseCon20K views
HBaseCon 2013: Full-Text Indexing for Apache HBase by Cloudera, Inc.
HBaseCon 2013: Full-Text Indexing for Apache HBaseHBaseCon 2013: Full-Text Indexing for Apache HBase
HBaseCon 2013: Full-Text Indexing for Apache HBase
Cloudera, Inc.7.3K views
Mapreduce over snapshots by enissoz
Mapreduce over snapshotsMapreduce over snapshots
Mapreduce over snapshots
enissoz6.4K views
HBase Read High Availability Using Timeline Consistent Region Replicas by enissoz
HBase  Read High Availability Using Timeline Consistent Region ReplicasHBase  Read High Availability Using Timeline Consistent Region Replicas
HBase Read High Availability Using Timeline Consistent Region Replicas
enissoz8.7K views
Hadoop World 2011: Advanced HBase Schema Design - Lars George, Cloudera by Cloudera, Inc.
Hadoop World 2011: Advanced HBase Schema Design - Lars George, ClouderaHadoop World 2011: Advanced HBase Schema Design - Lars George, Cloudera
Hadoop World 2011: Advanced HBase Schema Design - Lars George, Cloudera
Cloudera, Inc.8.8K views
HBaseCon 2012 | HBase, the Use Case in eBay Cassini by Cloudera, Inc.
HBaseCon 2012 | HBase, the Use Case in eBay Cassini HBaseCon 2012 | HBase, the Use Case in eBay Cassini
HBaseCon 2012 | HBase, the Use Case in eBay Cassini
Cloudera, Inc.6.1K views
Large-scale Web Apps @ Pinterest by HBaseCon
Large-scale Web Apps @ PinterestLarge-scale Web Apps @ Pinterest
Large-scale Web Apps @ Pinterest
HBaseCon4.1K views
HBaseCon 2013: Apache HBase Replication by Cloudera, Inc.
HBaseCon 2013: Apache HBase ReplicationHBaseCon 2013: Apache HBase Replication
HBaseCon 2013: Apache HBase Replication
Cloudera, Inc.4.7K views
Cross-Site BigTable using HBase by HBaseCon
Cross-Site BigTable using HBaseCross-Site BigTable using HBase
Cross-Site BigTable using HBase
HBaseCon3.5K views
HBase Data Modeling and Access Patterns with Kite SDK by HBaseCon
HBase Data Modeling and Access Patterns with Kite SDKHBase Data Modeling and Access Patterns with Kite SDK
HBase Data Modeling and Access Patterns with Kite SDK
HBaseCon4.7K views
Data Evolution in HBase by HBaseCon
Data Evolution in HBaseData Evolution in HBase
Data Evolution in HBase
HBaseCon5K views
HBase Backups by HBaseCon
HBase BackupsHBase Backups
HBase Backups
HBaseCon6.7K views
HBaseCon 2015: Trafodion - Integrating Operational SQL into HBase by HBaseCon
HBaseCon 2015: Trafodion - Integrating Operational SQL into HBaseHBaseCon 2015: Trafodion - Integrating Operational SQL into HBase
HBaseCon 2015: Trafodion - Integrating Operational SQL into HBase
HBaseCon3.3K views
HBase and HDFS: Understanding FileSystem Usage in HBase by enissoz
HBase and HDFS: Understanding FileSystem Usage in HBaseHBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBase
enissoz74K views
HBase for Architects by Nick Dimiduk
HBase for ArchitectsHBase for Architects
HBase for Architects
Nick Dimiduk33.7K views
HBase Status Report - Hadoop Summit Europe 2014 by larsgeorge
HBase Status Report - Hadoop Summit Europe 2014HBase Status Report - Hadoop Summit Europe 2014
HBase Status Report - Hadoop Summit Europe 2014
larsgeorge1.1K views

Viewers also liked

Apache Phoenix and HBase: Past, Present and Future of SQL over HBase by
Apache Phoenix and HBase: Past, Present and Future of SQL over HBaseApache Phoenix and HBase: Past, Present and Future of SQL over HBase
Apache Phoenix and HBase: Past, Present and Future of SQL over HBaseDataWorks Summit/Hadoop Summit
3.3K views41 slides
Apache phoenix: Past, Present and Future of SQL over HBAse by
Apache phoenix: Past, Present and Future of SQL over HBAseApache phoenix: Past, Present and Future of SQL over HBAse
Apache phoenix: Past, Present and Future of SQL over HBAseenissoz
6.3K views41 slides
HBaseCon 2015: Solving HBase Performance Problems with Apache HTrace by
HBaseCon 2015: Solving HBase Performance Problems with Apache HTraceHBaseCon 2015: Solving HBase Performance Problems with Apache HTrace
HBaseCon 2015: Solving HBase Performance Problems with Apache HTraceHBaseCon
4.5K views26 slides
HBaseCon 2015: HBase Performance Tuning @ Salesforce by
HBaseCon 2015: HBase Performance Tuning @ SalesforceHBaseCon 2015: HBase Performance Tuning @ Salesforce
HBaseCon 2015: HBase Performance Tuning @ SalesforceHBaseCon
6.1K views54 slides
HBaseCon 2015: Analyzing HBase Data with Apache Hive by
HBaseCon 2015: Analyzing HBase Data with Apache  HiveHBaseCon 2015: Analyzing HBase Data with Apache  Hive
HBaseCon 2015: Analyzing HBase Data with Apache HiveHBaseCon
7.9K views22 slides
HBaseCon 2015: HBase 2.0 and Beyond Panel by
HBaseCon 2015: HBase 2.0 and Beyond PanelHBaseCon 2015: HBase 2.0 and Beyond Panel
HBaseCon 2015: HBase 2.0 and Beyond PanelHBaseCon
5.3K views44 slides

Viewers also liked(20)

Apache phoenix: Past, Present and Future of SQL over HBAse by enissoz
Apache phoenix: Past, Present and Future of SQL over HBAseApache phoenix: Past, Present and Future of SQL over HBAse
Apache phoenix: Past, Present and Future of SQL over HBAse
enissoz6.3K views
HBaseCon 2015: Solving HBase Performance Problems with Apache HTrace by HBaseCon
HBaseCon 2015: Solving HBase Performance Problems with Apache HTraceHBaseCon 2015: Solving HBase Performance Problems with Apache HTrace
HBaseCon 2015: Solving HBase Performance Problems with Apache HTrace
HBaseCon4.5K views
HBaseCon 2015: HBase Performance Tuning @ Salesforce by HBaseCon
HBaseCon 2015: HBase Performance Tuning @ SalesforceHBaseCon 2015: HBase Performance Tuning @ Salesforce
HBaseCon 2015: HBase Performance Tuning @ Salesforce
HBaseCon6.1K views
HBaseCon 2015: Analyzing HBase Data with Apache Hive by HBaseCon
HBaseCon 2015: Analyzing HBase Data with Apache  HiveHBaseCon 2015: Analyzing HBase Data with Apache  Hive
HBaseCon 2015: Analyzing HBase Data with Apache Hive
HBaseCon7.9K views
HBaseCon 2015: HBase 2.0 and Beyond Panel by HBaseCon
HBaseCon 2015: HBase 2.0 and Beyond PanelHBaseCon 2015: HBase 2.0 and Beyond Panel
HBaseCon 2015: HBase 2.0 and Beyond Panel
HBaseCon5.3K views
Introduction to Spark - Phoenix Meetup 08-19-2014 by cdmaxime
Introduction to Spark - Phoenix Meetup 08-19-2014Introduction to Spark - Phoenix Meetup 08-19-2014
Introduction to Spark - Phoenix Meetup 08-19-2014
cdmaxime1.6K views
Apache Phoenix Query Server by Josh Elser
Apache Phoenix Query ServerApache Phoenix Query Server
Apache Phoenix Query Server
Josh Elser2.8K views
Apache Phoenix: Transforming HBase into a SQL Database by DataWorks Summit
Apache Phoenix: Transforming HBase into a SQL DatabaseApache Phoenix: Transforming HBase into a SQL Database
Apache Phoenix: Transforming HBase into a SQL Database
DataWorks Summit16.7K views
Hortonworks Technical Workshop: HBase and Apache Phoenix by Hortonworks
Hortonworks Technical Workshop: HBase and Apache Phoenix Hortonworks Technical Workshop: HBase and Apache Phoenix
Hortonworks Technical Workshop: HBase and Apache Phoenix
Hortonworks15K views
HBaseCon 2013: Apache HBase, Meet Ops. Ops, Meet Apache HBase. by Cloudera, Inc.
HBaseCon 2013: Apache HBase, Meet Ops. Ops, Meet Apache HBase.HBaseCon 2013: Apache HBase, Meet Ops. Ops, Meet Apache HBase.
HBaseCon 2013: Apache HBase, Meet Ops. Ops, Meet Apache HBase.
Cloudera, Inc.7.1K views
HBaseCon 2013: Apache HBase on Flash by Cloudera, Inc.
HBaseCon 2013: Apache HBase on FlashHBaseCon 2013: Apache HBase on Flash
HBaseCon 2013: Apache HBase on Flash
Cloudera, Inc.4.3K views
HBaseCon 2012 | Relaxed Transactions for HBase - Francis Liu, Yahoo! by Cloudera, Inc.
HBaseCon 2012 | Relaxed Transactions for HBase - Francis Liu, Yahoo!HBaseCon 2012 | Relaxed Transactions for HBase - Francis Liu, Yahoo!
HBaseCon 2012 | Relaxed Transactions for HBase - Francis Liu, Yahoo!
Cloudera, Inc.3.2K views
HBaseCon 2013: Evolving a First-Generation Apache HBase Deployment to Second... by Cloudera, Inc.
HBaseCon 2013:  Evolving a First-Generation Apache HBase Deployment to Second...HBaseCon 2013:  Evolving a First-Generation Apache HBase Deployment to Second...
HBaseCon 2013: Evolving a First-Generation Apache HBase Deployment to Second...
Cloudera, Inc.4.2K views
HBaseCon 2013: Apache Hadoop and Apache HBase for Real-Time Video Analytics by Cloudera, Inc.
HBaseCon 2013: Apache Hadoop and Apache HBase for Real-Time Video Analytics HBaseCon 2013: Apache Hadoop and Apache HBase for Real-Time Video Analytics
HBaseCon 2013: Apache Hadoop and Apache HBase for Real-Time Video Analytics
Cloudera, Inc.4.8K views
HBaseCon 2012 | HBase for the Worlds Libraries - OCLC by Cloudera, Inc.
HBaseCon 2012 | HBase for the Worlds Libraries - OCLCHBaseCon 2012 | HBase for the Worlds Libraries - OCLC
HBaseCon 2012 | HBase for the Worlds Libraries - OCLC
Cloudera, Inc.3.9K views
HBaseCon 2013: Being Smarter Than the Smart Meter by Cloudera, Inc.
HBaseCon 2013: Being Smarter Than the Smart MeterHBaseCon 2013: Being Smarter Than the Smart Meter
HBaseCon 2013: Being Smarter Than the Smart Meter
Cloudera, Inc.4.3K views
HBaseCon 2012 | Unique Sets on HBase and Hadoop - Elliot Clark, StumbleUpon by Cloudera, Inc.
HBaseCon 2012 | Unique Sets on HBase and Hadoop - Elliot Clark, StumbleUponHBaseCon 2012 | Unique Sets on HBase and Hadoop - Elliot Clark, StumbleUpon
HBaseCon 2012 | Unique Sets on HBase and Hadoop - Elliot Clark, StumbleUpon
Cloudera, Inc.3.4K views

Similar to HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer over HBase

HBaseCon2015-final by
HBaseCon2015-finalHBaseCon2015-final
HBaseCon2015-finalMaryann Xue
127 views35 slides
The Evolution of a Relational Database Layer over HBase by
The Evolution of a Relational Database Layer over HBaseThe Evolution of a Relational Database Layer over HBase
The Evolution of a Relational Database Layer over HBaseDataWorks Summit
2.3K views34 slides
HBaseCon2016-final by
HBaseCon2016-finalHBaseCon2016-final
HBaseCon2016-finalMaryann Xue
323 views38 slides
Apache Phoenix: Use Cases and New Features by
Apache Phoenix: Use Cases and New FeaturesApache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New FeaturesHBaseCon
4.7K views39 slides
eHarmony @ Hbase Conference 2016 by vijay vangapandu. by
eHarmony @ Hbase Conference 2016 by vijay vangapandu.eHarmony @ Hbase Conference 2016 by vijay vangapandu.
eHarmony @ Hbase Conference 2016 by vijay vangapandu.Vijaykumar Vangapandu
698 views38 slides
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing by
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingFabian Hueske
1.5K views39 slides

Similar to HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer over HBase(20)

HBaseCon2015-final by Maryann Xue
HBaseCon2015-finalHBaseCon2015-final
HBaseCon2015-final
Maryann Xue127 views
The Evolution of a Relational Database Layer over HBase by DataWorks Summit
The Evolution of a Relational Database Layer over HBaseThe Evolution of a Relational Database Layer over HBase
The Evolution of a Relational Database Layer over HBase
DataWorks Summit2.3K views
HBaseCon2016-final by Maryann Xue
HBaseCon2016-finalHBaseCon2016-final
HBaseCon2016-final
Maryann Xue323 views
Apache Phoenix: Use Cases and New Features by HBaseCon
Apache Phoenix: Use Cases and New FeaturesApache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New Features
HBaseCon4.7K views
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing by Fabian Hueske
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
Fabian Hueske1.5K views
Alluxio 2.0 & Near Real-time Big Data Platform w/ Spark & Alluxio by Alluxio, Inc.
Alluxio 2.0 & Near Real-time Big Data Platform w/ Spark & AlluxioAlluxio 2.0 & Near Real-time Big Data Platform w/ Spark & Alluxio
Alluxio 2.0 & Near Real-time Big Data Platform w/ Spark & Alluxio
Alluxio, Inc.1.5K views
An In-Depth Look at Putting the Sting in Hive by DataWorks Summit
An In-Depth Look at Putting the Sting in HiveAn In-Depth Look at Putting the Sting in Hive
An In-Depth Look at Putting the Sting in Hive
DataWorks Summit1.9K views
Stinger hadoop summit june 2013 by alanfgates
Stinger hadoop summit june 2013Stinger hadoop summit june 2013
Stinger hadoop summit june 2013
alanfgates2.6K views
HBaseCon 2013: Apache Drill - A Community-driven Initiative to Deliver ANSI S... by Cloudera, Inc.
HBaseCon 2013: Apache Drill - A Community-driven Initiative to Deliver ANSI S...HBaseCon 2013: Apache Drill - A Community-driven Initiative to Deliver ANSI S...
HBaseCon 2013: Apache Drill - A Community-driven Initiative to Deliver ANSI S...
Cloudera, Inc.4.7K views
Etu Solution Day 2014 Track-D: 掌握Impala和Spark by James Chen
Etu Solution Day 2014 Track-D: 掌握Impala和SparkEtu Solution Day 2014 Track-D: 掌握Impala和Spark
Etu Solution Day 2014 Track-D: 掌握Impala和Spark
James Chen691 views
Introduction to Impala by markgrover
Introduction to ImpalaIntroduction to Impala
Introduction to Impala
markgrover2.9K views
Architecture Patterns - Open Discussion by Nguyen Tung
Architecture Patterns - Open DiscussionArchitecture Patterns - Open Discussion
Architecture Patterns - Open Discussion
Nguyen Tung4.2K views
Fabian Hueske - Taking a look under the hood of Apache Flink’s relational APIs by Flink Forward
Fabian Hueske - Taking a look under the hood of Apache Flink’s relational APIsFabian Hueske - Taking a look under the hood of Apache Flink’s relational APIs
Fabian Hueske - Taking a look under the hood of Apache Flink’s relational APIs
Flink Forward423 views
Taking a look under the hood of Apache Flink's relational APIs. by Fabian Hueske
Taking a look under the hood of Apache Flink's relational APIs.Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.
Fabian Hueske2.7K views
Data all over the place! How SQL and Apache Calcite bring sanity to streaming... by Julian Hyde
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
Julian Hyde4K views
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc... by Databricks
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Databricks10.8K views
SQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers by Lucidworks
SQL Analytics for Search Engineers - Timothy Potter, LucidworksngineersSQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
SQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
Lucidworks173 views
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa... by StreamNative
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...
Change Data Capture to Data Lakes Using Apache Pulsar and Apache Hudi - Pulsa...
StreamNative510 views
Streaming Data Analytics with ksqlDB and Superset | Robert Stolz, Preset by HostedbyConfluent
Streaming Data Analytics with ksqlDB and Superset | Robert Stolz, PresetStreaming Data Analytics with ksqlDB and Superset | Robert Stolz, Preset
Streaming Data Analytics with ksqlDB and Superset | Robert Stolz, Preset
HostedbyConfluent2.8K views

More from HBaseCon

hbaseconasia2017: Building online HBase cluster of Zhihu based on Kubernetes by
hbaseconasia2017: Building online HBase cluster of Zhihu based on Kuberneteshbaseconasia2017: Building online HBase cluster of Zhihu based on Kubernetes
hbaseconasia2017: Building online HBase cluster of Zhihu based on KubernetesHBaseCon
3.9K views36 slides
hbaseconasia2017: HBase on Beam by
hbaseconasia2017: HBase on Beamhbaseconasia2017: HBase on Beam
hbaseconasia2017: HBase on BeamHBaseCon
1.3K views26 slides
hbaseconasia2017: HBase Disaster Recovery Solution at Huawei by
hbaseconasia2017: HBase Disaster Recovery Solution at Huaweihbaseconasia2017: HBase Disaster Recovery Solution at Huawei
hbaseconasia2017: HBase Disaster Recovery Solution at HuaweiHBaseCon
1.4K views21 slides
hbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinterest by
hbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinteresthbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinterest
hbaseconasia2017: Removable singularity: a story of HBase upgrade in PinterestHBaseCon
936 views42 slides
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程 by
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程HBaseCon
1.1K views21 slides
hbaseconasia2017: Apache HBase at Netease by
hbaseconasia2017: Apache HBase at Neteasehbaseconasia2017: Apache HBase at Netease
hbaseconasia2017: Apache HBase at NeteaseHBaseCon
754 views27 slides

More from HBaseCon(20)

hbaseconasia2017: Building online HBase cluster of Zhihu based on Kubernetes by HBaseCon
hbaseconasia2017: Building online HBase cluster of Zhihu based on Kuberneteshbaseconasia2017: Building online HBase cluster of Zhihu based on Kubernetes
hbaseconasia2017: Building online HBase cluster of Zhihu based on Kubernetes
HBaseCon3.9K views
hbaseconasia2017: HBase on Beam by HBaseCon
hbaseconasia2017: HBase on Beamhbaseconasia2017: HBase on Beam
hbaseconasia2017: HBase on Beam
HBaseCon1.3K views
hbaseconasia2017: HBase Disaster Recovery Solution at Huawei by HBaseCon
hbaseconasia2017: HBase Disaster Recovery Solution at Huaweihbaseconasia2017: HBase Disaster Recovery Solution at Huawei
hbaseconasia2017: HBase Disaster Recovery Solution at Huawei
HBaseCon1.4K views
hbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinterest by HBaseCon
hbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinteresthbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinterest
hbaseconasia2017: Removable singularity: a story of HBase upgrade in Pinterest
HBaseCon936 views
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程 by HBaseCon
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程
hbaseconasia2017: HareQL:快速HBase查詢工具的發展過程
HBaseCon1.1K views
hbaseconasia2017: Apache HBase at Netease by HBaseCon
hbaseconasia2017: Apache HBase at Neteasehbaseconasia2017: Apache HBase at Netease
hbaseconasia2017: Apache HBase at Netease
HBaseCon754 views
hbaseconasia2017: HBase在Hulu的使用和实践 by HBaseCon
hbaseconasia2017: HBase在Hulu的使用和实践hbaseconasia2017: HBase在Hulu的使用和实践
hbaseconasia2017: HBase在Hulu的使用和实践
HBaseCon878 views
hbaseconasia2017: 基于HBase的企业级大数据平台 by HBaseCon
hbaseconasia2017: 基于HBase的企业级大数据平台hbaseconasia2017: 基于HBase的企业级大数据平台
hbaseconasia2017: 基于HBase的企业级大数据平台
HBaseCon701 views
hbaseconasia2017: HBase at JD.com by HBaseCon
hbaseconasia2017: HBase at JD.comhbaseconasia2017: HBase at JD.com
hbaseconasia2017: HBase at JD.com
HBaseCon828 views
hbaseconasia2017: Large scale data near-line loading method and architecture by HBaseCon
hbaseconasia2017: Large scale data near-line loading method and architecturehbaseconasia2017: Large scale data near-line loading method and architecture
hbaseconasia2017: Large scale data near-line loading method and architecture
HBaseCon598 views
hbaseconasia2017: Ecosystems with HBase and CloudTable service at Huawei by HBaseCon
hbaseconasia2017: Ecosystems with HBase and CloudTable service at Huaweihbaseconasia2017: Ecosystems with HBase and CloudTable service at Huawei
hbaseconasia2017: Ecosystems with HBase and CloudTable service at Huawei
HBaseCon683 views
hbaseconasia2017: HBase Practice At XiaoMi by HBaseCon
hbaseconasia2017: HBase Practice At XiaoMihbaseconasia2017: HBase Practice At XiaoMi
hbaseconasia2017: HBase Practice At XiaoMi
HBaseCon1.8K views
hbaseconasia2017: hbase-2.0.0 by HBaseCon
hbaseconasia2017: hbase-2.0.0hbaseconasia2017: hbase-2.0.0
hbaseconasia2017: hbase-2.0.0
HBaseCon1.8K views
HBaseCon2017 Democratizing HBase by HBaseCon
HBaseCon2017 Democratizing HBaseHBaseCon2017 Democratizing HBase
HBaseCon2017 Democratizing HBase
HBaseCon897 views
HBaseCon2017 Removable singularity: a story of HBase upgrade in Pinterest by HBaseCon
HBaseCon2017 Removable singularity: a story of HBase upgrade in PinterestHBaseCon2017 Removable singularity: a story of HBase upgrade in Pinterest
HBaseCon2017 Removable singularity: a story of HBase upgrade in Pinterest
HBaseCon646 views
HBaseCon2017 Quanta: Quora's hierarchical counting system on HBase by HBaseCon
HBaseCon2017 Quanta: Quora's hierarchical counting system on HBaseHBaseCon2017 Quanta: Quora's hierarchical counting system on HBase
HBaseCon2017 Quanta: Quora's hierarchical counting system on HBase
HBaseCon608 views
HBaseCon2017 Transactions in HBase by HBaseCon
HBaseCon2017 Transactions in HBaseHBaseCon2017 Transactions in HBase
HBaseCon2017 Transactions in HBase
HBaseCon1.8K views
HBaseCon2017 Highly-Available HBase by HBaseCon
HBaseCon2017 Highly-Available HBaseHBaseCon2017 Highly-Available HBase
HBaseCon2017 Highly-Available HBase
HBaseCon1.1K views
HBaseCon2017 Apache HBase at Didi by HBaseCon
HBaseCon2017 Apache HBase at DidiHBaseCon2017 Apache HBase at Didi
HBaseCon2017 Apache HBase at Didi
HBaseCon996 views
HBaseCon2017 gohbase: Pure Go HBase Client by HBaseCon
HBaseCon2017 gohbase: Pure Go HBase ClientHBaseCon2017 gohbase: Pure Go HBase Client
HBaseCon2017 gohbase: Pure Go HBase Client
HBaseCon1.7K views

Recently uploaded

Electronic AWB - Electronic Air Waybill by
Electronic AWB - Electronic Air Waybill Electronic AWB - Electronic Air Waybill
Electronic AWB - Electronic Air Waybill Freightoscope
5 views1 slide
The Path to DevOps by
The Path to DevOpsThe Path to DevOps
The Path to DevOpsJohn Valentino
5 views6 slides
Generic or specific? Making sensible software design decisions by
Generic or specific? Making sensible software design decisionsGeneric or specific? Making sensible software design decisions
Generic or specific? Making sensible software design decisionsBert Jan Schrijver
7 views60 slides
What is API by
What is APIWhat is API
What is APIartembondar5
13 views15 slides
tecnologia18.docx by
tecnologia18.docxtecnologia18.docx
tecnologia18.docxnosi6702
5 views5 slides
Agile 101 by
Agile 101Agile 101
Agile 101John Valentino
12 views20 slides

Recently uploaded(20)

Electronic AWB - Electronic Air Waybill by Freightoscope
Electronic AWB - Electronic Air Waybill Electronic AWB - Electronic Air Waybill
Electronic AWB - Electronic Air Waybill
Freightoscope 5 views
Generic or specific? Making sensible software design decisions by Bert Jan Schrijver
Generic or specific? Making sensible software design decisionsGeneric or specific? Making sensible software design decisions
Generic or specific? Making sensible software design decisions
tecnologia18.docx by nosi6702
tecnologia18.docxtecnologia18.docx
tecnologia18.docx
nosi67025 views
Navigating container technology for enhanced security by Niklas Saari by Metosin Oy
Navigating container technology for enhanced security by Niklas SaariNavigating container technology for enhanced security by Niklas Saari
Navigating container technology for enhanced security by Niklas Saari
Metosin Oy15 views
Team Transformation Tactics for Holistic Testing and Quality (Japan Symposium... by Lisi Hocke
Team Transformation Tactics for Holistic Testing and Quality (Japan Symposium...Team Transformation Tactics for Holistic Testing and Quality (Japan Symposium...
Team Transformation Tactics for Holistic Testing and Quality (Japan Symposium...
Lisi Hocke35 views
Dapr Unleashed: Accelerating Microservice Development by Miroslav Janeski
Dapr Unleashed: Accelerating Microservice DevelopmentDapr Unleashed: Accelerating Microservice Development
Dapr Unleashed: Accelerating Microservice Development
Miroslav Janeski15 views
Ports-and-Adapters Architecture for Embedded HMI by Burkhard Stubert
Ports-and-Adapters Architecture for Embedded HMIPorts-and-Adapters Architecture for Embedded HMI
Ports-and-Adapters Architecture for Embedded HMI
Burkhard Stubert33 views
Dev-HRE-Ops - Addressing the _Last Mile DevOps Challenge_ in Highly Regulated... by TomHalpin9
Dev-HRE-Ops - Addressing the _Last Mile DevOps Challenge_ in Highly Regulated...Dev-HRE-Ops - Addressing the _Last Mile DevOps Challenge_ in Highly Regulated...
Dev-HRE-Ops - Addressing the _Last Mile DevOps Challenge_ in Highly Regulated...
TomHalpin96 views
Transport Management System - Shipment & Container Tracking by Freightoscope
Transport Management System - Shipment & Container TrackingTransport Management System - Shipment & Container Tracking
Transport Management System - Shipment & Container Tracking
Freightoscope 5 views
Unlocking the Power of AI in Product Management - A Comprehensive Guide for P... by NimaTorabi2
Unlocking the Power of AI in Product Management - A Comprehensive Guide for P...Unlocking the Power of AI in Product Management - A Comprehensive Guide for P...
Unlocking the Power of AI in Product Management - A Comprehensive Guide for P...
NimaTorabi216 views
aATP - New Correlation Confirmation Feature.pptx by EsatEsenek1
aATP - New Correlation Confirmation Feature.pptxaATP - New Correlation Confirmation Feature.pptx
aATP - New Correlation Confirmation Feature.pptx
EsatEsenek1205 views
Understanding HTML terminology by artembondar5
Understanding HTML terminologyUnderstanding HTML terminology
Understanding HTML terminology
artembondar57 views
AI and Ml presentation .pptx by FayazAli87
AI and Ml presentation .pptxAI and Ml presentation .pptx
AI and Ml presentation .pptx
FayazAli8714 views
Airline Booking Software by SharmiMehta
Airline Booking SoftwareAirline Booking Software
Airline Booking Software
SharmiMehta9 views

HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer over HBase

  • 1. The Evolution of a Relational Database Layer over HBase @ApachePhoenix http://phoenix.apache.org/ James Taylor (@JamesPlusPlus) Maryann Xue (@MaryannXue) V5
  • 2. About James • Architect at Salesforce.com – Part of the Big Data group • Lead of Apache Phoenix project • PMC member of Apache Calcite • Engineer and Product Manager at BEA Systems – XQuery-based federated query engine – SQL-based complex event processing engine • Various startups prior to that
  • 3. Agenda • What is Apache Phoenix? • State of the Union • A Deeper Look – Joins and Subquery Support • What’s New? • What’s Next? • Q&A
  • 4. What is Apache Phoenix? • A relational database layer for Apache HBase – Query engine • Transforms SQL queries into native HBase API calls • Pushes as much work as possible onto the cluster for parallel execution – Metadata repository • Typed access to data stored in HBase tables – A JDBC driver • A top level Apache Software Foundation project – Originally developed at Salesforce – Now a top-level project at the ASF (Happy Birthday!) – A growing community with momentum
  • 5. Where Does Phoenix Fit In? Sqoop RDBDataCollector Flume LogDataCollector Zookeeper Coordination YARN (MRv2) Cluster Resource Manager / MapReduce HDFS 2.0 Hadoop Distributed File System GraphX Graph analysis framework Phoenix Query execution engine HBase Distributed Database The Java Virtual Machine Hadoop Common JNI Spark Iterative In-Memory Computation MLLib Data mining Pig Data Manipulation Hive Structured Query Phoenix JDBC client
  • 6. State of the Union • Broad enough SQL support to run TPC queries – Joins, Sub-queries, Derived tables, etc. • Three different secondary indexing strategies – Immutable for write-once/append only data – Global for read-heavy mutable data – Local for write-heavy mutable or immutable data • Statistics driven parallel execution • Tracing and metrics for Monitoring & Management
  • 7. About Maryann • Software Engineer at Intel • PMC Member of Apache Phoenix project – Joins, Subqueries, Phoenix+Calcite Integration, etc. • IDH and Intel XML Libraries – The HBase part of Intel’s Distribution of Hadoop – XSLT compiler of Intel XML Libraries
  • 8. Join and Subquery Support • Grammar: inner join; left/right/full outer join; cross join • Additional: semi join; anti join • Algorithms: hash-join; sort-merge-join • Optimizations: – Predicate push-down – FK-to-PK join optimization – Global index with missing data columns – Correlated subquery rewrite
  • 9. TPC Example 1 Small-Quantity-Order Revenue Query (Q17) select sum(l_extendedprice) / 7.0 as avg_yearly from lineitem, part where p_partkey = l_partkey and p_brand = '[B]' and p_container = '[C]' and l_quantity < ( select 0.2 * avg(l_quantity) from lineitem where l_partkey = p_partkey ); CLIENT 4-WAY FULL SCAN OVER lineitem PARALLEL INNER JOIN TABLE 0 CLIENT 1-WAY FULL SCAN OVER part SERVER FILTER BY p_partkey = ‘[B]’ AND p_container = ‘[C]’ PARALLEL INNER JOIN TABLE 1 CLIENT 4-WAY FULL SCAN OVER lineitem SERVER AGGREGATE INTO DISTINCT ROWS BY l_partkey AFTER-JOIN SERVER FILTER BY l_quantity < $0
  • 10. TPC Example 2 Order Priority Checking Query (Q4) select o_orderpriority, count(*) as order_count from orders where o_orderdate >= date '[D]' and o_orderdate < date '[D]' + interval '3' month and exists ( select * from lineitem where l_orderkey = o_orderkey and l_commitdate < l_receiptdate ) group by o_orderpriority order by o_orderpriority; CLIENT 4-WAY FULL SCAN OVER orders SERVER FILTER o_orderdate >= ‘[D]’ AND o_orderdate < ‘[D]’ + 3(d) SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY o_orderpriority CLIENT MERGE SORT SKIP-SCAN JOIN TABLE 0 CLIENT 4-WAY FULL SCAN OVER lineitem SERVER FILTER BY l_commitdate < l_receiptdate DYNAMIC SERVER FILTER BY o_orderkey IN l_orderkey
  • 11. Join support - what can't we do? • Nested Loop Join • Statistics Guided Join Algorithm – Smartly choose the smaller table for the build side – Smartly switch between hash-join and sort-merge-join – Smartly turn on/off FK-to-PK join optimization
  • 12. What’s New? • HBase 1.0 Support • Functional Indexes
  • 13. Functional Indexes • Creating an index on an expression as opposed to just a column value. For example, the following will be a full table scan: SELECT AVG(response_time) FROM SERVER_METRICS WHERE DAYOFMONTH(create_time) = 1 • Adding the following functional index will turn it into a range scan: CREATE INDEX day_of_month_idx ON SERVER_METRICS (DAYOFMONTH(create_time)) INCLUDE (response_time)
  • 14. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions
  • 15. User Defined Functions • Extension points to Phoenix for domain-specific functions. For example, a geo-location application might load a set of UDFs like this: CREATE FUNCTION WOEID_DISTANCE(INTEGER,INTEGER) RETURNS INTEGER AS ‘org.apache.geo.woeidDistance’ USING JAR ‘/lib/geo/geoloc.jar’ • Querying, functional indexing, etc. then possible: SELECT * FROM woeid a JOIN woeid b ON a.country = b.country WHERE woeid_distance(a.ID,b.ID) < 5
  • 16. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions • Query Server with Thin Driver
  • 17. Query Server + Thin Driver • Offloads query planning and execution to different server(s) • Minimizes client dependencies – Enabler for ODBC driver (not available yet, though) • Connect like this instead: Connection conn = DriverManager.getConnection( “jdbc:phoenix:thin:url=http://localhost:8765”); • Still evolving, so no backward compatibility guarantees yet • For more information, see http://phoenix.apache.org/server.html
  • 18. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions • Query Server with Thin Driver • Union All support • Testing at scale with Pherf • MR index build • Spark integration • Date built-in functions – WEEK, DAYOFMONTH, etc. • Transactions (WIP - will be in next release)
  • 19. Transactions • Snapshot isolation model – Using Tephra (http://tephra.io/) – Supports REPEABLE_READ isolation level – Allows reading your own uncommitted data • Optional – Enabled on a table by table basis – No performance penalty when not used • Work in progress, but close to release – Come by the Phoenix booth for a demo – Try our txn branch – Will be available in next release
  • 20. Optimistic Concurrency Control • Avoids cost of locking rows and tables • No deadlocks or lock escalations • Cost of conflict detection and possible rollback is higher • Good if conflicts are rare: short transaction, disjoint partitioning of work • Conflict detection not always necessary: write-once/append- only data
  • 21. Tephra Architecture ZooKeeper Tx Manager (standby) HBase Master 1 Master 2 RS 1 RS 2 RS 4 RS 3 Client 1 Client 2 Client N Tx Manager (active)
  • 22. time out try abort failed roll back in HBase write to HBase do work Client Tx Manager none complete V abortsucceeded in progress start tx start start tx commit try commit check conflicts invalid X invalidate failed Transaction Lifecycle
  • 23. Tephra Architecture • TransactionAware client • Coordinates transaction lifecycle with manager • Communicates directly with HBase for reads and writes • Transaction Manager • Assigns transaction IDs • Maintains state on in-progress, committed and invalid transactions • Transaction Processor coprocessor • Applies server-side filtering for reads • Cleans up data from failed transactions, and no longer visible versions
  • 24. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions • Query Server with Thin Driver • Union All support • Testing at scale with Pherf • MR index build • Spark integration • Date built-in functions – WEEK, DAYOFMONTH, etc. • Transactions (WIP - will be in next release)
  • 25. What’s Next? • Is Phoenix done? • What about the Big Picture? – How can Phoenix be leveraged in the larger ecosystem? – Hive, Pig, Spark, MR integration with Phoenix exists today, but not a great story
  • 27. Introducing Apache Calcite • Query parser, compiler, and planner framework – SQL-92 compliant (ever argue SQL with Julian? :-) ) – Enables Phoenix to get missing SQL support • Pluggable cost-based optimizer framework – Sane way to model push down through rules • Interop with other Calcite adaptors – Not for free, but it becomes feasible – Already used by Drill, Hive, Kylin, Samza – Supports any JDBC source (i.e. RDBMS - remember them :-) ) – One cost-model to rule them all
  • 28. How does Phoenix plug in? Calcite Parser & Validator Calcite Query Optimizer Phoenix Query Plan Generator Phoenix Runtime Phoenix Tables over HBase JDBC Client SQL + Phoenix specific grammar Built-in rules + Phoenix specific rules
  • 29. Optimization Rules • AggregateRemoveRule • FilterAggregateTransposeRule • FilterJoinRule • FilterMergeRule • JoinCommuteRule • PhoenixFilterScanMergeRule • PhoenixJoinSingleAggregateMergeRule • …
  • 30. Query Example (filter push-down and smart join algorithm) LogicalFilter filter: $0 = ‘x’ LogicalJoin type: inner cond: $3 = $7 LogicalProject projects: $0, $5 LogicalTableScan table: A LogicalTableScan table: B PhoenixTableScan table: ‘a’ filter: $0 = ‘x’ PhoenixServerJoin type: inner cond: $3 = $1 PhoenixServerProject projects: $2, $0 Optimizer (with RelOptRules & ConvertRules) PhoenixTableScan table: ‘b’ PhoenixServerProject projects: $0, $2 PhoenixServerProject projects: $0, $3
  • 31. Query Example (filter push-down and smart join algorithm) ScanPlan table: ‘a’ skip-scan: pk0 = ‘x’ projects: pk0, c3 HashJoinPlan types {inner} join-keys: {$1} projects: $2, $0 Build hash-key: $1 Phoenix Implementor PhoenixTableScan table: ‘a’ filter: $0 = ‘x’ PhoenixServerJoin type: inner cond: $3 = $1 PhoenixServerProject projects: $2, $0 PhoenixTableScan table: ‘b’ PhoenixServerProject projects: $0, $2 PhoenixServerProject projects: $0, $3 ScanPlan table: ‘b’ projects: col0, col2 Probe
  • 32. Interoperibility Example • Joining data from Phoenix and mySQL EnumerableJoin PhoenixTableScan JdbcTableScan Phoenix Tables over HBase mySQL Database PhoenixToEnumerable Converter JdbcToEnumerable Converter
  • 33. Query Example 1 WITH m AS (SELECT * FROM dept_manager dm WHERE from_date = (SELECT max(from_date) FROM dept_manager dm2 WHERE dm.dept_no = dm2.dept_no)) SELECT m.dept_no, d.dept_name, e.first_name, e.last_name FROM employees e JOIN m ON e.emp_no = m.emp_no JOIN departments d ON d.dept_no = m.dept_no ORDER BY d.dept_no;
  • 34. Query Example 2 SELECT dept_no, title, count(*) FROM titles t JOIN dept_emp de ON t.emp_no = de.emp_no WHERE dept_no <= 'd006' GROUP BY rollup(dept_no, title) ORDER BY dept_no, title;

Editor's Notes

  1. Who here is already familiar with Phoenix? GitHub -> Incubator -> TLP 1000 msg / month -> 2000 1 year old today
  2. TPC = complex queries used to benchmark SQL databases against each other
  3. All types of; algorithms. FK-PK opt Useful in global index. Other opt Many TPC queries.
  4. a yearly average price for orders of a specific part brand and part container with a quantity less than 20% of the average quantity of orders for the same part. join + correlated subquery. Two opt in query plan: 1st one de-correlation. 2nd one predicate push-down.
  5. An example of EXISTS => semi-join Triggers another opt, FK-PK join opt In query plan, SKIP-SCAN-JOIN with a dynamic filter At runtime, a skip-scan not a full-scan on orders table
  6. Something missing. 2 join algorithms, hash and merge. Former faster vs. latter for two large tables. How to decide which algorithm? Can’t. Prioritize one. Can’t do the join side either. Are we going to? Yes. Table stats for choosing join algorithm and optimization.
  7. Jeffrey & Enis Thomas
  8. Rajeshbabu
  9. Nick & Julian
  10. Maryann, myself, and Alicia Cody & Mujtaba Ravi Josh Mahonin Alicia Thomas, myself, and Gary Helmling
  11. Slides courtesy of Gary and Andreas Go to Gary’s talk on CDAP at 4:10
  12. Ran out of room – didn’t even mention the 8x perf improvement for unordered, unaggregated queries by Samarth Fantastic work by a lot of people to pull this together
  13. Join ordering and other optimizations now possible
  14. Details of integration: Position and interact? 1. A customized Parser + Validator 2. Query Optimizer + own table stats + Phoenix rules. 3. Translation process 4. Phoenix Runtime
  15. A join query with a WHERE condition. Highlight filter push-down and swap of join tables, called FilterJoinTransposeRule and JoinCommuteRule. Conversion from Logical to Phoenix physical at the same time. Opt: Filter on table ‘A’ … The tree on the right => output A good example of how Calcite can make the decision of join algorithms easy. 
  16. Default implementation of backend: Enumerable RelNodes w/ adapters, run Phoenix + other data source. Example of joining Phoenix with JDBC: EnumerableJoin …  Can replace JDBC table with one from other data source.
  17. WITH: we don’t have for front-end but equivalent to derived table. Get the grammar from Calcite and run in Phoenix.
  18. ROLLUP group-by: part to Phoenix and rest to itself.