├── .gitignore
├── .travis.yml
├── README.md
├── configuration.md
├── lib
└── org
│ └── apache
│ └── mesos
│ └── mesos
│ ├── 0.14.0-20130806
│ ├── mesos-0.14.0-20130806.jar
│ └── mesos-0.14.0-20130806.pom
│ └── maven-metadata-local.xml
├── pom.xml
└── src
└── main
└── java
└── org
└── apache
├── hadoop
└── mapred
│ ├── MesosExecutor.java
│ ├── MesosScheduler.java
│ ├── MesosTracker.java
│ ├── ResourcePolicy.java
│ ├── ResourcePolicyFixed.java
│ └── ResourcePolicyVariable.java
└── mesos
└── hadoop
├── Metrics.java
└── Utils.java
/.gitignore:
--------------------------------------------------------------------------------
1 | target
2 | .vimrc
3 | .idea/
4 | *.iml
5 | *-pom.xml
6 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: java
2 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | Hadoop on Mesos
2 | ---------------
3 |
4 | [](https://travis-ci.org/mesos/hadoop)
5 |
6 | #### Overview ####
7 |
8 | To run _Hadoop on Mesos_ you need to add the `hadoop-mesos-0.1.0.jar`
9 | library to your Hadoop distribution (any distribution that uses protobuf > 2.5.0)
10 | and set some new configuration properties. Read on for details.
11 |
12 | The `pom.xml` included is configured and tested against CDH5 and MRv1. Hadoop on
13 | Mesos does not currently support YARN (and MRv2).
14 |
15 | #### Prerequisites ####
16 |
17 | To use the metrics feature (which uses the [CodaHale Metrics][CodaHale Metrics] library), you need to
18 | install `libsnappy`. The [`snappy-java`][snappy-java] package also includes a bundled version of
19 | `libsnappyjava`.
20 |
21 | [CodaHale Metrics]: http://metrics.codahale.com/
22 | [snappy-java]: https://github.com/xerial/snappy-java
23 |
24 | #### Build ####
25 |
26 | You can build `hadoop-mesos-0.1.0.jar` using Maven:
27 |
28 | ```shell
29 | mvn package
30 | ```
31 |
32 | If successful, the JAR will be at `target/hadoop-mesos-0.1.0.jar`.
33 |
34 | > NOTE: If you want to build against a different version of Mesos than
35 | > the default you'll need to update `mesos-version` in `pom.xml`.
36 |
37 | We plan to provide already built JARs at http://repository.apache.org
38 | in the near future!
39 |
40 | #### Package ####
41 |
42 | You'll need to download an existing Hadoop distribution. For this
43 | guide, we'll use [CDH5][CDH5.1.3]. First grab the tar archive and
44 | extract it.
45 |
46 | ```shell
47 | wget http://archive.cloudera.com/cdh5/cdh/5/hadoop-2.5.0-cdh5.2.0.tar.gz
48 | ...
49 | tar zxf hadoop-2.5.0-cdh5.2.0.tar.gz
50 | ```
51 |
52 | > **Take note**, the extracted directory is `hadoop-2.5.0-cdh5.2.0`.
53 |
54 | Now copy `hadoop-mesos-0.1.0.jar` into the `share/hadoop/common/lib` folder.
55 |
56 | ```shell
57 | cp /path/to/hadoop-mesos-0.1.0.jar hadoop-2.5.0-cdh5.2.0/share/hadoop/common/lib/
58 | ```
59 |
60 | Since CDH5 includes both MRv1 and MRv2 (YARN) and is configured for YARN by
61 | default, we need update the symlinks to point to the correct directories.
62 |
63 | ```shell
64 | cd hadoop-2.5.0-cdh5.2.0
65 |
66 | mv bin bin-mapreduce2
67 | mv examples examples-mapreduce2
68 | ln -s bin-mapreduce1 bin
69 | ln -s examples-mapreduce1 examples
70 |
71 | pushd etc
72 | mv hadoop hadoop-mapreduce2
73 | ln -s hadoop-mapreduce1 hadoop
74 | popd
75 |
76 | pushd share/hadoop
77 | rm mapreduce
78 | ln -s mapreduce1 mapreduce
79 | popd
80 | ```
81 |
82 | _That's it!_ You now have a _Hadoop on Mesos_ distribution!
83 |
84 | [CDH5.1.3]: http://www.cloudera.com/content/support/en/documentation/cdh5-documentation/cdh5-documentation-v5-latest.html
85 |
86 | #### Upload ####
87 |
88 | You'll want to upload your _Hadoop on Mesos_ distribution somewhere
89 | that Mesos can access in order to launch each `TaskTracker`. For
90 | example, if you're already running HDFS:
91 |
92 | ```
93 | $ tar czf hadoop-2.5.0-cdh5.2.0.tar.gz hadoop-2.5.0-cdh5.2.0
94 | $ hadoop fs -put hadoop-2.5.0-cdh5.2.0.tar.gz /hadoop-2.5.0-cdh5.2.0.tar.gz
95 | ```
96 |
97 | > **Consider** any permissions issues with your uploaded location
98 | > (i.e., on HDFS you'll probably want to make the file world
99 | > readable).
100 |
101 | Now you'll need to configure your `JobTracker` to launch each
102 | `TaskTracker` on Mesos!
103 |
104 | #### Configure ####
105 |
106 | Along with the normal configuration properties you might want to set
107 | to launch a `JobTracker`, you'll need to set some Mesos specific ones
108 | too.
109 |
110 | Here are the mandatory configuration properties for
111 | `conf/mapred-site.xml` (initialized to values representative of
112 | running in [pseudo distributed
113 | operation](http://hadoop.apache.org/docs/stable/single_node_setup.html#PseudoDistributed):
114 |
115 | ```
116 |
117 | mapred.job.tracker
118 | localhost:9001
119 |
120 |
121 | mapred.jobtracker.taskScheduler
122 | org.apache.hadoop.mapred.MesosScheduler
123 |
124 |
125 | mapred.mesos.taskScheduler
126 | org.apache.hadoop.mapred.JobQueueTaskScheduler
127 |
128 |
129 | mapred.mesos.master
130 | localhost:5050
131 |
132 |
133 | mapred.mesos.executor.uri
134 | hdfs://localhost:9000/hadoop-2.5.0-cdh5.2.0.tar.gz
135 |
136 | ```
137 |
138 | [More details on configuration propertios can be found here.](configuration.md)
139 |
140 | #### Start ####
141 |
142 | Now you can start the `JobTracker` but you'll need to include the path
143 | to the Mesos native library.
144 |
145 | On Linux:
146 |
147 | ```
148 | $ MESOS_NATIVE_LIBRARY=/path/to/libmesos.so hadoop jobtracker
149 | ```
150 |
151 | And on OS X:
152 |
153 | ```
154 | $ MESOS_NATIVE_LIBRARY=/path/to/libmesos.dylib hadoop jobtracker
155 | ```
156 |
157 | > **NOTE: You do not need to worry about distributing your Hadoop
158 | > configuration! All of the configuration properties read by the**
159 | > `JobTracker` **along with any necessary** `TaskTracker` **specific
160 | > _overrides_ will get serialized and passed to each** `TaskTracker`
161 | > **on startup.**
162 |
163 | #### Containers ####
164 |
165 | As of Mesos 0.19.0 you can now specify a container to be used when isolating a task on a Mesos Slave. If you're making use of this new container mechanism, you can configure the hadoop jobtracker to send a custom container image and set of options using two new JobConf options.
166 |
167 | This is purely opt-in, so omitting these jobconf options will cause no `ContainerInfo` to be sent to Mesos. Also, if you don't use these options there's no requirement to use version 0.19.0 of the mesos native library.
168 |
169 | This feature can be especially useful if your hadoop jobs have software dependencies on the slaves themselves, as using a container can isolate these dependencies between other users of a Mesos cluster.
170 |
171 | *It's important to note that the container/image you use does need to have the mesos native library installed already.*
172 |
173 | ```
174 |
175 | mapred.mesos.container.image
176 | docker:///ubuntu
177 |
178 |
179 | mapred.mesos.container.options
180 | -v,/foo/bar:/bar
181 |
182 | ```
183 |
184 | _Please email user@mesos.apache.org with questions!_
185 |
186 | ----------
187 |
--------------------------------------------------------------------------------
/configuration.md:
--------------------------------------------------------------------------------
1 | Configuration
2 | ---------------
3 |
4 | Mesos introduces some new parameters for configuring your Hadoop cluster.
5 | Below is a sample configuration with a description of the fields and their
6 | default values.
7 |
8 | ```
9 |
10 |
11 |
12 |
13 |
14 |
15 |
16 |
17 | mapred.job.tracker
18 | localhost:54311
19 |
20 |
21 | mapred.jobtracker.taskScheduler
22 | org.apache.hadoop.mapred.MesosScheduler
23 |
24 |
25 | mapred.mesos.taskScheduler
26 | org.apache.hadoop.mapred.JobQueueTaskScheduler
27 |
28 | This is the underlying task scheduler for the jobtracker. You may use
29 | other schedulers, like org.apache.hadoop.mapred.FairScheduler.
30 |
31 |
32 |
33 | mapred.mesos.master
34 | local
35 |
36 | This is the address of the Mesos master instance. If you're using
37 | Zookeeper for master election, use the Zookeeper address here (i.e.,
38 | zk://zk.apache.org:2181/hadoop/mesos).
39 |
40 |
41 |
42 | mapred.mesos.executor.uri
43 | hdfs://hdfs.name.node:port/hadoop.tar.gz
44 |
45 | This is the URI of the Hadoop on Mesos distribution.
46 | NOTE: You need to MANUALLY upload this yourself!
47 |
48 |
49 |
50 |
52 |
53 | mapred.mesos.slot.cpus
54 | 1
55 | This is the amount of CPU share allocated per slot. This number may be fractional (i.e., 0.5).
56 |
57 |
58 | mapred.mesos.slot.disk
59 | 1024
60 | This is the disk space required per slot. The value is in
61 | MiB.
62 |
63 |
64 | mapred.mesos.slot.mem
65 | 1024
66 |
67 | This is the total memory required for JVM overhead (10% of this value)
68 | and the heap (-Xmx) of the task. The value is in MiB.
69 |
70 |
71 |
72 |
73 |
74 | mapred.mesos.total.map.slots.minimum
75 | 0
76 |
77 | Mesos will attempt to make at least this many number of map slots
78 | available at a given time. This does not necessarily mean the slots will
79 | be idle, and this does not guarantee these slots will be available.
80 |
81 |
82 |
83 | mapred.mesos.total.reduce.slots.minimum
84 | 0
85 |
86 | Mesos will attempt to make at least this many number of reduce slots
87 | available at a given time. This does not necessarily mean the slots will
88 | be idle, and this does not guarantee these slots will be available.
89 |
90 |
91 |
92 | mapred.tasktracker.map.tasks.maximum
93 | 50
94 |
95 | This is the maximum number of tasks per task tracker. If you use the
96 | fixed resource policy, Mesos will always allocate this many slots per
97 | task tracker.
98 |
99 |
100 |
101 | mapred.tasktracker.reduce.tasks.maximum
102 | 50
103 |
104 | This is the maximum number of tasks per task tracker. If you use the
105 | fixed resource policy, Mesos will always allocate this many slots per
106 | task tracker.
107 |
108 |
109 |
110 | mapred.mesos.scheduler.policy.fixed
111 | false
112 |
113 | If this is set to true, Mesos will always allocate a fixed number of
114 | slots per task tracker based on the maximum map/reduce slot
115 | specification. If a resource offer is not large enough for the number of
116 | slots specified, that resource offer will be declined.
117 |
118 |
119 |
120 |
121 |
122 | mapred.mesos.checkpoint
123 | false
124 |
125 | This value enables/disables checkpointing for this framework.
126 |
127 |
128 |
129 | mapred.mesos.role
130 | *
131 |
132 | This is the Mesos framework role. This can be used in conjunction with
133 | Mesos reservations. Consult the Mesos documentation for details.
134 |
135 |
136 |
137 | mapred.mesos.role.strict
138 | false
139 |
140 | Force the framework to only ever accept resource offers that are of the
141 | role configured in "mapred.mesos.role".
142 |
143 |
144 |
145 |
146 |
147 | mapred.mesos.docker.image
148 | my-registry.com/image/foo:tag
149 |
150 | If you want the TaskTracker executor processes to start inside Docker containers,
151 | specify the docker image here.
152 |
153 |
154 |
155 | mapred.mesos.docker.network
156 | 1
157 |
158 | Use this option to change the networking configuration for containers. The
159 | default here is to use HOST networking (the container shares the network)
160 | with the host, no isolation.
161 | 1 = HOST, 2 = BRIDGE, 3 = NONE;
162 |
163 |
164 |
165 | mapred.mesos.docker.privileged
166 | false
167 |
168 | Enable the --privileged option on the executor containers.
169 |
170 |
171 |
172 | mapred.mesos.docker.force_pull_image
173 | false
174 |
175 | Tell the mesos slave to always check it has the latest version of the container
176 | image before starting the container.
177 |
178 |
179 |
180 | mapred.mesos.docker.parameters
181 |
182 |
183 | Comma separated list of command line arguments to pass directly to the
184 | docker run invocation. For example...
185 |
186 | "env,FOO=bar,env,BAZ=test"
187 |
188 |
189 |
190 | mapred.mesos.docker.volumes
191 |
192 |
193 | Comma separated list of volumes to mount into the container. The format for
194 | the volume definition is similar to the docker CLI, for example...
195 |
196 | "/host/path:/container/path:rw" (mount /host/path to /container/path read-write)
197 | "/host/path:/container/path:ro" (mount /host/path to /container/path read-only)
198 | "/host/path:ro" (mount /host/path to /host/path read-only)
199 |
200 |
201 |
202 |
203 |
204 | mapred.mesos.container.image
205 | docker:///ubuntu
206 |
207 | If you're using a custom Mesos Containerizer (like the External Containerizer)
208 | that uses images, you can set this option to cause Hadoop TaskTrackers to
209 | be launched within this container image.
210 |
211 |
212 |
213 | mapred.mesos.container.options
214 |
215 |
216 | Comma separated list of options to pass to the containerizer. The meaning
217 | of this entirely depends on the containerizer in use.
218 |
219 |
220 |
221 |
222 |
223 | mapred.mesos.tracker.idle.interval
224 | 5
225 |
226 | Internal (in seconds) to check for TaskTrackers that have idle
227 | slots. Default is 5 seconds.
228 |
229 |
230 |
231 | mapred.mesos.tracker.idle.checks
232 | 5
233 |
234 | After this many successful idle checks (meaning all slots *are* idle) the
235 | slots will be revoked from the TaskTracker.
236 |
237 |
238 |
239 |
240 |
241 | mapred.mesos.metrics.enabled
242 | false
243 |
244 | Set this to `true` to enable metric reporting with the Coda Hale Metrics
245 | library.
246 |
247 |
248 |
249 |
250 |
251 | mapred.mesos.metrics.csv.enabled
252 | false
253 |
254 | Set this to `true` to enable CSV reporting with the Coda Hale Metrics
255 | library.
256 |
257 |
258 |
259 | mapred.mesos.metrics.csv.path
260 | /path/to/metrics/csv/metrics.csv
261 |
262 | Set this to a file which will be created with CSV metrics data.
263 |
264 |
265 | mapred.mesos.metrics.csv.interval
266 | 60
267 |
268 |
269 |
270 |
271 |
272 | mapred.mesos.metrics.graphite.enabled
273 | false
274 |
275 | Set this to `true` to enable Graphite reporting with the Coda Hale Metrics
276 | library.
277 |
278 |
279 |
280 | mapred.mesos.metrics.graphite.host
281 | graphite.host.name
282 |
283 |
284 | mapred.mesos.metrics.graphite.port
285 | 2003
286 |
287 |
288 | mapred.mesos.metrics.graphite.prefix
289 | prefix
290 |
291 |
292 | mapred.mesos.metrics.graphite.interval
293 | 60
294 |
295 |
296 |
297 |
298 | mapred.mesos.metrics.cassandra.enabled
299 | false
300 |
301 | Set this to `true` to enable Cassandra reporting with the Coda Hale
302 | Metrics library.
303 |
304 |
305 |
306 | mapred.mesos.metrics.cassandra.hosts
307 | localhost
308 |
309 |
310 | mapred.mesos.metrics.cassandra.port
311 | 9042
312 |
313 |
314 | mapred.mesos.metrics.cassandra.interval
315 | 60
316 |
317 |
318 | mapred.mesos.metrics.cassandra.prefix
319 | prefix
320 |
321 |
322 | mapred.mesos.metrics.cassandra.ttl
323 | 864000
324 |
325 |
326 | mapred.mesos.metrics.cassandra.keyspace
327 | metrics
328 |
329 |
330 | mapred.mesos.metrics.cassandra.table
331 | metrics
332 |
333 |
334 | mapred.mesos.metrics.cassandra.consistency
335 | QUORUM
336 |
337 |
338 | ```
339 |
--------------------------------------------------------------------------------
/lib/org/apache/mesos/mesos/0.14.0-20130806/mesos-0.14.0-20130806.jar:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/mesos/hadoop/5a036b0daad52bdc27e6f156b64489de1d71485a/lib/org/apache/mesos/mesos/0.14.0-20130806/mesos-0.14.0-20130806.jar
--------------------------------------------------------------------------------
/lib/org/apache/mesos/mesos/0.14.0-20130806/mesos-0.14.0-20130806.pom:
--------------------------------------------------------------------------------
1 |
2 |
4 | 4.0.0
5 | org.apache.mesos
6 | mesos
7 | 0.14.0-20130806
8 | POM was created from install:install-file
9 |
10 |
--------------------------------------------------------------------------------
/lib/org/apache/mesos/mesos/maven-metadata-local.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 | org.apache.mesos
4 | mesos
5 |
6 | 0.14.0-20130806
7 |
8 | 0.14.0-20130806
9 |
10 | 20130807155313
11 |
12 |
13 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 | 4.0.0
4 |
5 | org.apache.mesos
6 | hadoop-mesos
7 | 0.1.0
8 |
9 |
10 | UTF-8
11 |
12 |
13 | 1.7
14 |
15 |
16 | 1.1.3
17 | 3.1
18 | 2.5.0-mr1-cdh5.2.0
19 | 0.23.1
20 | 2.5.0
21 | 3.1.0
22 | 1.0.5
23 |
24 |
25 | 4.11
26 | 1.9.5
27 |
28 |
29 |
30 |
31 |
32 | cloudera
33 | https://repository.cloudera.com/artifactory/cloudera-repos/
34 |
35 |
36 | clojars.org
37 | https://clojars.org/repo
38 |
39 |
40 |
41 |
42 |
43 | commons-logging
44 | commons-logging
45 | ${commons-logging.version}
46 |
47 |
48 | commons-httpclient
49 | commons-httpclient
50 | ${commons-httpclient.version}
51 |
52 |
53 | org.apache.hadoop
54 | hadoop-client
55 | ${hadoop-client.version}
56 | provided
57 |
58 |
59 | org.apache.mesos
60 |
61 | mesos
62 | ${mesos.version}
63 |
64 |
65 | com.google.protobuf
66 | protobuf-java
67 | ${protobuf.version}
68 |
69 |
70 | io.dropwizard.metrics
71 | metrics-graphite
72 | ${metrics.version}
73 |
74 |
75 | org.slf4j
76 | slf4j-api
77 |
78 |
79 |
80 |
81 | io.dropwizard.metrics
82 | metrics-jvm
83 | ${metrics.version}
84 |
85 |
86 | org.slf4j
87 | slf4j-api
88 |
89 |
90 |
91 |
92 | org.clojars.brenden
93 | metrics-cassandra
94 | ${metrics.version}
95 |
96 |
97 | org.slf4j
98 | slf4j-api
99 |
100 |
101 | org.codehaus.jackson
102 | jackson-core-asl
103 |
104 |
105 | org.codehaus.jackson
106 | jackson-mapper-asl
107 |
108 |
109 |
110 |
111 | org.xerial.snappy
112 | snappy-java
113 | ${snappy-java.version}
114 |
115 |
116 |
117 |
118 | junit
119 | junit
120 | ${junit.version}
121 | test
122 |
123 |
124 |
125 |
126 |
127 | org.apache.maven.plugins
128 | maven-compiler-plugin
129 | 3.1
130 |
131 | ${java.abi}
132 | ${java.abi}
133 | true
134 | true
135 |
136 |
137 |
138 |
139 | org.apache.maven.plugins
140 | maven-shade-plugin
141 | 2.2
142 |
143 |
144 |
145 | *:*
146 |
147 | META-INF/*.SF
148 | META-INF/*.DSA
149 | META-INF/*.RSA
150 |
151 |
152 |
153 |
154 |
155 |
156 | package
157 |
158 | shade
159 |
160 |
161 |
162 |
163 |
164 |
165 |
166 |
167 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/MesosExecutor.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | import org.apache.commons.logging.Log;
4 | import org.apache.commons.logging.LogFactory;
5 | import org.apache.mesos.Executor;
6 | import org.apache.mesos.ExecutorDriver;
7 | import org.apache.mesos.MesosExecutorDriver;
8 | import org.apache.mesos.Protos.*;
9 | import org.apache.mesos.Protos.TaskID;
10 | import org.apache.mesos.Protos.TaskStatus;
11 |
12 | import java.io.*;
13 |
14 | import java.lang.reflect.Field;
15 | import java.lang.ReflectiveOperationException;
16 |
17 | import java.util.concurrent.BlockingQueue;
18 | import java.util.concurrent.Executors;
19 | import java.util.concurrent.ScheduledExecutorService;
20 | import java.util.concurrent.TimeUnit;
21 |
22 | public class MesosExecutor implements Executor {
23 | public static final Log LOG = LogFactory.getLog(MesosExecutor.class);
24 | private SlaveInfo slaveInfo;
25 | private TaskTracker taskTracker;
26 |
27 | protected final ScheduledExecutorService timerScheduler =
28 | Executors.newScheduledThreadPool(1);
29 |
30 | public static void main(String[] args) {
31 | MesosExecutorDriver driver = new MesosExecutorDriver(new MesosExecutor());
32 | System.exit(driver.run() == Status.DRIVER_STOPPED ? 0 : 1);
33 | }
34 |
35 | private JobConf configure(final TaskInfo task) {
36 | JobConf conf = new JobConf(false);
37 | try {
38 | byte[] bytes = task.getData().toByteArray();
39 | conf.readFields(new DataInputStream(new ByteArrayInputStream(bytes)));
40 | } catch (IOException e) {
41 | LOG.warn("Failed to deserialize configuration.", e);
42 | System.exit(1);
43 | }
44 |
45 | // Output the configuration as XML for easy debugging.
46 | try {
47 | StringWriter writer = new StringWriter();
48 | conf.writeXml(writer);
49 | writer.flush();
50 | String xml = writer.getBuffer().toString();
51 | LOG.info("XML Configuration received:\n" +
52 | org.apache.mesos.hadoop.Utils.formatXml(xml));
53 | } catch (Exception e) {
54 | LOG.warn("Failed to output configuration as XML.", e);
55 | }
56 |
57 | // Get hostname from Mesos to make sure we match what it reports
58 | // to the JobTracker.
59 | conf.set("slave.host.name", slaveInfo.getHostname());
60 |
61 | String sandbox = System.getenv("MESOS_SANDBOX");
62 | if (sandbox == null || sandbox.equals("")) {
63 | sandbox = System.getenv("MESOS_DIRECTORY");
64 | }
65 |
66 | // Set the mapred.local directory inside the executor sandbox, so that
67 | // different TaskTrackers on the same host do not step on each other.
68 | conf.set("mapred.local.dir", sandbox + "/mapred");
69 |
70 | return conf;
71 | }
72 |
73 | @Override
74 | public void registered(ExecutorDriver driver, ExecutorInfo executorInfo,
75 | FrameworkInfo frameworkInfo, SlaveInfo slaveInfo) {
76 | LOG.info("Executor registered with the slave");
77 | this.slaveInfo = slaveInfo;
78 | }
79 |
80 | @Override
81 | public void launchTask(final ExecutorDriver driver, final TaskInfo task) {
82 | LOG.info("Launching task : " + task.getTaskId().getValue());
83 |
84 | // Get configuration from task data (prepared by the JobTracker).
85 | JobConf conf = configure(task);
86 |
87 | // NOTE: We need to manually set the context class loader here because,
88 | // the TaskTracker is unable to find LoginModule class otherwise.
89 | Thread.currentThread().setContextClassLoader(
90 | TaskTracker.class.getClassLoader());
91 |
92 | try {
93 | taskTracker = new TaskTracker(conf);
94 | } catch (IOException e) {
95 | LOG.fatal("Failed to start TaskTracker", e);
96 | System.exit(1);
97 | } catch (InterruptedException e) {
98 | LOG.fatal("Failed to start TaskTracker", e);
99 | System.exit(1);
100 | }
101 |
102 | // Spin up a TaskTracker in a new thread.
103 | new Thread("TaskTracker Run Thread") {
104 | @Override
105 | public void run() {
106 | try {
107 | taskTracker.run();
108 |
109 | // Send a TASK_FINISHED status update.
110 | // We do this here because we want to send it in a separate thread
111 | // than was used to call killTask().
112 | driver.sendStatusUpdate(TaskStatus.newBuilder()
113 | .setTaskId(task.getTaskId())
114 | .setState(TaskState.TASK_FINISHED)
115 | .build());
116 |
117 | // Give some time for the update to reach the slave.
118 | try {
119 | Thread.sleep(2000);
120 | } catch (InterruptedException e) {
121 | LOG.error("Failed to sleep TaskTracker thread", e);
122 | }
123 |
124 | // Stop the executor.
125 | driver.stop();
126 | } catch (Throwable t) {
127 | LOG.error("Caught exception, committing suicide.", t);
128 | driver.stop();
129 | System.exit(1);
130 | }
131 | }
132 | }.start();
133 |
134 | driver.sendStatusUpdate(TaskStatus.newBuilder()
135 | .setTaskId(task.getTaskId())
136 | .setState(TaskState.TASK_RUNNING).build());
137 | }
138 |
139 | @Override
140 | public void killTask(final ExecutorDriver driver, final TaskID taskId) {
141 | LOG.info("Killing task : " + taskId.getValue());
142 | if (taskTracker != null) {
143 | LOG.info("Revoking task tracker map/reduce slots");
144 | revokeSlots();
145 |
146 | // Send the TASK_FINISHED status
147 | new Thread("TaskFinishedUpdate") {
148 | @Override
149 | public void run() {
150 | driver.sendStatusUpdate(TaskStatus.newBuilder()
151 | .setTaskId(taskId)
152 | .setState(TaskState.TASK_FINISHED)
153 | .build());
154 | }
155 | }.start();
156 | }
157 | }
158 |
159 | @Override
160 | public void reregistered(ExecutorDriver driver, SlaveInfo slaveInfo) {
161 | LOG.info("Executor reregistered with the slave");
162 | }
163 |
164 | @Override
165 | public void disconnected(ExecutorDriver driver) {
166 | LOG.info("Executor disconnected from the slave");
167 | }
168 |
169 | @Override
170 | public void frameworkMessage(ExecutorDriver d, byte[] msg) {
171 | LOG.info("Executor received framework message of length: " + msg.length
172 | + " bytes");
173 | }
174 |
175 | @Override
176 | public void error(ExecutorDriver d, String message) {
177 | LOG.error("MesosExecutor.error: " + message);
178 | }
179 |
180 | @Override
181 | public void shutdown(ExecutorDriver d) {
182 | LOG.info("Executor asked to shutdown");
183 | }
184 |
185 | public void revokeSlots() {
186 | if (taskTracker == null) {
187 | LOG.error("Task tracker is not initialized");
188 | return;
189 | }
190 |
191 | int maxMapSlots = 0;
192 | int maxReduceSlots = 0;
193 |
194 | // TODO(tarnfeld): Sanity check that it's safe for us to change the slots.
195 | // Be sure there's nothing running and nothing in the launcher queue.
196 |
197 | // If we expect to have no slots, let's go ahead and terminate the task launchers
198 | if (maxMapSlots == 0) {
199 | try {
200 | Field launcherField = taskTracker.getClass().getDeclaredField("mapLauncher");
201 | launcherField.setAccessible(true);
202 |
203 | // Kill the current map task launcher
204 | TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
205 | launcher.notifySlots();
206 | launcher.interrupt();
207 | } catch (ReflectiveOperationException e) {
208 | LOG.fatal("Failed updating map slots due to error with reflection", e);
209 | }
210 | }
211 |
212 | if (maxReduceSlots == 0) {
213 | try {
214 | Field launcherField = taskTracker.getClass().getDeclaredField("reduceLauncher");
215 | launcherField.setAccessible(true);
216 |
217 | // Kill the current reduce task launcher
218 | TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
219 | launcher.notifySlots();
220 | launcher.interrupt();
221 | } catch (ReflectiveOperationException e) {
222 | LOG.fatal("Failed updating reduce slots due to error with reflection", e);
223 | }
224 | }
225 |
226 | // Configure the new slot counts on the task tracker
227 | taskTracker.setMaxMapSlots(maxMapSlots);
228 | taskTracker.setMaxReduceSlots(maxReduceSlots);
229 |
230 | // If we have zero slots left, commit suicide when no jobs are running
231 | if ((maxMapSlots + maxReduceSlots) == 0) {
232 | scheduleSuicideTimer();
233 | }
234 | }
235 |
236 | protected void scheduleSuicideTimer() {
237 | timerScheduler.schedule(new Runnable() {
238 | @Override
239 | public void run() {
240 | if (taskTracker == null) {
241 | return;
242 | }
243 |
244 | LOG.info("Checking to see if TaskTracker is idle");
245 |
246 | // If the task tracker is idle, all tasks have finished and task output
247 | // has been cleaned up.
248 | if (taskTracker.isIdle()) {
249 | LOG.warn("TaskTracker is idle, terminating");
250 |
251 | try {
252 | taskTracker.shutdown();
253 | } catch (IOException e) {
254 | LOG.error("Failed to shutdown TaskTracker", e);
255 | } catch (InterruptedException e) {
256 | LOG.error("Failed to shutdown TaskTracker", e);
257 | }
258 | }
259 | else {
260 | try {
261 | Field field = taskTracker.getClass().getDeclaredField("tasksToCleanup");
262 | field.setAccessible(true);
263 | BlockingQueue tasksToCleanup = ((BlockingQueue) field.get(taskTracker));
264 | LOG.info("TaskTracker has " + taskTracker.tasks.size() +
265 | " running tasks and " + tasksToCleanup +
266 | " tasks to clean up.");
267 | } catch (ReflectiveOperationException e) {
268 | LOG.fatal("Failed to get task counts from TaskTracker", e);
269 | }
270 |
271 | scheduleSuicideTimer();
272 | }
273 | }
274 | }, 1000, TimeUnit.MILLISECONDS);
275 | }
276 | }
277 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/MesosScheduler.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | import com.codahale.metrics.Meter;
4 | import org.apache.commons.httpclient.HttpHost;
5 | import org.apache.commons.logging.Log;
6 | import org.apache.commons.logging.LogFactory;
7 | import org.apache.hadoop.conf.Configuration;
8 | import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
9 | import org.apache.mesos.MesosSchedulerDriver;
10 | import org.apache.mesos.Protos;
11 | import org.apache.mesos.Protos.*;
12 | import org.apache.mesos.Scheduler;
13 | import org.apache.mesos.SchedulerDriver;
14 | import org.apache.mesos.hadoop.Metrics;
15 |
16 | import java.io.File;
17 | import java.io.IOException;
18 | import java.util.*;
19 | import java.util.concurrent.ConcurrentHashMap;
20 | import java.util.concurrent.Executors;
21 | import java.util.concurrent.ScheduledExecutorService;
22 | import java.util.concurrent.TimeUnit;
23 |
24 | public class MesosScheduler extends TaskScheduler implements Scheduler {
25 | public static final Log LOG = LogFactory.getLog(MesosScheduler.class);
26 |
27 | // This is the memory overhead for a jvm process. This needs to be added
28 | // to a jvm process's resource requirement, in addition to its heap size.
29 | public static final double JVM_MEM_OVERHEAD_PERCENT_DEFAULT = 0.25; // 25%.
30 |
31 | // NOTE: It appears that there's no real resource requirements for a
32 | // map / reduce slot. We therefore define a default slot as:
33 | // 1 cores.
34 | // 1024 MB memory.
35 | // 1 GB of disk space.
36 | public static final double SLOT_CPUS_DEFAULT = 1; // 1 cores.
37 | public static final int SLOT_DISK_DEFAULT = 1024; // 1 GB.
38 | public static final int SLOT_JVM_HEAP_DEFAULT = 1024; // 1024MB.
39 | public static final double TASKTRACKER_CPUS_DEFAULT = 1.0; // 1 core.
40 | public static final int TASKTRACKER_MEM_DEFAULT = 1024; // 1 GB.
41 | public static final int TASKTRACKER_DISK_DEFAULT = 1024; // 1 GB.
42 | // The default behavior in Hadoop is to use 4 slots per TaskTracker:
43 | public static final int MAP_SLOTS_DEFAULT = 2;
44 | public static final int REDUCE_SLOTS_DEFAULT = 2;
45 | // The amount of time to wait for task trackers to launch before
46 | // giving up.
47 | public static final long LAUNCH_TIMEOUT_MS = 300000; // 5 minutes
48 | public static final long PERIODIC_MS = 300000; // 5 minutes
49 | public static final long DEFAULT_IDLE_CHECK_INTERVAL = 5; // 5 seconds
50 | // Destroy task trackers after being idle for N idle checks
51 | public static final long DEFAULT_IDLE_REVOCATION_CHECKS = 5;
52 | private SchedulerDriver driver;
53 |
54 | protected TaskScheduler taskScheduler;
55 | protected JobTracker jobTracker;
56 | protected Configuration conf;
57 | protected File stateFile;
58 | // Count of the launched trackers for TaskID generation.
59 | protected long launchedTrackers = 0;
60 | // Use a fixed slot allocation policy?
61 | protected boolean policyIsFixed = false;
62 | protected ResourcePolicy policy;
63 |
64 | protected boolean enableMetrics = false;
65 | public Metrics metrics;
66 |
67 | // Maintains a mapping from {tracker host:port -> MesosTracker}.
68 | // Used for tracking the slots of each TaskTracker and the corresponding
69 | // Mesos TaskID.
70 | protected Map mesosTrackers =
71 | new ConcurrentHashMap();
72 |
73 | protected final ScheduledExecutorService timerScheduler =
74 | Executors.newScheduledThreadPool(1);
75 |
76 | protected JobInProgressListener jobListener = new JobInProgressListener() {
77 | @Override
78 | public void jobAdded(JobInProgress job) throws IOException {
79 | LOG.info("Added job " + job.getJobID());
80 | if (metrics != null) {
81 | metrics.jobTimerContexts.put(job.getJobID(), metrics.jobTimer.time());
82 | }
83 | }
84 |
85 | @Override
86 | public void jobRemoved(JobInProgress job) {
87 | LOG.info("Removed job " + job.getJobID());
88 | }
89 |
90 | @Override
91 | public void jobUpdated(JobChangeEvent event) {
92 | synchronized (MesosScheduler.this) {
93 | JobInProgress job = event.getJobInProgress();
94 |
95 | if (metrics != null) {
96 | Meter meter = metrics.jobStateMeter.get(job.getStatus().getRunState());
97 | if (meter != null) {
98 | meter.mark();
99 | }
100 | }
101 |
102 | // If we have flaky tasktrackers, kill them.
103 | final List flakyTrackers = job.getBlackListedTrackers();
104 | // Remove the task from the map. This is O(n^2), but there's no better
105 | // way to do it, AFAIK. The flakyTrackers list should usually be
106 | // small, so this is probably not bad.
107 | for (String hostname : flakyTrackers) {
108 | for (MesosTracker mesosTracker : mesosTrackers.values()) {
109 | if (mesosTracker.host.getHostName().startsWith(hostname)) {
110 | LOG.info("Killing Mesos task: " + mesosTracker.taskId + " on host "
111 | + mesosTracker.host + " because it has been marked as flaky");
112 | if (metrics != null) {
113 | metrics.flakyTrackerKilledMeter.mark();
114 | }
115 | killTracker(mesosTracker);
116 | }
117 | }
118 | }
119 |
120 | // If the job is complete, kill all the corresponding idle TaskTrackers.
121 | if (!job.isComplete()) {
122 | return;
123 | }
124 |
125 | if (metrics != null) {
126 | com.codahale.metrics.Timer.Context context = metrics.jobTimerContexts.get(job.getJobID());
127 | context.stop();
128 | metrics.jobTimerContexts.remove(job.getJobID());
129 | }
130 |
131 | LOG.info("Completed job : " + job.getJobID());
132 |
133 | // Remove the task from the map.
134 | final Set trackers = new HashSet(mesosTrackers.keySet());
135 | for (HttpHost tracker : trackers) {
136 | MesosTracker mesosTracker = mesosTrackers.get(tracker);
137 | mesosTracker.jobs.remove(job.getJobID());
138 |
139 | // If the TaskTracker doesn't have any running job tasks assigned,
140 | // kill it.
141 | if (mesosTracker.jobs.isEmpty() && mesosTracker.active) {
142 | LOG.info("Killing Mesos task: " + mesosTracker.taskId + " on host "
143 | + mesosTracker.host + " because it is no longer needed");
144 |
145 | killTracker(mesosTracker);
146 | }
147 | }
148 | }
149 | }
150 | };
151 |
152 | // TaskScheduler methods.
153 | @Override
154 | public synchronized void start() throws IOException {
155 | conf = getConf();
156 | String taskTrackerClass = conf.get("mapred.mesos.taskScheduler",
157 | "org.apache.hadoop.mapred.JobQueueTaskScheduler");
158 |
159 | try {
160 | taskScheduler =
161 | (TaskScheduler) Class.forName(taskTrackerClass).newInstance();
162 | taskScheduler.setConf(conf);
163 | taskScheduler.setTaskTrackerManager(taskTrackerManager);
164 | } catch (ClassNotFoundException e) {
165 | LOG.fatal("Failed to initialize the TaskScheduler", e);
166 | System.exit(1);
167 | } catch (InstantiationException e) {
168 | LOG.fatal("Failed to initialize the TaskScheduler", e);
169 | System.exit(1);
170 | } catch (IllegalAccessException e) {
171 | LOG.fatal("Failed to initialize the TaskScheduler", e);
172 | System.exit(1);
173 | }
174 |
175 | // Add the job listener to get job related updates.
176 | taskTrackerManager.addJobInProgressListener(jobListener);
177 |
178 | LOG.info("Starting MesosScheduler");
179 | jobTracker = (JobTracker) super.taskTrackerManager;
180 |
181 | String master = conf.get("mapred.mesos.master", "local");
182 |
183 | try {
184 | FrameworkInfo frameworkInfo = FrameworkInfo
185 | .newBuilder()
186 | .setUser("") // Let Mesos fill in the user.
187 | .setCheckpoint(conf.getBoolean("mapred.mesos.checkpoint", false))
188 | .setRole(conf.get("mapred.mesos.role", "*"))
189 | .setName("Hadoop: (RPC port: " + jobTracker.port + ","
190 | + " WebUI port: " + jobTracker.infoPort + ")").build();
191 |
192 | driver = new MesosSchedulerDriver(this, frameworkInfo, master);
193 | driver.start();
194 | } catch (Exception e) {
195 | // If the MesosScheduler can't be loaded, the JobTracker won't be useful
196 | // at all, so crash it now so that the user notices.
197 | LOG.fatal("Failed to start MesosScheduler", e);
198 | System.exit(1);
199 | }
200 |
201 | String file = conf.get("mapred.mesos.state.file", "");
202 | if (!file.equals("")) {
203 | this.stateFile = new File(file);
204 | }
205 |
206 | policyIsFixed = conf.getBoolean("mapred.mesos.scheduler.policy.fixed",
207 | policyIsFixed);
208 |
209 | if (policyIsFixed) {
210 | policy = new ResourcePolicyFixed(this);
211 | } else {
212 | policy = new ResourcePolicyVariable(this);
213 | }
214 |
215 | enableMetrics = conf.getBoolean("mapred.mesos.metrics.enabled",
216 | enableMetrics);
217 |
218 | if (enableMetrics) {
219 | metrics = new Metrics(conf);
220 | }
221 |
222 | taskScheduler.start();
223 | }
224 |
225 | @Override
226 | public synchronized void terminate() throws IOException {
227 | try {
228 | LOG.info("Stopping MesosScheduler");
229 | driver.stop();
230 | } catch (Exception e) {
231 | LOG.error("Failed to stop Mesos scheduler", e);
232 | }
233 |
234 | taskScheduler.terminate();
235 | }
236 |
237 | @Override
238 | public void checkJobSubmission(JobInProgress job) throws IOException {
239 | taskScheduler.checkJobSubmission(job);
240 | }
241 |
242 | @Override
243 | public List assignTasks(TaskTracker taskTracker)
244 | throws IOException {
245 | HttpHost tracker = new HttpHost(taskTracker.getStatus().getHost(),
246 | taskTracker.getStatus().getHttpPort());
247 |
248 | if (!mesosTrackers.containsKey(tracker)) {
249 | LOG.info("Unknown/exited TaskTracker: " + tracker + ". ");
250 | return null;
251 | }
252 |
253 | MesosTracker mesosTracker = mesosTrackers.get(tracker);
254 |
255 | // Make sure we're not asked to assign tasks to any task trackers that have
256 | // been stopped. This could happen while the task tracker has not been
257 | // removed from the cluster e.g still in the heartbeat timeout period.
258 | synchronized (this) {
259 | if (mesosTracker.stopped) {
260 | LOG.info("Asked to assign tasks to stopped tracker " + tracker + ".");
261 | return null;
262 | }
263 | }
264 |
265 | // Let the underlying task scheduler do the actual task scheduling.
266 | List tasks = taskScheduler.assignTasks(taskTracker);
267 |
268 | // The Hadoop Fair Scheduler is known to return null.
269 | if (tasks == null) {
270 | return null;
271 | }
272 |
273 | // Keep track of which TaskTracker contains which tasks.
274 | for (Task task : tasks) {
275 | mesosTracker.jobs.add(task.getJobID());
276 | }
277 |
278 | return tasks;
279 | }
280 |
281 | @Override
282 | public synchronized Collection getJobs(String queueName) {
283 | return taskScheduler.getJobs(queueName);
284 | }
285 |
286 | @Override
287 | public synchronized void refresh() throws IOException {
288 | taskScheduler.refresh();
289 | }
290 |
291 | // Mesos Scheduler methods.
292 | // These are synchronized, where possible. Some of these methods need to access the
293 | // JobTracker, which can lead to a deadlock:
294 | // See: https://issues.apache.org/jira/browse/MESOS-429
295 | // The workaround employed here is to unsynchronize those methods needing access to
296 | // the JobTracker state and use explicit synchronization instead as appropriate.
297 | // TODO(bmahler): Provide a cleaner solution to this issue. One solution is to
298 | // split up the Scheduler and TaskScheduler implementations in order to break the
299 | // locking cycle. This would require a synchronized class to store the shared
300 | // state across our Scheduler and TaskScheduler implementations, and provide
301 | // atomic operations as needed.
302 | @Override
303 | public synchronized void registered(SchedulerDriver schedulerDriver,
304 | FrameworkID frameworkID, MasterInfo masterInfo) {
305 | LOG.info("Registered as " + frameworkID.getValue()
306 | + " with master " + masterInfo);
307 | }
308 |
309 | @Override
310 | public synchronized void reregistered(SchedulerDriver schedulerDriver,
311 | MasterInfo masterInfo) {
312 | LOG.info("Re-registered with master " + masterInfo);
313 | }
314 |
315 | public void killTracker(MesosTracker tracker) {
316 | if (metrics != null) {
317 | metrics.killMeter.mark();
318 | }
319 | synchronized (this) {
320 | driver.killTask(tracker.taskId);
321 | }
322 | tracker.stop();
323 | if (mesosTrackers.get(tracker.host) == tracker) {
324 | mesosTrackers.remove(tracker.host);
325 | }
326 | }
327 |
328 | public synchronized void scheduleTimer(Runnable command,
329 | long delay,
330 | TimeUnit unit) {
331 | timerScheduler.schedule(command, delay, unit);
332 | }
333 |
334 | // For some reason, pendingMaps() and pendingReduces() doesn't return the
335 | // values we expect. We observed negative values, which may be related to
336 | // https://issues.apache.org/jira/browse/MAPREDUCE-1238. Below is the
337 | // algorithm that is used to calculate the pending tasks within the Hadoop
338 | // JobTracker sources (see 'printTaskSummary' in
339 | // src/org/apache/hadoop/mapred/jobdetails_jsp.java).
340 | public int getPendingTasks(TaskInProgress[] tasks) {
341 | int totalTasks = tasks.length;
342 | int runningTasks = 0;
343 | int finishedTasks = 0;
344 | int killedTasks = 0;
345 | for (int i = 0; i < totalTasks; ++i) {
346 | TaskInProgress task = tasks[i];
347 | if (task == null) {
348 | continue;
349 | }
350 | if (task.isComplete()) {
351 | finishedTasks += 1;
352 | } else if (task.isRunning()) {
353 | runningTasks += 1;
354 | } else if (task.wasKilled()) {
355 | killedTasks += 1;
356 | }
357 | }
358 | int pendingTasks = totalTasks - runningTasks - killedTasks - finishedTasks;
359 | return pendingTasks;
360 | }
361 |
362 | // This method uses explicit synchronization in order to avoid deadlocks when
363 | // accessing the JobTracker.
364 | @Override
365 | public void resourceOffers(SchedulerDriver schedulerDriver,
366 | List offers) {
367 | policy.resourceOffers(schedulerDriver, offers);
368 | }
369 |
370 | @Override
371 | public synchronized void offerRescinded(SchedulerDriver schedulerDriver,
372 | OfferID offerID) {
373 | LOG.warn("Rescinded offer: " + offerID.getValue());
374 | }
375 |
376 | @Override
377 | public synchronized void statusUpdate(SchedulerDriver schedulerDriver,
378 | Protos.TaskStatus taskStatus) {
379 | LOG.info("Status update of " + taskStatus.getTaskId().getValue()
380 | + " to " + taskStatus.getState().name()
381 | + " with message " + taskStatus.getMessage());
382 |
383 | // Remove the TaskTracker if the corresponding Mesos task has reached a
384 | // terminal state.
385 | switch (taskStatus.getState()) {
386 | case TASK_FINISHED:
387 | case TASK_FAILED:
388 | case TASK_KILLED:
389 | case TASK_LOST:
390 | // Make a copy to iterate over keys and delete values.
391 | Set trackers = new HashSet(mesosTrackers.keySet());
392 |
393 | // Remove the task from the map.
394 | for (HttpHost tracker : trackers) {
395 | if (mesosTrackers.get(tracker).taskId.equals(taskStatus.getTaskId())) {
396 | LOG.info("Removing terminated TaskTracker: " + tracker);
397 | mesosTrackers.get(tracker).stop();
398 | mesosTrackers.remove(tracker);
399 | }
400 | }
401 | break;
402 | case TASK_STAGING:
403 | case TASK_STARTING:
404 | case TASK_RUNNING:
405 | break;
406 | default:
407 | LOG.error("Unexpected TaskStatus: " + taskStatus.getState().name());
408 | break;
409 | }
410 |
411 | if (metrics != null) {
412 | Meter meter = metrics.taskStateMeter.get(taskStatus.getState());
413 | if (meter != null) {
414 | meter.mark();
415 | }
416 | }
417 | }
418 |
419 | @Override
420 | public synchronized void frameworkMessage(SchedulerDriver schedulerDriver,
421 | ExecutorID executorID, SlaveID slaveID, byte[] bytes) {
422 | LOG.info("Framework Message of " + bytes.length + " bytes"
423 | + " from executor " + executorID.getValue()
424 | + " on slave " + slaveID.getValue());
425 | }
426 |
427 | @Override
428 | public synchronized void disconnected(SchedulerDriver schedulerDriver) {
429 | LOG.warn("Disconnected from Mesos master.");
430 | }
431 |
432 | @Override
433 | public synchronized void slaveLost(SchedulerDriver schedulerDriver,
434 | SlaveID slaveID) {
435 | LOG.warn("Slave lost: " + slaveID.getValue());
436 | }
437 |
438 | @Override
439 | public synchronized void executorLost(SchedulerDriver schedulerDriver,
440 | ExecutorID executorID, SlaveID slaveID, int status) {
441 | LOG.warn("Executor " + executorID.getValue()
442 | + " lost with status " + status + " on slave " + slaveID);
443 | }
444 |
445 | @Override
446 | public synchronized void error(SchedulerDriver schedulerDriver, String s) {
447 | LOG.error("Error from scheduler driver: " + s);
448 | }
449 | }
450 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/MesosTracker.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | import org.apache.commons.httpclient.HttpHost;
4 | import org.apache.commons.logging.Log;
5 | import org.apache.commons.logging.LogFactory;
6 | import org.apache.mesos.Protos.TaskID;
7 |
8 | import java.util.Collection;
9 | import java.util.Collections;
10 | import java.util.HashSet;
11 | import java.util.Set;
12 | import java.util.concurrent.ConcurrentHashMap;
13 | import java.util.concurrent.TimeUnit;
14 |
15 | /**
16 | * Used to track the our launched TaskTrackers.
17 | */
18 | public class MesosTracker {
19 |
20 | public static final Log LOG = LogFactory.getLog(MesosScheduler.class);
21 | public volatile HttpHost host;
22 | public TaskID taskId;
23 | public long mapSlots;
24 | public long reduceSlots;
25 | public volatile long idleCounter = 0;
26 | public volatile long idleCheckInterval = 0;
27 | public volatile long idleCheckMax = 0;
28 | public volatile boolean active = false; // Set once tracked by the JobTracker.
29 | public volatile boolean stopped = false;
30 | public volatile MesosScheduler scheduler;
31 | // Tracks Hadoop jobs running on the tracker.
32 | public Set jobs = Collections.newSetFromMap(new ConcurrentHashMap());
33 | public com.codahale.metrics.Timer.Context context;
34 |
35 | public MesosTracker(HttpHost host, TaskID taskId, long mapSlots,
36 | long reduceSlots, MesosScheduler scheduler) {
37 | this.host = host;
38 | this.taskId = taskId;
39 | this.mapSlots = mapSlots;
40 | this.reduceSlots = reduceSlots;
41 | this.scheduler = scheduler;
42 |
43 | if (scheduler.metrics != null) {
44 | this.context = scheduler.metrics.trackerTimer.time();
45 | }
46 |
47 | this.idleCheckInterval = scheduler.conf.getLong("mapred.mesos.tracker.idle.interval",
48 | MesosScheduler.DEFAULT_IDLE_CHECK_INTERVAL);
49 | this.idleCheckMax = scheduler.conf.getLong("mapred.mesos.tracker.idle.checks",
50 | MesosScheduler.DEFAULT_IDLE_REVOCATION_CHECKS);
51 |
52 | scheduleStartupTimer();
53 | if (this.idleCheckInterval > 0 && this.idleCheckMax > 0) {
54 | scheduleIdleCheck();
55 | }
56 | }
57 |
58 | protected void scheduleStartupTimer() {
59 | scheduler.scheduleTimer(new Runnable() {
60 | @Override
61 | public void run() {
62 | if (MesosTracker.this.active) {
63 | // If the tracker activated while we were awaiting to acquire the
64 | // lock, start the periodic cleanup timer and return.
65 | schedulePeriodic();
66 | return;
67 | }
68 |
69 | // When the scheduler is busy or doesn't receive offers, it may
70 | // fail to mark some TaskTrackers as active even though they are.
71 | // Here we do a final check with the JobTracker to make sure this
72 | // TaskTracker is really not there before we kill it.
73 | final Collection taskTrackers =
74 | MesosTracker.this.scheduler.jobTracker.taskTrackers();
75 |
76 | for (TaskTrackerStatus status : taskTrackers) {
77 | HttpHost host = new HttpHost(status.getHost(), status.getHttpPort());
78 | if (status.getHealthStatus().isNodeHealthy() && MesosTracker.this.host.equals(host)) {
79 | schedulePeriodic();
80 | return;
81 | }
82 | }
83 |
84 | if (MesosTracker.this.scheduler.metrics != null) {
85 | MesosTracker.this.scheduler.metrics.launchTimeout.mark();
86 | }
87 | LOG.warn("Tracker " + MesosTracker.this.host + " failed to launch within " +
88 | MesosScheduler.LAUNCH_TIMEOUT_MS / 1000 + " seconds, killing it");
89 | MesosTracker.this.scheduler.killTracker(MesosTracker.this);
90 | }
91 | }, MesosScheduler.LAUNCH_TIMEOUT_MS, TimeUnit.MILLISECONDS);
92 | }
93 |
94 | protected void scheduleIdleCheck() {
95 | scheduler.scheduleTimer(new Runnable() {
96 | @Override
97 | public void run() {
98 | // We're not interested if the task tracker has been stopped.
99 | if (MesosTracker.this.stopped) {
100 | return;
101 | }
102 |
103 | // If the task tracker isn't active, wait until it is active.
104 | // TODO(tarnfeld): Do this based on some kind of lock/wait?
105 | if (!MesosTracker.this.active) {
106 | scheduleIdleCheck();
107 | return;
108 | }
109 |
110 | boolean trackerIsIdle = false;
111 |
112 | // We're only interested in TaskTrackers which have jobs assigned to them
113 | // but are completely idle. The MesosScheduler is in charge of destroying
114 | // task trackers that are not handling any jobs, so we can leave those alone.
115 | if (MesosTracker.this.idleCounter >= MesosTracker.this.idleCheckMax) {
116 | LOG.info("Killing idle tasktracker: " + MesosTracker.this.host);
117 | MesosTracker.this.scheduler.killTracker(MesosTracker.this);
118 | scheduleIdleCheck();
119 | return;
120 | }
121 |
122 | long idleMapSlots = 0;
123 | long idleReduceSlots = 0;
124 |
125 | Collection taskTrackers = scheduler.jobTracker.taskTrackers();
126 | for (TaskTrackerStatus status : taskTrackers) {
127 | HttpHost host = new HttpHost(status.getHost(), status.getHttpPort());
128 | if (host.toString().equals(MesosTracker.this.host.toString())) {
129 | idleMapSlots += status.getAvailableMapSlots();
130 | idleReduceSlots += status.getAvailableReduceSlots();
131 | break;
132 | }
133 | }
134 |
135 | trackerIsIdle = idleMapSlots == MesosTracker.this.mapSlots &&
136 | idleReduceSlots == MesosTracker.this.reduceSlots;
137 |
138 | if (trackerIsIdle) {
139 | LOG.info("TaskTracker appears idle right now: " + MesosTracker.this.host);
140 | MesosTracker.this.idleCounter += 1;
141 | } else {
142 | if (MesosTracker.this.idleCounter > 0) {
143 | LOG.info("TaskTracker is no longer idle: " + MesosTracker.this.host);
144 | }
145 | MesosTracker.this.idleCounter = 0;
146 | }
147 |
148 | scheduleIdleCheck();
149 | }
150 | }, MesosTracker.this.idleCheckInterval, TimeUnit.SECONDS);
151 | }
152 |
153 | protected void schedulePeriodic() {
154 | scheduler.scheduleTimer(new Runnable() {
155 | @Override
156 | public void run() {
157 | if (MesosTracker.this.scheduler.mesosTrackers.containsKey(host) &&
158 | MesosTracker.this == MesosTracker.this.scheduler.mesosTrackers.get(host)) {
159 | // Periodically check if the jobs assigned to this TaskTracker are
160 | // still running (lazy GC).
161 | final Set jobsCopy = new HashSet(MesosTracker.this.jobs);
162 | for (JobID id : jobsCopy) {
163 | JobStatus jobStatus = MesosTracker.this.scheduler.jobTracker.getJobStatus(id);
164 | if (jobStatus == null || jobStatus.isJobComplete()) {
165 | if (MesosTracker.this.scheduler.metrics != null) {
166 | MesosTracker.this.scheduler.metrics.periodicGC.mark();
167 | }
168 | MesosTracker.this.jobs.remove(id);
169 | }
170 | }
171 | schedulePeriodic();
172 | }
173 | }
174 | }, MesosScheduler.PERIODIC_MS, TimeUnit.MILLISECONDS);
175 | }
176 |
177 | public void stop() {
178 | active = false;
179 | stopped = true;
180 | if (context != null) {
181 | context.stop();
182 | }
183 | }
184 | }
185 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/ResourcePolicy.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | import org.apache.commons.httpclient.HttpHost;
4 | import org.apache.commons.logging.Log;
5 | import org.apache.commons.logging.LogFactory;
6 | import org.apache.hadoop.conf.Configuration;
7 | import org.apache.hadoop.mapreduce.TaskType;
8 | import org.apache.hadoop.util.StringUtils;
9 | import org.apache.mesos.Protos;
10 | import org.apache.mesos.Protos.*;
11 | import org.apache.mesos.Protos.TaskID;
12 | import org.apache.mesos.SchedulerDriver;
13 | import com.google.protobuf.ByteString;
14 |
15 | import java.io.*;
16 | import java.util.*;
17 |
18 | import static org.apache.hadoop.util.StringUtils.join;
19 |
20 | public class ResourcePolicy {
21 | public static final Log LOG = LogFactory.getLog(ResourcePolicy.class);
22 | public volatile MesosScheduler scheduler;
23 | public int neededMapSlots;
24 | public int neededReduceSlots;
25 | public long slots, mapSlots, reduceSlots;
26 | public int mapSlotsMax, reduceSlotsMax;
27 | double slotCpus;
28 | double slotDisk;
29 | int slotMem;
30 | long slotJVMHeap;
31 | int tasktrackerMem;
32 | long tasktrackerJVMHeap;
33 | // Minimum resource requirements for the container (TaskTracker + map/red
34 | // tasks).
35 | double containerCpus;
36 | double containerMem;
37 | double containerDisk;
38 | double cpus;
39 | double mem;
40 | double disk;
41 |
42 | public ResourcePolicy(MesosScheduler scheduler) {
43 | this.scheduler = scheduler;
44 |
45 | mapSlotsMax = scheduler.conf.getInt("mapred.tasktracker.map.tasks.maximum",
46 | MesosScheduler.MAP_SLOTS_DEFAULT);
47 | reduceSlotsMax =
48 | scheduler.conf.getInt("mapred.tasktracker.reduce.tasks.maximum",
49 | MesosScheduler.REDUCE_SLOTS_DEFAULT);
50 |
51 | slotCpus = scheduler.conf.getFloat("mapred.mesos.slot.cpus",
52 | (float) MesosScheduler.SLOT_CPUS_DEFAULT);
53 | slotDisk = scheduler.conf.getInt("mapred.mesos.slot.disk",
54 | MesosScheduler.SLOT_DISK_DEFAULT);
55 |
56 | slotMem = scheduler.conf.getInt("mapred.mesos.slot.mem",
57 | MesosScheduler.SLOT_JVM_HEAP_DEFAULT);
58 | slotJVMHeap = Math.round((double) slotMem /
59 | (MesosScheduler.JVM_MEM_OVERHEAD_PERCENT_DEFAULT + 1));
60 |
61 | tasktrackerMem = scheduler.conf.getInt("mapred.mesos.tasktracker.mem",
62 | MesosScheduler.TASKTRACKER_MEM_DEFAULT);
63 | tasktrackerJVMHeap = Math.round((double) tasktrackerMem /
64 | (MesosScheduler.JVM_MEM_OVERHEAD_PERCENT_DEFAULT + 1));
65 |
66 | containerCpus = scheduler.conf.getFloat("mapred.mesos.tasktracker.cpus",
67 | (float) MesosScheduler.TASKTRACKER_CPUS_DEFAULT);
68 | containerDisk = scheduler.conf.getInt("mapred.mesos.tasktracker.disk",
69 | MesosScheduler.TASKTRACKER_DISK_DEFAULT);
70 |
71 | containerMem = tasktrackerMem;
72 | }
73 |
74 | public void computeNeededSlots(List jobsInProgress,
75 | Collection taskTrackers) {
76 | // Compute the number of pending maps and reduces.
77 | int pendingMaps = 0;
78 | int pendingReduces = 0;
79 | int runningMaps = 0;
80 | int runningReduces = 0;
81 |
82 | for (JobInProgress progress : jobsInProgress) {
83 | // JobStatus.pendingMaps/Reduces may return the wrong value on
84 | // occasion. This seems to be safer.
85 | pendingMaps += scheduler.getPendingTasks(progress.getTasks(TaskType.MAP));
86 | pendingReduces += scheduler.getPendingTasks(progress.getTasks(TaskType.REDUCE));
87 | runningMaps += progress.runningMaps();
88 | runningReduces += progress.runningReduces();
89 |
90 | // If the task is waiting to launch the cleanup task, let us make sure we have
91 | // capacity to run the task.
92 | if (!progress.isCleanupLaunched()) {
93 | pendingMaps += scheduler.getPendingTasks(progress.getTasks(TaskType.JOB_CLEANUP));
94 | }
95 | }
96 |
97 | // Mark active (heartbeated) TaskTrackers and compute idle slots.
98 | int idleMapSlots = 0;
99 | int idleReduceSlots = 0;
100 | int unhealthyTrackers = 0;
101 |
102 | for (TaskTrackerStatus status : taskTrackers) {
103 | if (!status.getHealthStatus().isNodeHealthy()) {
104 | // Skip this node if it's unhealthy.
105 | ++unhealthyTrackers;
106 | continue;
107 | }
108 |
109 | HttpHost host = new HttpHost(status.getHost(), status.getHttpPort());
110 | if (scheduler.mesosTrackers.containsKey(host)) {
111 | scheduler.mesosTrackers.get(host).active = true;
112 | idleMapSlots += status.getAvailableMapSlots();
113 | idleReduceSlots += status.getAvailableReduceSlots();
114 | }
115 | }
116 |
117 | // Consider the TaskTrackers that have yet to become active as being idle,
118 | // otherwise we will launch excessive TaskTrackers.
119 | int inactiveMapSlots = 0;
120 | int inactiveReduceSlots = 0;
121 | for (MesosTracker tracker : scheduler.mesosTrackers.values()) {
122 | if (!tracker.active) {
123 | inactiveMapSlots += tracker.mapSlots;
124 | inactiveReduceSlots += tracker.reduceSlots;
125 | }
126 | }
127 |
128 | // To ensure Hadoop jobs begin promptly, we can specify a minimum number
129 | // of 'hot slots' to be available for use. This addresses the
130 | // TaskTracker spin up delay that exists with Hadoop on Mesos. This can
131 | // be a nuisance with lower latency applications, such as ad-hoc Hive
132 | // queries.
133 | int minimumMapSlots = scheduler.conf.getInt("mapred.mesos.total.map.slots.minimum", 0);
134 | int minimumReduceSlots =
135 | scheduler.conf.getInt("mapred.mesos.total.reduce.slots.minimum", 0);
136 |
137 | // Compute how many slots we need to allocate.
138 | neededMapSlots = Math.max(
139 | minimumMapSlots - (idleMapSlots + inactiveMapSlots),
140 | pendingMaps - (idleMapSlots + inactiveMapSlots));
141 | neededReduceSlots = Math.max(
142 | minimumReduceSlots - (idleReduceSlots + inactiveReduceSlots),
143 | pendingReduces - (idleReduceSlots + inactiveReduceSlots));
144 |
145 | LOG.info(join("\n", Arrays.asList(
146 | "JobTracker Status",
147 | " Pending Map Tasks: " + pendingMaps,
148 | " Pending Reduce Tasks: " + pendingReduces,
149 | " Running Map Tasks: " + runningMaps,
150 | " Running Reduce Tasks: " + runningReduces,
151 | " Idle Map Slots: " + idleMapSlots,
152 | " Idle Reduce Slots: " + idleReduceSlots,
153 | " Inactive Map Slots: " + inactiveMapSlots
154 | + " (launched but no hearbeat yet)",
155 | " Inactive Reduce Slots: " + inactiveReduceSlots
156 | + " (launched but no hearbeat yet)",
157 | " Needed Map Slots: " + neededMapSlots,
158 | " Needed Reduce Slots: " + neededReduceSlots,
159 | " Unhealthy Trackers: " + unhealthyTrackers)));
160 |
161 | if (scheduler.stateFile != null) {
162 | // Update state file
163 | synchronized (this) {
164 | Set hosts = new HashSet();
165 | for (MesosTracker tracker : scheduler.mesosTrackers.values()) {
166 | hosts.add(tracker.host.getHostName());
167 | }
168 | try {
169 | File tmp = new File(scheduler.stateFile.getAbsoluteFile() + ".tmp");
170 | FileWriter fstream = new FileWriter(tmp);
171 | fstream.write(join("\n", Arrays.asList(
172 | "time=" + System.currentTimeMillis(),
173 | "pendingMaps=" + pendingMaps,
174 | "pendingReduces=" + pendingReduces,
175 | "runningMaps=" + runningMaps,
176 | "runningReduces=" + runningReduces,
177 | "idleMapSlots=" + idleMapSlots,
178 | "idleReduceSlots=" + idleReduceSlots,
179 | "inactiveMapSlots=" + inactiveMapSlots,
180 | "inactiveReduceSlots=" + inactiveReduceSlots,
181 | "neededMapSlots=" + neededMapSlots,
182 | "neededReduceSlots=" + neededReduceSlots,
183 | "unhealthyTrackers=" + unhealthyTrackers,
184 | "hosts=" + join(",", hosts),
185 | "")));
186 | fstream.close();
187 | tmp.renameTo(scheduler.stateFile);
188 | } catch (Exception e) {
189 | LOG.error("Can't write state file: " + e.getMessage());
190 | }
191 | }
192 | }
193 | }
194 |
195 | // This method computes the number of slots to launch for this offer, and
196 | // returns true if the offer is sufficient.
197 | // Must be overridden.
198 | public boolean computeSlots() {
199 | return false;
200 | }
201 |
202 | public void resourceOffers(SchedulerDriver schedulerDriver,
203 | List offers) {
204 | final HttpHost jobTrackerAddress =
205 | new HttpHost(scheduler.jobTracker.getHostname(), scheduler.jobTracker.getTrackerPort());
206 |
207 | final Collection taskTrackers = scheduler.jobTracker.taskTrackers();
208 |
209 | final List jobsInProgress = new ArrayList();
210 | for (JobStatus status : scheduler.jobTracker.jobsToComplete()) {
211 | jobsInProgress.add(scheduler.jobTracker.getJob(status.getJobID()));
212 | }
213 |
214 | synchronized (this) {
215 | computeNeededSlots(jobsInProgress, taskTrackers);
216 |
217 | // Launch TaskTrackers to satisfy the slot requirements.
218 | for (Offer offer : offers) {
219 | if (neededMapSlots <= 0 && neededReduceSlots <= 0) {
220 | schedulerDriver.declineOffer(offer.getId());
221 | continue;
222 | }
223 |
224 | // Ensure these values aren't < 0.
225 | neededMapSlots = Math.max(0, neededMapSlots);
226 | neededReduceSlots = Math.max(0, neededReduceSlots);
227 |
228 | cpus = -1.0;
229 | mem = -1.0;
230 | disk = -1.0;
231 | Set ports = new HashSet();
232 | String cpuRole = new String("*");
233 | String memRole = cpuRole;
234 | String diskRole = cpuRole;
235 | String portsRole = cpuRole;
236 |
237 | // Pull out the cpus, memory, disk, and 2 ports from the offer.
238 | for (Resource resource : offer.getResourcesList()) {
239 | if (resource.getName().equals("cpus")
240 | && resource.getType() == Value.Type.SCALAR) {
241 | cpus = resource.getScalar().getValue();
242 | cpuRole = resource.getRole();
243 | } else if (resource.getName().equals("mem")
244 | && resource.getType() == Value.Type.SCALAR) {
245 | mem = resource.getScalar().getValue();
246 | memRole = resource.getRole();
247 | } else if (resource.getName().equals("disk")
248 | && resource.getType() == Value.Type.SCALAR) {
249 | disk = resource.getScalar().getValue();
250 | diskRole = resource.getRole();
251 | } else if (resource.getName().equals("ports")
252 | && resource.getType() == Value.Type.RANGES) {
253 | portsRole = resource.getRole();
254 | for (Value.Range range : resource.getRanges().getRangeList()) {
255 | Integer begin = (int) range.getBegin();
256 | Integer end = (int) range.getEnd();
257 | if (end < begin) {
258 | LOG.warn("Ignoring invalid port range: begin=" + begin + " end=" + end);
259 | continue;
260 | }
261 | while (begin <= end && ports.size() < 2) {
262 | int port = begin + (int)(Math.random() * ((end - begin) + 1));
263 | ports.add(port);
264 | begin += 1;
265 | }
266 | }
267 | }
268 | }
269 |
270 | // Verify the resource roles are what we need
271 | if (scheduler.conf.getBoolean("mapred.mesos.role.strict", false)) {
272 | String expectedRole = scheduler.conf.get("mapred.mesos.role", "*");
273 | if (!cpuRole.equals(expectedRole) ||
274 | !memRole.equals(expectedRole) ||
275 | !diskRole.equals(expectedRole) ||
276 | !portsRole.equals(expectedRole)) {
277 | LOG.info("Declining offer with invalid role " + expectedRole);
278 |
279 | schedulerDriver.declineOffer(offer.getId());
280 | continue;
281 | }
282 | }
283 |
284 | final boolean sufficient = computeSlots();
285 |
286 | double taskCpus = (mapSlots + reduceSlots) * slotCpus + containerCpus;
287 | double taskMem = (mapSlots + reduceSlots) * slotMem + containerMem;
288 | double taskDisk = (mapSlots + reduceSlots) * slotDisk + containerDisk;
289 |
290 | if (!sufficient || ports.size() < 2) {
291 | LOG.info(join("\n", Arrays.asList(
292 | "Declining offer with insufficient resources for a TaskTracker: ",
293 | " cpus: offered " + cpus + " needed at least " + taskCpus,
294 | " mem : offered " + mem + " needed at least " + taskMem,
295 | " disk: offered " + disk + " needed at least " + taskDisk,
296 | " ports: " + (ports.size() < 2
297 | ? " less than 2 offered"
298 | : " at least 2 (sufficient)"))));
299 |
300 | schedulerDriver.declineOffer(offer.getId());
301 | continue;
302 | }
303 |
304 | Iterator portIter = ports.iterator();
305 | HttpHost httpAddress = new HttpHost(offer.getHostname(), portIter.next());
306 | HttpHost reportAddress = new HttpHost(offer.getHostname(), portIter.next());
307 |
308 | // Check that this tracker is not already launched. This problem was
309 | // observed on a few occasions, but not reliably. The main symptom was
310 | // that entries in `mesosTrackers` were being lost, and task trackers
311 | // would be 'lost' mysteriously (probably because the ports were in
312 | // use). This problem has since gone away with a rewrite of the port
313 | // selection code, but the check + logging is left here.
314 | // TODO(brenden): Diagnose this to determine root cause.
315 | if (scheduler.mesosTrackers.containsKey(httpAddress)) {
316 | LOG.info(join("\n", Arrays.asList(
317 | "Declining offer because host/port combination is in use: ",
318 | " cpus: offered " + cpus + " needed " + taskCpus,
319 | " mem : offered " + mem + " needed " + taskMem,
320 | " disk: offered " + disk + " needed " + taskDisk,
321 | " ports: " + ports)));
322 |
323 | schedulerDriver.declineOffer(offer.getId());
324 | continue;
325 | }
326 |
327 | TaskID taskId = TaskID.newBuilder()
328 | .setValue("Task_Tracker_" + scheduler.launchedTrackers++).build();
329 |
330 | LOG.info("Launching task " + taskId.getValue() + " on "
331 | + httpAddress.toString() + " with mapSlots=" + mapSlots + " reduceSlots=" + reduceSlots);
332 |
333 | List defaultJvmOpts = Arrays.asList(
334 | "-XX:+UseConcMarkSweepGC",
335 | "-XX:+CMSParallelRemarkEnabled",
336 | "-XX:+CMSClassUnloadingEnabled",
337 | "-XX:+UseParNewGC",
338 | "-XX:TargetSurvivorRatio=80",
339 | "-XX:+UseTLAB",
340 | "-XX:ParallelGCThreads=2",
341 | "-XX:+AggressiveOpts",
342 | "-XX:+UseCompressedOops",
343 | "-XX:+UseFastEmptyMethods",
344 | "-XX:+UseFastAccessorMethods",
345 | "-Xss512k",
346 | "-XX:+AlwaysPreTouch",
347 | "-XX:CMSInitiatingOccupancyFraction=80"
348 | );
349 |
350 | String jvmOpts = scheduler.conf.get("mapred.mesos.executor.jvm.opts");
351 | if (jvmOpts == null) {
352 | jvmOpts = StringUtils.join(" ", defaultJvmOpts);
353 | }
354 |
355 | // Set up the environment for running the TaskTracker.
356 | Protos.Environment.Builder envBuilder = Protos.Environment
357 | .newBuilder()
358 | .addVariables(
359 | Protos.Environment.Variable.newBuilder()
360 | .setName("HADOOP_OPTS")
361 | .setValue(
362 | jvmOpts +
363 | " -Xmx" + tasktrackerJVMHeap + "m" +
364 | " -XX:NewSize=" + tasktrackerJVMHeap / 3 + "m -XX:MaxNewSize=" + (int)Math.floor
365 | (tasktrackerJVMHeap * 0.6) + "m"
366 | ));
367 |
368 | // Set java specific environment, appropriately.
369 | Map env = System.getenv();
370 | if (env.containsKey("JAVA_HOME")) {
371 | envBuilder.addVariables(Protos.Environment.Variable.newBuilder()
372 | .setName("JAVA_HOME")
373 | .setValue(env.get("JAVA_HOME")));
374 | }
375 |
376 | if (env.containsKey("JAVA_LIBRARY_PATH")) {
377 | envBuilder.addVariables(Protos.Environment.Variable.newBuilder()
378 | .setName("JAVA_LIBRARY_PATH")
379 | .setValue(env.get("JAVA_LIBRARY_PATH")));
380 | }
381 |
382 | // Command info differs when performing a local run.
383 | String master = scheduler.conf.get("mapred.mesos.master");
384 |
385 | if (master == null) {
386 | throw new RuntimeException(
387 | "Expecting configuration property 'mapred.mesos.master'");
388 | } else if (master == "local") {
389 | throw new RuntimeException(
390 | "Can not use 'local' for 'mapred.mesos.executor'");
391 | }
392 |
393 | String uri = scheduler.conf.get("mapred.mesos.executor.uri");
394 | String directory = scheduler.conf.get("mapred.mesos.executor.directory");
395 | boolean isUriSet = uri != null && !uri.equals("");
396 | boolean isDirectorySet = directory != null && !directory.equals("");
397 |
398 | if (!isUriSet && !isDirectorySet) {
399 | throw new RuntimeException(
400 | "Expecting configuration property 'mapred.mesos.executor'");
401 | } else if (isUriSet && isDirectorySet) {
402 | throw new RuntimeException(
403 | "Conflicting properties 'mapred.mesos.executor.uri' and 'mapred.mesos.executor.directory', only one can be set");
404 | } else if (!isDirectorySet) {
405 | LOG.info("URI: " + uri + ", name: " + new File(uri).getName());
406 |
407 | directory = new File(uri).getName().split("\\.")[0] + "*";
408 | } else if (!isUriSet) {
409 | LOG.info("mapred.mesos.executor.uri is not set, relying on configured 'mapred.mesos.executor.directory' for working Hadoop distribution");
410 | }
411 |
412 | String command = scheduler.conf.get("mapred.mesos.executor.command");
413 | if (command == null || command.equals("")) {
414 | command = "env ; ./bin/hadoop org.apache.hadoop.mapred.MesosExecutor";
415 | }
416 |
417 | CommandInfo.Builder commandInfo = CommandInfo.newBuilder();
418 | commandInfo
419 | .setEnvironment(envBuilder)
420 | .setValue(String.format("cd %s && %s", directory, command));
421 | if (uri != null) {
422 | commandInfo.addUris(CommandInfo.URI.newBuilder().setValue(uri));
423 | }
424 |
425 | // Populate old-style ContainerInfo if needed
426 | String containerImage = scheduler.conf.get("mapred.mesos.container.image");
427 | if (containerImage != null && !containerImage.equals("")) {
428 | commandInfo.setContainer(org.apache.mesos.hadoop.Utils.buildContainerInfo(scheduler.conf));
429 | }
430 |
431 | // Create a configuration from the current configuration and
432 | // override properties as appropriate for the TaskTracker.
433 | Configuration overrides = new Configuration(scheduler.conf);
434 |
435 | overrides.set("mapred.task.tracker.http.address",
436 | httpAddress.getHostName() + ':' + httpAddress.getPort());
437 |
438 | overrides.set("mapred.task.tracker.report.address",
439 | reportAddress.getHostName() + ':' + reportAddress.getPort());
440 |
441 | overrides.setLong("mapred.tasktracker.map.tasks.maximum", mapSlots);
442 | overrides.setLong("mapred.tasktracker.reduce.tasks.maximum", reduceSlots);
443 |
444 | // Build up the executor info
445 | ExecutorInfo.Builder executorBuilder = ExecutorInfo
446 | .newBuilder()
447 | .setExecutorId(ExecutorID.newBuilder().setValue(
448 | "executor_" + taskId.getValue()))
449 | .setName("Hadoop TaskTracker")
450 | .setSource(taskId.getValue())
451 | .addResources(
452 | Resource
453 | .newBuilder()
454 | .setName("cpus")
455 | .setType(Value.Type.SCALAR)
456 | .setRole(cpuRole)
457 | .setScalar(Value.Scalar.newBuilder().setValue(containerCpus)))
458 | .addResources(
459 | Resource
460 | .newBuilder()
461 | .setName("mem")
462 | .setType(Value.Type.SCALAR)
463 | .setRole(memRole)
464 | .setScalar(Value.Scalar.newBuilder().setValue(containerMem)))
465 | .addResources(
466 | Resource
467 | .newBuilder()
468 | .setName("disk")
469 | .setType(Value.Type.SCALAR)
470 | .setRole(diskRole)
471 | .setScalar(Value.Scalar.newBuilder().setValue(containerDisk)))
472 | .setCommand(commandInfo.build());
473 |
474 | // Add the docker container info if an image is specified
475 | String dockerImage = scheduler.conf.get("mapred.mesos.docker.image");
476 | if (dockerImage != null && !dockerImage.equals("")) {
477 | executorBuilder.setContainer(org.apache.mesos.hadoop.Utils.buildDockerContainerInfo(scheduler.conf));
478 | }
479 |
480 | ByteString taskData;
481 |
482 | try {
483 | taskData = org.apache.mesos.hadoop.Utils.confToBytes(overrides);
484 | } catch (IOException e) {
485 | LOG.error("Caught exception serializing configuration");
486 |
487 | // Skip this offer completely
488 | schedulerDriver.declineOffer(offer.getId());
489 | continue;
490 | }
491 |
492 | // Create the TaskTracker TaskInfo
493 | TaskInfo trackerTaskInfo = TaskInfo
494 | .newBuilder()
495 | .setName(taskId.getValue())
496 | .setTaskId(taskId)
497 | .setSlaveId(offer.getSlaveId())
498 | .addResources(
499 | Resource
500 | .newBuilder()
501 | .setName("ports")
502 | .setType(Value.Type.RANGES)
503 | .setRole(portsRole)
504 | .setRanges(
505 | Value.Ranges
506 | .newBuilder()
507 | .addRange(Value.Range.newBuilder()
508 | .setBegin(httpAddress.getPort())
509 | .setEnd(httpAddress.getPort()))
510 | .addRange(Value.Range.newBuilder()
511 | .setBegin(reportAddress.getPort())
512 | .setEnd(reportAddress.getPort()))))
513 | .addResources(
514 | Resource
515 | .newBuilder()
516 | .setName("cpus")
517 | .setType(Value.Type.SCALAR)
518 | .setRole(cpuRole)
519 | .setScalar(Value.Scalar.newBuilder().setValue(taskCpus - containerCpus)))
520 | .addResources(
521 | Resource
522 | .newBuilder()
523 | .setName("mem")
524 | .setType(Value.Type.SCALAR)
525 | .setRole(memRole)
526 | .setScalar(Value.Scalar.newBuilder().setValue(taskMem - containerCpus)))
527 | .setData(taskData)
528 | .setExecutor(executorBuilder.build())
529 | .build();
530 |
531 | // Add this tracker to Mesos tasks.
532 | scheduler.mesosTrackers.put(httpAddress, new MesosTracker(httpAddress, taskId,
533 | mapSlots, reduceSlots, scheduler));
534 |
535 | // Launch the task
536 | schedulerDriver.launchTasks(Arrays.asList(offer.getId()), Arrays.asList(trackerTaskInfo));
537 |
538 | neededMapSlots -= mapSlots;
539 | neededReduceSlots -= reduceSlots;
540 | }
541 |
542 | if (neededMapSlots <= 0 && neededReduceSlots <= 0) {
543 | LOG.info("Satisfied map and reduce slots needed.");
544 | } else {
545 | LOG.info("Unable to fully satisfy needed map/reduce slots: "
546 | + (neededMapSlots > 0 ? neededMapSlots + " map slots " : "")
547 | + (neededReduceSlots > 0 ? neededReduceSlots + " reduce slots " : "")
548 | + "remaining");
549 | }
550 | }
551 | }
552 | }
553 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/ResourcePolicyFixed.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | import org.apache.commons.logging.Log;
4 | import org.apache.commons.logging.LogFactory;
5 |
6 | public class ResourcePolicyFixed extends ResourcePolicy {
7 |
8 | public static final Log LOG = LogFactory.getLog(ResourcePolicyFixed.class);
9 |
10 | public ResourcePolicyFixed(MesosScheduler scheduler) {
11 | super(scheduler);
12 | }
13 |
14 | // This method computes the number of slots to launch for this offer, and
15 | // returns true if the offer is sufficient.
16 | @Override
17 | public boolean computeSlots() {
18 | mapSlots = mapSlotsMax;
19 | reduceSlots = reduceSlotsMax;
20 |
21 | slots = Integer.MAX_VALUE;
22 | slots = (int) Math.min(slots, (cpus - containerCpus) / slotCpus);
23 | slots = (int) Math.min(slots, (mem - containerMem) / slotMem);
24 | slots = (int) Math.min(slots, (disk - containerDisk) / slotDisk);
25 |
26 | // Is this offer too small for even the minimum slots?
27 | if (slots < mapSlots + reduceSlots || slots < 1) {
28 | return false;
29 | }
30 | return true;
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/hadoop/mapred/ResourcePolicyVariable.java:
--------------------------------------------------------------------------------
1 | package org.apache.hadoop.mapred;
2 |
3 | public class ResourcePolicyVariable extends ResourcePolicy {
4 | public ResourcePolicyVariable(MesosScheduler scheduler) {
5 | super(scheduler);
6 | }
7 |
8 | // This method computes the number of slots to launch for this offer, and
9 | // returns true if the offer is sufficient.
10 | @Override
11 | public boolean computeSlots() {
12 | // What's the minimum number of map and reduce slots we should try to
13 | // launch?
14 | mapSlots = 0;
15 | reduceSlots = 0;
16 |
17 | // Determine how many slots we can allocate.
18 | int slots = mapSlotsMax + reduceSlotsMax;
19 | slots = (int) Math.min(slots, (cpus - containerCpus) / slotCpus);
20 | slots = (int) Math.min(slots, (mem - containerMem) / slotMem);
21 | slots = (int) Math.min(slots, (disk - containerDisk) / slotDisk);
22 |
23 | // Is this offer too small for even the minimum slots?
24 | if (slots < 1) {
25 | return false;
26 | }
27 |
28 | // Is the number of slots we need sufficiently small? If so, we can
29 | // allocate exactly the number we need.
30 | if (slots >= neededMapSlots + neededReduceSlots && neededMapSlots <
31 | mapSlotsMax && neededReduceSlots < reduceSlotsMax) {
32 | mapSlots = neededMapSlots;
33 | reduceSlots = neededReduceSlots;
34 | } else {
35 | // Allocate slots fairly for this resource offer.
36 | double mapFactor = (double) neededMapSlots / (neededMapSlots + neededReduceSlots);
37 | double reduceFactor = (double) neededReduceSlots / (neededMapSlots + neededReduceSlots);
38 | // To avoid map/reduce slot starvation, don't allow more than 50%
39 | // spread between map/reduce slots when we need both mappers and
40 | // reducers.
41 | if (neededMapSlots > 0 && neededReduceSlots > 0) {
42 | if (mapFactor < 0.25) {
43 | mapFactor = 0.25;
44 | } else if (mapFactor > 0.75) {
45 | mapFactor = 0.75;
46 | }
47 | if (reduceFactor < 0.25) {
48 | reduceFactor = 0.25;
49 | } else if (reduceFactor > 0.75) {
50 | reduceFactor = 0.75;
51 | }
52 | }
53 | mapSlots = Math.min(Math.min((long)Math.max(Math.round(mapFactor * slots), 1), mapSlotsMax), neededMapSlots);
54 |
55 | // The remaining slots are allocated for reduces.
56 | slots -= mapSlots;
57 | reduceSlots = Math.min(Math.min(slots, reduceSlotsMax), neededReduceSlots);
58 | }
59 | return true;
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/mesos/hadoop/Metrics.java:
--------------------------------------------------------------------------------
1 | package org.apache.mesos.hadoop;
2 |
3 | import com.codahale.metrics.CsvReporter;
4 | import com.codahale.metrics.Meter;
5 | import com.codahale.metrics.MetricFilter;
6 | import com.codahale.metrics.MetricRegistry;
7 | import com.codahale.metrics.cassandra.Cassandra;
8 | import com.codahale.metrics.cassandra.CassandraReporter;
9 | import com.codahale.metrics.graphite.Graphite;
10 | import com.codahale.metrics.graphite.GraphiteReporter;
11 | import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
12 | import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
13 | import com.codahale.metrics.jvm.ThreadStatesGaugeSet;
14 | import org.apache.hadoop.conf.Configuration;
15 | import org.apache.hadoop.mapred.JobID;
16 | import org.apache.hadoop.mapred.JobStatus;
17 | import org.apache.mesos.Protos.TaskState;
18 |
19 | import java.io.File;
20 | import java.net.InetSocketAddress;
21 | import java.util.Arrays;
22 | import java.util.Map;
23 | import java.util.concurrent.ConcurrentHashMap;
24 | import java.util.concurrent.TimeUnit;
25 |
26 | public class Metrics {
27 | public MetricRegistry registry;
28 | public Meter killMeter, flakyTrackerKilledMeter, launchTimeout, periodicGC;
29 | public Map jobStateMeter =
30 | new ConcurrentHashMap();
31 | public Map taskStateMeter =
32 | new ConcurrentHashMap();
33 | public com.codahale.metrics.Timer jobTimer, trackerTimer;
34 | public Map jobTimerContexts =
35 | new ConcurrentHashMap();
36 |
37 | public Metrics(Configuration conf) {
38 | registry = new MetricRegistry();
39 |
40 | killMeter = registry.meter(MetricRegistry.name(Metrics.class, "killMeter"));
41 | flakyTrackerKilledMeter = registry.meter(MetricRegistry.name(Metrics.class, "flakyTrackerKilledMeter"));
42 | launchTimeout = registry.meter(MetricRegistry.name(Metrics.class, "launchTimeout"));
43 | periodicGC = registry.meter(MetricRegistry.name(Metrics.class, "periodicGC"));
44 | jobTimer = registry.timer(MetricRegistry.name(Metrics.class, "jobTimes"));
45 | trackerTimer = registry.timer(MetricRegistry.name(Metrics.class, "trackerTimes"));
46 |
47 | for (int i = 1; i <= 5; ++i) {
48 | jobStateMeter.put(i, registry.meter(MetricRegistry.name(Metrics.class, "jobState", JobStatus.getJobRunState(i))));
49 | }
50 |
51 | for (TaskState state : TaskState.values()) {
52 | taskStateMeter.put(state, registry.meter(MetricRegistry.name(Metrics.class, "taskState", state.name())));
53 | }
54 |
55 | registry.register(MetricRegistry.name(ThreadStatesGaugeSet.class), new ThreadStatesGaugeSet());
56 | registry.register(MetricRegistry.name(GarbageCollectorMetricSet.class), new GarbageCollectorMetricSet());
57 | registry.register(MetricRegistry.name(MemoryUsageGaugeSet.class), new MemoryUsageGaugeSet());
58 |
59 | final boolean csvEnabled = conf.getBoolean("mapred.mesos.metrics.csv.enabled", false);
60 | if (csvEnabled) {
61 | final String path = conf.get("mapred.mesos.metrics.csv.path");
62 | final int interval = conf.getInt("mapred.mesos.metrics.csv.interval", 60);
63 |
64 | CsvReporter csvReporter = CsvReporter.forRegistry(registry)
65 | .convertRatesTo(TimeUnit.SECONDS)
66 | .convertDurationsTo(TimeUnit.MILLISECONDS)
67 | .filter(MetricFilter.ALL)
68 | .build(new File(path));
69 | csvReporter.start(interval, TimeUnit.SECONDS);
70 | }
71 |
72 | final boolean graphiteEnabled = conf.getBoolean("mapred.mesos.metrics.graphite.enabled", false);
73 | if (graphiteEnabled) {
74 | final String host = conf.get("mapred.mesos.metrics.graphite.host");
75 | final int port = conf.getInt("mapred.mesos.metrics.graphite.port", 2003);
76 | final String prefix = conf.get("mapred.mesos.metrics.graphite.prefix");
77 | final int interval = conf.getInt("mapred.mesos.metrics.graphite.interval", 60);
78 |
79 | Graphite graphite = new Graphite(new InetSocketAddress(host, port));
80 | GraphiteReporter graphiteReporter = GraphiteReporter.forRegistry(registry)
81 | .prefixedWith(prefix)
82 | .convertRatesTo(TimeUnit.SECONDS)
83 | .convertDurationsTo(TimeUnit.MILLISECONDS)
84 | .filter(MetricFilter.ALL)
85 | .build(graphite);
86 | graphiteReporter.start(interval, TimeUnit.SECONDS);
87 | }
88 |
89 | final boolean cassandraEnabled = conf.getBoolean("mapred.mesos.metrics.cassandra.enabled", false);
90 | if (cassandraEnabled) {
91 | final String hosts = conf.get("mapred.mesos.metrics.cassandra.hosts");
92 | final int port = conf.getInt("mapred.mesos.metrics.cassandra.port", 9042);
93 | final String prefix = conf.get("mapred.mesos.metrics.cassandra.prefix");
94 | final int interval = conf.getInt("mapred.mesos.metrics.cassandra.interval", 60);
95 | final int ttl = conf.getInt("mapred.mesos.metrics.cassandra.ttl", 864000);
96 | final String keyspace = conf.get("mapred.mesos.metrics.cassandra.keyspace");
97 | final String table = conf.get("mapred.mesos.metrics.cassandra.table");
98 | final String consistency = conf.get("mapred.mesos.metrics.cassandra.consistency");
99 |
100 | Cassandra cassandra = new Cassandra(
101 | Arrays.asList(hosts.split(",")),
102 | keyspace,
103 | table,
104 | ttl,
105 | port,
106 | consistency);
107 |
108 | CassandraReporter cassandraReporter = CassandraReporter.forRegistry(registry)
109 | .prefixedWith(prefix)
110 | .convertRatesTo(TimeUnit.SECONDS)
111 | .convertDurationsTo(TimeUnit.MILLISECONDS)
112 | .filter(MetricFilter.ALL)
113 | .build(cassandra);
114 | cassandraReporter.start(interval, TimeUnit.SECONDS);
115 | }
116 | }
117 | }
118 |
--------------------------------------------------------------------------------
/src/main/java/org/apache/mesos/hadoop/Utils.java:
--------------------------------------------------------------------------------
1 |
2 | package org.apache.mesos.hadoop;
3 |
4 | import javax.xml.transform.*;
5 | import javax.xml.transform.stream.StreamResult;
6 | import javax.xml.transform.stream.StreamSource;
7 | import java.lang.IllegalArgumentException;
8 | import java.io.*;
9 |
10 | import com.google.protobuf.ByteString;
11 | import org.apache.hadoop.conf.Configuration;
12 |
13 | import org.apache.mesos.Protos.CommandInfo;
14 | import org.apache.mesos.Protos.ContainerInfo;
15 | import org.apache.mesos.Protos.ContainerInfo.DockerInfo;
16 | import org.apache.mesos.Protos.Parameter;
17 | import org.apache.mesos.Protos.Parameters;
18 | import org.apache.mesos.Protos.Volume;
19 |
20 | public class Utils {
21 |
22 | public static String formatXml(String source) throws TransformerException {
23 | Source xmlInput = new StreamSource(new StringReader(source));
24 | StringWriter stringWriter = new StringWriter();
25 | StreamResult xmlOutput = new StreamResult(stringWriter);
26 |
27 | TransformerFactory transformerFactory = TransformerFactory.newInstance();
28 | transformerFactory.setAttribute("indent-number", 2);
29 |
30 | Transformer transformer = transformerFactory.newTransformer();
31 | transformer.setOutputProperty(OutputKeys.INDENT, "yes");
32 | transformer.transform(xmlInput, xmlOutput);
33 |
34 | return xmlOutput.getWriter().toString();
35 | }
36 |
37 | public static ByteString confToBytes(Configuration conf) throws IOException {
38 | ByteArrayOutputStream baos = new ByteArrayOutputStream();
39 | conf.write(new DataOutputStream(baos));
40 | baos.flush();
41 |
42 | byte[] bytes = baos.toByteArray();
43 | return ByteString.copyFrom(bytes);
44 | }
45 |
46 | public static CommandInfo.ContainerInfo buildContainerInfo(Configuration conf) {
47 | String containerImage = conf.get("mapred.mesos.container.image");
48 | String[] containerOptions = conf.getStrings("mapred.mesos.container.options");
49 |
50 | CommandInfo.ContainerInfo.Builder containerInfo =
51 | CommandInfo.ContainerInfo.newBuilder();
52 |
53 | if (containerImage != null) {
54 | containerInfo.setImage(containerImage);
55 | }
56 |
57 | if (containerOptions != null) {
58 | for (int i = 0; i < containerOptions.length; i++) {
59 | containerInfo.addOptions(containerOptions[i]);
60 | }
61 | }
62 |
63 | return containerInfo.build();
64 | }
65 |
66 | public static ContainerInfo buildDockerContainerInfo(Configuration conf) {
67 | ContainerInfo.Builder containerInfoBuilder = ContainerInfo.newBuilder();
68 | DockerInfo.Builder dockerInfoBuilder = DockerInfo.newBuilder();
69 |
70 | dockerInfoBuilder.setImage(conf.get("mapred.mesos.docker.image"));
71 |
72 | switch (conf.getInt("mapred.mesos.docker.network", 1)) {
73 | case 1:
74 | dockerInfoBuilder.setNetwork(DockerInfo.Network.HOST);
75 | case 2:
76 | dockerInfoBuilder.setNetwork(DockerInfo.Network.BRIDGE);
77 | case 3:
78 | dockerInfoBuilder.setNetwork(DockerInfo.Network.NONE);
79 | default:
80 | dockerInfoBuilder.setNetwork(DockerInfo.Network.HOST);
81 | }
82 |
83 | dockerInfoBuilder.setPrivileged(conf.getBoolean("mapred.mesos.docker.privileged", false));
84 | dockerInfoBuilder.setForcePullImage(conf.getBoolean("mapred.mesos.docker.force_pull_image", false));
85 |
86 | // Parse out any additional docker CLI params
87 | String[] params = conf.getStrings("mapred.mesos.docker.parameters");
88 | if (params != null && params.length > 0) {
89 | // Make sure we have an even number of parameters
90 | if ((params.length % 2) != 0) {
91 | throw new IllegalArgumentException("The number of parameters should be even, k/v pairs");
92 | }
93 |
94 | Parameter.Builder paramBuilder = null;
95 | for (int i = 0; i < params.length; i++) {
96 | if (paramBuilder == null) {
97 | paramBuilder = Parameter.newBuilder();
98 | paramBuilder.setKey(params[i]);
99 | } else {
100 | paramBuilder.setValue(params[i]);
101 | dockerInfoBuilder.addParameters(paramBuilder.build());
102 | paramBuilder = null;
103 | }
104 | }
105 | }
106 |
107 | // Parse out any volumes that have been defined
108 | String[] volumes = conf.getStrings("mapred.mesos.docker.volumes");
109 | if (volumes != null && volumes.length > 0) {
110 | for (int i = 0; i < volumes.length; i++) {
111 | String[] parts = volumes[i].split(":");
112 |
113 | if (parts.length <= 1 || parts.length > 3) {
114 | throw new IllegalArgumentException("Invalid volume configuration (host_path:container_path:[rw|ro])");
115 | }
116 |
117 | Volume.Mode mode = Volume.Mode.RW;
118 | if (parts[parts.length - 1].equalsIgnoreCase("ro")) {
119 | mode = Volume.Mode.RO;
120 | }
121 |
122 | if (parts.length == 2) {
123 | containerInfoBuilder.addVolumes(
124 | Volume.newBuilder()
125 | .setContainerPath(parts[0])
126 | .setMode(mode)
127 | .build());
128 | } else {
129 | containerInfoBuilder.addVolumes(
130 | Volume.newBuilder()
131 | .setHostPath(parts[0])
132 | .setContainerPath(parts[1])
133 | .setMode(mode)
134 | .build());
135 | }
136 | }
137 | }
138 |
139 | containerInfoBuilder.setType(ContainerInfo.Type.DOCKER);
140 | containerInfoBuilder.setDocker(dockerInfoBuilder.build());
141 |
142 | return containerInfoBuilder.build();
143 | }
144 | }
145 |
--------------------------------------------------------------------------------