SlideShare ist ein Scribd-Unternehmen logo
1 von 34
The Evolution of a Relational Database Layer over HBase
@ApachePhoenix
http://phoenix.apache.org/
James Taylor (@JamesPlusPlus)
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
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
– 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

Weitere ähnliche Inhalte

Was ist angesagt?

Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingTill Rohrmann
 
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...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxFlink Forward
 
Hive 3 - a new horizon
Hive 3 - a new horizonHive 3 - a new horizon
Hive 3 - a new horizonThejas Nair
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
Cassandra Introduction & Features
Cassandra Introduction & FeaturesCassandra Introduction & Features
Cassandra Introduction & FeaturesDataStax Academy
 
Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Timothy Spann
 
Using LLVM to accelerate processing of data in Apache Arrow
Using LLVM to accelerate processing of data in Apache ArrowUsing LLVM to accelerate processing of data in Apache Arrow
Using LLVM to accelerate processing of data in Apache ArrowDataWorks Summit
 
Hadoop Security Architecture
Hadoop Security ArchitectureHadoop Security Architecture
Hadoop Security ArchitectureOwen O'Malley
 
Local Secondary Indexes in Apache Phoenix
Local Secondary Indexes in Apache PhoenixLocal Secondary Indexes in Apache Phoenix
Local Secondary Indexes in Apache PhoenixRajeshbabu Chintaguntla
 
MongoDB WiredTiger Internals
MongoDB WiredTiger InternalsMongoDB WiredTiger Internals
MongoDB WiredTiger InternalsNorberto Leite
 
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...Altinity Ltd
 
Introduction to Apache ZooKeeper
Introduction to Apache ZooKeeperIntroduction to Apache ZooKeeper
Introduction to Apache ZooKeeperSaurav Haloi
 
Introduction to Apache solr
Introduction to Apache solrIntroduction to Apache solr
Introduction to Apache solrKnoldus Inc.
 
How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...DataWorks Summit/Hadoop Summit
 
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...GetInData
 
MongoDB vs. Postgres Benchmarks
MongoDB vs. Postgres Benchmarks MongoDB vs. Postgres Benchmarks
MongoDB vs. Postgres Benchmarks EDB
 

Was ist angesagt? (20)

Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
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...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Mongo DB Presentation
Mongo DB PresentationMongo DB Presentation
Mongo DB Presentation
 
Intro to HBase
Intro to HBaseIntro to HBase
Intro to HBase
 
Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Hive 3 - a new horizon
Hive 3 - a new horizonHive 3 - a new horizon
Hive 3 - a new horizon
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
Cassandra Introduction & Features
Cassandra Introduction & FeaturesCassandra Introduction & Features
Cassandra Introduction & Features
 
Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4Introduction to Apache NiFi 1.11.4
Introduction to Apache NiFi 1.11.4
 
Using LLVM to accelerate processing of data in Apache Arrow
Using LLVM to accelerate processing of data in Apache ArrowUsing LLVM to accelerate processing of data in Apache Arrow
Using LLVM to accelerate processing of data in Apache Arrow
 
Hadoop Security Architecture
Hadoop Security ArchitectureHadoop Security Architecture
Hadoop Security Architecture
 
Rds data lake @ Robinhood
Rds data lake @ Robinhood Rds data lake @ Robinhood
Rds data lake @ Robinhood
 
Local Secondary Indexes in Apache Phoenix
Local Secondary Indexes in Apache PhoenixLocal Secondary Indexes in Apache Phoenix
Local Secondary Indexes in Apache Phoenix
 
MongoDB WiredTiger Internals
MongoDB WiredTiger InternalsMongoDB WiredTiger Internals
MongoDB WiredTiger Internals
 
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...
OSA Con 2022 - Arrow in Flight_ New Developments in Data Connectivity - David...
 
Introduction to Apache ZooKeeper
Introduction to Apache ZooKeeperIntroduction to Apache ZooKeeper
Introduction to Apache ZooKeeper
 
Introduction to Apache solr
Introduction to Apache solrIntroduction to Apache solr
Introduction to Apache solr
 
How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...
 
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
Best Practices for ETL with Apache NiFi on Kubernetes - Albert Lewandowski, G...
 
MongoDB vs. Postgres Benchmarks
MongoDB vs. Postgres Benchmarks MongoDB vs. Postgres Benchmarks
MongoDB vs. Postgres Benchmarks
 

Andere mochten auch

Apache Big Data EU 2015 - Phoenix
Apache Big Data EU 2015 - PhoenixApache Big Data EU 2015 - Phoenix
Apache Big Data EU 2015 - PhoenixNick Dimiduk
 
