Skip to content

Commit 4fd5bf1

Browse files
committed
Merge pull request #33 from duedil-ltd/feature/fluid-task-trackers
Idle TaskTrackers
2 parents 3d70fb4 + 8028724 commit 4fd5bf1

File tree

9 files changed

+320
-80
lines changed

9 files changed

+320
-80
lines changed

.gitignore

+1
Original file line numberDiff line numberDiff line change
@@ -2,3 +2,4 @@ target
22
.vimrc
33
.idea/
44
*.iml
5+
*-pom.xml

README.md

+5-6
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@ Hadoop on Mesos
55

66
#### Overview ####
77

8-
To run _Hadoop on Mesos_ you need to add the `hadoop-mesos-0.0.9.jar`
8+
To run _Hadoop on Mesos_ you need to add the `hadoop-mesos-0.1.0.jar`
99
library to your Hadoop distribution (any distribution that uses protobuf > 2.5.0)
1010
and set some new configuration properties. Read on for details.
1111

@@ -23,13 +23,13 @@ install `libsnappy`. The [`snappy-java`][snappy-java] package also includes a b
2323

2424
#### Build ####
2525

26-
You can build `hadoop-mesos-0.0.9.jar` using Maven:
26+
You can build `hadoop-mesos-0.1.0.jar` using Maven:
2727

2828
```shell
2929
mvn package
3030
```
3131

32-
If successful, the JAR will be at `target/hadoop-mesos-0.0.9.jar`.
32+
If successful, the JAR will be at `target/hadoop-mesos-0.1.0.jar`.
3333

3434
> NOTE: If you want to build against a different version of Mesos than
3535
> the default you'll need to update `mesos-version` in `pom.xml`.
@@ -51,10 +51,10 @@ tar zxf hadoop-2.5.0-cdh5.2.0.tar.gz
5151

5252
> **Take note**, the extracted directory is `hadoop-2.5.0-cdh5.2.0`.
5353
54-
Now copy `hadoop-mesos-0.0.9.jar` into the `share/hadoop/common/lib` folder.
54+
Now copy `hadoop-mesos-0.1.0.jar` into the `share/hadoop/common/lib` folder.
5555

5656
```shell
57-
cp /path/to/hadoop-mesos-0.0.9.jar hadoop-2.5.0-cdh5.2.0/share/hadoop/common/lib/
57+
cp /path/to/hadoop-mesos-0.1.0.jar hadoop-2.5.0-cdh5.2.0/share/hadoop/common/lib/
5858
```
5959

6060
Since CDH5 includes both MRv1 and MRv2 (YARN) and is configured for YARN by
@@ -183,5 +183,4 @@ This feature can be especially useful if your hadoop jobs have software dependen
183183

184184
_Please email [email protected] with questions!_
185185

186-
187186
----------

configuration.md

+18
Original file line numberDiff line numberDiff line change
@@ -161,6 +161,24 @@ default values.
161161
</description>
162162
</property>
163163
164+
<!-- TaskTracker Idle Slots Revocation -->
165+
<property>
166+
<name>mapred.mesos.tracker.idle.interval</name>
167+
<value>5</value>
168+
<description>
169+
Internal (in seconds) to check for TaskTrackers that have idle
170+
slots. Default is 5 seconds.
171+
</description>
172+
</property>
173+
<property>
174+
<name>mapred.mesos.tracker.idle.checks</name>
175+
<value>5</value>
176+
<description>
177+
After this many successful idle checks (meaning all slots *are* idle) the
178+
slots will be revoked from the TaskTracker.
179+
</description>
180+
</property>
181+
164182
<!-- Metrics -->
165183
<property>
166184
<name>mapred.mesos.metrics.enabled</name>

pom.xml

+1-1
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44

55
<groupId>org.apache.mesos</groupId>
66
<artifactId>hadoop-mesos</artifactId>
7-
<version>0.0.9</version>
7+
<version>0.1.0</version>
88

99
<properties>
1010
<encoding>UTF-8</encoding>

src/main/java/org/apache/hadoop/mapred/MesosExecutor.java

+119-10
Original file line numberDiff line numberDiff line change
@@ -11,11 +11,22 @@
1111

