SlideShare a Scribd company logo
1 of 47
Download to read offline
Apache Hive Hook
2013. 8
Minwoo Kim
michael.kim@nexr.com
Apache Hive Hook
• The reason why I made this is that Ryan asked me about
hive hook, but I couldn’t find any info about hook in hive
wiki.
• I hope this will be helpful to develop applications using Hive
when you want to get extra info while executing a query on
Hive.
• This document was written based on release-0.11 tag
• Source:
- https://github.com/apache/hive (mirror of apache hive)
What is a hook?
• As you know, this is about computer programming technique,
but ..
• Hooking
- Techniques for intercepting function calls or
messages or events in an operating system, applications,
and other software components.
• Hook
- Code that handles intercepted function calls, events or
messages
Hive provides some hooking
points
• pre-execution
• post-execution
• execution-failure
• pre- and post-driver-run
• pre- and post-semantic-analyze
• metastore-initialize
How to set up hooks in Hive
<property>
<name>hive.exec.pre.hooks</name>
<value></value>
<description>
Comma-separated list of pre-execution hooks to be invoked for each statement.
A pre-execution hook is specified as the name of a Java class which implements
the org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.
</description>
</property>
hive-site.xml
<property>
<name>hive.aux.jars.path</name>
<value></value>
</property>
Setting hook property
Setting path of jars contains implementations of hook interfaces or abstract class
You can use hive.added.jars.path instead of hive.aux.jars.path
Hive hook properties and interfaces
Property Interface or Abstract class
hive.exec.pre.hooks
org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext
(PreExecute is deprecated)
hive.exec.post.hooks
org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext
(PostExecute is deprecated)
hive.exec.failure.hooks org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext
hive.metastore.init.hooks org.apache.hadoop.hive.metastore.MetaStoreInitListener
hive.exec.driver.run.hooks org.apache.hadoop.hive.ql.HiveDriverRunHook
hive.semantic.analyzer.hook org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook
When those hooks fire?
• You can submit a query on Hive through the
following entry points
- CLIDriver main method (called by shell script)
- HCatCli main method (called by shell script)
- HiveServer (called by thrift client)
- HiveServer2 (called by thrift client or beeline)
CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd()
↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠
➔ processLocalCmd() ➔ Driver.run() ➠
CLIDriver
➔ is remote ?
yes
no
CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd()
↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠
➔ processLocalCmd() ➔ Driver.run() ➠
CLIDriver
➔ is remote ?
yes
no
HCatCli
HCatCli.main() ➔ processLine() ➔ processCmd()
➔ HCatDriver.run() ⤇ Driver.run() ➠
HiveServer.execute() ➔ Driver.run() ➠
HiveServer
CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd()
↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠
➔ processLocalCmd() ➔ Driver.run() ➠
CLIDriver
➔ is remote ?
yes
no
HCatCli
HCatCli.main() ➔ processLine() ➔ processCmd()
➔ HCatDriver.run() ⤇ Driver.run() ➠
HiveServer2
ThriftCLIService.ExecuteStatement() ➔ CLIService.executeStatement()
CLIService.executeStatement()
↳ SessionManager.getSession()
↳ HiveSession.executeStatement()
↳ OperationManager.newExecuteStatementOperation()
↳ SQLOperation.run() ➔ Driver.run() ➠
⤶
HiveServer2
ThriftCLIService.ExecuteStatement() ➔ CLIService.executeStatement()
CLIService.executeStatement()
↳ SessionManager.getSession()
↳ HiveSession.executeStatement()
↳ OperationManager.newExecuteStatementOperation()
↳ SQLOperation.run() ➔ Driver.run() ➠
• OperationManager.newExecuteStatementOperation() is like a kind of factory
- AddResourceOperation, DeleteResourceOperation, DfsOperation,
GetCatalogsOperation, GetColumnsOperation, GetFunctionsOperation,
GetSchemasOperation, GetTablesOperation, GetTableTypesOperation,
GetTypeInfoOperation, SetOperation, SQLOperation
⤶
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse() ↝ HiveParser
{
• HiveParser.g
- SelectClauseParser.g
- FromClauseParser.g
- IdentifiersParser.g
• ParseDriver.parse()
- Command String ➡ root of AST tree
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
• SemanticAnalyzerFactory.get(conf, ast)
- SemanticAnalyzer, ColumnStatsSemanticAnalyzer, ExplainSemanticAnalyzer,
ExportSemanticAnalyzer, FunctionSemanticAnalyzer,
ImportSemanticAnalyzer, LoadSemanticAnalyzer, MacroSemanticAnalyzer
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
• FilterOperator
• SelectOperator
• ForwardOperator
• FileSinkOperator
• ScriptOperator
• PTFOperator
• ReduceSinkOperator
• ExtractOperator
• GroupByOperator
• JoinOperator
• MapJoinOperator
• SMBMapJoinOperator
• LimitOperator
• TableScanOperator
• UnionOperator
• UDTFOperator
• LateralViewJoinOperator
• LateralViewForwardOperator
• HashTableDummyOperator
• HashTableSinkOperator
• DummyStoreOperator
• DemuxOperator
• MuxOperator
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
• PredicateTransitivePropagate
• PredicatePushDown
• PartitionPruner
• PartitionConditionRemover
• ListBucketingPruner
• ListBucketingPruner
• ColumnPruner
• SkewJoinOptimizer
• RewriteGBUsingIndex
• GroupByOptimizer
• SamplePruner
• MapJoinProcessor
• BucketMapJoinOptimizer
• BucketMapJoinOptimizer
• SortedMergeBucketMapJoinO
ptimizer
• BucketingSortingReduceSink
Optimizer
• UnionProcessor
• JoinReorder
• ReduceSinkDeDuplication
• NonBlockingOpDeDupProc
• GlobalLimitOptimizer
• CorrelationOptimizer
• SimpleFetchOptimizer
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
• MapRedTask
• FetchTask
• ConditionalTask
• ExplainTask
• CopyTask
• DDLTask
• MoveTask
• FunctionTask
• StatsTask
• ColumnStatsTask
• DependencyCollectionTask
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
↳ Driver.execute()
➔ loop (List<Task>)
⟳ Driver.launchTask()
➔ TaskRunner.runSequential() ➔ Task.executeTask()
➔ Task.execute()
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
↳ Driver.execute()
➔ loop (List<Task>)
⟳ Driver.launchTask()
➔ TaskRunner.runSequential() ➔ Task.executeTask()
➔ Task.execute()
➔ analyzeInternal()
• processPositionAlias()
• doPhase1()
• getMetaData()
• genPlan()
• Optimizer.optimize()
• MapReduceCompiler.compile()
{
• ex) MapRedTask.execute() ⤇ ExecDriver.execute() ➔ JobClient.submitJob()
ExecMapper, ExecReducer
➠ Driver.run()
➔ Driver.runInternal()
↳ Driver.compile()
↳ ParseDriver.parse()
↳ SemanticAnalyzer.analyze()
↳ Driver.execute()
➔ loop (List<Task>)
⟳ Driver.launchTask()
➔ TaskRunner.runSequential() ➔ Task.executeTask()
➔ Task.execute()
PRE- and POST-DRIVER-RUN
PRE- and POST-SEMANTIC-ANALYZE
PRE-, POST-EXEC and ON-FAILURE
HiveServer2.main() ➔ HiveServer2.start()
➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠
HiveServer2.main() ➔ HiveServer2.start()
➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠
➔ HiveSession.getMetaStoreClient()
➔ new HiveMetaStoreClient() ➠
CLIService.executeStatement()
⇒
GetColumnsOperation.run()
GetSchemasOperation.run()
GetTablesOperation.run()
HiveServer2.main() ➔ HiveServer2.start()
➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠
➔ HiveSession.getMetaStoreClient()
➔ new HiveMetaStoreClient() ➠
CLIService.executeStatement()
⇒
SemanticAnalyzer ↝ Hive ↝ getMSC() is invoked by many other methods in Hive object
Hive.getMSC() ➔ Hive.createMetaStoreClient() ➔ RetryingHMSHandler.getProxy() ➠
GetColumnsOperation.run()
GetSchemasOperation.run()
GetTablesOperation.run()
HiveServer2.main() ➔ HiveServer2.start()
➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠
➔ HiveSession.getMetaStoreClient()
➔ new HiveMetaStoreClient() ➠
➠ new HiveMetaStoreClient()
➔ HiveMetaStore.newHMSHandler()
➔ RetryingHMSHandler.getProxy()
➔ new RetryingHMSHandler()
➔ new HMSHandler() ➔ HMSHandler.init()
➔ HiveMetaStore.init()
CLIService.executeStatement()
⇒
MATASTORE-INIT
SemanticAnalyzer ↝ Hive ↝ getMSC() is invoked by many other methods in Hive object
Hive.getMSC() ➔ Hive.createMetaStoreClient() ➔ RetryingHMSHandler.getProxy() ➠
GetColumnsOperation.run()
GetSchemasOperation.run()
GetTablesOperation.run()
How Hive executes hooks
List<HiveDriverRunHook> driverRunHooks;
try {
driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS,
HiveDriverRunHook.class);
for (HiveDriverRunHook driverRunHook : driverRunHooks) {
driverRunHook.preDriverRun(hookContext);
}
} catch (Exception e) {
• Hive executes multiple hooks on each hook points.
ex. Driver.runInternal()
1. MetaStoreInitListener
public abstract class MetaStoreInitListener implements Configurable {
private Configuration conf;
public MetaStoreInitListener(Configuration config){
this.conf = config;
}
public abstract void onInit(MetaStoreInitContext context) throws MetaException;
@Override
public Configuration getConf() {
return this.conf;
}
@Override
public void setConf(Configuration config) {
this.conf = config;
}
}
1. MetaStoreInitListener
public abstract class MetaStoreInitListener implements Configurable {
private Configuration conf;
public MetaStoreInitListener(Configuration config){
this.conf = config;
}
public abstract void onInit(MetaStoreInitContext context) throws MetaException;
@Override
public Configuration getConf() {
return this.conf;
}
@Override
public void setConf(Configuration config) {
this.conf = config;
}
}
What MetaStoreInitContext got
• has Nothing!
- This hook just alarms you when metastore initialize.
(but you, of course, can get HiveConf by calling getConf())
public class MetaStoreInitContext {
}
2. HiveDriverRunHook
• preDriverRun
- Invoked before Hive begins any processing of a command in the Driver,
before compilation
• postDriverRun
- Invoked after Hive performs any processing of a command,
just before a response is returned to the entity calling the Driver.run()
public interface HiveDriverRunHook extends Hook {
public void preDriverRun(
HiveDriverRunHookContext hookContext) throws Exception;
public void postDriverRun(
HiveDriverRunHookContext hookContext) throws Exception;
}
What
HiveDriverRunHookContext got
• You can get command string from this hook context.
- This is the only thing that HiveDriverRunHookContext has.
public interface HiveDriverRunHookContext extends Configurable{
public String getCommand();
public void setCommand(String command);
}
3.AbstractSemanticAnalyzerHook
• You can get
- HiveSemanticAnalyzerHookContext and ASTNode (Root node of
abstract syntax tree) before analyze.
- HiveSemanticAnalyzerHookContext and List<Task> after analyze.
public abstract class AbstractSemanticAnalyzerHook implements
HiveSemanticAnalyzerHook {
public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext
context,ASTNode ast)
throws SemanticException {
return ast;
}
public void postAnalyze(HiveSemanticAnalyzerHookContext context,
List<Task<? extends Serializable>> rootTasks) throws
SemanticException {
}
}
What
HiveSemanticAnalyzerHookContext got
• Hive Object
- contains information about a set of data in HDFS organized for query
processing. (from comment)
• ReadEntity, WriteEntity
• update method will be invoked after the semantic analyzer completes.
public interface HiveSemanticAnalyzerHookContext extends Configurable{
public Hive getHive() throws HiveException;
public void update(BaseSemanticAnalyzer sem);
public Set<ReadEntity> getInputs();
public Set<WriteEntity> getOutputs();
}
How Hive executes analyzer
hooks
List<AbstractSemanticAnalyzerHook> saHooks =
getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class);
// ~ ellipsis ~
HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
hookCtx.setConf(conf);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
How Hive executes analyzer
hooks
List<AbstractSemanticAnalyzerHook> saHooks =
getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class);
// ~ ellipsis ~
HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
hookCtx.setConf(conf);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
How Hive executes analyzer
hooks
List<AbstractSemanticAnalyzerHook> saHooks =
getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class);
// ~ ellipsis ~
HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
hookCtx.setConf(conf);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
How Hive executes analyzer
hooks
List<AbstractSemanticAnalyzerHook> saHooks =
getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class);
// ~ ellipsis ~
HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
hookCtx.setConf(conf);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
How Hive executes analyzer
hooks
List<AbstractSemanticAnalyzerHook> saHooks =
getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class);
// ~ ellipsis ~
HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl();
hookCtx.setConf(conf);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
tree = hook.preAnalyze(hookCtx, tree);
}
sem.analyze(tree, ctx);
hookCtx.update(sem);
for (AbstractSemanticAnalyzerHook hook : saHooks) {
hook.postAnalyze(hookCtx, sem.getRootTasks());
}
4. ExecuteWithHookContext
• Can be used in the followings
- hive.exec.pre.hooks
- hive.exec.post.hooks
- hive.exec.failure.hooks
public interface ExecuteWithHookContext extends Hook {
/**
   *
   * @param hookContext
   * The hook context passed to each hooks.
   * @throws Exception
   */
void run(HookContext hookContext) throws Exception;
}
What HookContext got
• HookType
- PRE_EXEC_HOOK, POST_EXEC_HOOK, ON_FAILURE_HOOK
• QueryPlan
• HiveConf
• LineageInfo
• UserGroupInformation
• OperationName
• List<TaskRunner> completeTaskList
• Set<ReadEntity> inputs
• Set<WriteEntity> outputs
• Map<String, ContentSummary> inputPathToContentSummary
How Hive fires hooks without
executing query physically
• This has the effect of causing the pre/post execute hooks to fire.
ALTER TABLE table_name TOUCH [PARTITION partitionSpec];
MetaStore Event Listeners
Property Abstract Class
hive.metastore.pre.event.listeners MetaStorePreEventListener
hive.metastore.end.function.listeners MetaStoreEndFunctionListener
hive.metastore.event.listeners MetaStoreEventListener
package : org.apache.hadoop.hive.metastore
• I think those listeners look like hooks.
• I couldn’t find any particular differences between listeners and hooks while just taking a look.
The only thing I found is that listeners can’t affect query processing. It can only read.
• Anyway, it looks useful to let you know when a metastore do something.
MetaStoreEventListener
• The followings will be performed when a particular event occurs on a
metastore.
- onCreateTable
- onDropTable
- onAlterTable
- onDropPartition
- onAlterPartition
- onCreateDatabase
- onDropDatabase
- onLoadPartitionDone
If you need more details, see org.apache.hadoop.hive.metastore.MetaStoreEventListener
Be careful!
• Hooks
- can be a critical failure point!
(you should better catch runtime exceptions)
- are preformed synchronously.
- can affect query processing time.
Let's try it out
• Demo
- Don’t be surprised if it doesn’t work.
- That’s the way the demo is...
Thanks!
• Questions?
• Resources
- https://cwiki.apache.org/confluence/display/Hive/
- https://github.com/apache/hive

