SlideShare uma empresa Scribd logo
1 de 37
Baixar para ler offline
© Cloudera, Inc. All rights reserved.
WHAT’S NEW IN APACHE HIVE 3 FOR HDP 3.1
Jason Dere
Apache Hive PMC Member
jdere@cloudera.com
© Cloudera, Inc. All rights reserved. 2
AGENDA
Apache Hive 3
Data Analytics Studio
Coming Soon
© Cloudera, Inc. All rights reserved. 3
AGENDA
Apache Hive 3
Data Analytics Studio
Coming Soon
© Cloudera, Inc. All rights reserved.
Hive LLAP - MPP Performance at Hadoop Scale
Deep
Storage
Hadoop Cluster
LLAP Daemon
Query Executors
LLAP Daemon
Query Executors
LLAP Daemon
Query Executors
LLAP Daemon
Query Executors
Query
Coordinators
Coord-
inator
Coord-
inator
Coord-
inator
HiveServer2
(Query
Endpoint)
ODBC /
JDBC
SQL
Queries In-Memory Cache
(Shared Across All Users)
HDFS and Compatible S3 WASB Isilon
© Cloudera, Inc. All rights reserved.
Hive3: Focus on the EnterpriseDataWarehouse
BI tools
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
© Cloudera, Inc. All rights reserved.
New SQL Features
© Cloudera, Inc. All rights reserved.
Materializedview
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;
© Cloudera, Inc. All rights reserved.
Materializedview - 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
© Cloudera, Inc. 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
);
© Cloudera, Inc. 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;
© Cloudera, Inc. All rights reserved.
Connectors
© Cloudera, Inc. All rights reserved.
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
CREATE TABLE postgres_table (
id INT,
name varchar
);
CREATE EXTERNAL TABLE hive_table (
id INT,
name STRING
) 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",
"hive.jdbc.update.on.duplicate" = "true"
);
In Postgres
In Hive
© Cloudera, Inc. All rights reserved.
Druid Connector- Joins between Hive and realtime datain 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’;
© Cloudera, Inc. All rights reserved.
Kafkaconnector
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
© Cloudera, Inc. All rights reserved.
ACID v2
© Cloudera, Inc. 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
• Non-ORC formats supported (INSERT & SELECT only)
• Fully compatible with native cloud storage
© Cloudera, Inc. All rights reserved.
Workload Management
© Cloudera, Inc. All rights reserved.
LLAP workload management
⬢ Effectively share LLAP cluster resources
– Resource allocation per user policy; separate ETL and BI, etc.
⬢ Resource 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
© Cloudera, Inc. All rights reserved.
Guardrail Example
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;
© Cloudera, Inc. 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 TRIGGER 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
© Cloudera, Inc. All rights reserved.
Performance
© Cloudera, Inc. All rights reserved.
• Ran all 99 TPCDS queries
• Total query runtime have improved multifold in each release!
Benchmarkjourney
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
© Cloudera, Inc. All rights reserved.
• Performed by Postech University (Korea)
• Compares LLAP, Spark, Presto and Tez, and MR3
• Shows Hive3/LLAP fastest in aggregate and for most queries
• Indigo cluster: 20 nodes, 96GB, 2 disks, 3TB TPCDS
PostechUniversity benchmark
MR3 brenchmark
© Cloudera, Inc. 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
© Cloudera, Inc. 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
© Cloudera, Inc. 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
© Cloudera, Inc. All rights reserved. 27
AGENDA
Apache Hive 3
Data Analytics Studio
Coming Soon
© Cloudera, Inc. All rights reserved.
SOLUTIONS: Full featured Auto-complete, results
direct download, quick-data preview and many
other quality-of-life improvements
© Cloudera, Inc. All rights reserved.
SOLUTIONS: Pre-defined searches to quickly narrow
down problematic queries in a large cluster
© Cloudera, Inc. All rights reserved.
SOLUTIONS: Heuristic recommendation engine
Fully self-serviced query and storage optimization
© Cloudera, Inc. All rights reserved.
Query compare allows side-by-side
comparison of query details, explain
plan, configuration, execution
details
© Cloudera, Inc. All rights reserved.
SOLUTIONS: Data Analytics Studio gives database
heatmap, quickly discover and see what part of your
cluster is being utilized more
© Cloudera, Inc. All rights reserved.
One of the Extensible DataPlane Services
⬢ DAS 1.2 available now for HDP 3.1!
⬢ Replaces Hive & Tez Views
⬢ Monthly release cadence
⬢ Separate install from stack
Data Analytics Studio
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
EXTENSIBLE SERVICES
DATA
ANALYTICS
STUDIO
© Cloudera, Inc. All rights reserved. 34
AGENDA
Apache Hive 3
Data Analytics Studio
Coming Soon
© Cloudera, Inc. All rights reserved.
• Hive on Kubernetes
• Easy creation/deployment of new Hive compute clusters
• Integration with shared catalog/security/governance (SDX)
• Multiple versions of Hive
• Rolling patch upgrades
• Data Analytics Studio
• More recommendations, including materialized views
• New visualizations for query execution
Hive On CDP
© Cloudera, Inc. All rights reserved.
• Connectors
• Integration with managed streaming/relational services
• Query Scheduler
• Micro-batch streaming queries with Kafka
• Automatic materialized view maintenance
• Automatic statistics collection/update
• ACID
• Provide APIs for native integration with other apps (Impala, Spark, BigSQL)
Hive On CDP
© Cloudera, Inc. All rights reserved.
THANK YOU

