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.util.ArrayList; 025import java.util.Collection; 026import java.util.Collections; 027import java.util.List; 028import java.util.Properties; 029import java.util.concurrent.Callable; 030import java.util.concurrent.CancellationException; 031import java.util.concurrent.ExecutorService; 032import java.util.concurrent.Executors; 033import java.util.concurrent.Future; 034import java.util.concurrent.Semaphore; 035 036import cascading.flow.FlowNode; 037import cascading.flow.FlowProcess; 038import cascading.flow.local.LocalFlowStep; 039import cascading.flow.local.stream.graph.LocalStepStreamGraph; 040import cascading.flow.stream.duct.Duct; 041import cascading.flow.stream.graph.StreamGraph; 042import cascading.util.Util; 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045 046import static cascading.util.LogUtil.logCounters; 047import static cascading.util.LogUtil.logMemory; 048 049/** 050 * 051 */ 052public class LocalStepRunner implements Callable<Throwable> 053 { 054 private static final Logger LOG = LoggerFactory.getLogger( LocalStepRunner.class ); 055 056 private final FlowProcess<Properties> currentProcess; 057 058 private volatile boolean started = false; 059 private volatile boolean completed = false; 060 private volatile boolean successful = false; 061 private volatile boolean stopped = false; 062 063 private final FlowNode flowNode; 064 private final StreamGraph streamGraph; 065 private final Collection<Duct> heads; 066 private Throwable throwable = null; 067 068 private Semaphore markComplete = new Semaphore( 0 ); 069 private List<Future<Throwable>> futures = Collections.emptyList(); 070 071 public LocalStepRunner( FlowProcess<Properties> flowProcess, LocalFlowStep step ) 072 { 073 this.currentProcess = flowProcess; 074 this.flowNode = Util.getFirst( step.getFlowNodeGraph().vertexSet() ); 075 this.streamGraph = new LocalStepStreamGraph( this.currentProcess, step, flowNode ); 076 this.heads = streamGraph.getHeads(); 077 } 078 079 public FlowProcess<Properties> getFlowProcess() 080 { 081 return currentProcess; 082 } 083 084 public boolean isCompleted() 085 { 086 return completed; 087 } 088 089 public void blockUntilStopped() 090 { 091 if( !started || completed ) 092 return; 093 094 stopped = true; 095 096 for( Future<Throwable> future : futures ) 097 future.cancel( true ); 098 099 try 100 { 101 markComplete.acquire(); 102 } 103 catch( InterruptedException exception ) 104 { 105 // do nothing 106 } 107 } 108 109 public boolean isStopped() 110 { 111 return stopped; 112 } 113 114 public boolean isSuccessful() 115 { 116 return successful; 117 } 118 119 public Throwable getThrowable() 120 { 121 return throwable; 122 } 123 124 @Override 125 public Throwable call() throws Exception 126 { 127 started = true; 128 boolean attemptedCleanup = false; 129 130 try 131 { 132 try 133 { 134 streamGraph.prepare(); 135 136 logMemory( LOG, "flow node id: " + flowNode.getID() + ", mem on start" ); 137 } 138 catch( Throwable currentThrowable ) 139 { 140 try 141 { 142 if( !( currentThrowable instanceof OutOfMemoryError ) ) 143 LOG.error( "unable to prepare operation graph", currentThrowable ); 144 145 completed = true; 146 successful = false; 147 throwable = currentThrowable; 148 149 return throwable; 150 } 151 finally 152 { 153 markComplete.release(); 154 } 155 } 156 157 if( stopped ) 158 { 159 markComplete.release(); 160 161 return null; 162 } 163 164 try 165 { 166 futures = spawnHeads(); 167 168 for( Future<Throwable> future : futures ) 169 { 170 try 171 { 172 throwable = future.get(); 173 } 174 catch( CancellationException exception ) 175 { 176 break; 177 } 178 179 if( throwable != null ) 180 break; 181 } 182 183 futures = Collections.emptyList(); 184 } 185 catch( Throwable currentThrowable ) 186 { 187 if( !( currentThrowable instanceof OutOfMemoryError ) ) 188 LOG.error( "unable to complete step", currentThrowable ); 189 190 throwable = currentThrowable; 191 } 192 193 try 194 { 195 attemptedCleanup = true; // set so we don't try again regardless 196 197 if( !( throwable instanceof OutOfMemoryError ) ) 198 streamGraph.cleanup(); 199 } 200 catch( Throwable currentThrowable ) 201 { 202 if( !( currentThrowable instanceof OutOfMemoryError ) ) 203 LOG.error( "unable to cleanup operation graph", currentThrowable ); 204 205 if( throwable == null ) 206 throwable = currentThrowable; 207 } 208 209 completed = true; 210 successful = throwable == null; 211 212 return throwable; 213 } 214 finally 215 { 216 try 217 { 218 if( !attemptedCleanup ) 219 streamGraph.cleanup(); 220 } 221 catch( Throwable currentThrowable ) 222 { 223 if( !( currentThrowable instanceof OutOfMemoryError ) ) 224 LOG.error( "unable to cleanup operation graph", currentThrowable ); 225 226 if( throwable == null ) 227 throwable = currentThrowable; 228 229 successful = false; 230 } 231 finally 232 { 233 markComplete.release(); 234 } 235 236 String message = "flow node id: " + flowNode.getID(); 237 logMemory( LOG, message + ", mem on close" ); 238 logCounters( LOG, message + ", counter:", currentProcess ); 239 } 240 } 241 242 private List<Future<Throwable>> spawnHeads() 243 { 244 // todo: consider a CyclicBarrier to syn all threads after the openForRead 245 // todo: should find all Callable Ducts and spawn them, group ducts may run on a timer etc 246 ExecutorService executors = Executors.newFixedThreadPool( heads.size() ); 247 List<Future<Throwable>> futures = new ArrayList<Future<Throwable>>(); 248 249 for( Duct head : heads ) 250 futures.add( executors.submit( (Callable) head ) ); 251 252 executors.shutdown(); 253 254 return futures; 255 } 256 }