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.hadoop; 023 024import java.io.IOException; 025import java.util.Collection; 026import java.util.Collections; 027import java.util.HashSet; 028import java.util.Map; 029import java.util.Set; 030 031import cascading.CascadingException; 032import cascading.flow.FlowProcess; 033import cascading.flow.FlowSession; 034import cascading.flow.hadoop.util.HadoopUtil; 035import cascading.tap.Tap; 036import cascading.tuple.Fields; 037import cascading.tuple.TupleEntry; 038import cascading.tuple.TupleEntryCollector; 039import cascading.tuple.TupleEntryIterator; 040import org.apache.hadoop.conf.Configuration; 041import org.apache.hadoop.mapred.JobConf; 042import org.apache.hadoop.mapred.OutputCollector; 043import org.apache.hadoop.mapred.Reporter; 044import org.apache.hadoop.util.ReflectionUtils; 045 046/** 047 * Class HadoopFlowProcess is an implementation of {@link FlowProcess} for Hadoop. Use this interface to get direct 048 * access to the Hadoop JobConf and Reporter interfaces. 049 * <p> 050 * Be warned that coupling to this implementation will cause custom {@link cascading.operation.Operation}s to 051 * fail if they are executed on a system other than Hadoop. 052 * 053 * @see cascading.flow.FlowSession 054 * @see JobConf 055 * @see Reporter 056 */ 057public class HadoopFlowProcess extends FlowProcess<JobConf> implements MapRed 058 { 059 /** Field jobConf */ 060 final JobConf jobConf; 061 /** Field isMapper */ 062 private final boolean isMapper; 063 /** Field reporter */ 064 Reporter reporter = Reporter.NULL; 065 /** Field outputCollector */ 066 private OutputCollector outputCollector; 067 068 public HadoopFlowProcess() 069 { 070 this.jobConf = new JobConf(); 071 this.isMapper = true; 072 } 073 074 public HadoopFlowProcess( Configuration jobConf ) 075 { 076 this( new JobConf( jobConf ) ); 077 } 078 079 public HadoopFlowProcess( JobConf jobConf ) 080 { 081 this.jobConf = jobConf; 082 this.isMapper = true; 083 } 084 085 public HadoopFlowProcess( FlowSession flowSession, JobConf jobConf ) 086 { 087 super( flowSession ); 088 this.jobConf = jobConf; 089 this.isMapper = true; 090 } 091 092 /** 093 * Constructor HadoopFlowProcess creates a new HadoopFlowProcess instance. 094 * 095 * @param flowSession of type FlowSession 096 * @param jobConf of type JobConf 097 */ 098 public HadoopFlowProcess( FlowSession flowSession, JobConf jobConf, boolean isMapper ) 099 { 100 super( flowSession ); 101 this.jobConf = jobConf; 102 this.isMapper = isMapper; 103 } 104 105 public HadoopFlowProcess( HadoopFlowProcess flowProcess, JobConf jobConf ) 106 { 107 super( flowProcess ); 108 this.jobConf = jobConf; 109 this.isMapper = flowProcess.isMapper(); 110 this.reporter = flowProcess.getReporter(); 111 } 112 113 @Override 114 public FlowProcess copyWith( JobConf jobConf ) 115 { 116 return new HadoopFlowProcess( this, jobConf ); 117 } 118 119 /** 120 * Method getJobConf returns the jobConf of this HadoopFlowProcess object. 121 * 122 * @return the jobConf (type JobConf) of this HadoopFlowProcess object. 123 */ 124 public JobConf getJobConf() 125 { 126 return jobConf; 127 } 128 129 @Override 130 public JobConf getConfig() 131 { 132 return jobConf; 133 } 134 135 @Override 136 public JobConf getConfigCopy() 137 { 138 return HadoopUtil.copyJobConf( jobConf ); 139 } 140 141 /** 142 * Method isMapper returns true if this part of the FlowProcess is a MapReduce mapper. If false, it is a reducer. 143 * 144 * @return boolean 145 */ 146 public boolean isMapper() 147 { 148 return isMapper; 149 } 150 151 public int getCurrentNumMappers() 152 { 153 return getJobConf().getNumMapTasks(); 154 } 155 156 public int getCurrentNumReducers() 157 { 158 return getJobConf().getNumReduceTasks(); 159 } 160 161 /** 162 * Method getCurrentTaskNum returns the task number of this task. Task 0 is the first task. 163 * 164 * @return int 165 */ 166 @Override 167 public int getCurrentSliceNum() 168 { 169 return getJobConf().getInt( "mapred.task.partition", 0 ); 170 } 171 172 @Override 173 public int getNumProcessSlices() 174 { 175 if( isMapper() ) 176 return getCurrentNumMappers(); 177 else 178 return getCurrentNumReducers(); 179 } 180 181 /** 182 * Method setReporter sets the reporter of this HadoopFlowProcess object. 183 * 184 * @param reporter the reporter of this HadoopFlowProcess object. 185 */ 186 public void setReporter( Reporter reporter ) 187 { 188 if( reporter == null ) 189 this.reporter = Reporter.NULL; 190 else 191 this.reporter = reporter; 192 } 193 194 @Override 195 public Reporter getReporter() 196 { 197 return reporter; 198 } 199 200 public void setOutputCollector( OutputCollector outputCollector ) 201 { 202 this.outputCollector = outputCollector; 203 } 204 205 public OutputCollector getOutputCollector() 206 { 207 return outputCollector; 208 } 209 210 @Override 211 public Object getProperty( String key ) 212 { 213 return jobConf.get( key ); 214 } 215 216 @Override 217 public Collection<String> getPropertyKeys() 218 { 219 Set<String> keys = new HashSet<String>(); 220 221 for( Map.Entry<String, String> entry : jobConf ) 222 keys.add( entry.getKey() ); 223 224 return Collections.unmodifiableSet( keys ); 225 } 226 227 @Override 228 public Object newInstance( String className ) 229 { 230 if( className == null || className.isEmpty() ) 231 return null; 232 233 try 234 { 235 Class type = (Class) HadoopFlowProcess.class.getClassLoader().loadClass( className.toString() ); 236 237 return ReflectionUtils.newInstance( type, jobConf ); 238 } 239 catch( ClassNotFoundException exception ) 240 { 241 throw new CascadingException( "unable to load class: " + className.toString(), exception ); 242 } 243 } 244 245 @Override 246 public void keepAlive() 247 { 248 getReporter().progress(); 249 } 250 251 @Override 252 public void increment( Enum counter, long amount ) 253 { 254 getReporter().incrCounter( counter, amount ); 255 } 256 257 @Override 258 public void increment( String group, String counter, long amount ) 259 { 260 getReporter().incrCounter( group, counter, amount ); 261 } 262 263 @Override 264 public long getCounterValue( Enum counter ) 265 { 266 return getReporter().getCounter( counter ).getValue(); 267 } 268 269 @Override 270 public long getCounterValue( String group, String counter ) 271 { 272 return getReporter().getCounter( group, counter ).getValue(); 273 } 274 275 @Override 276 public void setStatus( String status ) 277 { 278 getReporter().setStatus( status ); 279 } 280 281 @Override 282 public boolean isCounterStatusInitialized() 283 { 284 return getReporter() != null; 285 } 286 287 @Override 288 public TupleEntryIterator openTapForRead( Tap tap ) throws IOException 289 { 290 return tap.openForRead( this ); 291 } 292 293 @Override 294 public TupleEntryCollector openTapForWrite( Tap tap ) throws IOException 295 { 296 return tap.openForWrite( this, null ); // do not honor sinkmode as this may be opened across tasks 297 } 298 299 @Override 300 public TupleEntryCollector openTrapForWrite( Tap trap ) throws IOException 301 { 302 JobConf jobConf = HadoopUtil.copyJobConf( getJobConf() ); 303 304 int stepNum = jobConf.getInt( "cascading.flow.step.num", 0 ); 305 String partname; 306 307 if( jobConf.getBoolean( "mapred.task.is.map", true ) ) 308 partname = String.format( "-m-%05d-", stepNum ); 309 else 310 partname = String.format( "-r-%05d-", stepNum ); 311 312 jobConf.set( "cascading.tapcollector.partname", "%s%spart" + partname + "%05d" ); 313 314 return trap.openForWrite( new HadoopFlowProcess( this, jobConf ), null ); // do not honor sinkmode as this may be opened across tasks 315 } 316 317 @Override 318 public TupleEntryCollector openSystemIntermediateForWrite() throws IOException 319 { 320 return new TupleEntryCollector( Fields.size( 2 ) ) 321 { 322 @Override 323 protected void collect( TupleEntry tupleEntry ) 324 { 325 try 326 { 327 getOutputCollector().collect( tupleEntry.getObject( 0 ), tupleEntry.getObject( 1 ) ); 328 } 329 catch( IOException exception ) 330 { 331 throw new CascadingException( "failed collecting key and value", exception ); 332 } 333 } 334 }; 335 } 336 337 @Override 338 public <C> C copyConfig( C config ) 339 { 340 return HadoopUtil.copyJobConf( config ); 341 } 342 343 @Override 344 public <C> Map<String, String> diffConfigIntoMap( C defaultConfig, C updatedConfig ) 345 { 346 return HadoopUtil.getConfig( (Configuration) defaultConfig, (Configuration) updatedConfig ); 347 } 348 349 @Override 350 public JobConf mergeMapIntoConfig( JobConf defaultConfig, Map<String, String> map ) 351 { 352 return HadoopUtil.mergeConf( defaultConfig, map, false ); 353 } 354 }