public void runPreHooks(HookContext hookContext) throws HiveException { initialize(); invokeGeneralHook(preExecHooks, PerfLogger.PRE_HOOK, hookContext); }
public void runFailureHooks(HookContext hookContext) throws HiveException { initialize(); invokeGeneralHook(onFailureHooks, PerfLogger.FAILURE_HOOK, hookContext); }
public void runPostExecHooks(HookContext hookContext) throws HiveException { initialize(); invokeGeneralHook(postExecHooks, PerfLogger.POST_HOOK, hookContext); }
public void runPreDriverHooks(HiveDriverRunHookContext hookContext) throws HiveException { initialize(); try { for (HiveDriverRunHook driverRunHook : driverRunHooks) { driverRunHook.preDriverRun(hookContext); } } catch (HiveException e) { throw e; } catch (Exception e) { throw new HiveException("Error while invoking PreDriverHooks:" + HiveStringUtils.stringifyException(e), e); } }
public void runPostDriverHooks(HiveDriverRunHookContext hookContext) throws HiveException { initialize(); try { for (HiveDriverRunHook driverRunHook : driverRunHooks) { driverRunHook.postDriverRun(hookContext); } } catch (HiveException e) { throw e; } catch (Exception e) { throw new HiveException("Error while invoking PostDriverHooks:" + HiveStringUtils.stringifyException(e), e); } }
public ASTNode runPreAnalyzeHooks(HiveSemanticAnalyzerHookContext hookCtx, ASTNode tree) throws HiveException { initialize(); try { for (HiveSemanticAnalyzerHook hook : saHooks) { tree = hook.preAnalyze(hookCtx, tree); } return tree; } catch (HiveException e) { throw e; } catch (Exception e) { throw new HiveException("Error while invoking PreAnalyzeHooks:" + HiveStringUtils.stringifyException(e), e); } }
public void runPostAnalyzeHooks(HiveSemanticAnalyzerHookContext hookCtx, List<Task<? extends Serializable>> allRootTasks) throws HiveException { initialize(); try { for (HiveSemanticAnalyzerHook hook : saHooks) { hook.postAnalyze(hookCtx, allRootTasks); } } catch (HiveException e) { throw e; } catch (Exception e) { throw new HiveException("Error while invoking PostAnalyzeHooks:" + HiveStringUtils.stringifyException(e), e); } }
/** * Dispatches {@link QueryLifeTimeHook#beforeCompile(QueryLifeTimeHookContext)}. * * @param command the Hive command that is being run */ void runBeforeCompileHook(String command) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command).build(); for (QueryLifeTimeHook hook : queryHooks) { hook.beforeCompile(qhc); } } }
/** * Dispatches {@link QueryLifeTimeHook#afterCompile(QueryLifeTimeHookContext, boolean)}. * * @param command the Hive command that is being run * @param compileError true if there was an error while compiling the command, false otherwise */ void runAfterCompilationHook(String command, boolean compileError) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command).build(); for (QueryLifeTimeHook hook : queryHooks) { hook.afterCompile(qhc, compileError); } } }
/** * If {@link QueryLifeTimeHookWithParseHooks} have been loaded via the {@link HooksLoader} then invoke the * {@link QueryLifeTimeHookWithParseHooks#beforeParse(QueryLifeTimeHookContext)} method for each * {@link QueryLifeTimeHookWithParseHooks}. * * @param command the Hive command that is being run */ void runBeforeParseHook(String command) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command).build(); for (QueryLifeTimeHook hook : queryHooks) { if (hook instanceof QueryLifeTimeHookWithParseHooks) { ((QueryLifeTimeHookWithParseHooks) hook).beforeParse(qhc); } } } }
/** * Dispatches {@link QueryLifeTimeHook#afterExecution(QueryLifeTimeHookContext, boolean)}. * * @param command the Hive command that is being run * @param hookContext the {@link HookContext} of the command being run * @param executionError true if there was an error while executing the command, false otherwise */ void runAfterExecutionHook(String command, HookContext hookContext, boolean executionError) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command) .withHookContext(hookContext).build(); for (QueryLifeTimeHook hook : queryHooks) { hook.afterExecution(qhc, executionError); } } }
/** * If {@link QueryLifeTimeHookWithParseHooks} have been loaded via the {@link HooksLoader} then invoke the * {@link QueryLifeTimeHookWithParseHooks#afterParse(QueryLifeTimeHookContext, boolean)} method for each * {@link QueryLifeTimeHookWithParseHooks}. * * @param command the Hive command that is being run * @param parseError true if there was an error while parsing the command, false otherwise */ void runAfterParseHook(String command, boolean parseError) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command).build(); for (QueryLifeTimeHook hook : queryHooks) { if (hook instanceof QueryLifeTimeHookWithParseHooks) { ((QueryLifeTimeHookWithParseHooks) hook).afterParse(qhc, parseError); } } } }
/** * Dispatches {@link QueryLifeTimeHook#beforeExecution(QueryLifeTimeHookContext)}. * * @param command the Hive command that is being run * @param hookContext the {@link HookContext} of the command being run */ void runBeforeExecutionHook(String command, HookContext hookContext) { initialize(); if (!queryHooks.isEmpty()) { QueryLifeTimeHookContext qhc = new QueryLifeTimeHookContextImpl.Builder().withHiveConf(conf).withCommand(command) .withHookContext(hookContext).build(); for (QueryLifeTimeHook hook : queryHooks) { hook.beforeExecution(qhc); } } }