More Related Content

What's hot

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
 
ORC File & Vectorization - Improving Hive Data Storage and Query Performance
ORC File & Vectorization - Improving Hive Data Storage and Query PerformanceORC File & Vectorization - Improving Hive Data Storage and Query Performance
ORC File & Vectorization - Improving Hive Data Storage and Query PerformanceDataWorks Summit
 
Enabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkEnabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkKazuaki Ishizaki
 
Apache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingApache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingDataWorks Summit
 
Apache Tez – Present and Future
Apache Tez – Present and FutureApache Tez – Present and Future
Apache Tez – Present and FutureDataWorks Summit
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache SparkDatabricks
 
a Secure Public Cache for YARN Application Resources
a Secure Public Cache for YARN Application Resourcesa Secure Public Cache for YARN Application Resources
a Secure Public Cache for YARN Application ResourcesDataWorks Summit
 
Understanding Memory Management In Spark For Fun And Profit
Understanding Memory Management In Spark For Fun And ProfitUnderstanding Memory Management In Spark For Fun And Profit
Understanding Memory Management In Spark For Fun And ProfitSpark Summit
 
From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.Taras Matyashovsky
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsAnton Kirillov
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsDatabricks
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Databricks
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsDatabricks
 
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
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Flink Forward
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introductioncolorant
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkPatrick Wendell
 
Bucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing ShuffleBucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing ShuffleDatabricks
 

What's hot (20)

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
 
ORC File & Vectorization - Improving Hive Data Storage and Query Performance
ORC File & Vectorization - Improving Hive Data Storage and Query PerformanceORC File & Vectorization - Improving Hive Data Storage and Query Performance
ORC File & Vectorization - Improving Hive Data Storage and Query Performance
 
Enabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache SparkEnabling Vectorized Engine in Apache Spark
Enabling Vectorized Engine in Apache Spark
 
Apache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data ProcessingApache Tez - A unifying Framework for Hadoop Data Processing
Apache Tez - A unifying Framework for Hadoop Data Processing
 
Optimizing Hive Queries
Optimizing Hive QueriesOptimizing Hive Queries
Optimizing Hive Queries
 
Apache Tez – Present and Future
Apache Tez – Present and FutureApache Tez – Present and Future
Apache Tez – Present and Future
 
Memory Management in Apache Spark
Memory Management in Apache SparkMemory Management in Apache Spark
Memory Management in Apache Spark
 
a Secure Public Cache for YARN Application Resources
a Secure Public Cache for YARN Application Resourcesa Secure Public Cache for YARN Application Resources
a Secure Public Cache for YARN Application Resources
 
Hive: Loading Data
Hive: Loading DataHive: Loading Data
Hive: Loading Data
 