HBaseCon 2012 | HBase Schema Design - Ian Varley, Salesforce
HBaseCon 2012 | HBase Schema Design - Ian Varley, SalesforceHBaseCon 2012 | HBase Schema Design - Ian Varley, Salesforce
HBaseCon 2012 | HBase Schema Design - Ian Varley, SalesforceCloudera, Inc.
 
Breaking with relational dbms and dating with hbase
Breaking with relational dbms and dating with hbaseBreaking with relational dbms and dating with hbase
Breaking with relational dbms and dating with hbaseGaurav Kohli
 
Apache Ambari - What's New in 1.4.2
Apache Ambari - What's New in 1.4.2Apache Ambari - What's New in 1.4.2
Apache Ambari - What's New in 1.4.2Hortonworks
 
Seo for-bloggers-2014
Seo for-bloggers-2014Seo for-bloggers-2014
Seo for-bloggers-2014Rand Fishkin
 
Hadoop World 2011: Advanced HBase Schema Design
Hadoop World 2011: Advanced HBase Schema DesignHadoop World 2011: Advanced HBase Schema Design
Hadoop World 2011: Advanced HBase Schema DesignCloudera, Inc.
 
Evaluating NoSQL Performance: Time for Benchmarking
Evaluating NoSQL Performance: Time for BenchmarkingEvaluating NoSQL Performance: Time for Benchmarking
Evaluating NoSQL Performance: Time for BenchmarkingSergey Bushik
 
Near-realtime analytics with Kafka and HBase
Near-realtime analytics with Kafka and HBaseNear-realtime analytics with Kafka and HBase
Near-realtime analytics with Kafka and HBasedave_revell
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 ReleaseNick Dimiduk
 
Netflix Webkit-Based UI for TV Devices
Netflix Webkit-Based UI for TV DevicesNetflix Webkit-Based UI for TV Devices
Netflix Webkit-Based UI for TV DevicesMatt McCarthy
 
Couchbase Performance Benchmarking
Couchbase Performance BenchmarkingCouchbase Performance Benchmarking
Couchbase Performance BenchmarkingRenat Khasanshyn
 
Realtime Analytics with Hadoop and HBase
Realtime Analytics with Hadoop and HBaseRealtime Analytics with Hadoop and HBase
Realtime Analytics with Hadoop and HBaselarsgeorge
 
Deploying and Managing Hadoop Clusters with AMBARI
Deploying and Managing Hadoop Clusters with AMBARIDeploying and Managing Hadoop Clusters with AMBARI
Deploying and Managing Hadoop Clusters with AMBARIDataWorks Summit
 
Apache HBase for Architects
Apache HBase for ArchitectsApache HBase for Architects
Apache HBase for ArchitectsNick Dimiduk
 
Apache-Flink-What-How-Why-Who-Where-by-Slim-Baltagi
Apache-Flink-What-How-Why-Who-Where-by-Slim-BaltagiApache-Flink-What-How-Why-Who-Where-by-Slim-Baltagi
Apache-Flink-What-How-Why-Who-Where-by-Slim-BaltagiSlim Baltagi
 
HBaseCon 2015: Analyzing HBase Data with Apache Hive
HBaseCon 2015: Analyzing HBase Data with Apache  HiveHBaseCon 2015: Analyzing HBase Data with Apache  Hive
HBaseCon 2015: Analyzing HBase Data with Apache HiveHBaseCon
 
HBase for Architects
HBase for ArchitectsHBase for Architects
HBase for ArchitectsNick Dimiduk
 
20090713 Hbase Schema Design Case Studies
20090713 Hbase Schema Design Case Studies20090713 Hbase Schema Design Case Studies
20090713 Hbase Schema Design Case StudiesEvan Liu
 
Intro to HBase Internals & Schema Design (for HBase users)
Intro to HBase Internals & Schema Design (for HBase users)Intro to HBase Internals & Schema Design (for HBase users)
Intro to HBase Internals & Schema Design (for HBase users)alexbaranau
 

Andere mochten auch (20)

Apache Big Data EU 2015 - Phoenix
Apache Big Data EU 2015 - PhoenixApache Big Data EU 2015 - Phoenix
Apache Big Data EU 2015 - Phoenix
 
Introducción a Apache HBase
Introducción a Apache HBaseIntroducción a Apache HBase
Introducción a Apache HBase
 
HBaseCon 2012 | HBase Schema Design - Ian Varley, Salesforce
HBaseCon 2012 | HBase Schema Design - Ian Varley, SalesforceHBaseCon 2012 | HBase Schema Design - Ian Varley, Salesforce
HBaseCon 2012 | HBase Schema Design - Ian Varley, Salesforce
 
