embulk-executor-mapreduce 0.1.0
Sign up to get free protection for your applications and to get access to all the features.
- checksums.yaml +7 -0
- data/build.gradle +2 -0
- data/classpath/activation-1.1.jar +0 -0
- data/classpath/apacheds-i18n-2.0.0-M15.jar +0 -0
- data/classpath/apacheds-kerberos-codec-2.0.0-M15.jar +0 -0
- data/classpath/api-asn1-api-1.0.0-M20.jar +0 -0
- data/classpath/api-util-1.0.0-M20.jar +0 -0
- data/classpath/avro-1.7.4.jar +0 -0
- data/classpath/commons-beanutils-1.7.0.jar +0 -0
- data/classpath/commons-cli-1.2.jar +0 -0
- data/classpath/commons-codec-1.6.jar +0 -0
- data/classpath/commons-collections-3.2.1.jar +0 -0
- data/classpath/commons-compress-1.4.1.jar +0 -0
- data/classpath/commons-configuration-1.6.jar +0 -0
- data/classpath/commons-digester-1.8.jar +0 -0
- data/classpath/commons-httpclient-3.1.jar +0 -0
- data/classpath/commons-io-2.4.jar +0 -0
- data/classpath/commons-lang-2.6.jar +0 -0
- data/classpath/commons-logging-1.1.3.jar +0 -0
- data/classpath/commons-math3-3.1.1.jar +0 -0
- data/classpath/commons-net-3.1.jar +0 -0
- data/classpath/curator-client-2.6.0.jar +0 -0
- data/classpath/curator-framework-2.6.0.jar +0 -0
- data/classpath/curator-recipes-2.6.0.jar +0 -0
- data/classpath/embulk-executor-mapreduce-0.1.0.jar +0 -0
- data/classpath/gson-2.2.4.jar +0 -0
- data/classpath/hadoop-annotations-2.6.0.jar +0 -0
- data/classpath/hadoop-auth-2.6.0.jar +0 -0
- data/classpath/hadoop-client-2.6.0.jar +0 -0
- data/classpath/hadoop-common-2.6.0.jar +0 -0
- data/classpath/hadoop-hdfs-2.6.0.jar +0 -0
- data/classpath/hadoop-mapreduce-client-app-2.6.0.jar +0 -0
- data/classpath/hadoop-mapreduce-client-common-2.6.0.jar +0 -0
- data/classpath/hadoop-mapreduce-client-core-2.6.0.jar +0 -0
- data/classpath/hadoop-mapreduce-client-jobclient-2.6.0.jar +0 -0
- data/classpath/hadoop-mapreduce-client-shuffle-2.6.0.jar +0 -0
- data/classpath/hadoop-yarn-api-2.6.0.jar +0 -0
- data/classpath/hadoop-yarn-client-2.6.0.jar +0 -0
- data/classpath/hadoop-yarn-common-2.6.0.jar +0 -0
- data/classpath/hadoop-yarn-server-common-2.6.0.jar +0 -0
- data/classpath/hadoop-yarn-server-nodemanager-2.6.0.jar +0 -0
- data/classpath/htrace-core-3.0.4.jar +0 -0
- data/classpath/httpclient-4.2.5.jar +0 -0
- data/classpath/httpcore-4.2.4.jar +0 -0
- data/classpath/jackson-core-asl-1.9.13.jar +0 -0
- data/classpath/jackson-jaxrs-1.9.13.jar +0 -0
- data/classpath/jackson-mapper-asl-1.9.13.jar +0 -0
- data/classpath/jackson-xc-1.9.13.jar +0 -0
- data/classpath/jaxb-api-2.2.2.jar +0 -0
- data/classpath/jaxb-impl-2.2.3-1.jar +0 -0
- data/classpath/jersey-client-1.9.jar +0 -0
- data/classpath/jersey-core-1.9.jar +0 -0
- data/classpath/jersey-guice-1.9.jar +0 -0
- data/classpath/jersey-json-1.9.jar +0 -0
- data/classpath/jersey-server-1.9.jar +0 -0
- data/classpath/jettison-1.1.jar +0 -0
- data/classpath/jetty-util-6.1.26.jar +0 -0
- data/classpath/jline-0.9.94.jar +0 -0
- data/classpath/jsr305-1.3.9.jar +0 -0
- data/classpath/leveldbjni-all-1.8.jar +0 -0
- data/classpath/netty-3.7.0.Final.jar +0 -0
- data/classpath/paranamer-2.3.jar +0 -0
- data/classpath/protobuf-java-2.5.0.jar +0 -0
- data/classpath/servlet-api-2.5.jar +0 -0
- data/classpath/snappy-java-1.0.4.1.jar +0 -0
- data/classpath/stax-api-1.0-2.jar +0 -0
- data/classpath/xmlenc-0.52.jar +0 -0
- data/classpath/xz-1.0.jar +0 -0
- data/classpath/zookeeper-3.4.6.jar +0 -0
- data/lib/embulk/executor/mapreduce.rb +3 -0
- data/src/main/java/org/embulk/executor/mapreduce/AttemptState.java +154 -0
- data/src/main/java/org/embulk/executor/mapreduce/BufferWritable.java +74 -0
- data/src/main/java/org/embulk/executor/mapreduce/BufferedPagePartitioner.java +158 -0
- data/src/main/java/org/embulk/executor/mapreduce/EmbulkInputFormat.java +37 -0
- data/src/main/java/org/embulk/executor/mapreduce/EmbulkInputSplit.java +61 -0
- data/src/main/java/org/embulk/executor/mapreduce/EmbulkMapReduce.java +359 -0
- data/src/main/java/org/embulk/executor/mapreduce/EmbulkPartitioningMapReduce.java +303 -0
- data/src/main/java/org/embulk/executor/mapreduce/EmbulkRecordReader.java +63 -0
- data/src/main/java/org/embulk/executor/mapreduce/MapReduceExecutor.java +391 -0
- data/src/main/java/org/embulk/executor/mapreduce/MapReduceExecutorTask.java +60 -0
- data/src/main/java/org/embulk/executor/mapreduce/PageWritable.java +66 -0
- data/src/main/java/org/embulk/executor/mapreduce/PartitionKey.java +11 -0
- data/src/main/java/org/embulk/executor/mapreduce/Partitioner.java +11 -0
- data/src/main/java/org/embulk/executor/mapreduce/Partitioning.java +12 -0
- data/src/main/java/org/embulk/executor/mapreduce/PluginArchive.java +189 -0
- data/src/main/java/org/embulk/executor/mapreduce/RemoteTaskFailedException.java +10 -0
- data/src/main/java/org/embulk/executor/mapreduce/SetContextClassLoader.java +19 -0
- data/src/main/java/org/embulk/executor/mapreduce/TimestampPartitioning.java +291 -0
- metadata +131 -0
@@ -0,0 +1,303 @@
|
|
1
|
+
package org.embulk.executor.mapreduce;
|
2
|
+
|
3
|
+
import java.util.List;
|
4
|
+
import java.util.Iterator;
|
5
|
+
import java.io.IOException;
|
6
|
+
import com.google.common.base.Optional;
|
7
|
+
import com.google.common.collect.ImmutableList;
|
8
|
+
import org.apache.hadoop.fs.Path;
|
9
|
+
import org.apache.hadoop.io.NullWritable;
|
10
|
+
import org.apache.hadoop.io.IntWritable;
|
11
|
+
import org.apache.hadoop.conf.Configuration;
|
12
|
+
import org.apache.hadoop.mapreduce.Mapper;
|
13
|
+
import org.apache.hadoop.mapreduce.Reducer;
|
14
|
+
import org.embulk.config.ModelManager;
|
15
|
+
import org.embulk.config.CommitReport;
|
16
|
+
import org.embulk.config.ConfigDiff;
|
17
|
+
import org.embulk.config.TaskSource;
|
18
|
+
import org.embulk.config.ConfigSource;
|
19
|
+
import org.embulk.spi.Exec;
|
20
|
+
import org.embulk.spi.ExecAction;
|
21
|
+
import org.embulk.spi.ExecSession;
|
22
|
+
import org.embulk.spi.Schema;
|
23
|
+
import org.embulk.spi.Page;
|
24
|
+
import org.embulk.spi.PageReader;
|
25
|
+
import org.embulk.spi.PageOutput;
|
26
|
+
import org.embulk.spi.BufferAllocator;
|
27
|
+
import org.embulk.spi.InputPlugin;
|
28
|
+
import org.embulk.spi.OutputPlugin;
|
29
|
+
import org.embulk.spi.FilterPlugin;
|
30
|
+
import org.embulk.spi.ProcessTask;
|
31
|
+
import org.embulk.spi.TransactionalPageOutput;
|
32
|
+
import org.embulk.spi.util.Filters;
|
33
|
+
import org.embulk.spi.util.Executors;
|
34
|
+
import org.embulk.executor.mapreduce.EmbulkMapReduce.SessionRunner;
|
35
|
+
import org.embulk.executor.mapreduce.BufferedPagePartitioner.PartitionedPageOutput;
|
36
|
+
import org.embulk.executor.mapreduce.EmbulkMapReduce.AttemptStateUpdateHandler;
|
37
|
+
import static org.embulk.executor.mapreduce.MapReduceExecutor.newPartitioning;
|
38
|
+
|
39
|
+
public class EmbulkPartitioningMapReduce
|
40
|
+
{
|
41
|
+
public static class EmbulkPartitioningMapper
|
42
|
+
extends Mapper<IntWritable, NullWritable, BufferWritable, PageWritable>
|
43
|
+
{
|
44
|
+
private Context context;
|
45
|
+
private SessionRunner runner;
|
46
|
+
|
47
|
+
@Override
|
48
|
+
public void setup(Context context) throws IOException
|
49
|
+
{
|
50
|
+
this.context = context;
|
51
|
+
this.runner = new SessionRunner(context);
|
52
|
+
runner.readPluginArchive().restoreLoadPathsTo(runner.getScriptingContainer());
|
53
|
+
}
|
54
|
+
|
55
|
+
@Override
|
56
|
+
public void map(IntWritable key, NullWritable value, final Context context) throws IOException, InterruptedException
|
57
|
+
{
|
58
|
+
final int taskIndex = key.get();
|
59
|
+
|
60
|
+
runner.execSession(new ExecAction<Void>() {
|
61
|
+
public Void run() throws Exception
|
62
|
+
{
|
63
|
+
process(context, taskIndex);
|
64
|
+
return null;
|
65
|
+
}
|
66
|
+
});
|
67
|
+
}
|
68
|
+
|
69
|
+
private void process(final Context context, int taskIndex) throws IOException, InterruptedException
|
70
|
+
{
|
71
|
+
ProcessTask task = runner.getMapReduceExecutorTask().getProcessTask();
|
72
|
+
ExecSession exec = runner.getExecSession();
|
73
|
+
|
74
|
+
// input and filters run at mapper
|
75
|
+
InputPlugin inputPlugin = exec.newPlugin(InputPlugin.class, task.getInputPluginType());
|
76
|
+
List<FilterPlugin> filterPlugins = Filters.newFilterPlugins(exec, task.getFilterPluginTypes());
|
77
|
+
|
78
|
+
// output writes pages with partitioning key to the Context
|
79
|
+
Partitioning partitioning = newPartitioning(runner.getMapReduceExecutorTask().getPartitioningType().get());
|
80
|
+
final Partitioner partitioner = partitioning.newPartitioner(runner.getMapReduceExecutorTask().getPartitioningTask().get());
|
81
|
+
OutputPlugin outputPlugin = new MapperOutputPlugin(
|
82
|
+
runner.getBufferAllocator(), partitioner,
|
83
|
+
128, // TODO configurable
|
84
|
+
new PartitionedPageOutput() {
|
85
|
+
private final BufferWritable keyWritable = new BufferWritable();
|
86
|
+
private final PageWritable valueWritable = new PageWritable();
|
87
|
+
|
88
|
+
{
|
89
|
+
keyWritable.set(partitioner.newKeyBuffer());
|
90
|
+
}
|
91
|
+
|
92
|
+
@Override
|
93
|
+
public void add(PartitionKey key, Page value)
|
94
|
+
{
|
95
|
+
try {
|
96
|
+
key.dump(keyWritable.get());
|
97
|
+
valueWritable.set(value);
|
98
|
+
context.write(keyWritable, valueWritable);
|
99
|
+
} catch (IOException | InterruptedException ex) {
|
100
|
+
throw new RuntimeException(ex);
|
101
|
+
} finally {
|
102
|
+
value.release();
|
103
|
+
}
|
104
|
+
}
|
105
|
+
|
106
|
+
@Override
|
107
|
+
public void finish()
|
108
|
+
{ }
|
109
|
+
|
110
|
+
@Override
|
111
|
+
public void close()
|
112
|
+
{ }
|
113
|
+
});
|
114
|
+
|
115
|
+
AttemptStateUpdateHandler handler = new AttemptStateUpdateHandler(runner,
|
116
|
+
new AttemptState(context.getTaskAttemptID(), Optional.of(taskIndex), Optional.<Integer>absent()));
|
117
|
+
|
118
|
+
try {
|
119
|
+
Executors.process(exec, taskIndex,
|
120
|
+
inputPlugin, task.getInputSchema(), task.getInputTaskSource(),
|
121
|
+
filterPlugins, task.getFilterSchemas(), task.getFilterTaskSources(),
|
122
|
+
outputPlugin, task.getOutputSchema(), task.getOutputTaskSource(),
|
123
|
+
handler);
|
124
|
+
} catch (Throwable ex) {
|
125
|
+
try {
|
126
|
+
handler.setException(ex);
|
127
|
+
} catch (Throwable e) {
|
128
|
+
e.addSuppressed(ex);
|
129
|
+
throw e;
|
130
|
+
}
|
131
|
+
//if (task.getTaskRecovery()) {
|
132
|
+
// throw ex;
|
133
|
+
//}
|
134
|
+
}
|
135
|
+
}
|
136
|
+
}
|
137
|
+
|
138
|
+
public static class EmbulkPartitioningReducer
|
139
|
+
extends Reducer<BufferWritable, PageWritable, NullWritable, NullWritable>
|
140
|
+
{
|
141
|
+
private Context context;
|
142
|
+
private SessionRunner runner;
|
143
|
+
private AttemptStateUpdateHandler handler;
|
144
|
+
private TransactionalPageOutput output;
|
145
|
+
private boolean failed = false;
|
146
|
+
|
147
|
+
@Override
|
148
|
+
public void setup(final Context context) throws IOException, InterruptedException
|
149
|
+
{
|
150
|
+
this.context = context;
|
151
|
+
this.runner = new SessionRunner(context);
|
152
|
+
runner.readPluginArchive().restoreLoadPathsTo(runner.getScriptingContainer());
|
153
|
+
|
154
|
+
runner.execSession(new ExecAction<Void>() {
|
155
|
+
public Void run() throws Exception
|
156
|
+
{
|
157
|
+
int taskIndex = context.getTaskAttemptID().getTaskID().getId();
|
158
|
+
|
159
|
+
ProcessTask task = runner.getMapReduceExecutorTask().getProcessTask();
|
160
|
+
ExecSession exec = runner.getExecSession();
|
161
|
+
OutputPlugin outputPlugin = exec.newPlugin(OutputPlugin.class, task.getOutputPluginType());
|
162
|
+
|
163
|
+
handler = new AttemptStateUpdateHandler(runner,
|
164
|
+
new AttemptState(context.getTaskAttemptID(), Optional.<Integer>absent(), Optional.of(taskIndex)));
|
165
|
+
|
166
|
+
output = outputPlugin.open(task.getOutputTaskSource(), task.getExecutorSchema(), taskIndex);
|
167
|
+
|
168
|
+
handler.started();
|
169
|
+
|
170
|
+
return null;
|
171
|
+
}
|
172
|
+
});
|
173
|
+
}
|
174
|
+
|
175
|
+
@Override
|
176
|
+
public void reduce(BufferWritable key, final Iterable<PageWritable> values, final Context context)
|
177
|
+
throws IOException, InterruptedException
|
178
|
+
{
|
179
|
+
runner.execSession(new ExecAction<Void>() {
|
180
|
+
public Void run() throws Exception
|
181
|
+
{
|
182
|
+
process(context, values);
|
183
|
+
return null;
|
184
|
+
}
|
185
|
+
});
|
186
|
+
}
|
187
|
+
|
188
|
+
private void process(final Context context, Iterable<PageWritable> values) throws IOException, InterruptedException
|
189
|
+
{
|
190
|
+
try {
|
191
|
+
for (PageWritable value : values) {
|
192
|
+
output.add(value.get());
|
193
|
+
}
|
194
|
+
} catch (Throwable ex) {
|
195
|
+
failed = true;
|
196
|
+
try {
|
197
|
+
handler.setException(ex);
|
198
|
+
} catch (Throwable e) {
|
199
|
+
e.addSuppressed(ex);
|
200
|
+
throw e;
|
201
|
+
}
|
202
|
+
}
|
203
|
+
}
|
204
|
+
|
205
|
+
protected void cleanup(Context context) throws IOException, InterruptedException
|
206
|
+
{
|
207
|
+
runner.execSession(new ExecAction<Void>() {
|
208
|
+
public Void run() throws Exception
|
209
|
+
{
|
210
|
+
try {
|
211
|
+
if (!failed) {
|
212
|
+
output.finish();
|
213
|
+
CommitReport report = output.commit();
|
214
|
+
handler.outputCommitted(report);
|
215
|
+
}
|
216
|
+
} finally {
|
217
|
+
output.close();
|
218
|
+
}
|
219
|
+
return null;
|
220
|
+
}
|
221
|
+
});
|
222
|
+
}
|
223
|
+
}
|
224
|
+
|
225
|
+
private static class MapperOutputPlugin
|
226
|
+
implements OutputPlugin
|
227
|
+
{
|
228
|
+
private final BufferAllocator bufferAllocator;
|
229
|
+
private final Partitioner partitioner;
|
230
|
+
private final int maxPageBufferCount;
|
231
|
+
private final PartitionedPageOutput output;
|
232
|
+
|
233
|
+
public MapperOutputPlugin(BufferAllocator bufferAllocator,
|
234
|
+
Partitioner partitioner, int maxPageBufferCount,
|
235
|
+
PartitionedPageOutput output)
|
236
|
+
{
|
237
|
+
this.bufferAllocator = bufferAllocator;
|
238
|
+
this.partitioner = partitioner;
|
239
|
+
this.maxPageBufferCount = maxPageBufferCount;
|
240
|
+
this.output = output;
|
241
|
+
}
|
242
|
+
|
243
|
+
public ConfigDiff transaction(ConfigSource config,
|
244
|
+
Schema schema, int taskCount,
|
245
|
+
OutputPlugin.Control control)
|
246
|
+
{
|
247
|
+
// won't be called
|
248
|
+
throw new RuntimeException("");
|
249
|
+
}
|
250
|
+
|
251
|
+
public ConfigDiff resume(TaskSource taskSource,
|
252
|
+
Schema schema, int taskCount,
|
253
|
+
OutputPlugin.Control control)
|
254
|
+
{
|
255
|
+
// won't be called
|
256
|
+
throw new RuntimeException("");
|
257
|
+
}
|
258
|
+
|
259
|
+
public void cleanup(TaskSource taskSource,
|
260
|
+
Schema schema, int taskCount,
|
261
|
+
List<CommitReport> successCommitReports)
|
262
|
+
{
|
263
|
+
// won't be called
|
264
|
+
throw new RuntimeException("");
|
265
|
+
}
|
266
|
+
|
267
|
+
public TransactionalPageOutput open(TaskSource taskSource, final Schema schema, int taskIndex)
|
268
|
+
{
|
269
|
+
return new TransactionalPageOutput() {
|
270
|
+
private final BufferedPagePartitioner bufferedPartitioner = new BufferedPagePartitioner(
|
271
|
+
bufferAllocator, schema, partitioner, maxPageBufferCount, output);
|
272
|
+
private final PageReader reader = new PageReader(schema);
|
273
|
+
|
274
|
+
public void add(Page page)
|
275
|
+
{
|
276
|
+
reader.setPage(page);
|
277
|
+
while (reader.nextRecord()) {
|
278
|
+
bufferedPartitioner.add(reader);
|
279
|
+
}
|
280
|
+
}
|
281
|
+
|
282
|
+
public void finish()
|
283
|
+
{
|
284
|
+
bufferedPartitioner.finish();
|
285
|
+
}
|
286
|
+
|
287
|
+
public void close()
|
288
|
+
{
|
289
|
+
reader.close();
|
290
|
+
bufferedPartitioner.close();
|
291
|
+
}
|
292
|
+
|
293
|
+
public void abort()
|
294
|
+
{ }
|
295
|
+
|
296
|
+
public CommitReport commit()
|
297
|
+
{
|
298
|
+
return Exec.newCommitReport();
|
299
|
+
}
|
300
|
+
};
|
301
|
+
}
|
302
|
+
}
|
303
|
+
}
|
@@ -0,0 +1,63 @@
|
|
1
|
+
package org.embulk.executor.mapreduce;
|
2
|
+
|
3
|
+
import org.apache.hadoop.io.IntWritable;
|
4
|
+
import org.apache.hadoop.io.NullWritable;
|
5
|
+
import org.apache.hadoop.mapreduce.InputSplit;
|
6
|
+
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
7
|
+
import org.apache.hadoop.mapreduce.RecordReader;
|
8
|
+
|
9
|
+
public class EmbulkRecordReader
|
10
|
+
extends RecordReader<IntWritable, NullWritable>
|
11
|
+
{
|
12
|
+
private final int[] taskIndexes;
|
13
|
+
private int offset;
|
14
|
+
|
15
|
+
private final IntWritable currentKey = new IntWritable();
|
16
|
+
|
17
|
+
public EmbulkRecordReader(EmbulkInputSplit split)
|
18
|
+
{
|
19
|
+
this.taskIndexes = split.getTaskIndexes();
|
20
|
+
this.offset = -1;
|
21
|
+
}
|
22
|
+
|
23
|
+
@Override
|
24
|
+
public void initialize(InputSplit split, TaskAttemptContext context)
|
25
|
+
{ }
|
26
|
+
|
27
|
+
@Override
|
28
|
+
public boolean nextKeyValue()
|
29
|
+
{
|
30
|
+
offset++;
|
31
|
+
if (taskIndexes.length <= offset) {
|
32
|
+
return false;
|
33
|
+
}
|
34
|
+
currentKey.set(taskIndexes[offset]);
|
35
|
+
return true;
|
36
|
+
}
|
37
|
+
|
38
|
+
@Override
|
39
|
+
public float getProgress()
|
40
|
+
{
|
41
|
+
if (taskIndexes.length == 0) {
|
42
|
+
return (float) 1.0;
|
43
|
+
}
|
44
|
+
return offset / (float) taskIndexes.length;
|
45
|
+
}
|
46
|
+
|
47
|
+
@Override
|
48
|
+
public IntWritable getCurrentKey()
|
49
|
+
{
|
50
|
+
return currentKey;
|
51
|
+
}
|
52
|
+
|
53
|
+
@Override
|
54
|
+
public NullWritable getCurrentValue()
|
55
|
+
{
|
56
|
+
return NullWritable.get();
|
57
|
+
}
|
58
|
+
|
59
|
+
@Override
|
60
|
+
public void close()
|
61
|
+
{
|
62
|
+
}
|
63
|
+
}
|