@Override protected FlowStepJob createFlowStepJob( ClientState clientState, FlowProcess<TezConfiguration> flowProcess, TezConfiguration initializedStepConfig ) { DAG dag = createDAG( flowProcess, initializedStepConfig ); return new Hadoop2TezFlowStepJob( clientState, this, initializedStepConfig, dag ); }
public Hadoop2TezFlowStepJob( ClientState clientState, BaseFlowStep<TezConfiguration> flowStep, TezConfiguration currentConf, DAG dag ) { super( clientState, currentConf, flowStep, getJobPollingInterval( currentConf ), getStoreInterval( currentConf ), getChildDetailsBlockingDuration( currentConf ) ); this.dag = dag; if( flowStep.isDebugEnabled() ) flowStep.logDebug( "using polling interval: " + pollingInterval ); }
protected boolean internalNonBlockingIsSuccessful() throws IOException { return isDagStatusComplete() && getDagStatusState() == DAGStatus.State.SUCCEEDED; }
protected void internalBlockOnStop() throws IOException { if( isDagStatusComplete() ) return; try { if( dagClient != null ) dagClient.tryKillDAG(); // sometimes throws an NPE } catch( Exception exception ) { flowStep.logWarn( "exception during attempt to kill dag", exception ); } stopDAGClient(); stopTezClient(); }
protected void internalNonBlockingStart() throws IOException { try { if( !isTimelineServiceEnabled( jobConfiguration ) ) flowStep.logWarn( "'" + YarnConfiguration.TIMELINE_SERVICE_ENABLED + "' is disabled, please enable to capture detailed metrics of completed flows, this may require starting the YARN timeline server daemon" ); TezConfiguration workingConf = new TezConfiguration( jobConfiguration ); // this could be problematic flowStep.logInfo( "tez session mode enabled: " + workingConf.getBoolean( TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT ) ); prepareEnsureStagingDir( workingConf ); tezClient = TezClient.create( flowStep.getName(), workingConf, ( (Hadoop2TezFlowStep) flowStep ).getAllLocalResources(), null ); tezClient.start(); dagClient = tezClient.submitDAG( dag ); dagId = Util.returnInstanceFieldIfExistsSafe( dagClient, "dagId" ); flowStep.logInfo( "submitted tez dag to app master: {}, with dag id: {}", tezClient.getAppMasterApplicationId(), dagId ); } catch( TezException exception ) { this.throwable = exception; throw new CascadingException( exception ); } }
private boolean isDagStatusComplete() { DAGStatus dagStatus = getDagStatus(); if( dagStatus == null ) flowStep.logWarn( "getDagStatus returned null" ); return dagStatus != null && dagStatus.isCompleted(); }
private Path prepareEnsureStagingDir( TezConfiguration workingConf ) throws IOException { String stepStagingPath = createStepStagingPath(); workingConf.set( TezConfiguration.TEZ_AM_STAGING_DIR, stepStagingPath ); Path stagingDir = new Path( stepStagingPath ); FileSystem fileSystem = FileSystem.get( workingConf ); stagingDir = fileSystem.makeQualified( stagingDir ); TokenCache.obtainTokensForNamenodes( new Credentials(), new Path[]{stagingDir}, workingConf ); TezClientUtils.ensureStagingDirExists( workingConf, stagingDir ); if( fileSystem.getScheme().startsWith( "file:/" ) ) new File( stagingDir.toUri() ).mkdirs(); return stagingDir; }
protected boolean internalNonBlockingIsComplete() throws IOException { return isDagStatusComplete(); }
private void stopTezClient() { try { if( tezClient == null ) return; if( isRemoteExecution() ) { tezClient.stop(); // will shutdown the session return; } // the Tez LocalClient will frequently hang on #stop(), this causes tests to never complete Boolean result = Util.submitWithTimeout( new Callable<Boolean>() { @Override public Boolean call() throws Exception { tezClient.stop(); return true; } }, 5, TimeUnit.MINUTES ); if( result == null || !result ) flowStep.logWarn( "tezClient#stop() timed out after 5 minutes, cancelling call, continuing" ); } catch( Exception exception ) { flowStep.logWarn( "exception during attempt to cleanup client", exception ); } }
protected void internalBlockOnStop() throws IOException { if( isDagStatusComplete() ) return; try { if( dagClient != null ) dagClient.tryKillDAG(); // sometimes throws an NPE } catch( Exception exception ) { flowStep.logWarn( "exception during attempt to kill dag", exception ); } stopDAGClient(); stopTezClient(); }
protected void internalNonBlockingStart() throws IOException { try { if( !isTimelineServiceEnabled( jobConfiguration ) ) flowStep.logWarn( "'" + YarnConfiguration.TIMELINE_SERVICE_ENABLED + "' is disabled, please enable to capture detailed metrics of completed flows, this may require starting the YARN timeline server daemon" ); TezConfiguration workingConf = new TezConfiguration( jobConfiguration ); // this could be problematic flowStep.logInfo( "tez session mode enabled: " + workingConf.getBoolean( TezConfiguration.TEZ_AM_SESSION_MODE, TezConfiguration.TEZ_AM_SESSION_MODE_DEFAULT ) ); prepareEnsureStagingDir( workingConf ); tezClient = TezClient.create( flowStep.getName(), workingConf, ( (Hadoop2TezFlowStep) flowStep ).getAllLocalResources(), null ); tezClient.start(); dagClient = tezClient.submitDAG( dag ); dagId = Util.returnInstanceFieldIfExistsSafe( dagClient, "dagId" ); flowStep.logInfo( "submitted tez dag to app master: {}, with dag id: {}", tezClient.getAppMasterApplicationId(), dagId ); } catch( TezException exception ) { this.throwable = exception; throw new CascadingException( exception ); } }
private boolean isDagStatusComplete() { DAGStatus dagStatus = getDagStatus(); if( dagStatus == null ) flowStep.logWarn( "getDagStatus returned null" ); return dagStatus != null && dagStatus.isCompleted(); }
private Path prepareEnsureStagingDir( TezConfiguration workingConf ) throws IOException { String stepStagingPath = createStepStagingPath(); workingConf.set( TezConfiguration.TEZ_AM_STAGING_DIR, stepStagingPath ); Path stagingDir = new Path( stepStagingPath ); FileSystem fileSystem = FileSystem.get( workingConf ); stagingDir = fileSystem.makeQualified( stagingDir ); TokenCache.obtainTokensForNamenodes( new Credentials(), new Path[]{stagingDir}, workingConf ); TezClientUtils.ensureStagingDirExists( workingConf, stagingDir ); if( fileSystem.getScheme().startsWith( "file:/" ) ) new File( stagingDir.toUri() ).mkdirs(); return stagingDir; }
protected boolean internalNonBlockingIsComplete() throws IOException { return isDagStatusComplete(); }
private void stopTezClient() { try { if( tezClient == null ) return; if( isRemoteExecution() ) { tezClient.stop(); // will shutdown the session return; } // the Tez LocalClient will frequently hang on #stop(), this causes tests to never complete Boolean result = Util.submitWithTimeout( new Callable<Boolean>() { @Override public Boolean call() throws Exception { tezClient.stop(); return true; } }, 5, TimeUnit.MINUTES ); if( result == null || !result ) flowStep.logWarn( "tezClient#stop() timed out after 5 minutes, cancelling call, continuing" ); } catch( Exception exception ) { flowStep.logWarn( "exception during attempt to cleanup client", exception ); } }
public Hadoop2TezFlowStepJob( ClientState clientState, BaseFlowStep<TezConfiguration> flowStep, TezConfiguration currentConf, DAG dag ) { super( clientState, currentConf, flowStep, getJobPollingInterval( currentConf ), getStoreInterval( currentConf ), getChildDetailsBlockingDuration( currentConf ) ); this.dag = dag; if( flowStep.isDebugEnabled() ) flowStep.logDebug( "using polling interval: " + pollingInterval ); }
protected boolean internalNonBlockingIsSuccessful() throws IOException { return isDagStatusComplete() && getDagStatusState() == DAGStatus.State.SUCCEEDED; }
private DAGStatus.State getDagStatusState() { DAGStatus dagStatus = getDagStatus(); if( dagStatus == null ) { flowStep.logWarn( "getDagStatus returned null" ); return null; } DAGStatus.State state = dagStatus.getState(); if( state == null ) flowStep.logWarn( "dagStatus#getState returned null" ); return state; }
@Override protected FlowStepJob createFlowStepJob( ClientState clientState, FlowProcess<TezConfiguration> flowProcess, TezConfiguration initializedStepConfig ) { DAG dag = createDAG( flowProcess, initializedStepConfig ); return new Hadoop2TezFlowStepJob( clientState, this, initializedStepConfig, dag ); }
protected boolean internalIsStartedRunning() { // this is an alternative, seems to be set in tests sooner // but unsure if the tasks are actually engaged return getDagStatusState() == DAGStatus.State.RUNNING || isDagStatusComplete(); /* DAGStatus dagStatus = getDagStatus(); if( dagStatus == null ) return false; Progress dagProgress = dagStatus.getDAGProgress(); // not strictly true if( dagProgress == null ) return false; // same as showing progress in map/reduce int completed = dagProgress.getRunningTaskCount() + dagProgress.getFailedTaskCount() + dagProgress.getKilledTaskCount() + dagProgress.getSucceededTaskCount(); return completed > 0; */ } }