1212
import java.io.*;
1313

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+
1422
public class MesosExecutor implements Executor {
1523
public static final Log LOG = LogFactory.getLog(MesosExecutor.class);
1624
private SlaveInfo slaveInfo;
1725
private TaskTracker taskTracker;
1826

27+
protected final ScheduledExecutorService timerScheduler =
28+
Executors.newScheduledThreadPool(1);
29+
1930
public static void main(String[] args) {
2031
MesosExecutorDriver driver = new MesosExecutorDriver(new MesosExecutor());
2132
System.exit(driver.run() == Status.DRIVER_STOPPED ? 0 : 1);
@@ -37,10 +48,8 @@ private JobConf configure(final TaskInfo task) {
3748
conf.writeXml(writer);
3849
writer.flush();
3950
String xml = writer.getBuffer().toString();
40-
String xmlFormatted =
41-
org.apache.mesos.hadoop.Utils.formatXml(xml);
4251
LOG.info("XML Configuration received:\n" +
43-
xmlFormatted);
52+
org.apache.mesos.hadoop.Utils.formatXml(xml));
4453
} catch (Exception e) {
4554
LOG.warn("Failed to output configuration as XML.", e);
4655
}
@@ -123,14 +132,22 @@ public void run() {
123132
}
124133

125134
@Override
126-
public void killTask(ExecutorDriver driver, TaskID taskId) {
135+
public void killTask(final ExecutorDriver driver, final TaskID taskId) {
127136
LOG.info("Killing task : " + taskId.getValue());
128-
try {
129-
taskTracker.shutdown();
130-
} catch (IOException e) {
131-
LOG.error("Failed to shutdown TaskTracker", e);
132-
} catch (InterruptedException e) {
133-
LOG.error("Failed to shutdown TaskTracker", e);
137+
if (taskTracker != null) {
138+
LOG.info("Revoking task tracker map/reduce slots");
139+
revokeSlots();
140+
141+
// Send the TASK_FINISHED status
142+
new Thread("TaskFinishedUpdate") {
143+
@Override
144+
public void run() {
145+
driver.sendStatusUpdate(TaskStatus.newBuilder()
146+
.setTaskId(taskId)
147+
.setState(TaskState.TASK_FINISHED)
148+
.build());
149+
}
150+
}.start();
134151
}
135152
}
136153

@@ -159,4 +176,96 @@ public void error(ExecutorDriver d, String message) {
159176
public void shutdown(ExecutorDriver d) {
160177
LOG.info("Executor asked to shutdown");
161178
}
179+
180+
public void revokeSlots() {
181+
if (taskTracker == null) {
182+
LOG.error("Task tracker is not initialized");
183+
return;
184+
}
185+
186+
int maxMapSlots = 0;
187+
int maxReduceSlots = 0;
188+
189+
// TODO(tarnfeld): Sanity check that it's safe for us to change the slots.
190+
// Be sure there's nothing running and nothing in the launcher queue.
191+
192+
// If we expect to have no slots, let's go ahead and terminate the task launchers
193+
if (maxMapSlots == 0) {
194+
try {
195+
Field launcherField = taskTracker.getClass().getDeclaredField("mapLauncher");
196+
launcherField.setAccessible(true);
197+
198+
// Kill the current map task launcher
199+
TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
200+
launcher.notifySlots();
201+
launcher.interrupt();
202+
} catch (ReflectiveOperationException e) {
203+
LOG.fatal("Failed updating map slots due to error with reflection", e);
204+
}
205+
}
206+
207+
if (maxReduceSlots == 0) {
208+
try {
209+
Field launcherField = taskTracker.getClass().getDeclaredField("reduceLauncher");
210+
launcherField.setAccessible(true);
211+
212+
// Kill the current reduce task launcher
213+
TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
214+
launcher.notifySlots();
215+
launcher.interrupt();
216+
} catch (ReflectiveOperationException e) {
217+
LOG.fatal("Failed updating reduce slots due to error with reflection", e);
218+
}
219+
}
220+
221+
// Configure the new slot counts on the task tracker
222+
taskTracker.setMaxMapSlots(maxMapSlots);
223+
taskTracker.setMaxReduceSlots(maxReduceSlots);
224+
225+
// If we have zero slots left, commit suicide when no jobs are running
226+
if ((maxMapSlots + maxReduceSlots) == 0) {
227+
scheduleSuicideTimer();
228+
}
229+
}
230+
231+
protected void scheduleSuicideTimer() {
232+
timerScheduler.schedule(new Runnable() {
233+
@Override
234+
public void run() {
235+
if (taskTracker == null) {
236+
return;
237+
}
238+
239+
LOG.info("Checking to see if TaskTracker is idle");
240+
241+
// If the task tracker is idle, all tasks have finished and task output
242+
// has been cleaned up.
243+
if (taskTracker.isIdle()) {
244+
LOG.warn("TaskTracker is idle, terminating");
245+
246+
try {
247+
taskTracker.shutdown();
248+
} catch (IOException e) {
249+
LOG.error("Failed to shutdown TaskTracker", e);
250+
} catch (InterruptedException e) {
251+
LOG.error("Failed to shutdown TaskTracker", e);
252+
}
253+
}
254+
else {
255+
try {
256+
Field field = taskTracker.getClass().getDeclaredField("tasksToCleanup");
257+
field.setAccessible(true);
258+
BlockingQueue<TaskTrackerAction> tasksToCleanup = ((BlockingQueue<TaskTrackerAction>) field.get(taskTracker));
259+
LOG.info("TaskTracker has " + taskTracker.tasks.size() +
260+
" running tasks and " + tasksToCleanup +
261+
" tasks to clean up.");
262+
} catch (ReflectiveOperationException e) {
263+
LOG.fatal("Failed to get task counts from TaskTracker", e);
264+
}
265+
266+
scheduleSuicideTimer();
267+
}
268+
}
269+
}, 1000, TimeUnit.MILLISECONDS);
270+
}
162271
}

src/main/java/org/apache/hadoop/mapred/MesosScheduler.java

+19-2
Original file line numberDiff line numberDiff line change
@@ -36,15 +36,19 @@ public class MesosScheduler extends TaskScheduler implements Scheduler {
3636
public static final double SLOT_CPUS_DEFAULT = 1; // 1 cores.
3737
public static final int SLOT_DISK_DEFAULT = 1024; // 1 GB.
3838
public static final int SLOT_JVM_HEAP_DEFAULT = 1024; // 1024MB.
39-
public static final double TASKTRACKER_CPUS = 1.0; // 1 core.
39+
public static final double TASKTRACKER_CPUS_DEFAULT = 1.0; // 1 core.
4040
public static final int TASKTRACKER_MEM_DEFAULT = 1024; // 1 GB.
41+
public static final int TASKTRACKER_DISK_DEFAULT = 1024; // 1 GB.
4142
// The default behavior in Hadoop is to use 4 slots per TaskTracker:
4243
public static final int MAP_SLOTS_DEFAULT = 2;
4344
public static final int REDUCE_SLOTS_DEFAULT = 2;
4445
// The amount of time to wait for task trackers to launch before
4546
// giving up.
4647
public static final long LAUNCH_TIMEOUT_MS = 300000; // 5 minutes
4748
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;
4852
private SchedulerDriver driver;
4953

5054
protected TaskScheduler taskScheduler;
@@ -245,6 +249,19 @@ public List<Task> assignTasks(TaskTracker taskTracker)
245249
LOG.info("Unknown/exited TaskTracker: " + tracker + ". ");
246250
return null;
247251
}
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+
248265
// Let the underlying task scheduler do the actual task scheduling.
249266
List<Task> tasks = taskScheduler.assignTasks(taskTracker);
250267

@@ -255,7 +272,7 @@ public List<Task> assignTasks(TaskTracker taskTracker)
255272

256273
// Keep track of which TaskTracker contains which tasks.
257274
for (Task task : tasks) {
258-
mesosTrackers.get(tracker).jobs.add(task.getJobID());
275+
mesosTracker.jobs.add(task.getJobID());
259276
}
260277

261278
return tasks;

0 commit comments

Comments
 (0)