embulk-executor-mapreduce 0.1.2 → 0.1.3
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
checksums.yaml
CHANGED
@@ -1,7 +1,7 @@
|
|
1
1
|
---
|
2
2
|
SHA1:
|
3
|
-
metadata.gz:
|
4
|
-
data.tar.gz:
|
3
|
+
metadata.gz: dc83806412506bc567037cdf24a1c247a99abf13
|
4
|
+
data.tar.gz: 10a3dd696c3729f58a0c4e2b0ec5b0217cbdfcc1
|
5
5
|
SHA512:
|
6
|
-
metadata.gz:
|
7
|
-
data.tar.gz:
|
6
|
+
metadata.gz: f4ef4b1809a3acf01d0cf449efd4bc026fc77e60bd3aba8708102b0118c06a4b5a7bfd06ce70f497d0121da86ddfe9012dc20ab152c0c192ba0dad1eb80065be
|
7
|
+
data.tar.gz: 0999ab7bc7eb9fa1e71e61212c6be680c9e5fa232ea1cd4d57f45a8e7228e41ea4d4ee8c1e794de64b5caf77232264126979560fe09988bb55d264bfa2839e70
|
Binary file
|
@@ -1,15 +1,18 @@
|
|
1
1
|
package org.embulk.executor.mapreduce;
|
2
2
|
|
3
|
-
import java.io.EOFException;
|
4
|
-
import java.io.InterruptedIOException;
|
5
3
|
import java.util.List;
|
6
4
|
import java.util.ArrayList;
|
5
|
+
import java.util.concurrent.Callable;
|
7
6
|
import java.util.concurrent.ExecutionException;
|
8
7
|
import java.io.File;
|
9
8
|
import java.io.IOException;
|
9
|
+
import java.io.EOFException;
|
10
|
+
import java.io.InterruptedIOException;
|
11
|
+
import java.lang.reflect.InvocationTargetException;
|
10
12
|
import com.google.inject.Injector;
|
11
13
|
import com.google.common.base.Optional;
|
12
14
|
import com.google.common.base.Throwables;
|
15
|
+
import com.google.common.base.Throwables;
|
13
16
|
import com.google.common.collect.ImmutableList;
|
14
17
|
import com.fasterxml.jackson.core.JsonFactory;
|
15
18
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
@@ -24,6 +27,7 @@ import org.apache.hadoop.io.NullWritable;
|
|
24
27
|
import org.apache.hadoop.conf.Configuration;
|
25
28
|
import org.apache.hadoop.mapreduce.Job;
|
26
29
|
import org.apache.hadoop.mapreduce.JobContext;
|
30
|
+
import org.apache.hadoop.mapreduce.Counters;
|
27
31
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
28
32
|
import org.apache.hadoop.mapreduce.Mapper;
|
29
33
|
import org.apache.hadoop.mapreduce.Reducer;
|
@@ -48,11 +52,14 @@ import static org.embulk.spi.util.RetryExecutor.retryExecutor;
|
|
48
52
|
|
49
53
|
public class EmbulkMapReduce
|
50
54
|
{
|
55
|
+
private static final String SYSTEM_CONFIG_SERVICE_CLASS = "mapreduce_service_class";
|
56
|
+
|
51
57
|
private static final String CK_SYSTEM_CONFIG = "embulk.mapreduce.systemConfig";
|
52
58
|
private static final String CK_STATE_DIRECTORY_PATH = "embulk.mapreduce.stateDirectorypath";
|
53
59
|
private static final String CK_TASK_COUNT = "embulk.mapreduce.taskCount";
|
54
60
|
private static final String CK_TASK = "embulk.mapreduce.task";
|
55
61
|
private static final String CK_PLUGIN_ARCHIVE_SPECS = "embulk.mapreduce.pluginArchive.specs";
|
62
|
+
|
56
63
|
private static final String PLUGIN_ARCHIVE_FILE_NAME = "gems.zip";
|
57
64
|
|
58
65
|
public static void setSystemConfig(Configuration config, ModelManager modelManager, ConfigSource systemConfig)
|
@@ -105,51 +112,157 @@ public class EmbulkMapReduce
|
|
105
112
|
public static Injector newEmbulkInstance(Configuration config)
|
106
113
|
{
|
107
114
|
ConfigSource systemConfig = getSystemConfig(config);
|
108
|
-
|
115
|
+
String serviceClassName = systemConfig.get(String.class, SYSTEM_CONFIG_SERVICE_CLASS, "org.embulk.EmbulkService");
|
116
|
+
|
117
|
+
try {
|
118
|
+
Object obj;
|
119
|
+
if (serviceClassName.equals("org.embulk.EmbulkService")) {
|
120
|
+
obj = new EmbulkService(systemConfig);
|
121
|
+
} else {
|
122
|
+
Class<?> serviceClass = Class.forName(serviceClassName);
|
123
|
+
obj = serviceClass.getConstructor(ConfigSource.class).newInstance(systemConfig);
|
124
|
+
}
|
125
|
+
|
126
|
+
if (obj instanceof EmbulkService) {
|
127
|
+
return ((EmbulkService) obj).getInjector();
|
128
|
+
} else {
|
129
|
+
return (Injector) obj.getClass().getMethod("getInjector").invoke(obj);
|
130
|
+
}
|
131
|
+
|
132
|
+
} catch (InvocationTargetException ex) {
|
133
|
+
throw Throwables.propagate(ex.getCause());
|
134
|
+
} catch (ClassNotFoundException | NoSuchMethodException | InstantiationException | IllegalAccessException | IllegalArgumentException ex) {
|
135
|
+
throw Throwables.propagate(ex);
|
136
|
+
}
|
109
137
|
}
|
110
138
|
|
111
|
-
public static
|
112
|
-
Path stateDir) throws IOException
|
139
|
+
public static class JobStatus
|
113
140
|
{
|
114
|
-
|
115
|
-
|
116
|
-
|
117
|
-
|
118
|
-
|
119
|
-
|
120
|
-
|
121
|
-
|
122
|
-
|
141
|
+
private final boolean completed;
|
142
|
+
private final float mapProgress;
|
143
|
+
private final float reduceProgress;
|
144
|
+
|
145
|
+
public JobStatus(boolean completed, float mapProgress, float reduceProgress)
|
146
|
+
{
|
147
|
+
this.completed = completed;
|
148
|
+
this.mapProgress = mapProgress;
|
149
|
+
this.reduceProgress = reduceProgress;
|
150
|
+
}
|
151
|
+
|
152
|
+
public boolean isComplete()
|
153
|
+
{
|
154
|
+
return completed;
|
155
|
+
}
|
156
|
+
|
157
|
+
public float getMapProgress()
|
158
|
+
{
|
159
|
+
return mapProgress;
|
160
|
+
}
|
161
|
+
|
162
|
+
public float getReduceProgress()
|
163
|
+
{
|
164
|
+
return reduceProgress;
|
165
|
+
}
|
166
|
+
}
|
167
|
+
|
168
|
+
public static JobStatus getJobStatus(final Job job) throws IOException
|
169
|
+
{
|
170
|
+
return hadoopOperationWithRetry("getting job status", new Callable<JobStatus>() {
|
171
|
+
public JobStatus call() throws IOException
|
172
|
+
{
|
173
|
+
return new JobStatus(job.isComplete(), job.mapProgress(), job.reduceProgress());
|
174
|
+
}
|
175
|
+
});
|
176
|
+
}
|
177
|
+
|
178
|
+
public static Counters getJobCounters(final Job job) throws IOException
|
179
|
+
{
|
180
|
+
return hadoopOperationWithRetry("getting job counters", new Callable<Counters>() {
|
181
|
+
public Counters call() throws IOException
|
182
|
+
{
|
183
|
+
return job.getCounters();
|
184
|
+
}
|
185
|
+
});
|
186
|
+
}
|
187
|
+
|
188
|
+
public static List<TaskAttemptID> listAttempts(final Configuration config,
|
189
|
+
final Path stateDir) throws IOException
|
190
|
+
{
|
191
|
+
return hadoopOperationWithRetry("getting list of attempt state files on "+stateDir, new Callable<List<TaskAttemptID>>() {
|
192
|
+
public List<TaskAttemptID> call() throws IOException
|
193
|
+
{
|
194
|
+
FileStatus[] stats = stateDir.getFileSystem(config).listStatus(stateDir);
|
195
|
+
ImmutableList.Builder<TaskAttemptID> builder = ImmutableList.builder();
|
196
|
+
for (FileStatus stat : stats) {
|
197
|
+
if (stat.getPath().getName().startsWith("attempt_") && stat.isFile()) {
|
198
|
+
String name = stat.getPath().getName();
|
199
|
+
TaskAttemptID id;
|
200
|
+
try {
|
201
|
+
id = TaskAttemptID.forName(name);
|
202
|
+
} catch (Exception ex) {
|
203
|
+
// ignore this file
|
204
|
+
continue;
|
205
|
+
}
|
206
|
+
builder.add(id);
|
207
|
+
}
|
123
208
|
}
|
209
|
+
return builder.build();
|
124
210
|
}
|
125
|
-
}
|
126
|
-
return builder.build();
|
211
|
+
});
|
127
212
|
}
|
128
213
|
|
129
|
-
public static
|
130
|
-
|
214
|
+
public static void writePluginArchive(final Configuration config, final Path stateDir,
|
215
|
+
final PluginArchive archive, final ModelManager modelManager) throws IOException
|
131
216
|
{
|
132
|
-
|
133
|
-
|
134
|
-
|
135
|
-
|
136
|
-
|
137
|
-
|
138
|
-
|
217
|
+
final Path path = new Path(stateDir, PLUGIN_ARCHIVE_FILE_NAME);
|
218
|
+
hadoopOperationWithRetry("writing plugin archive to "+path, new Callable<Void>() {
|
219
|
+
public Void call() throws IOException
|
220
|
+
{
|
221
|
+
stateDir.getFileSystem(config).mkdirs(stateDir);
|
222
|
+
try (FSDataOutputStream out = path.getFileSystem(config).create(path, true)) {
|
223
|
+
List<PluginArchive.GemSpec> specs = archive.dump(out);
|
224
|
+
config.set(CK_PLUGIN_ARCHIVE_SPECS, modelManager.writeObject(specs));
|
225
|
+
}
|
226
|
+
return null;
|
227
|
+
}
|
228
|
+
});
|
139
229
|
}
|
140
230
|
|
141
|
-
public static
|
142
|
-
|
231
|
+
public static PluginArchive readPluginArchive(final File localDirectory, final Configuration config,
|
232
|
+
Path stateDir, final ModelManager modelManager) throws IOException
|
143
233
|
{
|
144
|
-
Path path = new Path(stateDir, PLUGIN_ARCHIVE_FILE_NAME);
|
145
|
-
|
146
|
-
|
147
|
-
|
148
|
-
|
234
|
+
final Path path = new Path(stateDir, PLUGIN_ARCHIVE_FILE_NAME);
|
235
|
+
return hadoopOperationWithRetry("reading plugin archive file from "+path, new Callable<PluginArchive>() {
|
236
|
+
public PluginArchive call() throws IOException
|
237
|
+
{
|
238
|
+
List<PluginArchive.GemSpec> specs = modelManager.readObject(
|
239
|
+
new ArrayList<PluginArchive.GemSpec>() {}.getClass(),
|
240
|
+
config.get(CK_PLUGIN_ARCHIVE_SPECS));
|
241
|
+
try (FSDataInputStream in = path.getFileSystem(config).open(path)) {
|
242
|
+
return PluginArchive.load(localDirectory, specs, in);
|
243
|
+
}
|
244
|
+
}
|
245
|
+
});
|
246
|
+
}
|
247
|
+
|
248
|
+
public static void writeAttemptStateFile(final Configuration config,
|
249
|
+
Path stateDir, final AttemptState state, final ModelManager modelManager) throws IOException
|
250
|
+
{
|
251
|
+
final Path path = new Path(stateDir, state.getAttemptId().toString());
|
252
|
+
hadoopOperationWithRetry("writing attempt state file to "+path, new Callable<Void>() {
|
253
|
+
public Void call() throws IOException
|
254
|
+
{
|
255
|
+
try (FSDataOutputStream out = path.getFileSystem(config).create(path, true)) {
|
256
|
+
state.writeTo(out, modelManager);
|
257
|
+
}
|
258
|
+
return null;
|
259
|
+
}
|
260
|
+
});
|
149
261
|
}
|
150
262
|
|
151
263
|
public static AttemptState readAttemptStateFile(final Configuration config,
|
152
|
-
Path stateDir, TaskAttemptID id, final ModelManager modelManager
|
264
|
+
Path stateDir, TaskAttemptID id, final ModelManager modelManager,
|
265
|
+
final boolean concurrentWriteIsPossible) throws IOException
|
153
266
|
{
|
154
267
|
final Logger log = Exec.getLogger(EmbulkMapReduce.class);
|
155
268
|
final Path path = new Path(stateDir, id.toString());
|
@@ -160,34 +273,46 @@ public class EmbulkMapReduce
|
|
160
273
|
.withMaxRetryWait(20 * 1000)
|
161
274
|
.runInterruptible(new Retryable<AttemptState>() {
|
162
275
|
@Override
|
163
|
-
public AttemptState call() throws IOException
|
276
|
+
public AttemptState call() throws IOException
|
277
|
+
{
|
164
278
|
try (FSDataInputStream in = path.getFileSystem(config).open(path)) {
|
165
279
|
return AttemptState.readFrom(in, modelManager);
|
166
280
|
}
|
167
281
|
}
|
168
282
|
|
169
283
|
@Override
|
170
|
-
public boolean isRetryableException(Exception exception)
|
171
|
-
|
172
|
-
//
|
173
|
-
//
|
174
|
-
//
|
284
|
+
public boolean isRetryableException(Exception exception)
|
285
|
+
{
|
286
|
+
// AttemptState.readFrom throws 4 types of exceptions:
|
287
|
+
//
|
288
|
+
// concurrentWriteIsPossible == true:
|
289
|
+
// a) EOFException: race between readFrom and writeTo. See comments on AttemptState.readFrom.
|
290
|
+
// b) EOFException: file exists but its format is invalid because this task is retried and last job/attempt left corrupted files (such as empty, partially written, etc)
|
291
|
+
// c) IOException "Cannot obtain block length for LocatedBlock": HDFS-1058. See https://github.com/embulk/embulk-executor-mapreduce/pull/3
|
292
|
+
// d) IOException: FileSystem is not working
|
293
|
+
// concurrentWriteIsPossible == false:
|
294
|
+
// e) EOFException: file exists but its format is invalid because this task is retried and last job/attempt left corrupted files (such as empty, partially written, etc)
|
295
|
+
// f) IOException: FileSystem is not working
|
175
296
|
//
|
176
|
-
|
177
|
-
|
297
|
+
if (exception instanceof EOFException && !concurrentWriteIsPossible) {
|
298
|
+
// e) is not recoverable.
|
299
|
+
return false;
|
300
|
+
}
|
178
301
|
return true;
|
179
302
|
}
|
180
303
|
|
181
304
|
@Override
|
182
305
|
public void onRetry(Exception exception, int retryCount, int retryLimit, int retryWait)
|
183
|
-
throws RetryGiveupException
|
184
|
-
|
306
|
+
throws RetryGiveupException
|
307
|
+
{
|
308
|
+
log.warn("Retrying opening state file {} ({}/{}) error: {}",
|
309
|
+
path, retryCount, retryLimit, exception);
|
185
310
|
}
|
186
311
|
|
187
312
|
@Override
|
188
313
|
public void onGiveup(Exception firstException, Exception lastException)
|
189
|
-
throws RetryGiveupException
|
190
|
-
}
|
314
|
+
throws RetryGiveupException
|
315
|
+
{ }
|
191
316
|
});
|
192
317
|
} catch (RetryGiveupException e) {
|
193
318
|
Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
|
@@ -197,13 +322,45 @@ public class EmbulkMapReduce
|
|
197
322
|
}
|
198
323
|
}
|
199
324
|
|
200
|
-
|
201
|
-
Path stateDir, AttemptState state, ModelManager modelManager) throws IOException
|
325
|
+
private static <T> T hadoopOperationWithRetry(final String message, final Callable<T> callable) throws IOException
|
202
326
|
{
|
203
|
-
|
204
|
-
|
205
|
-
|
206
|
-
|
327
|
+
final Logger log = Exec.getLogger(EmbulkMapReduce.class);
|
328
|
+
try {
|
329
|
+
return retryExecutor()
|
330
|
+
.withRetryLimit(5)
|
331
|
+
.withInitialRetryWait(2 * 1000)
|
332
|
+
.withMaxRetryWait(20 * 1000)
|
333
|
+
.runInterruptible(new Retryable<T>() {
|
334
|
+
@Override
|
335
|
+
public T call() throws Exception
|
336
|
+
{
|
337
|
+
return callable.call();
|
338
|
+
}
|
339
|
+
|
340
|
+
@Override
|
341
|
+
public boolean isRetryableException(Exception exception)
|
342
|
+
{
|
343
|
+
return true;
|
344
|
+
}
|
345
|
+
|
346
|
+
@Override
|
347
|
+
public void onRetry(Exception exception, int retryCount, int retryLimit, int retryWait)
|
348
|
+
throws RetryGiveupException
|
349
|
+
{
|
350
|
+
log.warn("Retrying {} ({}/{}) error: {}",
|
351
|
+
message, retryCount, retryLimit, exception);
|
352
|
+
}
|
353
|
+
|
354
|
+
@Override
|
355
|
+
public void onGiveup(Exception firstException, Exception lastException)
|
356
|
+
throws RetryGiveupException
|
357
|
+
{ }
|
358
|
+
});
|
359
|
+
} catch (RetryGiveupException e) {
|
360
|
+
Throwables.propagateIfInstanceOf(e.getCause(), IOException.class);
|
361
|
+
throw Throwables.propagate(e.getCause());
|
362
|
+
} catch (InterruptedException e) {
|
363
|
+
throw new InterruptedIOException();
|
207
364
|
}
|
208
365
|
}
|
209
366
|
|
@@ -1,9 +1,11 @@
|
|
1
1
|
package org.embulk.executor.mapreduce;
|
2
2
|
|
3
3
|
import java.util.List;
|
4
|
-
import java.util.
|
4
|
+
import java.util.Collection;
|
5
5
|
import java.util.Set;
|
6
|
+
import java.util.Map;
|
6
7
|
import java.util.HashSet;
|
8
|
+
import java.util.HashMap;
|
7
9
|
import java.io.File;
|
8
10
|
import java.io.IOException;
|
9
11
|
import java.io.EOFException;
|
@@ -29,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
|
|
29
31
|
import org.apache.hadoop.mapreduce.JobContext;
|
30
32
|
import org.apache.hadoop.mapreduce.Cluster;
|
31
33
|
import org.apache.hadoop.mapreduce.Job;
|
34
|
+
import org.apache.hadoop.mapreduce.JobID;
|
32
35
|
import org.apache.hadoop.mapreduce.Counters;
|
33
36
|
import org.apache.hadoop.mapreduce.TaskType;
|
34
37
|
import org.apache.hadoop.mapreduce.TaskAttemptID;
|
@@ -37,6 +40,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
37
40
|
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
38
41
|
import org.embulk.exec.ForSystemConfig;
|
39
42
|
import org.embulk.config.ConfigSource;
|
43
|
+
import org.embulk.config.CommitReport;
|
40
44
|
import org.embulk.config.ConfigException;
|
41
45
|
import org.embulk.config.TaskSource;
|
42
46
|
import org.embulk.config.ModelManager;
|
@@ -116,6 +120,67 @@ public class MapReduceExecutor
|
|
116
120
|
}
|
117
121
|
}
|
118
122
|
|
123
|
+
private static class TaskReportSet
|
124
|
+
{
|
125
|
+
private Map<Integer, AttemptReport> inputTaskReports = new HashMap<>();
|
126
|
+
private Map<Integer, AttemptReport> outputTaskReports = new HashMap<>();
|
127
|
+
|
128
|
+
private final JobID runningJobId;
|
129
|
+
|
130
|
+
public TaskReportSet(JobID runningJobId)
|
131
|
+
{
|
132
|
+
this.runningJobId = runningJobId;
|
133
|
+
}
|
134
|
+
|
135
|
+
public Collection<AttemptReport> getLatestInputAttemptReports()
|
136
|
+
{
|
137
|
+
return inputTaskReports.values();
|
138
|
+
}
|
139
|
+
|
140
|
+
public Collection<AttemptReport> getLatestOutputAttemptReports()
|
141
|
+
{
|
142
|
+
return outputTaskReports.values();
|
143
|
+
}
|
144
|
+
|
145
|
+
public void update(AttemptReport report)
|
146
|
+
{
|
147
|
+
if (report.getInputTaskIndex().isPresent()) {
|
148
|
+
int taskIndex = report.getInputTaskIndex().get();
|
149
|
+
AttemptReport past = inputTaskReports.get(taskIndex);
|
150
|
+
if (past == null || checkOverwrite(past, report)) {
|
151
|
+
inputTaskReports.put(taskIndex, report);
|
152
|
+
}
|
153
|
+
}
|
154
|
+
if (report.getOutputTaskIndex().isPresent()) {
|
155
|
+
int taskIndex = report.getOutputTaskIndex().get();
|
156
|
+
AttemptReport past = outputTaskReports.get(taskIndex);
|
157
|
+
if (past == null || checkOverwrite(past, report)) {
|
158
|
+
outputTaskReports.put(taskIndex, report);
|
159
|
+
}
|
160
|
+
}
|
161
|
+
}
|
162
|
+
|
163
|
+
private boolean checkOverwrite(AttemptReport past, AttemptReport report)
|
164
|
+
{
|
165
|
+
// if already committed successfully, use it
|
166
|
+
if (!past.isOutputCommitted() && report.isOutputCommitted()) {
|
167
|
+
return true;
|
168
|
+
}
|
169
|
+
|
170
|
+
// Here expects that TaskAttemptID.compareTo returns <= 0 if attempt is started later.
|
171
|
+
// However, it returns unexpected result if 2 jobs run on different JobTrackers because
|
172
|
+
// JobID includes start time of a JobTracker with sequence number in the JobTracker
|
173
|
+
// rather than start time of a job. To mitigate this problem, this code assumes that
|
174
|
+
// attempts of the running job is always newer.
|
175
|
+
boolean pastRunning = past.getTaskAttempId().getJobID().equals(runningJobId);
|
176
|
+
boolean reportRunning = report.getTaskAttempId().getJobID().equals(runningJobId);
|
177
|
+
if (!pastRunning && reportRunning) {
|
178
|
+
return true;
|
179
|
+
}
|
180
|
+
return past.getTaskAttempId().compareTo(report.getTaskAttempId()) <= 0;
|
181
|
+
}
|
182
|
+
}
|
183
|
+
|
119
184
|
void run(MapReduceExecutorTask task,
|
120
185
|
int mapTaskCount, int reduceTaskCount, ProcessState state)
|
121
186
|
{
|
@@ -163,14 +228,7 @@ public class MapReduceExecutor
|
|
163
228
|
EmbulkMapReduce.setMapTaskCount(job.getConfiguration(), mapTaskCount); // used by EmbulkInputFormat
|
164
229
|
EmbulkMapReduce.setStateDirectoryPath(job.getConfiguration(), stateDir);
|
165
230
|
|
166
|
-
// create state dir
|
167
|
-
try {
|
168
|
-
stateDir.getFileSystem(job.getConfiguration()).mkdirs(stateDir);
|
169
|
-
} catch (IOException ex) {
|
170
|
-
throw new RuntimeException(ex);
|
171
|
-
}
|
172
|
-
|
173
|
-
// archive plugins
|
231
|
+
// archive plugins (also create state dir)
|
174
232
|
PluginArchive archive = new PluginArchive.Builder()
|
175
233
|
.addLoadedRubyGems(jruby)
|
176
234
|
.build();
|
@@ -211,26 +269,33 @@ public class MapReduceExecutor
|
|
211
269
|
|
212
270
|
try {
|
213
271
|
job.submit();
|
272
|
+
TaskReportSet reportSet = new TaskReportSet(job.getJobID());
|
214
273
|
|
215
274
|
int interval = Job.getCompletionPollInterval(job.getConfiguration());
|
216
|
-
while (
|
275
|
+
while (true) {
|
276
|
+
EmbulkMapReduce.JobStatus status = EmbulkMapReduce.getJobStatus(job);
|
277
|
+
if (status.isComplete()) {
|
278
|
+
break;
|
279
|
+
}
|
280
|
+
log.info(String.format("map %.1f%% reduce %.1f%%",
|
281
|
+
status.getMapProgress() * 100, status.getReduceProgress() * 100));
|
282
|
+
|
217
283
|
//if (job.getState() == JobStatus.State.PREP) {
|
218
284
|
// continue;
|
219
285
|
//}
|
220
|
-
log.info(String.format("map %.1f%% reduce %.1f%%",
|
221
|
-
job.mapProgress() * 100, job.reduceProgress() * 100));
|
222
286
|
Thread.sleep(interval);
|
223
287
|
|
224
|
-
updateProcessState(job,
|
288
|
+
updateProcessState(job, reportSet, stateDir, state, modelManager, true);
|
225
289
|
}
|
226
290
|
|
227
|
-
|
228
|
-
// condition of AttemptReport.readFrom and .writeTo does not happen here.
|
291
|
+
EmbulkMapReduce.JobStatus status = EmbulkMapReduce.getJobStatus(job);
|
229
292
|
log.info(String.format("map %.1f%% reduce %.1f%%",
|
230
|
-
|
231
|
-
updateProcessState
|
293
|
+
status.getMapProgress() * 100, status.getReduceProgress() * 100));
|
294
|
+
// Here sets inProgress=false to updateProcessState method to tell that race
|
295
|
+
// condition of AttemptReport.readFrom and .writeTo does not happen here.
|
296
|
+
updateProcessState(job, reportSet, stateDir, state, modelManager, false);
|
232
297
|
|
233
|
-
Counters counters =
|
298
|
+
Counters counters = EmbulkMapReduce.getJobCounters(job);
|
234
299
|
if (counters != null) {
|
235
300
|
log.info(counters.toString());
|
236
301
|
}
|
@@ -292,50 +357,39 @@ public class MapReduceExecutor
|
|
292
357
|
+ String.format("%09d", time.getNano());
|
293
358
|
}
|
294
359
|
|
295
|
-
private void updateProcessState(Job job,
|
296
|
-
ProcessState state, ModelManager modelManager, boolean
|
360
|
+
private void updateProcessState(Job job, TaskReportSet reportSet, Path stateDir,
|
361
|
+
ProcessState state, ModelManager modelManager, boolean inProgress) throws IOException
|
297
362
|
{
|
298
|
-
List<AttemptReport> reports = getAttemptReports(job.getConfiguration(), stateDir, modelManager
|
363
|
+
List<AttemptReport> reports = getAttemptReports(job.getConfiguration(), stateDir, modelManager,
|
364
|
+
inProgress, job.getJobID());
|
299
365
|
|
300
366
|
for (AttemptReport report : reports) {
|
301
|
-
if (report
|
302
|
-
|
303
|
-
}
|
304
|
-
if (!report.isAvailable()) {
|
305
|
-
if (skipUnavailable) {
|
306
|
-
continue;
|
307
|
-
} else {
|
308
|
-
throw report.getUnavailableException();
|
309
|
-
}
|
310
|
-
}
|
311
|
-
AttemptState attempt = report.getAttemptState();
|
312
|
-
if (attempt.getInputTaskIndex().isPresent()) {
|
313
|
-
updateState(state.getInputTaskState(attempt.getInputTaskIndex().get()), attempt, true);
|
314
|
-
}
|
315
|
-
if (attempt.getOutputTaskIndex().isPresent()) {
|
316
|
-
updateState(state.getOutputTaskState(attempt.getOutputTaskIndex().get()), attempt, false);
|
367
|
+
if (report.isAvailable()) {
|
368
|
+
reportSet.update(report);
|
317
369
|
}
|
318
370
|
}
|
371
|
+
|
372
|
+
for (AttemptReport report : reportSet.getLatestInputAttemptReports()) {
|
373
|
+
updateTaskState(state.getInputTaskState(report.getInputTaskIndex().get()), report.getAttemptState(), true);
|
374
|
+
}
|
375
|
+
|
376
|
+
for (AttemptReport report : reportSet.getLatestOutputAttemptReports()) {
|
377
|
+
updateTaskState(state.getOutputTaskState(report.getOutputTaskIndex().get()), report.getAttemptState(), true);
|
378
|
+
}
|
319
379
|
}
|
320
380
|
|
321
|
-
private static void
|
381
|
+
private static void updateTaskState(TaskState state, AttemptState attempt, boolean isInput)
|
322
382
|
{
|
323
383
|
state.start();
|
384
|
+
Optional<CommitReport> commitReport = isInput ? attempt.getInputCommitReport() : attempt.getOutputCommitReport();
|
385
|
+
boolean committed = commitReport.isPresent();
|
324
386
|
if (attempt.getException().isPresent()) {
|
325
387
|
if (!state.isCommitted()) {
|
326
388
|
state.setException(new RemoteTaskFailedException(attempt.getException().get()));
|
327
389
|
}
|
328
|
-
} else if (
|
329
|
-
(isInput && attempt.getInputCommitReport().isPresent()) ||
|
330
|
-
(!isInput && attempt.getOutputCommitReport().isPresent())) {
|
331
|
-
state.resetException();
|
332
|
-
}
|
333
|
-
if (isInput && attempt.getInputCommitReport().isPresent()) {
|
334
|
-
state.setCommitReport(attempt.getInputCommitReport().get());
|
335
|
-
state.finish();
|
336
390
|
}
|
337
|
-
if (
|
338
|
-
state.setCommitReport(
|
391
|
+
if (commitReport.isPresent()) {
|
392
|
+
state.setCommitReport(commitReport.get());
|
339
393
|
state.finish();
|
340
394
|
}
|
341
395
|
}
|
@@ -370,6 +424,16 @@ public class MapReduceExecutor
|
|
370
424
|
return unavailableException;
|
371
425
|
}
|
372
426
|
|
427
|
+
public Optional<Integer> getInputTaskIndex()
|
428
|
+
{
|
429
|
+
return attemptState == null ? Optional.<Integer>absent() : attemptState.getInputTaskIndex();
|
430
|
+
}
|
431
|
+
|
432
|
+
public Optional<Integer> getOutputTaskIndex()
|
433
|
+
{
|
434
|
+
return attemptState == null ? Optional.<Integer>absent() : attemptState.getOutputTaskIndex();
|
435
|
+
}
|
436
|
+
|
373
437
|
public boolean isInputCommitted()
|
374
438
|
{
|
375
439
|
return attemptState != null && attemptState.getInputCommitReport().isPresent();
|
@@ -380,28 +444,39 @@ public class MapReduceExecutor
|
|
380
444
|
return attemptState != null && attemptState.getOutputCommitReport().isPresent();
|
381
445
|
}
|
382
446
|
|
447
|
+
public TaskAttemptID getTaskAttempId()
|
448
|
+
{
|
449
|
+
return attemptId;
|
450
|
+
}
|
451
|
+
|
383
452
|
public AttemptState getAttemptState()
|
384
453
|
{
|
385
454
|
return attemptState;
|
386
455
|
}
|
387
456
|
}
|
388
457
|
|
389
|
-
private static final int TASK_EVENT_FETCH_SIZE = 100;
|
390
|
-
|
391
458
|
private static List<AttemptReport> getAttemptReports(Configuration config,
|
392
|
-
Path stateDir, ModelManager modelManager
|
459
|
+
Path stateDir, ModelManager modelManager,
|
460
|
+
boolean jobIsRunning, JobID runningJobId) throws IOException
|
393
461
|
{
|
394
462
|
ImmutableList.Builder<AttemptReport> builder = ImmutableList.builder();
|
395
463
|
for (TaskAttemptID aid : EmbulkMapReduce.listAttempts(config, stateDir)) {
|
464
|
+
boolean concurrentWriteIsPossible = aid.getJobID().equals(runningJobId) && jobIsRunning;
|
396
465
|
try {
|
397
466
|
AttemptState state = EmbulkMapReduce.readAttemptStateFile(config,
|
398
|
-
stateDir, aid, modelManager);
|
467
|
+
stateDir, aid, modelManager, concurrentWriteIsPossible);
|
399
468
|
builder.add(new AttemptReport(aid, state));
|
400
469
|
} catch (IOException ex) {
|
401
|
-
//
|
402
|
-
|
403
|
-
|
404
|
-
|
470
|
+
// See comments on readAttemptStateFile for the possible error causes.
|
471
|
+
if (!concurrentWriteIsPossible) {
|
472
|
+
if (!(ex instanceof EOFException)) {
|
473
|
+
// f) HDFS is broken. This is critical problem which should throw an exception
|
474
|
+
throw new RuntimeException(ex);
|
475
|
+
}
|
476
|
+
// HDFS is working but file is corrupted. It is always possible that the directly
|
477
|
+
// contains corrupted file created by past attempts of retried task or job. Ignore it.
|
478
|
+
}
|
479
|
+
// if concurrentWriteIsPossible, there're no ways to tell the cause. Ignore it.
|
405
480
|
builder.add(new AttemptReport(aid, ex));
|
406
481
|
}
|
407
482
|
}
|
metadata
CHANGED
@@ -1,14 +1,14 @@
|
|
1
1
|
--- !ruby/object:Gem::Specification
|
2
2
|
name: embulk-executor-mapreduce
|
3
3
|
version: !ruby/object:Gem::Version
|
4
|
-
version: 0.1.
|
4
|
+
version: 0.1.3
|
5
5
|
platform: ruby
|
6
6
|
authors:
|
7
7
|
- Sadayuki Furuhashi
|
8
8
|
autorequire:
|
9
9
|
bindir: bin
|
10
10
|
cert_chain: []
|
11
|
-
date: 2015-
|
11
|
+
date: 2015-07-02 00:00:00.000000000 Z
|
12
12
|
dependencies: []
|
13
13
|
description: Executes tasks on Hadoop.
|
14
14
|
email:
|
@@ -60,7 +60,7 @@ files:
|
|
60
60
|
- classpath/curator-client-2.6.0.jar
|
61
61
|
- classpath/curator-framework-2.6.0.jar
|
62
62
|
- classpath/curator-recipes-2.6.0.jar
|
63
|
-
- classpath/embulk-executor-mapreduce-0.1.
|
63
|
+
- classpath/embulk-executor-mapreduce-0.1.3.jar
|
64
64
|
- classpath/gson-2.2.4.jar
|
65
65
|
- classpath/hadoop-annotations-2.6.0.jar
|
66
66
|
- classpath/hadoop-auth-2.6.0.jar
|