Understanding Memory Management In Spark For Fun And Profit
Understanding Memory Management In Spark For Fun And ProfitUnderstanding Memory Management In Spark For Fun And Profit
Understanding Memory Management In Spark For Fun And Profit
 
From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.
 
Apache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & InternalsApache Spark in Depth: Core Concepts, Architecture & Internals
Apache Spark in Depth: Core Concepts, Architecture & Internals
 
Optimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL JoinsOptimizing Apache Spark SQL Joins
Optimizing Apache Spark SQL Joins
 
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
Everyday I'm Shuffling - Tips for Writing Better Spark Programs, Strata San J...
 
Understanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIsUnderstanding Query Plans and Spark UIs
Understanding Query Plans and Spark UIs
 
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
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
 
Spark shuffle introduction
Spark shuffle introductionSpark shuffle introduction
Spark shuffle introduction
 
Tuning and Debugging in Apache Spark
Tuning and Debugging in Apache SparkTuning and Debugging in Apache Spark
Tuning and Debugging in Apache Spark
 
Bucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing ShuffleBucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
Bucketing 2.0: Improve Spark SQL Performance by Removing Shuffle
 

Viewers also liked

Cloudera Impala 1.0
Cloudera Impala 1.0Cloudera Impala 1.0
Cloudera Impala 1.0Minwoo Kim
 
Hive Authorization
Hive AuthorizationHive Authorization
Hive AuthorizationMinwoo Kim
 
Couchbase Usecase : Mobile Game Cookierun
Couchbase Usecase : Mobile Game CookierunCouchbase Usecase : Mobile Game Cookierun
Couchbase Usecase : Mobile Game CookierunMinwoo Kim
 
Hbasecon2013 Wrap Up
Hbasecon2013 Wrap UpHbasecon2013 Wrap Up
Hbasecon2013 Wrap UpMinwoo Kim
 
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈Minwoo Kim
 
introduce of Hadoop map reduce
introduce of Hadoop map reduceintroduce of Hadoop map reduce
introduce of Hadoop map reduceDaeyong Shin
 
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지Changje Jeong
 
쿠키런 1년, 서버개발 분투기
쿠키런 1년, 서버개발 분투기쿠키런 1년, 서버개발 분투기
쿠키런 1년, 서버개발 분투기Brian Hong
 
Hadoop Overview 1
Hadoop Overview 1Hadoop Overview 1
Hadoop Overview 1Kay Kim
 
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.LtdPlus21 Co.ltd
 
Hive introduction 介绍
Hive  introduction 介绍Hive  introduction 介绍
Hive introduction 介绍ablozhou
 
User-Defined Table Generating Functions
User-Defined Table Generating FunctionsUser-Defined Table Generating Functions
User-Defined Table Generating Functionspauly1
 
Datacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConDatacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConamarsri
 
하이브 최적화 방안
하이브 최적화 방안하이브 최적화 방안
하이브 최적화 방안Teddy Choi
 
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for HadoopSeungYong Baek
 
Ten tools for ten big data areas 04_Apache Hive
Ten tools for ten big data areas 04_Apache HiveTen tools for ten big data areas 04_Apache Hive
Ten tools for ten big data areas 04_Apache HiveWill Du
 
Python과 Git으로 만드는 모바일 게임 패치 시스템
Python과 Git으로 만드는 모바일 게임 패치 시스템Python과 Git으로 만드는 모바일 게임 패치 시스템
Python과 Git으로 만드는 모바일 게임 패치 시스템Youngtaek Oh
 
An intriduction to hive
An intriduction to hiveAn intriduction to hive
An intriduction to hiveReza Ameri
 

Viewers also liked (20)

Cloudera Impala 1.0
Cloudera Impala 1.0Cloudera Impala 1.0
Cloudera Impala 1.0
 
Hive Authorization
Hive AuthorizationHive Authorization
Hive Authorization
 
Couchbase Usecase : Mobile Game Cookierun
Couchbase Usecase : Mobile Game CookierunCouchbase Usecase : Mobile Game Cookierun
Couchbase Usecase : Mobile Game Cookierun
 
Hbasecon2013 Wrap Up
Hbasecon2013 Wrap UpHbasecon2013 Wrap Up
Hbasecon2013 Wrap Up
 
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈
KGC 2014 가볍고 유연하게 데이터 분석하기 : 쿠키런 사례 중심 , 데브시스터즈
 
introduce of Hadoop map reduce
introduce of Hadoop map reduceintroduce of Hadoop map reduce
introduce of Hadoop map reduce
 
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지
AWS를 활용하여 Daily Report 만들기 : 로그 수집부터 자동화된 분석까지
 
쿠키런 1년, 서버개발 분투기
쿠키런 1년, 서버개발 분투기쿠키런 1년, 서버개발 분투기
쿠키런 1년, 서버개발 분투기
 
Hadoop Overview 1
Hadoop Overview 1Hadoop Overview 1
Hadoop Overview 1
 
Hive begins
Hive beginsHive begins
Hive begins
 
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd
플러스21 회사소개서 version 2016.02_company brief about Plus 21 Co.Ltd
 
Hive introduction 介绍
Hive  introduction 介绍Hive  introduction 介绍
Hive introduction 介绍
 
User-Defined Table Generating Functions
User-Defined Table Generating FunctionsUser-Defined Table Generating Functions
User-Defined Table Generating Functions
 
Datacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheConDatacubes in Apache Hive at ApacheCon
Datacubes in Apache Hive at ApacheCon
 
Advanced topics in hive
Advanced topics in hiveAdvanced topics in hive
Advanced topics in hive
 
하이브 최적화 방안
하이브 최적화 방안하이브 최적화 방안
하이브 최적화 방안
 
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop
하둡 알아보기(Learn about Hadoop basic), NetApp FAS NFS Connector for Hadoop
 
