/
HadoopFlowProcess.java
326 lines (282 loc) · 8.18 KB
/
HadoopFlowProcess.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
/*
* Copyright (c) 2007-2012 Concurrent, Inc. All Rights Reserved.
*
* Project and contact information: http://www.cascading.org/
*
* This file is part of the Cascading project.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package cascading.flow.hadoop;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import cascading.CascadingException;
import cascading.flow.FlowProcess;
import cascading.flow.FlowSession;
import cascading.tap.Tap;
import cascading.tuple.Fields;
import cascading.tuple.TupleEntry;
import cascading.tuple.TupleEntryCollector;
import cascading.tuple.TupleEntryIterator;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.Reporter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Class HadoopFlowProcess is an implementation of {@link FlowProcess} for Hadoop. Use this interface to get direct
* access to the Hadoop JobConf and Reporter interfaces.
* <p/>
* Be warned that coupling to this implementation will cause custom {@link cascading.operation.Operation}s to
* fail if they are executed on a system other than Hadoop.
*
* @see cascading.flow.FlowSession
* @see JobConf
* @see Reporter
*/
public class HadoopFlowProcess extends FlowProcess<JobConf>
{
/** Field LOG */
private static final Logger LOG = LoggerFactory.getLogger( HadoopFlowProcess.class );
/** Field jobConf */
final JobConf jobConf;
/** Field isMapper */
private final boolean isMapper;
/** Field reporter */
Reporter reporter = Reporter.NULL;
private OutputCollector outputCollector;
private CompressionCodec codec;
private String codecs;
private String compress;
public HadoopFlowProcess()
{
this.jobConf = new JobConf();
this.isMapper = true;
}
public HadoopFlowProcess( JobConf jobConf )
{
this.jobConf = jobConf;
this.isMapper = true;
}
public HadoopFlowProcess( FlowSession flowSession, JobConf jobConf )
{
super( flowSession );
this.jobConf = jobConf;
this.isMapper = true;
}
/**
* Constructor HadoopFlowProcess creates a new HadoopFlowProcess instance.
*
* @param flowSession of type FlowSession
* @param jobConf of type JobConf
*/
public HadoopFlowProcess( FlowSession flowSession, JobConf jobConf, boolean isMapper )
{
super( flowSession );
this.jobConf = jobConf;
this.isMapper = isMapper;
}
public HadoopFlowProcess( HadoopFlowProcess flowProcess, JobConf jobConf )
{
super( flowProcess.getCurrentSession() );
this.jobConf = jobConf;
this.isMapper = flowProcess.isMapper();
this.reporter = flowProcess.getReporter();
}
@Override
public FlowProcess copyWith( JobConf jobConf )
{
return new HadoopFlowProcess( this, jobConf );
}
/**
* Method getJobConf returns the jobConf of this HadoopFlowProcess object.
*
* @return the jobConf (type JobConf) of this HadoopFlowProcess object.
*/
public JobConf getJobConf()
{
return jobConf;
}
@Override
public JobConf getConfigCopy()
{
return new JobConf( jobConf );
}
/**
* Method isMapper returns true if this part of the FlowProcess is a MapReduce mapper. If false, it is a reducer.
*
* @return boolean
*/
public boolean isMapper()
{
return isMapper;
}
public int getCurrentNumMappers()
{
return getJobConf().getNumMapTasks();
}
public int getCurrentNumReducers()
{
return getJobConf().getNumReduceTasks();
}
/**
* Method getCurrentTaskNum returns the task number of this task. Task 0 is the first task.
*
* @return int
*/
@Override
public int getCurrentTaskNum()
{
return getJobConf().getInt( "mapred.task.partition", 0 );
}
@Override
public int getNumConcurrentTasks()
{
if( isMapper() )
return getCurrentNumMappers();
else
return getCurrentNumReducers();
}
/**
* Method setReporter sets the reporter of this HadoopFlowProcess object.
*
* @param reporter the reporter of this HadoopFlowProcess object.
*/
public void setReporter( Reporter reporter )
{
this.reporter = reporter;
}
/**
* Method getReporter returns the reporter of this HadoopFlowProcess object.
*
* @return the reporter (type Reporter) of this HadoopFlowProcess object.
*/
public Reporter getReporter()
{
return reporter;
}
private final Reporter getReporterOrFail()
{
if( reporter == null )
throw new IllegalStateException( "unable to access the hadoop reporter, it is not available until the first map/reduce invocation" );
return reporter;
}
public void setOutputCollector( OutputCollector outputCollector )
{
this.outputCollector = outputCollector;
}
public OutputCollector getOutputCollector()
{
return outputCollector;
}
@Override
public Object getProperty( String key )
{
return jobConf.get( key );
}
@Override
public Collection<String> getPropertyKeys()
{
Set<String> keys = new HashSet<String>();
for( Map.Entry<String, String> entry : jobConf )
keys.add( entry.getKey() );
return Collections.unmodifiableSet( keys );
}
@Override
public void keepAlive()
{
getReporterOrFail().progress();
}
@Override
public void increment( Enum counter, long amount )
{
getReporterOrFail().incrCounter( counter, amount );
}
@Override
public void increment( String group, String counter, long amount )
{
getReporterOrFail().incrCounter( group, counter, amount );
}
@Override
public void setStatus( String status )
{
getReporterOrFail().setStatus( status );
}
@Override
public boolean isCounterStatusInitialized()
{
return getReporter() != null;
}
@Override
public TupleEntryIterator openTapForRead( Tap tap ) throws IOException
{
return tap.openForRead( this );
}
@Override
public TupleEntryCollector openTapForWrite( Tap tap ) throws IOException
{
return tap.openForWrite( this, outputCollector );
}
@Override
public TupleEntryCollector openTrapForWrite( Tap trap ) throws IOException
{
JobConf jobConf = new JobConf( getJobConf() );
int stepNum = jobConf.getInt( "cascading.flow.step.num", 0 );
String partname;
if( jobConf.getBoolean( "mapred.task.is.map", true ) )
partname = String.format( "-m-%05d-", stepNum );
else
partname = String.format( "-r-%05d-", stepNum );
jobConf.set( "cascading.tapcollector.partname", "%s%spart" + partname + "%05d" );
return trap.openForWrite( new HadoopFlowProcess( this, jobConf ), null );
}
@Override
public TupleEntryCollector openSystemIntermediateForWrite() throws IOException
{
return new TupleEntryCollector( Fields.size( 2 ) )
{
@Override
protected void collect( TupleEntry tupleEntry )
{
try
{
getOutputCollector().collect( tupleEntry.get( 0 ), tupleEntry.get( 1 ) );
}
catch( IOException exception )
{
throw new CascadingException( "failed collecting key and value", exception );
}
}
};
}
@Override
public JobConf copyConfig( JobConf jobConf )
{
return new JobConf( jobConf );
}
@Override
public Map<String, String> diffConfigIntoMap( JobConf defaultConfig, JobConf updatedConfig )
{
return HadoopUtil.getConfig( defaultConfig, updatedConfig );
}
@Override
public JobConf mergeMapIntoConfig( JobConf defaultConfig, Map<String, String> map )
{
return HadoopUtil.mergeConf( defaultConfig, map, false );
}
}