Mais conteúdo relacionado

Mais procurados

Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Databricks
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudNoritaka Sekiyama
 
Facebook Messages & HBase
Facebook Messages & HBaseFacebook Messages & HBase
Facebook Messages & HBase强 王
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversScyllaDB
 
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
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewenconfluent
 
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOx
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOxInfluxDB IOx Tech Talks: Query Processing in InfluxDB IOx
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOxInfluxData
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDatabricks
 
Hive User Meeting August 2009 Facebook
Hive User Meeting August 2009 FacebookHive User Meeting August 2009 Facebook
Hive User Meeting August 2009 Facebookragho
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
What is in a Lucene index?
What is in a Lucene index?What is in a Lucene index?
What is in a Lucene index?lucenerevolution
 
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 DebeziumTathastu.ai
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark Summit
 
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan Agrawal
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan AgrawalApache Spark Based Reliable Data Ingestion in Datalake with Gagan Agrawal
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan AgrawalDatabricks
 
Introduction to Apache NiFi dws19 DWS - DC 2019
Introduction to Apache NiFi   dws19 DWS - DC 2019Introduction to Apache NiFi   dws19 DWS - DC 2019
Introduction to Apache NiFi dws19 DWS - DC 2019Timothy Spann
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaJiangjie Qin
 
Running Apache NiFi with Apache Spark : Integration Options
Running Apache NiFi with Apache Spark : Integration OptionsRunning Apache NiFi with Apache Spark : Integration Options
Running Apache NiFi with Apache Spark : Integration OptionsTimothy Spann
 

Mais procurados (20)

Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0Deep Dive into the New Features of Apache Spark 3.0
Deep Dive into the New Features of Apache Spark 3.0
 
File Format Benchmark - Avro, JSON, ORC & Parquet
File Format Benchmark - Avro, JSON, ORC & ParquetFile Format Benchmark - Avro, JSON, ORC & Parquet
File Format Benchmark - Avro, JSON, ORC & Parquet
 
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the CloudAmazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
Amazon S3 Best Practice and Tuning for Hadoop/Spark in the Cloud
 
Facebook Messages & HBase
Facebook Messages & HBaseFacebook Messages & HBase
Facebook Messages & HBase
 
Apache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the CoversApache Iceberg: An Architectural Look Under the Covers
Apache Iceberg: An Architectural Look Under the Covers
 
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...
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
 
Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem Apache NiFi in the Hadoop Ecosystem
Apache NiFi in the Hadoop Ecosystem
 
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOx
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOxInfluxDB IOx Tech Talks: Query Processing in InfluxDB IOx
InfluxDB IOx Tech Talks: Query Processing in InfluxDB IOx
 
Deep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache SparkDeep Dive: Memory Management in Apache Spark
Deep Dive: Memory Management in Apache Spark
 
Hive User Meeting August 2009 Facebook
Hive User Meeting August 2009 FacebookHive User Meeting August 2009 Facebook
Hive User Meeting August 2009 Facebook
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
What is in a Lucene index?
What is in a Lucene index?What is in a Lucene index?
What is in a Lucene index?
 
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
 
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
Spark + Parquet In Depth: Spark Summit East Talk by Emily Curtin and Robbie S...
 