Ten tools for ten big data areas 04_Apache Hive
Ten tools for ten big data areas 04_Apache HiveTen tools for ten big data areas 04_Apache Hive
Ten tools for ten big data areas 04_Apache Hive
 
Python과 Git으로 만드는 모바일 게임 패치 시스템
Python과 Git으로 만드는 모바일 게임 패치 시스템Python과 Git으로 만드는 모바일 게임 패치 시스템
Python과 Git으로 만드는 모바일 게임 패치 시스템
 
An intriduction to hive
An intriduction to hiveAn intriduction to hive
An intriduction to hive
 

Similar to Apache Hive Hook

J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"
J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"
J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"Daniel Bryant
 
Hive Anatomy
Hive AnatomyHive Anatomy
Hive Anatomynzhang
 
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak   CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak PROIDEA
 
Automatisation in development and testing - within budget [IronCamp prague 20...
Automatisation in development and testing - within budget [IronCamp prague 20...Automatisation in development and testing - within budget [IronCamp prague 20...
Automatisation in development and testing - within budget [IronCamp prague 20...David Lukac
 
Google Back To Front: From Gears to App Engine and Beyond
Google Back To Front: From Gears to App Engine and BeyondGoogle Back To Front: From Gears to App Engine and Beyond
Google Back To Front: From Gears to App Engine and Beyonddion
 
CPAN 模組二三事
CPAN 模組二三事CPAN 模組二三事
CPAN 模組二三事Lin Yo-An
 
Complex Made Simple: Sleep Better with TorqueBox
Complex Made Simple: Sleep Better with TorqueBoxComplex Made Simple: Sleep Better with TorqueBox
Complex Made Simple: Sleep Better with TorqueBoxbobmcwhirter
 
Deploying Symfony | symfony.cat
Deploying Symfony | symfony.catDeploying Symfony | symfony.cat
Deploying Symfony | symfony.catPablo Godel
 
PyCon AU 2012 - Debugging Live Python Web Applications
PyCon AU 2012 - Debugging Live Python Web ApplicationsPyCon AU 2012 - Debugging Live Python Web Applications
PyCon AU 2012 - Debugging Live Python Web ApplicationsGraham Dumpleton
 
2015-GopherCon-Talk-Uptime.pdf
2015-GopherCon-Talk-Uptime.pdf2015-GopherCon-Talk-Uptime.pdf
2015-GopherCon-Talk-Uptime.pdfUtabeUtabe
 
Fisl - Deployment
Fisl - DeploymentFisl - Deployment
Fisl - DeploymentFabio Akita
 
Python在豆瓣的应用
Python在豆瓣的应用Python在豆瓣的应用
Python在豆瓣的应用Qiangning Hong
 
Operating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionOperating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionDatabricks
 
php & performance
 php & performance php & performance
php & performancesimon8410
 
The Ring programming language version 1.6 book - Part 42 of 189
The Ring programming language version 1.6 book - Part 42 of 189The Ring programming language version 1.6 book - Part 42 of 189
The Ring programming language version 1.6 book - Part 42 of 189Mahmoud Samir Fayed
 

Similar to Apache Hive Hook (20)

J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"
J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"
J1 2015 "Debugging Java Apps in Containers: No Heavy Welding Gear Required"
 
Hive Anatomy
Hive AnatomyHive Anatomy
Hive Anatomy
 
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak   CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak
CONFidence 2015: DTrace + OSX = Fun - Andrzej Dyjak
 
Osd ctw spark
Osd ctw sparkOsd ctw spark
Osd ctw spark
 
Automatisation in development and testing - within budget [IronCamp prague 20...
Automatisation in development and testing - within budget [IronCamp prague 20...Automatisation in development and testing - within budget [IronCamp prague 20...
Automatisation in development and testing - within budget [IronCamp prague 20...
 
Google Back To Front: From Gears to App Engine and Beyond
Google Back To Front: From Gears to App Engine and BeyondGoogle Back To Front: From Gears to App Engine and Beyond
Google Back To Front: From Gears to App Engine and Beyond
 
CPAN 模組二三事
CPAN 模組二三事CPAN 模組二三事
CPAN 模組二三事
 
Complex Made Simple: Sleep Better with TorqueBox
Complex Made Simple: Sleep Better with TorqueBoxComplex Made Simple: Sleep Better with TorqueBox
Complex Made Simple: Sleep Better with TorqueBox
 
groovy & grails - lecture 9
groovy & grails - lecture 9groovy & grails - lecture 9
groovy & grails - lecture 9
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache Apex
 
Sprockets
SprocketsSprockets
Sprockets
 
Deploying Symfony | symfony.cat
Deploying Symfony | symfony.catDeploying Symfony | symfony.cat
Deploying Symfony | symfony.cat
 
PyCon AU 2012 - Debugging Live Python Web Applications
PyCon AU 2012 - Debugging Live Python Web ApplicationsPyCon AU 2012 - Debugging Live Python Web Applications
PyCon AU 2012 - Debugging Live Python Web Applications
 
2015-GopherCon-Talk-Uptime.pdf
2015-GopherCon-Talk-Uptime.pdf2015-GopherCon-Talk-Uptime.pdf
2015-GopherCon-Talk-Uptime.pdf
 
Fisl - Deployment
Fisl - DeploymentFisl - Deployment
Fisl - Deployment
 
Python在豆瓣的应用
Python在豆瓣的应用Python在豆瓣的应用
Python在豆瓣的应用
 
Apache Flink Hands On
Apache Flink Hands OnApache Flink Hands On
Apache Flink Hands On
 
Operating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in ProductionOperating and Supporting Delta Lake in Production
Operating and Supporting Delta Lake in Production
 
php & performance
 php & performance php & performance
php & performance
 
The Ring programming language version 1.6 book - Part 42 of 189
The Ring programming language version 1.6 book - Part 42 of 189The Ring programming language version 1.6 book - Part 42 of 189
The Ring programming language version 1.6 book - Part 42 of 189
 

Recently uploaded

Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelDeepika Singh
 
WSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfOrbitshub
 
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...Orbitshub
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdfSandro Moreira
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Angeliki Cooney
 
DBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDropbox
 
ICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesrafiqahmad00786416
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsNanddeep Nachan
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherRemote DBA Services
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...apidays
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc
 
Vector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptxVector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptxRemote DBA Services
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...apidays
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...apidays
 
Exploring Multimodal Embeddings with Milvus
Exploring Multimodal Embeddings with MilvusExploring Multimodal Embeddings with Milvus
Exploring Multimodal Embeddings with MilvusZilliz
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...Zilliz
 
Platformless Horizons for Digital Adaptability
Platformless Horizons for Digital AdaptabilityPlatformless Horizons for Digital Adaptability
Platformless Horizons for Digital AdaptabilityWSO2
 
FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024The Digital Insurer
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProduct Anonymous
 

Recently uploaded (20)

Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot ModelMcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
Mcleodganj Call Girls 🥰 8617370543 Service Offer VIP Hot Model
 
WSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering DevelopersWSO2's API Vision: Unifying Control, Empowering Developers
WSO2's API Vision: Unifying Control, Empowering Developers
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
 
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
Navigating the Deluge_ Dubai Floods and the Resilience of Dubai International...
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
 
DBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor PresentationDBX First Quarter 2024 Investor Presentation
DBX First Quarter 2024 Investor Presentation
 
ICT role in 21st century education and its challenges
ICT role in 21st century education and its challengesICT role in 21st century education and its challenges
ICT role in 21st century education and its challenges
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectors
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
Apidays New York 2024 - APIs in 2030: The Risk of Technological Sleepwalk by ...
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
Vector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptxVector Search -An Introduction in Oracle Database 23ai.pptx
Vector Search -An Introduction in Oracle Database 23ai.pptx
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
 
Exploring Multimodal Embeddings with Milvus
Exploring Multimodal Embeddings with MilvusExploring Multimodal Embeddings with Milvus
Exploring Multimodal Embeddings with Milvus
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
 
Platformless Horizons for Digital Adaptability
Platformless Horizons for Digital AdaptabilityPlatformless Horizons for Digital Adaptability
Platformless Horizons for Digital Adaptability
 
FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 

Apache Hive Hook

  • 1. Apache Hive Hook 2013. 8 Minwoo Kim michael.kim@nexr.com
  • 2. Apache Hive Hook • The reason why I made this is that Ryan asked me about hive hook, but I couldn’t find any info about hook in hive wiki. • I hope this will be helpful to develop applications using Hive when you want to get extra info while executing a query on Hive. • This document was written based on release-0.11 tag • Source: - https://github.com/apache/hive (mirror of apache hive)
  • 3. What is a hook? • As you know, this is about computer programming technique, but .. • Hooking - Techniques for intercepting function calls or messages or events in an operating system, applications, and other software components. • Hook - Code that handles intercepted function calls, events or messages
  • 4. Hive provides some hooking points • pre-execution • post-execution • execution-failure • pre- and post-driver-run • pre- and post-semantic-analyze • metastore-initialize
  • 5. How to set up hooks in Hive <property> <name>hive.exec.pre.hooks</name> <value></value> <description> Comma-separated list of pre-execution hooks to be invoked for each statement. A pre-execution hook is specified as the name of a Java class which implements the org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface. </description> </property> hive-site.xml <property> <name>hive.aux.jars.path</name> <value></value> </property> Setting hook property Setting path of jars contains implementations of hook interfaces or abstract class You can use hive.added.jars.path instead of hive.aux.jars.path
  • 6. Hive hook properties and interfaces Property Interface or Abstract class hive.exec.pre.hooks org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext (PreExecute is deprecated) hive.exec.post.hooks org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext (PostExecute is deprecated) hive.exec.failure.hooks org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext hive.metastore.init.hooks org.apache.hadoop.hive.metastore.MetaStoreInitListener hive.exec.driver.run.hooks org.apache.hadoop.hive.ql.HiveDriverRunHook hive.semantic.analyzer.hook org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook
  • 7. When those hooks fire? • You can submit a query on Hive through the following entry points - CLIDriver main method (called by shell script) - HCatCli main method (called by shell script) - HiveServer (called by thrift client) - HiveServer2 (called by thrift client or beeline)
  • 8. CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd() ↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠ ➔ processLocalCmd() ➔ Driver.run() ➠ CLIDriver ➔ is remote ? yes no
  • 9. CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd() ↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠ ➔ processLocalCmd() ➔ Driver.run() ➠ CLIDriver ➔ is remote ? yes no HCatCli HCatCli.main() ➔ processLine() ➔ processCmd() ➔ HCatDriver.run() ⤇ Driver.run() ➠
  • 10. HiveServer.execute() ➔ Driver.run() ➠ HiveServer CLIDriver.main() ➔ run() ➔ executeDriver() ➔ processLine() ➔ processCmd() ↳ CliSessionState.getClient() ↳ HiveClient.execute() ➠ ➔ processLocalCmd() ➔ Driver.run() ➠ CLIDriver ➔ is remote ? yes no HCatCli HCatCli.main() ➔ processLine() ➔ processCmd() ➔ HCatDriver.run() ⤇ Driver.run() ➠
  • 11. HiveServer2 ThriftCLIService.ExecuteStatement() ➔ CLIService.executeStatement() CLIService.executeStatement() ↳ SessionManager.getSession() ↳ HiveSession.executeStatement() ↳ OperationManager.newExecuteStatementOperation() ↳ SQLOperation.run() ➔ Driver.run() ➠ ⤶
  • 12. HiveServer2 ThriftCLIService.ExecuteStatement() ➔ CLIService.executeStatement() CLIService.executeStatement() ↳ SessionManager.getSession() ↳ HiveSession.executeStatement() ↳ OperationManager.newExecuteStatementOperation() ↳ SQLOperation.run() ➔ Driver.run() ➠ • OperationManager.newExecuteStatementOperation() is like a kind of factory - AddResourceOperation, DeleteResourceOperation, DfsOperation, GetCatalogsOperation, GetColumnsOperation, GetFunctionsOperation, GetSchemasOperation, GetTablesOperation, GetTableTypesOperation, GetTypeInfoOperation, SetOperation, SQLOperation ⤶
  • 13. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse()
  • 14. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↝ HiveParser { • HiveParser.g - SelectClauseParser.g - FromClauseParser.g - IdentifiersParser.g • ParseDriver.parse() - Command String ➡ root of AST tree
  • 15. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() • SemanticAnalyzerFactory.get(conf, ast) - SemanticAnalyzer, ColumnStatsSemanticAnalyzer, ExplainSemanticAnalyzer, ExportSemanticAnalyzer, FunctionSemanticAnalyzer, ImportSemanticAnalyzer, LoadSemanticAnalyzer, MacroSemanticAnalyzer
  • 16. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() {
  • 17. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() • FilterOperator • SelectOperator • ForwardOperator • FileSinkOperator • ScriptOperator • PTFOperator • ReduceSinkOperator • ExtractOperator • GroupByOperator • JoinOperator • MapJoinOperator • SMBMapJoinOperator • LimitOperator • TableScanOperator • UnionOperator • UDTFOperator • LateralViewJoinOperator • LateralViewForwardOperator • HashTableDummyOperator • HashTableSinkOperator • DummyStoreOperator • DemuxOperator • MuxOperator ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() {
  • 18. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() • PredicateTransitivePropagate • PredicatePushDown • PartitionPruner • PartitionConditionRemover • ListBucketingPruner • ListBucketingPruner • ColumnPruner • SkewJoinOptimizer • RewriteGBUsingIndex • GroupByOptimizer • SamplePruner • MapJoinProcessor • BucketMapJoinOptimizer • BucketMapJoinOptimizer • SortedMergeBucketMapJoinO ptimizer • BucketingSortingReduceSink Optimizer • UnionProcessor • JoinReorder • ReduceSinkDeDuplication • NonBlockingOpDeDupProc • GlobalLimitOptimizer • CorrelationOptimizer • SimpleFetchOptimizer ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() {
  • 19. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() • MapRedTask • FetchTask • ConditionalTask • ExplainTask • CopyTask • DDLTask • MoveTask • FunctionTask • StatsTask • ColumnStatsTask • DependencyCollectionTask ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() {
  • 20. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() ↳ Driver.execute() ➔ loop (List<Task>) ⟳ Driver.launchTask() ➔ TaskRunner.runSequential() ➔ Task.executeTask() ➔ Task.execute() ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() {
  • 21. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() ↳ Driver.execute() ➔ loop (List<Task>) ⟳ Driver.launchTask() ➔ TaskRunner.runSequential() ➔ Task.executeTask() ➔ Task.execute() ➔ analyzeInternal() • processPositionAlias() • doPhase1() • getMetaData() • genPlan() • Optimizer.optimize() • MapReduceCompiler.compile() { • ex) MapRedTask.execute() ⤇ ExecDriver.execute() ➔ JobClient.submitJob() ExecMapper, ExecReducer
  • 22. ➠ Driver.run() ➔ Driver.runInternal() ↳ Driver.compile() ↳ ParseDriver.parse() ↳ SemanticAnalyzer.analyze() ↳ Driver.execute() ➔ loop (List<Task>) ⟳ Driver.launchTask() ➔ TaskRunner.runSequential() ➔ Task.executeTask() ➔ Task.execute() PRE- and POST-DRIVER-RUN PRE- and POST-SEMANTIC-ANALYZE PRE-, POST-EXEC and ON-FAILURE
  • 23. HiveServer2.main() ➔ HiveServer2.start() ➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠
  • 24. HiveServer2.main() ➔ HiveServer2.start() ➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠ ➔ HiveSession.getMetaStoreClient() ➔ new HiveMetaStoreClient() ➠ CLIService.executeStatement() ⇒ GetColumnsOperation.run() GetSchemasOperation.run() GetTablesOperation.run()
  • 25. HiveServer2.main() ➔ HiveServer2.start() ➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠ ➔ HiveSession.getMetaStoreClient() ➔ new HiveMetaStoreClient() ➠ CLIService.executeStatement() ⇒ SemanticAnalyzer ↝ Hive ↝ getMSC() is invoked by many other methods in Hive object Hive.getMSC() ➔ Hive.createMetaStoreClient() ➔ RetryingHMSHandler.getProxy() ➠ GetColumnsOperation.run() GetSchemasOperation.run() GetTablesOperation.run()
  • 26. HiveServer2.main() ➔ HiveServer2.start() ➔ CLIService.start() ➔ new HiveMetaStoreClient() ➠ ➔ HiveSession.getMetaStoreClient() ➔ new HiveMetaStoreClient() ➠ ➠ new HiveMetaStoreClient() ➔ HiveMetaStore.newHMSHandler() ➔ RetryingHMSHandler.getProxy() ➔ new RetryingHMSHandler() ➔ new HMSHandler() ➔ HMSHandler.init() ➔ HiveMetaStore.init() CLIService.executeStatement() ⇒ MATASTORE-INIT SemanticAnalyzer ↝ Hive ↝ getMSC() is invoked by many other methods in Hive object Hive.getMSC() ➔ Hive.createMetaStoreClient() ➔ RetryingHMSHandler.getProxy() ➠ GetColumnsOperation.run() GetSchemasOperation.run() GetTablesOperation.run()
  • 27. How Hive executes hooks List<HiveDriverRunHook> driverRunHooks; try { driverRunHooks = getHooks(HiveConf.ConfVars.HIVE_DRIVER_RUN_HOOKS, HiveDriverRunHook.class); for (HiveDriverRunHook driverRunHook : driverRunHooks) { driverRunHook.preDriverRun(hookContext); } } catch (Exception e) { • Hive executes multiple hooks on each hook points. ex. Driver.runInternal()
  • 28. 1. MetaStoreInitListener public abstract class MetaStoreInitListener implements Configurable { private Configuration conf; public MetaStoreInitListener(Configuration config){ this.conf = config; } public abstract void onInit(MetaStoreInitContext context) throws MetaException; @Override public Configuration getConf() { return this.conf; } @Override public void setConf(Configuration config) { this.conf = config; } }
  • 29. 1. MetaStoreInitListener public abstract class MetaStoreInitListener implements Configurable { private Configuration conf; public MetaStoreInitListener(Configuration config){ this.conf = config; } public abstract void onInit(MetaStoreInitContext context) throws MetaException; @Override public Configuration getConf() { return this.conf; } @Override public void setConf(Configuration config) { this.conf = config; } }
  • 30. What MetaStoreInitContext got • has Nothing! - This hook just alarms you when metastore initialize. (but you, of course, can get HiveConf by calling getConf()) public class MetaStoreInitContext { }
  • 31. 2. HiveDriverRunHook • preDriverRun - Invoked before Hive begins any processing of a command in the Driver, before compilation • postDriverRun - Invoked after Hive performs any processing of a command, just before a response is returned to the entity calling the Driver.run() public interface HiveDriverRunHook extends Hook { public void preDriverRun( HiveDriverRunHookContext hookContext) throws Exception; public void postDriverRun( HiveDriverRunHookContext hookContext) throws Exception; }
  • 32. What HiveDriverRunHookContext got • You can get command string from this hook context. - This is the only thing that HiveDriverRunHookContext has. public interface HiveDriverRunHookContext extends Configurable{ public String getCommand(); public void setCommand(String command); }
  • 33. 3.AbstractSemanticAnalyzerHook • You can get - HiveSemanticAnalyzerHookContext and ASTNode (Root node of abstract syntax tree) before analyze. - HiveSemanticAnalyzerHookContext and List<Task> after analyze. public abstract class AbstractSemanticAnalyzerHook implements HiveSemanticAnalyzerHook { public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context,ASTNode ast) throws SemanticException { return ast; } public void postAnalyze(HiveSemanticAnalyzerHookContext context, List<Task<? extends Serializable>> rootTasks) throws SemanticException { } }
  • 34. What HiveSemanticAnalyzerHookContext got • Hive Object - contains information about a set of data in HDFS organized for query processing. (from comment) • ReadEntity, WriteEntity • update method will be invoked after the semantic analyzer completes. public interface HiveSemanticAnalyzerHookContext extends Configurable{ public Hive getHive() throws HiveException; public void update(BaseSemanticAnalyzer sem); public Set<ReadEntity> getInputs(); public Set<WriteEntity> getOutputs(); }
  • 35. How Hive executes analyzer hooks List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class); // ~ ellipsis ~ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); hookCtx.setConf(conf); for (AbstractSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } sem.analyze(tree, ctx); hookCtx.update(sem); for (AbstractSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, sem.getRootTasks()); }
  • 36. How Hive executes analyzer hooks List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class); // ~ ellipsis ~ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); hookCtx.setConf(conf); for (AbstractSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } sem.analyze(tree, ctx); hookCtx.update(sem); for (AbstractSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, sem.getRootTasks()); }
  • 37. How Hive executes analyzer hooks List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class); // ~ ellipsis ~ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); hookCtx.setConf(conf); for (AbstractSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } sem.analyze(tree, ctx); hookCtx.update(sem); for (AbstractSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, sem.getRootTasks()); }
  • 38. How Hive executes analyzer hooks List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class); // ~ ellipsis ~ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); hookCtx.setConf(conf); for (AbstractSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } sem.analyze(tree, ctx); hookCtx.update(sem); for (AbstractSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, sem.getRootTasks()); }
  • 39. How Hive executes analyzer hooks List<AbstractSemanticAnalyzerHook> saHooks = getHooks(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK, AbstractSemanticAnalyzerHook.class); // ~ ellipsis ~ HiveSemanticAnalyzerHookContext hookCtx = new HiveSemanticAnalyzerHookContextImpl(); hookCtx.setConf(conf); for (AbstractSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } sem.analyze(tree, ctx); hookCtx.update(sem); for (AbstractSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, sem.getRootTasks()); }
  • 40. 4. ExecuteWithHookContext • Can be used in the followings - hive.exec.pre.hooks - hive.exec.post.hooks - hive.exec.failure.hooks public interface ExecuteWithHookContext extends Hook { /**    *    * @param hookContext    * The hook context passed to each hooks.    * @throws Exception    */ void run(HookContext hookContext) throws Exception; }
  • 41. What HookContext got • HookType - PRE_EXEC_HOOK, POST_EXEC_HOOK, ON_FAILURE_HOOK • QueryPlan • HiveConf • LineageInfo • UserGroupInformation • OperationName • List<TaskRunner> completeTaskList • Set<ReadEntity> inputs • Set<WriteEntity> outputs • Map<String, ContentSummary> inputPathToContentSummary
  • 42. How Hive fires hooks without executing query physically • This has the effect of causing the pre/post execute hooks to fire. ALTER TABLE table_name TOUCH [PARTITION partitionSpec];
  • 43. MetaStore Event Listeners Property Abstract Class hive.metastore.pre.event.listeners MetaStorePreEventListener hive.metastore.end.function.listeners MetaStoreEndFunctionListener hive.metastore.event.listeners MetaStoreEventListener package : org.apache.hadoop.hive.metastore • I think those listeners look like hooks. • I couldn’t find any particular differences between listeners and hooks while just taking a look. The only thing I found is that listeners can’t affect query processing. It can only read. • Anyway, it looks useful to let you know when a metastore do something.
  • 44. MetaStoreEventListener • The followings will be performed when a particular event occurs on a metastore. - onCreateTable - onDropTable - onAlterTable - onDropPartition - onAlterPartition - onCreateDatabase - onDropDatabase - onLoadPartitionDone If you need more details, see org.apache.hadoop.hive.metastore.MetaStoreEventListener
  • 45. Be careful! • Hooks - can be a critical failure point! (you should better catch runtime exceptions) - are preformed synchronously. - can affect query processing time.
  • 46. Let's try it out • Demo - Don’t be surprised if it doesn’t work. - That’s the way the demo is...
  • 47. Thanks! • Questions? • Resources - https://cwiki.apache.org/confluence/display/Hive/ - https://github.com/apache/hive