Breaking with relational dbms and dating with hbase
Breaking with relational dbms and dating with hbaseBreaking with relational dbms and dating with hbase
Breaking with relational dbms and dating with hbase
 
Apache Ambari - What's New in 1.4.2
Apache Ambari - What's New in 1.4.2Apache Ambari - What's New in 1.4.2
Apache Ambari - What's New in 1.4.2
 
Seo for-bloggers-2014
Seo for-bloggers-2014Seo for-bloggers-2014
Seo for-bloggers-2014
 
Hadoop World 2011: Advanced HBase Schema Design
Hadoop World 2011: Advanced HBase Schema DesignHadoop World 2011: Advanced HBase Schema Design
Hadoop World 2011: Advanced HBase Schema Design
 
Evaluating NoSQL Performance: Time for Benchmarking
Evaluating NoSQL Performance: Time for BenchmarkingEvaluating NoSQL Performance: Time for Benchmarking
Evaluating NoSQL Performance: Time for Benchmarking
 
Near-realtime analytics with Kafka and HBase
Near-realtime analytics with Kafka and HBaseNear-realtime analytics with Kafka and HBase
Near-realtime analytics with Kafka and HBase
 
Apache HBase 1.0 Release
Apache HBase 1.0 ReleaseApache HBase 1.0 Release
Apache HBase 1.0 Release
 
Netflix Webkit-Based UI for TV Devices
Netflix Webkit-Based UI for TV DevicesNetflix Webkit-Based UI for TV Devices
Netflix Webkit-Based UI for TV Devices
 
Couchbase Performance Benchmarking
Couchbase Performance BenchmarkingCouchbase Performance Benchmarking
Couchbase Performance Benchmarking
 
Realtime Analytics with Hadoop and HBase
Realtime Analytics with Hadoop and HBaseRealtime Analytics with Hadoop and HBase
Realtime Analytics with Hadoop and HBase
 
Deploying and Managing Hadoop Clusters with AMBARI
Deploying and Managing Hadoop Clusters with AMBARIDeploying and Managing Hadoop Clusters with AMBARI
Deploying and Managing Hadoop Clusters with AMBARI
 
Apache HBase for Architects
Apache HBase for ArchitectsApache HBase for Architects
Apache HBase for Architects
 
Apache-Flink-What-How-Why-Who-Where-by-Slim-Baltagi
Apache-Flink-What-How-Why-Who-Where-by-Slim-BaltagiApache-Flink-What-How-Why-Who-Where-by-Slim-Baltagi
Apache-Flink-What-How-Why-Who-Where-by-Slim-Baltagi
 
HBaseCon 2015: Analyzing HBase Data with Apache Hive
HBaseCon 2015: Analyzing HBase Data with Apache  HiveHBaseCon 2015: Analyzing HBase Data with Apache  Hive
HBaseCon 2015: Analyzing HBase Data with Apache Hive
 
HBase for Architects
HBase for ArchitectsHBase for Architects
HBase for Architects
 
20090713 Hbase Schema Design Case Studies
20090713 Hbase Schema Design Case Studies20090713 Hbase Schema Design Case Studies
20090713 Hbase Schema Design Case Studies
 
Intro to HBase Internals & Schema Design (for HBase users)
Intro to HBase Internals & Schema Design (for HBase users)Intro to HBase Internals & Schema Design (for HBase users)
Intro to HBase Internals & Schema Design (for HBase users)
 

Ähnlich wie The Evolution of a Relational Database Layer over HBase

HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...
HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...
HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...HBaseCon
 
HBaseCon2015-final
HBaseCon2015-finalHBaseCon2015-final
HBaseCon2015-finalMaryann Xue
 
HBaseCon2016-final
HBaseCon2016-finalHBaseCon2016-final
HBaseCon2016-finalMaryann Xue
 
What's New in .Net 4.5
What's New in .Net 4.5What's New in .Net 4.5
What's New in .Net 4.5Malam Team
 
ONE FOR ALL! Using Apache Calcite to make SQL smart
ONE FOR ALL! Using Apache Calcite to make SQL smartONE FOR ALL! Using Apache Calcite to make SQL smart
ONE FOR ALL! Using Apache Calcite to make SQL smartEvans Ye
 
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...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...Julian Hyde
 
Apache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New FeaturesApache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New FeaturesHBaseCon
 
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.
Taking a look under the hood of Apache Flink's relational APIs.Fabian Hueske
 
Fabian 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 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 APIsFlink Forward
 
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveApache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveXu Jiang
 
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibaba
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibabahbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibaba
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at AlibabaMichael Stack
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoopclairvoyantllc
 
