001/*
002 * Copyright (c) 2016-2017 Chris K Wensel <chris@wensel.net>. All Rights Reserved.
003 * Copyright (c) 2007-2017 Xplenty, Inc. All Rights Reserved.
004 *
005 * Project and contact information: http://www.cascading.org/
006 *
007 * This file is part of the Cascading project.
008 *
009 * Licensed under the Apache License, Version 2.0 (the "License");
010 * you may not use this file except in compliance with the License.
011 * You may obtain a copy of the License at
012 *
013 *     http://www.apache.org/licenses/LICENSE-2.0
014 *
015 * Unless required by applicable law or agreed to in writing, software
016 * distributed under the License is distributed on an "AS IS" BASIS,
017 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
018 * See the License for the specific language governing permissions and
019 * limitations under the License.
020 */
021
022package cascading.flow.local.planner;
023
024import java.io.IOException;
025import java.util.Properties;
026import java.util.concurrent.ExecutorService;
027import java.util.concurrent.Executors;
028import java.util.concurrent.Future;
029
030import cascading.flow.local.LocalFlowProcess;
031import cascading.flow.local.LocalFlowStep;
032import cascading.flow.planner.FlowStepJob;
033import cascading.management.state.ClientState;
034import cascading.stats.FlowNodeStats;
035import cascading.stats.FlowStepStats;
036import cascading.stats.local.LocalStepStats;
037
038/**
039 *
040 */
041public class LocalFlowStepJob extends FlowStepJob<Properties>
042  {
043  private final LocalStepRunner stackRunner;
044  private Future<Throwable> future;
045
046  public LocalFlowStepJob( ClientState clientState, LocalFlowProcess flowProcess, LocalFlowStep flowStep )
047    {
048    super( clientState, flowStep.getConfig(), flowStep, 200, 1000, 1000 * 60 );
049    flowProcess.setStepStats( (LocalStepStats) this.flowStepStats );
050    this.stackRunner = new LocalStepRunner( flowProcess, flowStep );
051    }
052
053  @Override
054  protected FlowStepStats createStepStats( ClientState clientState )
055    {
056    return new LocalStepStats( flowStep, clientState );
057    }
058
059  @Override
060  protected boolean isRemoteExecution()
061    {
062    return false;
063    }
064
065  @Override
066  protected String internalJobId()
067    {
068    return "flow";
069    }
070
071  @Override
072  protected void internalNonBlockingStart() throws IOException
073    {
074    ExecutorService executors = Executors.newFixedThreadPool( 1 );
075
076    future = executors.submit( stackRunner );
077
078    executors.shutdown();
079    }
080
081  @Override
082  protected void updateNodeStatus( FlowNodeStats flowNodeStats )
083    {
084    }
085
086  @Override
087  protected boolean internalIsStartedRunning()
088    {
089    return future != null;
090    }
091
092  @Override
093  protected boolean internalNonBlockingIsComplete() throws IOException
094    {
095    return stackRunner.isCompleted();
096    }
097
098  @Override
099  protected Throwable getThrowable()
100    {
101    return stackRunner.getThrowable();
102    }
103
104  @Override
105  protected boolean internalNonBlockingIsSuccessful() throws IOException
106    {
107    return stackRunner.isSuccessful();
108    }
109
110  @Override
111  protected void internalBlockOnStop() throws IOException
112    {
113    stackRunner.blockUntilStopped();
114    }
115
116  @Override
117  protected void dumpDebugInfo()
118    {
119    }
120  }