SlideShare a Scribd company logo
1 © Hortonworks Inc. 2011–2018. All rights reserved
© Hortonworks Inc. 2011 – 2017
Apache Hive 3: A new horizon
Gunther Hagleitner, Ashutosh Chauhan, Gopal
Vijayaraghavan, Thejas Nair, Will Xu
2 © Hortonworks Inc. 2011–2018. All rights reserved
7000 analysts, 80ms average latency, 1PB data.
250k BI queries per hour
On demand deep reporting in the cloud over
100Tb in minutes.
© Hortonworks Inc. 2011- 2018. All rights reserved | 3
Agenda
● Data Analytics Studio
● Apache Hive 3
● Hive-Spark interoperability
● Performance
● Look ahead
© Hortonworks Inc. 2011- 2018. All rights reserved | 4
Data Analytics Studio
© Hortonworks Inc. 2011- 2018. All rights reserved | 5
Self-service question #1: Why is my query slow?
Noisy neighbors Poor schema Inefficient queries Unstable demand
Smart query
log search
Storage
Optimizations
Query
Optimizations
Demand
Shifting
Hortonworks Data Analytics Studio
7 © Hortonworks Inc. 2011–2018. All rights reserved
One of the Extensible DataPlane Services
⬢ DAS 1.0 available now for HDP 3.0!
⬢ Monthly release cadence
⬢ Replaces Hive & Tez Views
⬢ Separate install from stack
Hortonworks Data Analytics Studio
HORTONWORKS DATAPLANE SERVICE
DATA SOURCE INTEGRATION
DATA SERVICES CATALOG
…DATA
LIFECYCLE
MANAGER
DATA
STEWARD
STUDIO
+OTHER
(partner)
SECURITY CONTROLS
CORE CAPABILITIES
MULTIPLE CLUSTERS AND SOURCES
MULTIHYBRID
*not yet available, coming soon
EXTENSIBLE SERVICES
IBM DSX*
DATA
ANALYTICS
STUDIO
© Hortonworks Inc. 2011- 2018. All rights reserved | 8
Apache Hive 3
9 © Hortonworks Inc. 2011–2018. All rights reserved
Hive3: EDW analyst pipeline
Tableau
BI systems
Materialized
view
Surrogate
key
Constraints
Query
Result
Cache
Workload
management
• Results return
from HDFS/cache
directly
• Reduce load from
repetitive queries
• Allows more
queries to be run
in parallel
• Reduce resource
starvation in large
clusters
• Active/Passive HA
• More “tools” for
optimizer to use
• More ”tools” for
DBAs to
tune/optimize
• Invisible tuning of
DB from users’
perspective
• ACID v2 is as fast as
regular tables
• Hive 3 is optimized
for S3/WASB/GCP
• Support for
JDBC/Kafka/Druid
out of the box
ACID v2
Cloud
Storage
Connectors
© Hortonworks Inc. 2011- 2018. All rights reserved | 10
Connectors
11 © Hortonworks Inc. 2011–2018. All rights reserved
Hive-1010: Information schema & sysdb
Question:
Find which tables have a column with ‘ssn’
as part of the column name?
use information_schema;
SELECT table_schema, table_name
FROM information_schema.columns
WHERE column_name LIKE '%ssn%';
Question:
Find the biggest tables in the system.
use sys;
SELECT tbl_name, total_size
FROM table_stats_view v, tbls t
WHERE t.tbl_id = v.tbl_id ORDER BY
cast(v.total_size as int) DESC LIMIT 3;
12 © Hortonworks Inc. 2011–2018. All rights reserved
HIVE-1555: JDBC connector
• How did we build the information_schema?
• We mapped the metastore into Hive’s table space!
• Uses Hive-JDBC connector
• Read-only for now
• Supports automatic pushdown of full subqueries
• Cost-based optimizer decides part of query runs in
RDBMS versus Hive
• Joins, aggregates, filters, projections, etc
13 © Hortonworks Inc. 2011–2018. All rights reserved
JDBC Table mapping example
CREATE TABLE postgres_table
(
id INT,
name varchar(20)
);
CREATE EXTERNAL TABLE hive_table
(
id INT,
name varchar(20)
)
STORED BY
'org.apache.hive.storage.jdbc.JdbcStorageHandler'
TBLPROPERTIES (
"hive.sql.database.type" = "POSTGRES",
"hive.sql.jdbc.driver"="org.postgresql.Driver",
"hive.sql.jdbc.url"="jdbc:postgresql://...",
"hive.sql.dbcp.username"="jdbctest",
"hive.sql.dbcp.password"="",
"hive.sql.query"="select * from postgres_table",
"hive.sql.column.mapping" = "id=ID, name=NAME"
);
In Postgres In Hive
14 © Hortonworks Inc. 2011–2018. All rights reserved
Druid Connector
Realtime Node
Realtime Node
Realtime Node
Broker HiveServer2
Instantly analyze kafka data with milliseconds latency
15 © Hortonworks Inc. 2011–2018. All rights reserved
Druid Connector - Joins between Hive and realtime data in Druid
Bloom filter pushdown greatly reduces data transfer
Send promotional email to all customers from CA who purchased more than 1000$ worth of merchandise today.
create external table sales(`__time` timestamp, quantity int, sales_price double,customer_id bigint, item_id int, store_id int)
stored by 'org.apache.hadoop.hive.druid.DruidStorageHandler'
tblproperties ( "kafka.bootstrap.servers" = "localhost:9092", "kafka.topic" = "sales-topic",
"druid.kafka.ingestion.maxRowsInMemory" = "5");
create table customers (customer_id bigint, first_name string, last_name string, email string, state string);
select email from customers join sales using customer_id where to_date(sales.__time) = date ‘2018-09-06’
and quantity * sales_price > 1000 and customers.state = ‘CA’;
16 © Hortonworks Inc. 2011–2018. All rights reserved
Kafka Connector
LLAP Node
LLAP Node
LLAP Node
Query
Coordinator
HiveServer2
Ad-hoc / Ingest / Transform
17 © Hortonworks Inc. 2011–2018. All rights reserved
Kafka connector
Transformation over stream in real time
I want to have moving average over sliding window in kafka from stock ticker kafka stream.
create external table
tickers (`__time` timestamp , stock_id bigint, stock_sym varchar(4), price decimal (10,2), exhange_id int)
stored by 'org.apache.hadoop.hive.kafka.KafkaStorageHandler’
tblproperties ("kafka.topic" = "stock-topic", "kafka.bootstrap.servers"="localhost:9092",
"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe");
create external table
moving_avg (`__time` timestamp , stock_id bigint, avg_price decimal (10,2)
stored by 'org.apache.hadoop.hive.kafka.KafkaStorageHandler'
tblproperties ("kafka.topic" = "averages-topic", "kafka.bootstrap.servers"="localhost:9092",
"kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe");
Insert into table moving_avg select CURRENT_TIMESTAMP, stock_id, avg(price) group by stock_id,
from tickers where __timestamp > to_unix_timestamp(CURRENT_TIMESTAMP - 5 minutes) * 1000
© Hortonworks Inc. 2011- 2018. All rights reserved | 18
Table types
19 © Hortonworks Inc. 2011–2018. All rights reserved
Managed and External Tables
• Hive 3 cleans up semantics of managed and external tables
• External: Outside control and management of data
• Managed: Fully under Hive control, ACID only
• Non-native tables are external
• ACID: Full IUD on ORC, Insert-only on other formats
• Defaults have changed
• Managed: ORC + ACID
• External: TextFile
• Two tablespaces with different permissions & ownership
20 © Hortonworks Inc. 2011–2018. All rights reserved
Differences between external and managed tables
• Storage based auth (doAs=true) is supported for external tables
• Ranger and SBA can co-exist in HDP 3 (Ranger is default)
• Script to convert from file permissions to Ranger policies on tables
Note: SBA in HDP 3 requires ACL in HDFS. ACL is turned on by default in HDP3
Hive managed table
ACID on by default
No SBA, Ranger auth only
Statistics and other optimizations apply
Spark access via
HiveWarehouseConnector
External tables
No ACID, Text by default
SBA possible
Some optimizations unavailable
Spark direct file access
21 © Hortonworks Inc. 2011–2018. All rights reserved
ACID v2
V1: CREATE TABLE hello_acid (load_date date, key int, value int)
CLUSTERED BY(key) INTO 3 BUCKETS
STORED AS ORC TBLPROPERTIES ('transactional'='true');
V2: CREATE TABLE hello_acid_v2 (load_date date, key int, value int);
• Performance just as good as non-ACID tables
• No bucketing required
• Fully compatible with native cloud storage
© Hortonworks Inc. 2011- 2018. All rights reserved | 22
SQL Enhancements
23 © Hortonworks Inc. 2011–2018. All rights reserved
Materialized view
Optimizing workloads and queries without changing the SQL
SELECT distinct dest,origin
FROM flights;
SELECT origin, count(*)
FROM flights
GROUP BY origin
HAVING origin = ‘OAK’;
CREATE MATERIALIZED VIEW flight_agg
AS
SELECT dest,origin,count(*)
FROM flights
GROUP BY dest,origin;
24 © Hortonworks Inc. 2011–2018. All rights reserved
Materialized view - Maintenance
• Partial table rewrites are supported
• Typical: Denormalize last month of data only
• Rewrite engine will produce union of latest and historical data
• Updates to base tables
• Invalidates views, but
• Can choose to allow stale views (max staleness) for performance
• Can partial match views and compute delta after updates
• Incremental updates
• Common classes of views allow for incremental updates
• Others need full refresh
25 © Hortonworks Inc. 2011–2018. All rights reserved
Constraints & defaults
• Helps optimizer to produce better plans
• BI tool integrations
• Data Integrity
• hive.constraint.notnull.enforce = true
• SQL compatibility & offload scenarios
Example:
CREATE TABLE Persons (
ID Int NOT NULL,
Name String NOT NULL,
Age Int,
Creator String DEFAULT CURRENT_USER(),
CreateDate Date DEFAULT CURRENT_DATE(),
PRIMARY KEY (ID) DISABLE NOVALIDATE
);
CREATE TABLE BusinessUnit (
ID Int NOT NULL,
Head Int NOT NULL,
Creator String DEFAULT CURRENT_USER(),
CreateDate Date DEFAULT CURRENT_DATE(),
PRIMARY KEY (ID) DISABLE NOVALIDATE,
CONSTRAINT fk FOREIGN KEY (Head)
REFERENCES Persons(ID) DISABLE
NOVALIDATE
);
26 © Hortonworks Inc. 2011–2018. All rights reserved
Default clause & Surrogate keys
• Multiple approaches
• Sequence number is dense & increasing, but: Bottleneck in distributed DBMS
• UUID is easy & distributable, but large and slow
• Surrogate key UDF is easy & distributable & fast, but: No sequence and has gaps
CREATE TABLE AIRLINES_V2
(ID BIGINT DEFAULT SURROGATE_KEY(),
CODE STRING,
DESCRIPTION STRING,
PRIMARY KEY (ID) DISABLE NOVALIDATE);
INSERT INTO AIRLINES_V2 (CODE, DESCRIPTION) SELECT * FROM AIRLINES;
ALTER TABLE FLIGHTS ADD COLUMNS (carrier_sk BIGINT);
MERGE INTO FLIGHTS f USING AIRLINES_V2 a ON f.uniquecarrier = a.code
WHEN MATCHED THEN UPDATE SET carrier_sk = a.id;
27 © Hortonworks Inc. 2011–2018. All rights reserved
⬢ Solution
● Query fails because of stats estimation error
● Runtime sends observed statistics back to
coordinator
● Statistics overrides are created at session, server
or global level
● Query is replanned and resubmitted
Hive-17626: Optimizer is learning from planning mistakes
⬢ Symptoms
● Memory exhaustion due to under
provisioning
● Excessive runtime (future)
● Excessive spilling (future)
© Hortonworks Inc. 2011- 2018. All rights reserved | 28
Multitenancy
29 © Hortonworks Inc. 2011–2018. All rights reserved
HIVE-17481: LLAP workload management
⬢ Effectively share LLAP cluster resources
– Resource allocation per user policy; separate ETL and BI, etc.
⬢ Resources based guardrails
– Protect against long running queries, high memory usage
⬢ Improved, query-aware scheduling
– Scheduler is aware of query characteristics, types, etc.
– Fragments easy to pre-empt compared to containers
– Queries get guaranteed fractions of the cluster, but can use
empty space
30 © Hortonworks Inc. 2011–2018. All rights reserved
Common Triggers
● ELAPSED_TIME
● EXECUTION_TIME
● TOTAL_TASKS
● HDFS_BYTES_READ, HDFS_BYTES_WRITTEN
● CREATED FILES
● CREATED_DYNAMIC_PARTITIONS
Example
CREATE RESOURCE PLAN guardrail;
CREATE TRIGGER guardrail.long_running WHEN EXECUTION_TIME > 2000 DO KILL;
ALTER TRIGGER guardrail.long_running ADD TO UNMANAGED;
ALTER RESOURCE PLAN guardrail ENABLE ACTIVATE;
Guardrail Example
31 © Hortonworks Inc. 2011–2018. All rights reserved
Resource plans example
CREATE RESOURCE PLAN daytime;
CREATE POOL daytime.bi WITH ALLOC_FRACTION=0.8, QUERY_PARALLELISM=5;
CREATE POOL daytime.etl WITH ALLOC_FRACTION=0.2, QUERY_PARALLELISM=20;
CREATE RULE downgrade IN daytime WHEN total_runtime > 3000 THEN MOVE etl;
ADD RULE downgrade TO bi;
CREATE APPLICATION MAPPING tableau in daytime TO bi;
ALTER PLAN daytime SET default pool= etl;
APPLY PLAN daytime;
daytime
bi: 80% etl: 20%
Downgrade when total_runtime>3000
© Hortonworks Inc. 2011- 2018. All rights reserved | 32
BI caching
33 © Hortonworks Inc. 2011–2018. All rights reserved
HIVE-18513: Query result cache
Returns results directly from storage (e.g.
HDFS) without actually executing the query
If
the same query had ran before
Important for dashboards, reports etc.
where repetitive queries is common
Without cache
With cache
34 © Hortonworks Inc. 2011–2018. All rights reserved
HIVE-18513: Query result cache details
• hive.query.results.cache.enabled=true (on by default)
• Works only on hive managed tables
• If you JOIN an external table with Hive managed table, Hive will fall back to executing the full
query. Because Hive can’t know if external table data has changed
• Works with ACID
• That means if Hive table has been updated, the query will be rerun automatically
• Is different from LLAP cache
• LLAP cache is a data cache. That means multiple queries can benefit by avoiding reading from disk.
Speeds up the read path.
• Result cache effectively bypasses execution of query
• Stored at /tmp/hive/__resultcache__/, default space is 2GB, LRU eviction
• Tunable setting hive.query.results.cache.max.size (bytes)
35 © Hortonworks Inc. 2011–2018. All rights reserved
Metastore Cache
• With query execution time being < 1 sec, compilation time starts to dominate
• Metadata retrieval is often significant part of compilation time. Most of it is in RDBMS
queries.
• Cloud RDBMS As a Service is often slower, and frequent queries leads to throttling.
• Metadata cache speeds compilation time by around 50% with onprem mysql.
Significantly more improvement with cloud RDBMS.
• Cache is consistent in single metastore setup, eventually consistent with HA setup.
Consistent HA setup support is in the works.
© Hortonworks Inc. 2011- 2018. All rights reserved | 36
Phew. That was a lot.
37 © Hortonworks Inc. 2011–2018. All rights reserved
Hive 3 feature summary
⬢ EDW offload
– Surrogate key and constraints
– Information schema
– Materialized views
⬢ Perf
– Query result & metastore caches
– LLAP workload management
⬢ Real-time capabilities with Kafka
– Ingest in ACID tables
– Instantly query using Druid
⬢ Unified SQL
– JDBC connector
– Druid connector
– Spark-hive connector
⬢ Table types
– ACID v2 and on by default
– External v Managed
⬢ Cloud
– AWS/GCP/Azure cloud storage
natively supported now
© Hortonworks Inc. 2011- 2018. All rights reserved | 38
Spark-Hive connect
39 © Hortonworks Inc. 2011–2018. All rights reserved
Hive is broadening its reach
SQL over Hadoop
• External tables only
• No ACID
• No LLAP
• SBA OK (doAs=True)
• Some perf penalty
Hive as EDW
• ACID
• LLAP
• doAs=False
• Managed tables
• Column-level security
• Stats, better perf etc.
External table/
Direct
Hive Warehouse
Connector
Spark
45 © Hortonworks Inc. 2011–2018. All rights reserved
Driver
MetaStore
HiveServer2
LLAP DaemonsExecutors
Spark
Meta
Hive
Meta
Executors LLAP Daemons
Isolate Spark and Hive Catalogs/Tables
Leverage connector for Spark <-> Hive
Uses Apache Arrow for fast data transfer
HWC
HWC
47 © Hortonworks Inc. 2011–2018. All rights reserved
Driver
MetaStore
HiveServer2
LLAP DaemonsExecutors
Spark
Meta
Hive
Meta
HWC (Thrift JDBC)
Executors LLAP Daemons
a) hive.executeUpdate(“INSERT INTO s SELECT * FROM t”)
1. Driver submits update op to HiveServer2
2. Process update through Tez and/or LLAP
3. HWC returns true on success
1
2
3
49 © Hortonworks Inc. 2011–2018. All rights reserved
Driver
MetaStore
HiveServer2
LLAP DaemonsExecutors
Spark
Meta
Hive
Meta
Executors LLAP Daemons
b) df.write.format(HIVE_WAREHOUSE_CONNECTOR).save()
1.Driver launches DataWriter tasks
2.Tasks write ORC files
3.On commit, Driver executes LOAD DATA INTO TABLE
HDFS
/tmp
1
2
3
ACID
Tables
51 © Hortonworks Inc. 2011–2018. All rights reserved
Driver
MetaStore
HiveServer+Tez
Executors
Spark
Meta
Hive
Meta
Executors
c) df.write.format(STREAM_TO_STREAM).start()
1.Driver launches DataWriter tasks
2.Tasks open Txns
3.Write rows to ACID tables in Tx
ACID
Tables1
2
3
© Hortonworks Inc. 2011- 2018. All rights reserved | 52
Performance
53 © Hortonworks Inc. 2011–2018. All rights reserved
• Ran all 99 TPCDS queries
• Total query runtime have improved multifold in each release!
Benchmark journey
TPCDS 10TB scale on 10 node cluster
HDP 2.5
Hive1
HDP 2.5
LLAP
HDP 2.6
LLAP
25x 3x 2x
HDP 3.0
LLAP
2016 20182017
ACID
tables
54 © Hortonworks Inc. 2011–2018. All rights reserved
• Faster analytical queries with improved vectorization in HDP 3.0
• Vectorized execution of PTF, rollup and grouping sets.
• Perf gain compared to HDP 2.6
• TPCDS query67 ~ 10x!
• TPCDS query36 ~ 30x!
• TPCDS query27 ~ 20x!
OLAP Vectorization
55 © Hortonworks Inc. 2011–2018. All rights reserved
SELECT * FROM
( SELECT AVG(ss_list_price) B1_LP,
COUNT(ss_list_price) B1_CNT ,COUNT(DISTINCT
ss_list_price) B1_CNTD
FROM store_sales
WHERE ss_quantity BETWEEN 0 AND 5 AND
(ss_list_price BETWEEN 11 and 11+10 OR
ss_coupon_amt BETWEEN 460 and 460+1000 OR
ss_wholesale_cost BETWEEN 14 and 14+20)) B1,
( SELECT AVG(ss_list_price) B2_LP,
COUNT(ss_list_price) B2_CNT ,COUNT(DISTINCT
ss_list_price) B2_CNTD
FROM store_sales
WHERE ss_quantity BETWEEN 6 AND 10 AND
(ss_list_price BETWEEN 91 and 91+10 OR
ss_coupon_amt BETWEEN 1430 and 1430+1000 OR
ss_wholesale_cost BETWEEN 32 and 32+20)) B2,
. . .
LIMIT 100;
TPCDS SQL query 28 joins 6 instances of store_sales table
Shared scan - 4x improvement!
RS RS RS RS RS
Scan
store_sales
Combined OR’ed B1-B6 Filters
B1 Filter B2 Filter B3 Filter B4 Filter B5 Filter
Join
56 © Hortonworks Inc. 2011–2018. All rights reserved
• Dramatically improves performance of very selective joins
• Builds a bloom filter from one side of join and filters rows from other side
• Skips scan and further evaluation of rows that would not qualify the join
Dynamic Semijoin Reduction - 7x improvement for q72
SELECT …
FROM sales JOIN time ON
sales.time_id = time.time_id
WHERE time.year = 2014 AND
time.quarter IN ('Q1', 'Q2’)
Reduced scan on sales
© Hortonworks Inc. 2011- 2018. All rights reserved | 57
We’ve made it! Questions?
© Hortonworks Inc. 2011- 2018. All rights reserved | 58
Oh. One more thing.
59 © Hortonworks Inc. 2011–2018. All rights reserved
⬢ Hive on Kubernetes solves:
– Hive/LLAP side install (to main cluster)
– Multiple versions of Hive
– Multiple warehouse & compute instances
– Dynamic configuration and secrets management
– Stateful and work preserving restarts (cache)
– Rolling restart for upgrades. Fast rollback to
previous good state.
Hive on Kubernetes (WIP)
Kubernetes Hosting Environments
AWS GCP
Data OS
CPU / MEMORY /
STORAGE
OPENSHIFTAZURE
CLOUD PROVIDERS
ON-
PREM/HYB
RID
DATA PLANE SERVICES
Cluster Lifecycle Manager Data Analytics Studio (DAS) Organizational Services
COMPUTE CLUSTER
SHARED SERVICES
Ranger
Atlas
Metastore
Tiller API Server
DAS Web Service
Query Coordinators
Query Executors
Registry
Blobstore
Indexe
r
RDBMS
Hive Server
Long-running kubernetes cluster
Inter-cluster communication Intra-cluster communication
Ingress Controller or
Load Balancer
Internal Service Endpoint for
ReplicaSet or StatefulSet
Ephemeral kubernetes cluster

More Related Content

What's hot

Hive tuning
Hive tuningHive tuning
Hive tuning
Michael Zhang
 
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
 
Intro to HBase
Intro to HBaseIntro to HBase
Intro to HBase
alexbaranau
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0
Cloudera, Inc.
 
Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing
DataWorks Summit
 
Druid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best PracticesDruid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best Practices
DataWorks Summit
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
Databricks
 
Building robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and DebeziumBuilding robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and Debezium
Tathastu.ai
 
Achieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on TezAchieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on Tez
DataWorks Summit/Hadoop Summit
 
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
StampedeCon
 
ORC improvement in Apache Spark 2.3
ORC improvement in Apache Spark 2.3ORC improvement in Apache Spark 2.3
ORC improvement in Apache Spark 2.3
DataWorks Summit
 
Securing Hadoop with Apache Ranger
Securing Hadoop with Apache RangerSecuring Hadoop with Apache Ranger
Securing Hadoop with Apache Ranger
DataWorks Summit
 
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
DataWorks Summit
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
DataWorks Summit
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxData
 
Apache Phoenix and HBase: Past, Present and Future of SQL over HBase
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 HBase
DataWorks Summit/Hadoop Summit
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
Cloudera, Inc.
 
Apache hive introduction
Apache hive introductionApache hive introduction
Apache hive introduction
Mahmood Reza Esmaili Zand
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
Databricks
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
DataWorks Summit
 

What's hot (20)

Hive tuning
Hive tuningHive tuning
Hive tuning
 
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...
 
Intro to HBase
Intro to HBaseIntro to HBase
Intro to HBase
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0
 
Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing Apache Tez: Accelerating Hadoop Query Processing
Apache Tez: Accelerating Hadoop Query Processing
 
Druid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best PracticesDruid and Hive Together : Use Cases and Best Practices
Druid and Hive Together : Use Cases and Best Practices
 
The Apache Spark File Format Ecosystem
The Apache Spark File Format EcosystemThe Apache Spark File Format Ecosystem
The Apache Spark File Format Ecosystem
 
Building robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and DebeziumBuilding robust CDC pipeline with Apache Hudi and Debezium
Building robust CDC pipeline with Apache Hudi and Debezium
 
Achieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on TezAchieving 100k Queries per Hour on Hive on Tez
Achieving 100k Queries per Hour on Hive on Tez
 
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
Choosing an HDFS data storage format- Avro vs. Parquet and more - StampedeCon...
 
ORC improvement in Apache Spark 2.3
ORC improvement in Apache Spark 2.3ORC improvement in Apache Spark 2.3
ORC improvement in Apache Spark 2.3
 
Securing Hadoop with Apache Ranger
Securing Hadoop with Apache RangerSecuring Hadoop with Apache Ranger
Securing Hadoop with Apache Ranger
 
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
Interactive real-time dashboards on data streams using Kafka, Druid, and Supe...
 
What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?What is new in Apache Hive 3.0?
What is new in Apache Hive 3.0?
 
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
InfluxDB IOx Tech Talks: Query Engine Design and the Rust-Based DataFusion in...
 
Apache Phoenix and HBase: Past, Present and Future of SQL over HBase
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 HBase
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Apache hive introduction
Apache hive introductionApache hive introduction
Apache hive introduction
 
Intro to Delta Lake
Intro to Delta LakeIntro to Delta Lake
Intro to Delta Lake
 
Apache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data ProcessingApache Tez - A New Chapter in Hadoop Data Processing
Apache Tez - A New Chapter in Hadoop Data Processing
 

Similar to Hive 3 - a new horizon

Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizon
Artem Ervits
 
Fast SQL on Hadoop, really?
Fast SQL on Hadoop, really?Fast SQL on Hadoop, really?
Fast SQL on Hadoop, really?
DataWorks Summit
 
What's New in Apache Hive 3.0 - Tokyo
What's New in Apache Hive 3.0 - TokyoWhat's New in Apache Hive 3.0 - Tokyo
What's New in Apache Hive 3.0 - Tokyo
DataWorks Summit
 
What's New in Apache Hive 3.0?
What's New in Apache Hive 3.0?What's New in Apache Hive 3.0?
What's New in Apache Hive 3.0?
DataWorks Summit
 
What is New in Apache Hive 3.0?
What is New in Apache Hive 3.0?What is New in Apache Hive 3.0?
What is New in Apache Hive 3.0?
DataWorks Summit
 
Hive 3 New Horizons DataWorks Summit Melbourne February 2019
Hive 3 New Horizons DataWorks Summit Melbourne February 2019Hive 3 New Horizons DataWorks Summit Melbourne February 2019
Hive 3 New Horizons DataWorks Summit Melbourne February 2019
alanfgates
 
What's New in Apache Hive
What's New in Apache HiveWhat's New in Apache Hive
What's New in Apache Hive
DataWorks Summit
 
Hive Performance Dataworks Summit Melbourne February 2019
Hive Performance Dataworks Summit Melbourne February 2019Hive Performance Dataworks Summit Melbourne February 2019
Hive Performance Dataworks Summit Melbourne February 2019
alanfgates
 
Fast SQL on Hadoop, Really?
Fast SQL on Hadoop, Really?Fast SQL on Hadoop, Really?
Fast SQL on Hadoop, Really?
DataWorks Summit
 
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
HortonworksJapan
 
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data PlatformModernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
Hortonworks
 
SQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for ImpalaSQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for Impala
markgrover
 
Hortonworks Technical Workshop: What's New in HDP 2.3
Hortonworks Technical Workshop: What's New in HDP 2.3Hortonworks Technical Workshop: What's New in HDP 2.3
Hortonworks Technical Workshop: What's New in HDP 2.3
Hortonworks
 
Cloudera Operational DB (Apache HBase & Apache Phoenix)
Cloudera Operational DB (Apache HBase & Apache Phoenix)Cloudera Operational DB (Apache HBase & Apache Phoenix)
Cloudera Operational DB (Apache HBase & Apache Phoenix)
Timothy Spann
 
Interactive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using DruidInteractive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using Druid
DataWorks Summit
 
An Apache Hive Based Data Warehouse
An Apache Hive Based Data WarehouseAn Apache Hive Based Data Warehouse
An Apache Hive Based Data Warehouse
DataWorks Summit
 
Hive edw-dataworks summit-eu-april-2017
Hive edw-dataworks summit-eu-april-2017Hive edw-dataworks summit-eu-april-2017
Hive edw-dataworks summit-eu-april-2017
alanfgates
 
Hp Converged Systems and Hortonworks - Webinar Slides
Hp Converged Systems and Hortonworks - Webinar SlidesHp Converged Systems and Hortonworks - Webinar Slides
Hp Converged Systems and Hortonworks - Webinar Slides
Hortonworks
 
Sharing metadata across the data lake and streams
Sharing metadata across the data lake and streamsSharing metadata across the data lake and streams
Sharing metadata across the data lake and streams
DataWorks Summit
 
Standalone metastore-dws-sjc-june-2018
Standalone metastore-dws-sjc-june-2018Standalone metastore-dws-sjc-june-2018
Standalone metastore-dws-sjc-june-2018
alanfgates
 

Similar to Hive 3 - a new horizon (20)

Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizon
 
Fast SQL on Hadoop, really?
Fast SQL on Hadoop, really?Fast SQL on Hadoop, really?
Fast SQL on Hadoop, really?
 
What's New in Apache Hive 3.0 - Tokyo
What's New in Apache Hive 3.0 - TokyoWhat's New in Apache Hive 3.0 - Tokyo
What's New in Apache Hive 3.0 - Tokyo
 
What's New in Apache Hive 3.0?
What's New in Apache Hive 3.0?What's New in Apache Hive 3.0?
What's New in Apache Hive 3.0?
 
What is New in Apache Hive 3.0?
What is New in Apache Hive 3.0?What is New in Apache Hive 3.0?
What is New in Apache Hive 3.0?
 
Hive 3 New Horizons DataWorks Summit Melbourne February 2019
Hive 3 New Horizons DataWorks Summit Melbourne February 2019Hive 3 New Horizons DataWorks Summit Melbourne February 2019
Hive 3 New Horizons DataWorks Summit Melbourne February 2019
 
What's New in Apache Hive
What's New in Apache HiveWhat's New in Apache Hive
What's New in Apache Hive
 
Hive Performance Dataworks Summit Melbourne February 2019
Hive Performance Dataworks Summit Melbourne February 2019Hive Performance Dataworks Summit Melbourne February 2019
Hive Performance Dataworks Summit Melbourne February 2019
 
Fast SQL on Hadoop, Really?
Fast SQL on Hadoop, Really?Fast SQL on Hadoop, Really?
Fast SQL on Hadoop, Really?
 
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
Hive 3.0 - HDPの最新バージョンで実現する新機能とパフォーマンス改善
 
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data PlatformModernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
Modernize Your Existing EDW with IBM Big SQL & Hortonworks Data Platform
 
SQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for ImpalaSQL Engines for Hadoop - The case for Impala
SQL Engines for Hadoop - The case for Impala
 
Hortonworks Technical Workshop: What's New in HDP 2.3
Hortonworks Technical Workshop: What's New in HDP 2.3Hortonworks Technical Workshop: What's New in HDP 2.3
Hortonworks Technical Workshop: What's New in HDP 2.3
 
Cloudera Operational DB (Apache HBase & Apache Phoenix)
Cloudera Operational DB (Apache HBase & Apache Phoenix)Cloudera Operational DB (Apache HBase & Apache Phoenix)
Cloudera Operational DB (Apache HBase & Apache Phoenix)
 
Interactive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using DruidInteractive Analytics at Scale in Apache Hive Using Druid
Interactive Analytics at Scale in Apache Hive Using Druid
 
An Apache Hive Based Data Warehouse
An Apache Hive Based Data WarehouseAn Apache Hive Based Data Warehouse
An Apache Hive Based Data Warehouse
 
Hive edw-dataworks summit-eu-april-2017
Hive edw-dataworks summit-eu-april-2017Hive edw-dataworks summit-eu-april-2017
Hive edw-dataworks summit-eu-april-2017
 
Hp Converged Systems and Hortonworks - Webinar Slides
Hp Converged Systems and Hortonworks - Webinar SlidesHp Converged Systems and Hortonworks - Webinar Slides
Hp Converged Systems and Hortonworks - Webinar Slides
 
Sharing metadata across the data lake and streams
Sharing metadata across the data lake and streamsSharing metadata across the data lake and streams
Sharing metadata across the data lake and streams
 
Standalone metastore-dws-sjc-june-2018
Standalone metastore-dws-sjc-june-2018Standalone metastore-dws-sjc-june-2018
Standalone metastore-dws-sjc-june-2018
 

Recently uploaded

Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
OnePlan Solutions
 
CrushFTP 10.4.0.29 PC Software - WhizNews
CrushFTP 10.4.0.29 PC Software - WhizNewsCrushFTP 10.4.0.29 PC Software - WhizNews
CrushFTP 10.4.0.29 PC Software - WhizNews
Eman Nisar
 
AI-driven Automation_ Transforming DevOps Practices.docx
AI-driven Automation_ Transforming DevOps Practices.docxAI-driven Automation_ Transforming DevOps Practices.docx
AI-driven Automation_ Transforming DevOps Practices.docx
zoondiacom
 
Unlocking the Future of Artificial Intelligence
Unlocking the Future of Artificial IntelligenceUnlocking the Future of Artificial Intelligence
Unlocking the Future of Artificial Intelligence
dorinIonescu
 
How to Secure Your Kubernetes Software Supply Chain at Scale
How to Secure Your Kubernetes Software Supply Chain at ScaleHow to Secure Your Kubernetes Software Supply Chain at Scale
How to Secure Your Kubernetes Software Supply Chain at Scale
Anchore
 
02. Ruby Basic slides - Ruby Core Teaching
02. Ruby Basic slides - Ruby Core Teaching02. Ruby Basic slides - Ruby Core Teaching
02. Ruby Basic slides - Ruby Core Teaching
quanhoangd129
 
The Politics of Agile Development.pptx
The  Politics of  Agile Development.pptxThe  Politics of  Agile Development.pptx
The Politics of Agile Development.pptx
NMahendiran
 
07. Ruby String Slides - Ruby Core Teaching
07. Ruby String Slides - Ruby Core Teaching07. Ruby String Slides - Ruby Core Teaching
07. Ruby String Slides - Ruby Core Teaching
quanhoangd129
 
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
John Gallagher
 
Literals - A Machine Independent Feature
Literals - A Machine Independent FeatureLiterals - A Machine Independent Feature
Literals - A Machine Independent Feature
21h16charis
 
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
vijayatibirds
 
vSAN_Tutorial_Presentation with important topics
vSAN_Tutorial_Presentation with important  topicsvSAN_Tutorial_Presentation with important  topics
vSAN_Tutorial_Presentation with important topics
abhilashspt
 
Fantastic Design Patterns and Where to use them No Notes.pdf
Fantastic Design Patterns and Where to use them No Notes.pdfFantastic Design Patterns and Where to use them No Notes.pdf
Fantastic Design Patterns and Where to use them No Notes.pdf
6m9p7qnjj8
 
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
Andre Hora
 
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
Shane Coughlan
 
Top 10 ERP Companies in UAE Banibro IT Solutions.pdf
Top 10 ERP Companies in UAE Banibro IT Solutions.pdfTop 10 ERP Companies in UAE Banibro IT Solutions.pdf
Top 10 ERP Companies in UAE Banibro IT Solutions.pdf
Banibro IT Solutions
 
03. Ruby Variables & Regex - Ruby Core Teaching
03. Ruby Variables & Regex - Ruby Core Teaching03. Ruby Variables & Regex - Ruby Core Teaching
03. Ruby Variables & Regex - Ruby Core Teaching
quanhoangd129
 
B.Sc. Computer Science Department PPT 2024
B.Sc. Computer Science Department PPT 2024B.Sc. Computer Science Department PPT 2024
B.Sc. Computer Science Department PPT 2024
vmsdeptcom
 
05. Ruby Control Structures - Ruby Core Teaching
05. Ruby Control Structures - Ruby Core Teaching05. Ruby Control Structures - Ruby Core Teaching
05. Ruby Control Structures - Ruby Core Teaching
quanhoangd129
 
Fixing Git Catastrophes - Nebraska.Code()
Fixing Git Catastrophes - Nebraska.Code()Fixing Git Catastrophes - Nebraska.Code()
Fixing Git Catastrophes - Nebraska.Code()
Gene Gotimer
 

Recently uploaded (20)

Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
Bring Strategic Portfolio Management to Monday.com using OnePlan - Webinar 18...
 
CrushFTP 10.4.0.29 PC Software - WhizNews
CrushFTP 10.4.0.29 PC Software - WhizNewsCrushFTP 10.4.0.29 PC Software - WhizNews
CrushFTP 10.4.0.29 PC Software - WhizNews
 
AI-driven Automation_ Transforming DevOps Practices.docx
AI-driven Automation_ Transforming DevOps Practices.docxAI-driven Automation_ Transforming DevOps Practices.docx
AI-driven Automation_ Transforming DevOps Practices.docx
 
Unlocking the Future of Artificial Intelligence
Unlocking the Future of Artificial IntelligenceUnlocking the Future of Artificial Intelligence
Unlocking the Future of Artificial Intelligence
 
How to Secure Your Kubernetes Software Supply Chain at Scale
How to Secure Your Kubernetes Software Supply Chain at ScaleHow to Secure Your Kubernetes Software Supply Chain at Scale
How to Secure Your Kubernetes Software Supply Chain at Scale
 
02. Ruby Basic slides - Ruby Core Teaching
02. Ruby Basic slides - Ruby Core Teaching02. Ruby Basic slides - Ruby Core Teaching
02. Ruby Basic slides - Ruby Core Teaching
 
The Politics of Agile Development.pptx
The  Politics of  Agile Development.pptxThe  Politics of  Agile Development.pptx
The Politics of Agile Development.pptx
 
07. Ruby String Slides - Ruby Core Teaching
07. Ruby String Slides - Ruby Core Teaching07. Ruby String Slides - Ruby Core Teaching
07. Ruby String Slides - Ruby Core Teaching
 
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
Fix Production Bugs Quickly - The Power of Structured Logging in Ruby on Rail...
 
Literals - A Machine Independent Feature
Literals - A Machine Independent FeatureLiterals - A Machine Independent Feature
Literals - A Machine Independent Feature
 
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
iBirds Services - Comprehensive Salesforce CRM and Software Development Solut...
 
vSAN_Tutorial_Presentation with important topics
vSAN_Tutorial_Presentation with important  topicsvSAN_Tutorial_Presentation with important  topics
vSAN_Tutorial_Presentation with important topics
 
Fantastic Design Patterns and Where to use them No Notes.pdf
Fantastic Design Patterns and Where to use them No Notes.pdfFantastic Design Patterns and Where to use them No Notes.pdf
Fantastic Design Patterns and Where to use them No Notes.pdf
 
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
Monitoring the Execution of 14K Tests: Methods Tend to Have One Path that Is ...
 
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
OpenChain Webinar: IAV, TimeToAct and ISO/IEC 5230 - Third-Party Certificatio...
 
Top 10 ERP Companies in UAE Banibro IT Solutions.pdf
Top 10 ERP Companies in UAE Banibro IT Solutions.pdfTop 10 ERP Companies in UAE Banibro IT Solutions.pdf
Top 10 ERP Companies in UAE Banibro IT Solutions.pdf
 
03. Ruby Variables & Regex - Ruby Core Teaching
03. Ruby Variables & Regex - Ruby Core Teaching03. Ruby Variables & Regex - Ruby Core Teaching
03. Ruby Variables & Regex - Ruby Core Teaching
 
B.Sc. Computer Science Department PPT 2024
B.Sc. Computer Science Department PPT 2024B.Sc. Computer Science Department PPT 2024
B.Sc. Computer Science Department PPT 2024
 
05. Ruby Control Structures - Ruby Core Teaching
05. Ruby Control Structures - Ruby Core Teaching05. Ruby Control Structures - Ruby Core Teaching
05. Ruby Control Structures - Ruby Core Teaching
 
Fixing Git Catastrophes - Nebraska.Code()
Fixing Git Catastrophes - Nebraska.Code()Fixing Git Catastrophes - Nebraska.Code()
Fixing Git Catastrophes - Nebraska.Code()
 

Hive 3 - a new horizon

  • 1. 1 © Hortonworks Inc. 2011–2018. All rights reserved © Hortonworks Inc. 2011 – 2017 Apache Hive 3: A new horizon Gunther Hagleitner, Ashutosh Chauhan, Gopal Vijayaraghavan, Thejas Nair, Will Xu
  • 2. 2 © Hortonworks Inc. 2011–2018. All rights reserved 7000 analysts, 80ms average latency, 1PB data. 250k BI queries per hour On demand deep reporting in the cloud over 100Tb in minutes.
  • 3. © Hortonworks Inc. 2011- 2018. All rights reserved | 3 Agenda ● Data Analytics Studio ● Apache Hive 3 ● Hive-Spark interoperability ● Performance ● Look ahead
  • 4. © Hortonworks Inc. 2011- 2018. All rights reserved | 4 Data Analytics Studio
  • 5. © Hortonworks Inc. 2011- 2018. All rights reserved | 5 Self-service question #1: Why is my query slow? Noisy neighbors Poor schema Inefficient queries Unstable demand Smart query log search Storage Optimizations Query Optimizations Demand Shifting Hortonworks Data Analytics Studio
  • 6. 7 © Hortonworks Inc. 2011–2018. All rights reserved One of the Extensible DataPlane Services ⬢ DAS 1.0 available now for HDP 3.0! ⬢ Monthly release cadence ⬢ Replaces Hive & Tez Views ⬢ Separate install from stack Hortonworks Data Analytics Studio HORTONWORKS DATAPLANE SERVICE DATA SOURCE INTEGRATION DATA SERVICES CATALOG …DATA LIFECYCLE MANAGER DATA STEWARD STUDIO +OTHER (partner) SECURITY CONTROLS CORE CAPABILITIES MULTIPLE CLUSTERS AND SOURCES MULTIHYBRID *not yet available, coming soon EXTENSIBLE SERVICES IBM DSX* DATA ANALYTICS STUDIO
  • 7. © Hortonworks Inc. 2011- 2018. All rights reserved | 8 Apache Hive 3
  • 8. 9 © Hortonworks Inc. 2011–2018. All rights reserved Hive3: EDW analyst pipeline Tableau BI systems Materialized view Surrogate key Constraints Query Result Cache Workload management • Results return from HDFS/cache directly • Reduce load from repetitive queries • Allows more queries to be run in parallel • Reduce resource starvation in large clusters • Active/Passive HA • More “tools” for optimizer to use • More ”tools” for DBAs to tune/optimize • Invisible tuning of DB from users’ perspective • ACID v2 is as fast as regular tables • Hive 3 is optimized for S3/WASB/GCP • Support for JDBC/Kafka/Druid out of the box ACID v2 Cloud Storage Connectors
  • 9. © Hortonworks Inc. 2011- 2018. All rights reserved | 10 Connectors
  • 10. 11 © Hortonworks Inc. 2011–2018. All rights reserved Hive-1010: Information schema & sysdb Question: Find which tables have a column with ‘ssn’ as part of the column name? use information_schema; SELECT table_schema, table_name FROM information_schema.columns WHERE column_name LIKE '%ssn%'; Question: Find the biggest tables in the system. use sys; SELECT tbl_name, total_size FROM table_stats_view v, tbls t WHERE t.tbl_id = v.tbl_id ORDER BY cast(v.total_size as int) DESC LIMIT 3;
  • 11. 12 © Hortonworks Inc. 2011–2018. All rights reserved HIVE-1555: JDBC connector • How did we build the information_schema? • We mapped the metastore into Hive’s table space! • Uses Hive-JDBC connector • Read-only for now • Supports automatic pushdown of full subqueries • Cost-based optimizer decides part of query runs in RDBMS versus Hive • Joins, aggregates, filters, projections, etc
  • 12. 13 © Hortonworks Inc. 2011–2018. All rights reserved JDBC Table mapping example CREATE TABLE postgres_table ( id INT, name varchar(20) ); CREATE EXTERNAL TABLE hive_table ( id INT, name varchar(20) ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( "hive.sql.database.type" = "POSTGRES", "hive.sql.jdbc.driver"="org.postgresql.Driver", "hive.sql.jdbc.url"="jdbc:postgresql://...", "hive.sql.dbcp.username"="jdbctest", "hive.sql.dbcp.password"="", "hive.sql.query"="select * from postgres_table", "hive.sql.column.mapping" = "id=ID, name=NAME" ); In Postgres In Hive
  • 13. 14 © Hortonworks Inc. 2011–2018. All rights reserved Druid Connector Realtime Node Realtime Node Realtime Node Broker HiveServer2 Instantly analyze kafka data with milliseconds latency
  • 14. 15 © Hortonworks Inc. 2011–2018. All rights reserved Druid Connector - Joins between Hive and realtime data in Druid Bloom filter pushdown greatly reduces data transfer Send promotional email to all customers from CA who purchased more than 1000$ worth of merchandise today. create external table sales(`__time` timestamp, quantity int, sales_price double,customer_id bigint, item_id int, store_id int) stored by 'org.apache.hadoop.hive.druid.DruidStorageHandler' tblproperties ( "kafka.bootstrap.servers" = "localhost:9092", "kafka.topic" = "sales-topic", "druid.kafka.ingestion.maxRowsInMemory" = "5"); create table customers (customer_id bigint, first_name string, last_name string, email string, state string); select email from customers join sales using customer_id where to_date(sales.__time) = date ‘2018-09-06’ and quantity * sales_price > 1000 and customers.state = ‘CA’;
  • 15. 16 © Hortonworks Inc. 2011–2018. All rights reserved Kafka Connector LLAP Node LLAP Node LLAP Node Query Coordinator HiveServer2 Ad-hoc / Ingest / Transform
  • 16. 17 © Hortonworks Inc. 2011–2018. All rights reserved Kafka connector Transformation over stream in real time I want to have moving average over sliding window in kafka from stock ticker kafka stream. create external table tickers (`__time` timestamp , stock_id bigint, stock_sym varchar(4), price decimal (10,2), exhange_id int) stored by 'org.apache.hadoop.hive.kafka.KafkaStorageHandler’ tblproperties ("kafka.topic" = "stock-topic", "kafka.bootstrap.servers"="localhost:9092", "kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe"); create external table moving_avg (`__time` timestamp , stock_id bigint, avg_price decimal (10,2) stored by 'org.apache.hadoop.hive.kafka.KafkaStorageHandler' tblproperties ("kafka.topic" = "averages-topic", "kafka.bootstrap.servers"="localhost:9092", "kafka.serde.class"="org.apache.hadoop.hive.serde2.JsonSerDe"); Insert into table moving_avg select CURRENT_TIMESTAMP, stock_id, avg(price) group by stock_id, from tickers where __timestamp > to_unix_timestamp(CURRENT_TIMESTAMP - 5 minutes) * 1000
  • 17. © Hortonworks Inc. 2011- 2018. All rights reserved | 18 Table types
  • 18. 19 © Hortonworks Inc. 2011–2018. All rights reserved Managed and External Tables • Hive 3 cleans up semantics of managed and external tables • External: Outside control and management of data • Managed: Fully under Hive control, ACID only • Non-native tables are external • ACID: Full IUD on ORC, Insert-only on other formats • Defaults have changed • Managed: ORC + ACID • External: TextFile • Two tablespaces with different permissions & ownership
  • 19. 20 © Hortonworks Inc. 2011–2018. All rights reserved Differences between external and managed tables • Storage based auth (doAs=true) is supported for external tables • Ranger and SBA can co-exist in HDP 3 (Ranger is default) • Script to convert from file permissions to Ranger policies on tables Note: SBA in HDP 3 requires ACL in HDFS. ACL is turned on by default in HDP3 Hive managed table ACID on by default No SBA, Ranger auth only Statistics and other optimizations apply Spark access via HiveWarehouseConnector External tables No ACID, Text by default SBA possible Some optimizations unavailable Spark direct file access
  • 20. 21 © Hortonworks Inc. 2011–2018. All rights reserved ACID v2 V1: CREATE TABLE hello_acid (load_date date, key int, value int) CLUSTERED BY(key) INTO 3 BUCKETS STORED AS ORC TBLPROPERTIES ('transactional'='true'); V2: CREATE TABLE hello_acid_v2 (load_date date, key int, value int); • Performance just as good as non-ACID tables • No bucketing required • Fully compatible with native cloud storage
  • 21. © Hortonworks Inc. 2011- 2018. All rights reserved | 22 SQL Enhancements
  • 22. 23 © Hortonworks Inc. 2011–2018. All rights reserved Materialized view Optimizing workloads and queries without changing the SQL SELECT distinct dest,origin FROM flights; SELECT origin, count(*) FROM flights GROUP BY origin HAVING origin = ‘OAK’; CREATE MATERIALIZED VIEW flight_agg AS SELECT dest,origin,count(*) FROM flights GROUP BY dest,origin;
  • 23. 24 © Hortonworks Inc. 2011–2018. All rights reserved Materialized view - Maintenance • Partial table rewrites are supported • Typical: Denormalize last month of data only • Rewrite engine will produce union of latest and historical data • Updates to base tables • Invalidates views, but • Can choose to allow stale views (max staleness) for performance • Can partial match views and compute delta after updates • Incremental updates • Common classes of views allow for incremental updates • Others need full refresh
  • 24. 25 © Hortonworks Inc. 2011–2018. All rights reserved Constraints & defaults • Helps optimizer to produce better plans • BI tool integrations • Data Integrity • hive.constraint.notnull.enforce = true • SQL compatibility & offload scenarios Example: CREATE TABLE Persons ( ID Int NOT NULL, Name String NOT NULL, Age Int, Creator String DEFAULT CURRENT_USER(), CreateDate Date DEFAULT CURRENT_DATE(), PRIMARY KEY (ID) DISABLE NOVALIDATE ); CREATE TABLE BusinessUnit ( ID Int NOT NULL, Head Int NOT NULL, Creator String DEFAULT CURRENT_USER(), CreateDate Date DEFAULT CURRENT_DATE(), PRIMARY KEY (ID) DISABLE NOVALIDATE, CONSTRAINT fk FOREIGN KEY (Head) REFERENCES Persons(ID) DISABLE NOVALIDATE );
  • 25. 26 © Hortonworks Inc. 2011–2018. All rights reserved Default clause & Surrogate keys • Multiple approaches • Sequence number is dense & increasing, but: Bottleneck in distributed DBMS • UUID is easy & distributable, but large and slow • Surrogate key UDF is easy & distributable & fast, but: No sequence and has gaps CREATE TABLE AIRLINES_V2 (ID BIGINT DEFAULT SURROGATE_KEY(), CODE STRING, DESCRIPTION STRING, PRIMARY KEY (ID) DISABLE NOVALIDATE); INSERT INTO AIRLINES_V2 (CODE, DESCRIPTION) SELECT * FROM AIRLINES; ALTER TABLE FLIGHTS ADD COLUMNS (carrier_sk BIGINT); MERGE INTO FLIGHTS f USING AIRLINES_V2 a ON f.uniquecarrier = a.code WHEN MATCHED THEN UPDATE SET carrier_sk = a.id;
  • 26. 27 © Hortonworks Inc. 2011–2018. All rights reserved ⬢ Solution ● Query fails because of stats estimation error ● Runtime sends observed statistics back to coordinator ● Statistics overrides are created at session, server or global level ● Query is replanned and resubmitted Hive-17626: Optimizer is learning from planning mistakes ⬢ Symptoms ● Memory exhaustion due to under provisioning ● Excessive runtime (future) ● Excessive spilling (future)
  • 27. © Hortonworks Inc. 2011- 2018. All rights reserved | 28 Multitenancy
  • 28. 29 © Hortonworks Inc. 2011–2018. All rights reserved HIVE-17481: LLAP workload management ⬢ Effectively share LLAP cluster resources – Resource allocation per user policy; separate ETL and BI, etc. ⬢ Resources based guardrails – Protect against long running queries, high memory usage ⬢ Improved, query-aware scheduling – Scheduler is aware of query characteristics, types, etc. – Fragments easy to pre-empt compared to containers – Queries get guaranteed fractions of the cluster, but can use empty space
  • 29. 30 © Hortonworks Inc. 2011–2018. All rights reserved Common Triggers ● ELAPSED_TIME ● EXECUTION_TIME ● TOTAL_TASKS ● HDFS_BYTES_READ, HDFS_BYTES_WRITTEN ● CREATED FILES ● CREATED_DYNAMIC_PARTITIONS Example CREATE RESOURCE PLAN guardrail; CREATE TRIGGER guardrail.long_running WHEN EXECUTION_TIME > 2000 DO KILL; ALTER TRIGGER guardrail.long_running ADD TO UNMANAGED; ALTER RESOURCE PLAN guardrail ENABLE ACTIVATE; Guardrail Example
  • 30. 31 © Hortonworks Inc. 2011–2018. All rights reserved Resource plans example CREATE RESOURCE PLAN daytime; CREATE POOL daytime.bi WITH ALLOC_FRACTION=0.8, QUERY_PARALLELISM=5; CREATE POOL daytime.etl WITH ALLOC_FRACTION=0.2, QUERY_PARALLELISM=20; CREATE RULE downgrade IN daytime WHEN total_runtime > 3000 THEN MOVE etl; ADD RULE downgrade TO bi; CREATE APPLICATION MAPPING tableau in daytime TO bi; ALTER PLAN daytime SET default pool= etl; APPLY PLAN daytime; daytime bi: 80% etl: 20% Downgrade when total_runtime>3000
  • 31. © Hortonworks Inc. 2011- 2018. All rights reserved | 32 BI caching
  • 32. 33 © Hortonworks Inc. 2011–2018. All rights reserved HIVE-18513: Query result cache Returns results directly from storage (e.g. HDFS) without actually executing the query If the same query had ran before Important for dashboards, reports etc. where repetitive queries is common Without cache With cache
  • 33. 34 © Hortonworks Inc. 2011–2018. All rights reserved HIVE-18513: Query result cache details • hive.query.results.cache.enabled=true (on by default) • Works only on hive managed tables • If you JOIN an external table with Hive managed table, Hive will fall back to executing the full query. Because Hive can’t know if external table data has changed • Works with ACID • That means if Hive table has been updated, the query will be rerun automatically • Is different from LLAP cache • LLAP cache is a data cache. That means multiple queries can benefit by avoiding reading from disk. Speeds up the read path. • Result cache effectively bypasses execution of query • Stored at /tmp/hive/__resultcache__/, default space is 2GB, LRU eviction • Tunable setting hive.query.results.cache.max.size (bytes)
  • 34. 35 © Hortonworks Inc. 2011–2018. All rights reserved Metastore Cache • With query execution time being < 1 sec, compilation time starts to dominate • Metadata retrieval is often significant part of compilation time. Most of it is in RDBMS queries. • Cloud RDBMS As a Service is often slower, and frequent queries leads to throttling. • Metadata cache speeds compilation time by around 50% with onprem mysql. Significantly more improvement with cloud RDBMS. • Cache is consistent in single metastore setup, eventually consistent with HA setup. Consistent HA setup support is in the works.
  • 35. © Hortonworks Inc. 2011- 2018. All rights reserved | 36 Phew. That was a lot.
  • 36. 37 © Hortonworks Inc. 2011–2018. All rights reserved Hive 3 feature summary ⬢ EDW offload – Surrogate key and constraints – Information schema – Materialized views ⬢ Perf – Query result & metastore caches – LLAP workload management ⬢ Real-time capabilities with Kafka – Ingest in ACID tables – Instantly query using Druid ⬢ Unified SQL – JDBC connector – Druid connector – Spark-hive connector ⬢ Table types – ACID v2 and on by default – External v Managed ⬢ Cloud – AWS/GCP/Azure cloud storage natively supported now
  • 37. © Hortonworks Inc. 2011- 2018. All rights reserved | 38 Spark-Hive connect
  • 38. 39 © Hortonworks Inc. 2011–2018. All rights reserved Hive is broadening its reach SQL over Hadoop • External tables only • No ACID • No LLAP • SBA OK (doAs=True) • Some perf penalty Hive as EDW • ACID • LLAP • doAs=False • Managed tables • Column-level security • Stats, better perf etc. External table/ Direct Hive Warehouse Connector Spark
  • 39. 45 © Hortonworks Inc. 2011–2018. All rights reserved Driver MetaStore HiveServer2 LLAP DaemonsExecutors Spark Meta Hive Meta Executors LLAP Daemons Isolate Spark and Hive Catalogs/Tables Leverage connector for Spark <-> Hive Uses Apache Arrow for fast data transfer HWC HWC
  • 40. 47 © Hortonworks Inc. 2011–2018. All rights reserved Driver MetaStore HiveServer2 LLAP DaemonsExecutors Spark Meta Hive Meta HWC (Thrift JDBC) Executors LLAP Daemons a) hive.executeUpdate(“INSERT INTO s SELECT * FROM t”) 1. Driver submits update op to HiveServer2 2. Process update through Tez and/or LLAP 3. HWC returns true on success 1 2 3
  • 41. 49 © Hortonworks Inc. 2011–2018. All rights reserved Driver MetaStore HiveServer2 LLAP DaemonsExecutors Spark Meta Hive Meta Executors LLAP Daemons b) df.write.format(HIVE_WAREHOUSE_CONNECTOR).save() 1.Driver launches DataWriter tasks 2.Tasks write ORC files 3.On commit, Driver executes LOAD DATA INTO TABLE HDFS /tmp 1 2 3 ACID Tables
  • 42. 51 © Hortonworks Inc. 2011–2018. All rights reserved Driver MetaStore HiveServer+Tez Executors Spark Meta Hive Meta Executors c) df.write.format(STREAM_TO_STREAM).start() 1.Driver launches DataWriter tasks 2.Tasks open Txns 3.Write rows to ACID tables in Tx ACID Tables1 2 3
  • 43. © Hortonworks Inc. 2011- 2018. All rights reserved | 52 Performance
  • 44. 53 © Hortonworks Inc. 2011–2018. All rights reserved • Ran all 99 TPCDS queries • Total query runtime have improved multifold in each release! Benchmark journey TPCDS 10TB scale on 10 node cluster HDP 2.5 Hive1 HDP 2.5 LLAP HDP 2.6 LLAP 25x 3x 2x HDP 3.0 LLAP 2016 20182017 ACID tables
  • 45. 54 © Hortonworks Inc. 2011–2018. All rights reserved • Faster analytical queries with improved vectorization in HDP 3.0 • Vectorized execution of PTF, rollup and grouping sets. • Perf gain compared to HDP 2.6 • TPCDS query67 ~ 10x! • TPCDS query36 ~ 30x! • TPCDS query27 ~ 20x! OLAP Vectorization
  • 46. 55 © Hortonworks Inc. 2011–2018. All rights reserved SELECT * FROM ( SELECT AVG(ss_list_price) B1_LP, COUNT(ss_list_price) B1_CNT ,COUNT(DISTINCT ss_list_price) B1_CNTD FROM store_sales WHERE ss_quantity BETWEEN 0 AND 5 AND (ss_list_price BETWEEN 11 and 11+10 OR ss_coupon_amt BETWEEN 460 and 460+1000 OR ss_wholesale_cost BETWEEN 14 and 14+20)) B1, ( SELECT AVG(ss_list_price) B2_LP, COUNT(ss_list_price) B2_CNT ,COUNT(DISTINCT ss_list_price) B2_CNTD FROM store_sales WHERE ss_quantity BETWEEN 6 AND 10 AND (ss_list_price BETWEEN 91 and 91+10 OR ss_coupon_amt BETWEEN 1430 and 1430+1000 OR ss_wholesale_cost BETWEEN 32 and 32+20)) B2, . . . LIMIT 100; TPCDS SQL query 28 joins 6 instances of store_sales table Shared scan - 4x improvement! RS RS RS RS RS Scan store_sales Combined OR’ed B1-B6 Filters B1 Filter B2 Filter B3 Filter B4 Filter B5 Filter Join
  • 47. 56 © Hortonworks Inc. 2011–2018. All rights reserved • Dramatically improves performance of very selective joins • Builds a bloom filter from one side of join and filters rows from other side • Skips scan and further evaluation of rows that would not qualify the join Dynamic Semijoin Reduction - 7x improvement for q72 SELECT … FROM sales JOIN time ON sales.time_id = time.time_id WHERE time.year = 2014 AND time.quarter IN ('Q1', 'Q2’) Reduced scan on sales
  • 48. © Hortonworks Inc. 2011- 2018. All rights reserved | 57 We’ve made it! Questions?
  • 49. © Hortonworks Inc. 2011- 2018. All rights reserved | 58 Oh. One more thing.
  • 50. 59 © Hortonworks Inc. 2011–2018. All rights reserved ⬢ Hive on Kubernetes solves: – Hive/LLAP side install (to main cluster) – Multiple versions of Hive – Multiple warehouse & compute instances – Dynamic configuration and secrets management – Stateful and work preserving restarts (cache) – Rolling restart for upgrades. Fast rollback to previous good state. Hive on Kubernetes (WIP) Kubernetes Hosting Environments AWS GCP Data OS CPU / MEMORY / STORAGE OPENSHIFTAZURE CLOUD PROVIDERS ON- PREM/HYB RID DATA PLANE SERVICES Cluster Lifecycle Manager Data Analytics Studio (DAS) Organizational Services COMPUTE CLUSTER SHARED SERVICES Ranger Atlas Metastore Tiller API Server DAS Web Service Query Coordinators Query Executors Registry Blobstore Indexe r RDBMS Hive Server Long-running kubernetes cluster Inter-cluster communication Intra-cluster communication Ingress Controller or Load Balancer Internal Service Endpoint for ReplicaSet or StatefulSet Ephemeral kubernetes cluster