eHarmony @ Hbase Conference 2016 by vijay vangapandu.
eHarmony @ Hbase Conference 2016 by vijay vangapandu.eHarmony @ Hbase Conference 2016 by vijay vangapandu.
eHarmony @ Hbase Conference 2016 by vijay vangapandu.Vijaykumar Vangapandu
 
Distributed Kafka Architecture Taboola Scale
Distributed Kafka Architecture Taboola ScaleDistributed Kafka Architecture Taboola Scale
Distributed Kafka Architecture Taboola ScaleApache Kafka TLV
 
Distributed & Highly Available server applications in Java and Scala
Distributed & Highly Available server applications in Java and ScalaDistributed & Highly Available server applications in Java and Scala
Distributed & Highly Available server applications in Java and ScalaMax Alexejev
 
phoenix-on-calcite-nyc-meetup
phoenix-on-calcite-nyc-meetupphoenix-on-calcite-nyc-meetup
phoenix-on-calcite-nyc-meetupMaryann Xue
 
SQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
SQL Analytics for Search Engineers - Timothy Potter, LucidworksngineersSQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
SQL Analytics for Search Engineers - Timothy Potter, LucidworksngineersLucidworks
 
A Smarter Pig: Building a SQL interface to Pig using Apache Calcite
A Smarter Pig: Building a SQL interface to Pig using Apache CalciteA Smarter Pig: Building a SQL interface to Pig using Apache Calcite
A Smarter Pig: Building a SQL interface to Pig using Apache CalciteSalesforce Engineering
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaCloudera, Inc.
 
Managing multi tenant resource toward Hive 2.0
Managing multi tenant resource toward Hive 2.0Managing multi tenant resource toward Hive 2.0
Managing multi tenant resource toward Hive 2.0Kai Sasaki
 

Ähnlich wie The Evolution of a Relational Database Layer over HBase (20)

HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...
HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...
HBaseCon 2015: Apache Phoenix - The Evolution of a Relational Database Layer ...
 
HBaseCon2015-final
HBaseCon2015-finalHBaseCon2015-final
HBaseCon2015-final
 
HBaseCon2016-final
HBaseCon2016-finalHBaseCon2016-final
HBaseCon2016-final
 
What's New in .Net 4.5
What's New in .Net 4.5What's New in .Net 4.5
What's New in .Net 4.5
 
ONE FOR ALL! Using Apache Calcite to make SQL smart
ONE FOR ALL! Using Apache Calcite to make SQL smartONE FOR ALL! Using Apache Calcite to make SQL smart
ONE FOR ALL! Using Apache Calcite to make SQL smart
 
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...
Data all over the place! How SQL and Apache Calcite bring sanity to streaming...
 
Apache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New FeaturesApache Phoenix: Use Cases and New Features
Apache Phoenix: Use Cases and New Features
 
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.
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
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
 
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep DiveApache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive
 
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibaba
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibabahbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibaba
hbaseconasia2019 Phoenix Improvements and Practices on Cloud HBase at Alibaba
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoop
 
eHarmony @ Hbase Conference 2016 by vijay vangapandu.
eHarmony @ Hbase Conference 2016 by vijay vangapandu.eHarmony @ Hbase Conference 2016 by vijay vangapandu.
eHarmony @ Hbase Conference 2016 by vijay vangapandu.
 
Distributed Kafka Architecture Taboola Scale
Distributed Kafka Architecture Taboola ScaleDistributed Kafka Architecture Taboola Scale
Distributed Kafka Architecture Taboola Scale
 
Distributed & Highly Available server applications in Java and Scala
Distributed & Highly Available server applications in Java and ScalaDistributed & Highly Available server applications in Java and Scala
Distributed & Highly Available server applications in Java and Scala
 
phoenix-on-calcite-nyc-meetup
phoenix-on-calcite-nyc-meetupphoenix-on-calcite-nyc-meetup
phoenix-on-calcite-nyc-meetup
 
SQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
SQL Analytics for Search Engineers - Timothy Potter, LucidworksngineersSQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
SQL Analytics for Search Engineers - Timothy Potter, Lucidworksngineers
 
A Smarter Pig: Building a SQL interface to Pig using Apache Calcite
A Smarter Pig: Building a SQL interface to Pig using Apache CalciteA Smarter Pig: Building a SQL interface to Pig using Apache Calcite
A Smarter Pig: Building a SQL interface to Pig using Apache Calcite
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Managing multi tenant resource toward Hive 2.0
Managing multi tenant resource toward Hive 2.0Managing multi tenant resource toward Hive 2.0
Managing multi tenant resource toward Hive 2.0
 