Dataflow with Apache NiFi
Dataflow with Apache NiFiDataflow with Apache NiFi
Dataflow with Apache NiFi
 
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan Agrawal
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan AgrawalApache Spark Based Reliable Data Ingestion in Datalake with Gagan Agrawal
Apache Spark Based Reliable Data Ingestion in Datalake with Gagan Agrawal
 
Introduction to Apache NiFi dws19 DWS - DC 2019
Introduction to Apache NiFi   dws19 DWS - DC 2019Introduction to Apache NiFi   dws19 DWS - DC 2019
Introduction to Apache NiFi dws19 DWS - DC 2019
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 
Running Apache NiFi with Apache Spark : Integration Options
Running Apache NiFi with Apache Spark : Integration OptionsRunning Apache NiFi with Apache Spark : Integration Options
Running Apache NiFi with Apache Spark : Integration Options
 

Semelhante a What's New in Apache Hive

Impala tech-talk by Dimitris Tsirogiannis
Impala tech-talk by Dimitris TsirogiannisImpala tech-talk by Dimitris Tsirogiannis
Impala tech-talk by Dimitris TsirogiannisFelicia Haggarty
 
Spark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit
 
Introduction to Apache Kudu
Introduction to Apache KuduIntroduction to Apache Kudu
Introduction to Apache KuduJeff Holoman
 
Intro to Apache Kudu (short) - Big Data Application Meetup
Intro to Apache Kudu (short) - Big Data Application MeetupIntro to Apache Kudu (short) - Big Data Application Meetup
Intro to Apache Kudu (short) - Big Data Application MeetupMike Percy
 
Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizonArtem Ervits
 
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...Cloudera, Inc.
 
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'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 - TokyoDataWorks Summit
 
Introduction to Kudu - StampedeCon 2016
Introduction to Kudu - StampedeCon 2016Introduction to Kudu - StampedeCon 2016
Introduction to Kudu - StampedeCon 2016StampedeCon
 
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...Hadoop / Spark Conference Japan
 
VMworld 2013: Virtualizing Databases: Doing IT Right
VMworld 2013: Virtualizing Databases: Doing IT Right VMworld 2013: Virtualizing Databases: Doing IT Right
VMworld 2013: Virtualizing Databases: Doing IT Right VMworld
 
Kudu: Fast Analytics on Fast Data
Kudu: Fast Analytics on Fast DataKudu: Fast Analytics on Fast Data
Kudu: Fast Analytics on Fast Datamichaelguia
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaCloudera, Inc.
 
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...Data Con LA
 
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Cloudera, Inc.
 
Tips, Tricks & Best Practices for large scale HDInsight Deployments
Tips, Tricks & Best Practices for large scale HDInsight DeploymentsTips, Tricks & Best Practices for large scale HDInsight Deployments
Tips, Tricks & Best Practices for large scale HDInsight DeploymentsAshish Thapliyal
 
Kudu: New Hadoop Storage for Fast Analytics on Fast Data
Kudu: New Hadoop Storage for Fast Analytics on Fast DataKudu: New Hadoop Storage for Fast Analytics on Fast Data
Kudu: New Hadoop Storage for Fast Analytics on Fast DataCloudera, Inc.
 

Semelhante a What's New in Apache Hive (20)

Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizon
 
Impala tech-talk by Dimitris Tsirogiannis
Impala tech-talk by Dimitris TsirogiannisImpala tech-talk by Dimitris Tsirogiannis
Impala tech-talk by Dimitris Tsirogiannis
 
Spark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike PercySpark Summit EU talk by Mike Percy
Spark Summit EU talk by Mike Percy
 
Introduction to Apache Kudu
Introduction to Apache KuduIntroduction to Apache Kudu
Introduction to Apache Kudu
 
Intro to Apache Kudu (short) - Big Data Application Meetup
Intro to Apache Kudu (short) - Big Data Application MeetupIntro to Apache Kudu (short) - Big Data Application Meetup
Intro to Apache Kudu (short) - Big Data Application Meetup
 
Hive 3 a new horizon
Hive 3  a new horizonHive 3  a new horizon
Hive 3 a new horizon
 
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...
Apache Kudu (Incubating): New Hadoop Storage for Fast Analytics on Fast Data ...
 
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'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
 
Introduction to Kudu - StampedeCon 2016
Introduction to Kudu - StampedeCon 2016Introduction to Kudu - StampedeCon 2016
Introduction to Kudu - StampedeCon 2016
 
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...
Apache Kudu Fast Analytics on Fast Data (Hadoop / Spark Conference Japan 2016...
 
VMworld 2013: Virtualizing Databases: Doing IT Right
VMworld 2013: Virtualizing Databases: Doing IT Right VMworld 2013: Virtualizing Databases: Doing IT Right
VMworld 2013: Virtualizing Databases: Doing IT Right
 
Kudu: Fast Analytics on Fast Data
Kudu: Fast Analytics on Fast DataKudu: Fast Analytics on Fast Data
Kudu: Fast Analytics on Fast Data
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...
Big Data Day LA 2016/ Big Data Track - How To Use Impala and Kudu To Optimize...
 
SFHUG Kudu Talk
SFHUG Kudu TalkSFHUG Kudu Talk
SFHUG Kudu Talk
 
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
Simplifying Hadoop with RecordService, A Secure and Unified Data Access Path ...
 
Spark etl
Spark etlSpark etl
Spark etl
 
Tips, Tricks & Best Practices for large scale HDInsight Deployments
Tips, Tricks & Best Practices for large scale HDInsight DeploymentsTips, Tricks & Best Practices for large scale HDInsight Deployments
Tips, Tricks & Best Practices for large scale HDInsight Deployments
 
Kudu: New Hadoop Storage for Fast Analytics on Fast Data
Kudu: New Hadoop Storage for Fast Analytics on Fast DataKudu: New Hadoop Storage for Fast Analytics on Fast Data
Kudu: New Hadoop Storage for Fast Analytics on Fast Data
 

Mais de 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
 

Mais de 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
 

Último

Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Drew Madelung
 
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...Neo4j
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptxHampshireHUG
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProduct Anonymous
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?Igalia
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slidevu2urc
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerThousandEyes
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century educationjfdjdjcjdnsjd
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...Martijn de Jong
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...apidays
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationRadu Cotescu
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdfhans926745
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024Rafal Los
 
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...DianaGray10
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationMichael W. Hawkins
 
Advantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessAdvantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessPixlogix Infotech
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdflior mazor
 
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc
 

Último (20)

Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
 
A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024
 
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
Connector Corner: Accelerate revenue generation using UiPath API-centric busi...
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 
Advantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessAdvantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your Business
 
GenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdfGenAI Risks & Security Meetup 01052024.pdf
GenAI Risks & Security Meetup 01052024.pdf
 
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law DevelopmentsTrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
TrustArc Webinar - Stay Ahead of US State Data Privacy Law Developments
 

What's New in Apache Hive

  • 1. © Cloudera, Inc. All rights reserved. WHAT’S NEW IN APACHE HIVE 3 FOR HDP 3.1 Jason Dere Apache Hive PMC Member jdere@cloudera.com
  • 2. © Cloudera, Inc. All rights reserved. 2 AGENDA Apache Hive 3 Data Analytics Studio Coming Soon
  • 3. © Cloudera, Inc. All rights reserved. 3 AGENDA Apache Hive 3 Data Analytics Studio Coming Soon
  • 4. © Cloudera, Inc. All rights reserved. Hive LLAP - MPP Performance at Hadoop Scale Deep Storage Hadoop Cluster LLAP Daemon Query Executors LLAP Daemon Query Executors LLAP Daemon Query Executors LLAP Daemon Query Executors Query Coordinators Coord- inator Coord- inator Coord- inator HiveServer2 (Query Endpoint) ODBC / JDBC SQL Queries In-Memory Cache (Shared Across All Users) HDFS and Compatible S3 WASB Isilon
  • 5. © Cloudera, Inc. All rights reserved. Hive3: Focus on the EnterpriseDataWarehouse BI tools 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
  • 6. © Cloudera, Inc. All rights reserved. New SQL Features
  • 7. © Cloudera, Inc. All rights reserved. Materializedview 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;
  • 8. © Cloudera, Inc. All rights reserved. Materializedview - 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
  • 9. © Cloudera, Inc. 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 );
  • 10. © Cloudera, Inc. 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. © Cloudera, Inc. All rights reserved. Connectors
  • 12. © Cloudera, Inc. All rights reserved. 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 CREATE TABLE postgres_table ( id INT, name varchar ); CREATE EXTERNAL TABLE hive_table ( id INT, name STRING ) 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", "hive.jdbc.update.on.duplicate" = "true" ); In Postgres In Hive
  • 13. © Cloudera, Inc. All rights reserved. Druid Connector- Joins between Hive and realtime datain 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’;
  • 14. © Cloudera, Inc. All rights reserved. Kafkaconnector 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
  • 15. © Cloudera, Inc. All rights reserved. ACID v2
  • 16. © Cloudera, Inc. 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 • Non-ORC formats supported (INSERT & SELECT only) • Fully compatible with native cloud storage
  • 17. © Cloudera, Inc. All rights reserved. Workload Management
  • 18. © Cloudera, Inc. All rights reserved. LLAP workload management ⬢ Effectively share LLAP cluster resources – Resource allocation per user policy; separate ETL and BI, etc. ⬢ Resource 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
  • 19. © Cloudera, Inc. All rights reserved. Guardrail Example 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;
  • 20. © Cloudera, Inc. 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 TRIGGER 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
  • 21. © Cloudera, Inc. All rights reserved. Performance
  • 22. © Cloudera, Inc. All rights reserved. • Ran all 99 TPCDS queries • Total query runtime have improved multifold in each release! Benchmarkjourney 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
  • 23. © Cloudera, Inc. All rights reserved. • Performed by Postech University (Korea) • Compares LLAP, Spark, Presto and Tez, and MR3 • Shows Hive3/LLAP fastest in aggregate and for most queries • Indigo cluster: 20 nodes, 96GB, 2 disks, 3TB TPCDS PostechUniversity benchmark MR3 brenchmark
  • 24. © Cloudera, Inc. 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
  • 25. © Cloudera, Inc. 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
  • 26. © Cloudera, Inc. 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
  • 27. © Cloudera, Inc. All rights reserved. 27 AGENDA Apache Hive 3 Data Analytics Studio Coming Soon
  • 28. © Cloudera, Inc. All rights reserved. SOLUTIONS: Full featured Auto-complete, results direct download, quick-data preview and many other quality-of-life improvements
  • 29. © Cloudera, Inc. All rights reserved. SOLUTIONS: Pre-defined searches to quickly narrow down problematic queries in a large cluster
  • 30. © Cloudera, Inc. All rights reserved. SOLUTIONS: Heuristic recommendation engine Fully self-serviced query and storage optimization
  • 31. © Cloudera, Inc. All rights reserved. Query compare allows side-by-side comparison of query details, explain plan, configuration, execution details
  • 32. © Cloudera, Inc. All rights reserved. SOLUTIONS: Data Analytics Studio gives database heatmap, quickly discover and see what part of your cluster is being utilized more
  • 33. © Cloudera, Inc. All rights reserved. One of the Extensible DataPlane Services ⬢ DAS 1.2 available now for HDP 3.1! ⬢ Replaces Hive & Tez Views ⬢ Monthly release cadence ⬢ Separate install from stack Data Analytics Studio 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 EXTENSIBLE SERVICES DATA ANALYTICS STUDIO
  • 34. © Cloudera, Inc. All rights reserved. 34 AGENDA Apache Hive 3 Data Analytics Studio Coming Soon
  • 35. © Cloudera, Inc. All rights reserved. • Hive on Kubernetes • Easy creation/deployment of new Hive compute clusters • Integration with shared catalog/security/governance (SDX) • Multiple versions of Hive • Rolling patch upgrades • Data Analytics Studio • More recommendations, including materialized views • New visualizations for query execution Hive On CDP
  • 36. © Cloudera, Inc. All rights reserved. • Connectors • Integration with managed streaming/relational services • Query Scheduler • Micro-batch streaming queries with Kafka • Automatic materialized view maintenance • Automatic statistics collection/update • ACID • Provide APIs for native integration with other apps (Impala, Spark, BigSQL) Hive On CDP
  • 37. © Cloudera, Inc. All rights reserved. THANK YOU