Mehr von DataWorks Summit

Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisDataWorks Summit
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiDataWorks Summit
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...DataWorks Summit
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...DataWorks Summit
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal SystemDataWorks Summit
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExampleDataWorks Summit
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberDataWorks Summit
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixDataWorks Summit
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiDataWorks Summit
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsDataWorks Summit
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureDataWorks Summit
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EngineDataWorks Summit
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...DataWorks Summit
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudDataWorks Summit
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiDataWorks Summit
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerDataWorks Summit
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...DataWorks Summit
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouDataWorks Summit
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkDataWorks Summit
 

Mehr von DataWorks Summit (20)

Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache Ratis
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal System
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist Example
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at Uber
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability Improvements
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant Architecture
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything Engine
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google Cloud
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near You
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
 

Kürzlich hochgeladen

Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentPim van der Noll
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfIngrid Airi González
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Mark Goldstein
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxLoriGlavin3
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024Lonnie McRorey
 
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...Wes McKinney
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxLoriGlavin3
 
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...Scott Andery
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .Alan Dix
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesAssure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesThousandEyes
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsSergiu Bodiu
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteDianaGray10
 
2024 April Patch Tuesday
2024 April Patch Tuesday2024 April Patch Tuesday
2024 April Patch TuesdayIvanti
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesThousandEyes
 
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Alkin Tezuysal
 
UiPath Community: Communication Mining from Zero to Hero
UiPath Community: Communication Mining from Zero to HeroUiPath Community: Communication Mining from Zero to Hero
UiPath Community: Communication Mining from Zero to HeroUiPathCommunity
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxLoriGlavin3
 
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...AliaaTarek5
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfpanagenda
 

Kürzlich hochgeladen (20)

Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
 
Generative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdfGenerative Artificial Intelligence: How generative AI works.pdf
Generative Artificial Intelligence: How generative AI works.pdf
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
 
The State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptxThe State of Passkeys with FIDO Alliance.pptx
The State of Passkeys with FIDO Alliance.pptx
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024
 
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...
The Future Roadmap for the Composable Data Stack - Wes McKinney - Data Counci...
 
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptxA Deep Dive on Passkeys: FIDO Paris Seminar.pptx
A Deep Dive on Passkeys: FIDO Paris Seminar.pptx
 
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...
Enhancing User Experience - Exploring the Latest Features of Tallyman Axis Lo...
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyesAssure Ecommerce and Retail Operations Uptime with ThousandEyes
Assure Ecommerce and Retail Operations Uptime with ThousandEyes
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platforms
 
Take control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test SuiteTake control of your SAP testing with UiPath Test Suite
Take control of your SAP testing with UiPath Test Suite
 
2024 April Patch Tuesday
2024 April Patch Tuesday2024 April Patch Tuesday
2024 April Patch Tuesday
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
 
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
Unleashing Real-time Insights with ClickHouse_ Navigating the Landscape in 20...
 
UiPath Community: Communication Mining from Zero to Hero
UiPath Community: Communication Mining from Zero to HeroUiPath Community: Communication Mining from Zero to Hero
UiPath Community: Communication Mining from Zero to Hero
 
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptxUse of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
Use of FIDO in the Payments and Identity Landscape: FIDO Paris Seminar.pptx
 
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...
(How to Program) Paul Deitel, Harvey Deitel-Java How to Program, Early Object...
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
 

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) 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. 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
  • 8. 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
  • 9. 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
  • 10. 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
  • 11. What’s New? • HBase 1.0 Support • Functional Indexes
  • 12. 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)
  • 13. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions
  • 14. 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
  • 15. What’s New? • HBase 1.0 Support • Functional Indexes • User Defined Functions • Query Server with Thin Driver
  • 16. 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
  • 17. 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)
  • 18. 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 – Try our txn branch – Will be available in next release
  • 19. 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
  • 20. 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)
  • 21. 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
  • 22. 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
  • 23. 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)
  • 24. 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
  • 26. 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
  • 27. 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
  • 28. Optimization Rules • AggregateRemoveRule • FilterAggregateTransposeRule • FilterJoinRule • FilterMergeRule • JoinCommuteRule • PhoenixFilterScanMergeRule • PhoenixJoinSingleAggregateMergeRule • …
  • 29. 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
  • 30. 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
  • 31. Interoperibility Example • Joining data from Phoenix and mySQL EnumerableJoin PhoenixTableScan JdbcTableScan Phoenix Tables over HBase mySQL Database PhoenixToEnumerable Converter JdbcToEnumerable Converter
  • 32. 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;
  • 33. 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;

Hinweis der Redaktion

  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.