Skip to content

Commit

Permalink
Merge pull request #33 from duedil-ltd/feature/fluid-task-trackers
Browse files Browse the repository at this point in the history
Idle TaskTrackers
  • Loading branch information
tarnfeld committed Mar 29, 2015
2 parents 3d70fb4 + 8028724 commit 4fd5bf1
Show file tree
Hide file tree
Showing 9 changed files with 320 additions and 80 deletions.
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -2,3 +2,4 @@ target
.vimrc
.idea/
*.iml
*-pom.xml
11 changes: 5 additions & 6 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ Hadoop on Mesos

#### Overview ####

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

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

#### Build ####

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

```shell
mvn package
```

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

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

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

```shell
cp /path/to/hadoop-mesos-0.0.9.jar hadoop-2.5.0-cdh5.2.0/share/hadoop/common/lib/
cp /path/to/hadoop-mesos-0.1.0.jar hadoop-2.5.0-cdh5.2.0/share/hadoop/common/lib/
```

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

_Please email [email protected] with questions!_


----------
18 changes: 18 additions & 0 deletions configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,24 @@ default values.
</description>
</property>
<!-- TaskTracker Idle Slots Revocation -->
<property>
<name>mapred.mesos.tracker.idle.interval</name>
<value>5</value>
<description>
Internal (in seconds) to check for TaskTrackers that have idle
slots. Default is 5 seconds.
</description>
</property>
<property>
<name>mapred.mesos.tracker.idle.checks</name>
<value>5</value>
<description>
After this many successful idle checks (meaning all slots *are* idle) the
slots will be revoked from the TaskTracker.
</description>
</property>
<!-- Metrics -->
<property>
<name>mapred.mesos.metrics.enabled</name>
Expand Down
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,7 @@

<groupId>org.apache.mesos</groupId>
<artifactId>hadoop-mesos</artifactId>
<version>0.0.9</version>
<version>0.1.0</version>

<properties>
<encoding>UTF-8</encoding>
Expand Down
129 changes: 119 additions & 10 deletions src/main/java/org/apache/hadoop/mapred/MesosExecutor.java
Original file line number Diff line number Diff line change
Expand Up @@ -11,11 +11,22 @@

import java.io.*;

import java.lang.reflect.Field;
import java.lang.ReflectiveOperationException;

import java.util.concurrent.BlockingQueue;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;

public class MesosExecutor implements Executor {
public static final Log LOG = LogFactory.getLog(MesosExecutor.class);
private SlaveInfo slaveInfo;
private TaskTracker taskTracker;

protected final ScheduledExecutorService timerScheduler =
Executors.newScheduledThreadPool(1);

public static void main(String[] args) {
MesosExecutorDriver driver = new MesosExecutorDriver(new MesosExecutor());
System.exit(driver.run() == Status.DRIVER_STOPPED ? 0 : 1);
Expand All @@ -37,10 +48,8 @@ private JobConf configure(final TaskInfo task) {
conf.writeXml(writer);
writer.flush();
String xml = writer.getBuffer().toString();
String xmlFormatted =
org.apache.mesos.hadoop.Utils.formatXml(xml);
LOG.info("XML Configuration received:\n" +
xmlFormatted);
org.apache.mesos.hadoop.Utils.formatXml(xml));
} catch (Exception e) {
LOG.warn("Failed to output configuration as XML.", e);
}
Expand Down Expand Up @@ -123,14 +132,22 @@ public void run() {
}

@Override
public void killTask(ExecutorDriver driver, TaskID taskId) {
public void killTask(final ExecutorDriver driver, final TaskID taskId) {
LOG.info("Killing task : " + taskId.getValue());
try {
taskTracker.shutdown();
} catch (IOException e) {
LOG.error("Failed to shutdown TaskTracker", e);
} catch (InterruptedException e) {
LOG.error("Failed to shutdown TaskTracker", e);
if (taskTracker != null) {
LOG.info("Revoking task tracker map/reduce slots");
revokeSlots();

// Send the TASK_FINISHED status
new Thread("TaskFinishedUpdate") {
@Override
public void run() {
driver.sendStatusUpdate(TaskStatus.newBuilder()
.setTaskId(taskId)
.setState(TaskState.TASK_FINISHED)
.build());
}
}.start();
}
}

Expand Down Expand Up @@ -159,4 +176,96 @@ public void error(ExecutorDriver d, String message) {
public void shutdown(ExecutorDriver d) {
LOG.info("Executor asked to shutdown");
}

public void revokeSlots() {
if (taskTracker == null) {
LOG.error("Task tracker is not initialized");
return;
}

int maxMapSlots = 0;
int maxReduceSlots = 0;

// TODO(tarnfeld): Sanity check that it's safe for us to change the slots.
// Be sure there's nothing running and nothing in the launcher queue.

// If we expect to have no slots, let's go ahead and terminate the task launchers
if (maxMapSlots == 0) {
try {
Field launcherField = taskTracker.getClass().getDeclaredField("mapLauncher");
launcherField.setAccessible(true);

// Kill the current map task launcher
TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
launcher.notifySlots();
launcher.interrupt();
} catch (ReflectiveOperationException e) {
LOG.fatal("Failed updating map slots due to error with reflection", e);
}
}

if (maxReduceSlots == 0) {
try {
Field launcherField = taskTracker.getClass().getDeclaredField("reduceLauncher");
launcherField.setAccessible(true);

// Kill the current reduce task launcher
TaskTracker.TaskLauncher launcher = ((TaskTracker.TaskLauncher) launcherField.get(taskTracker));
launcher.notifySlots();
launcher.interrupt();
} catch (ReflectiveOperationException e) {
LOG.fatal("Failed updating reduce slots due to error with reflection", e);
}
}

// Configure the new slot counts on the task tracker
taskTracker.setMaxMapSlots(maxMapSlots);
taskTracker.setMaxReduceSlots(maxReduceSlots);

// If we have zero slots left, commit suicide when no jobs are running
if ((maxMapSlots + maxReduceSlots) == 0) {
scheduleSuicideTimer();
}
}

protected void scheduleSuicideTimer() {
timerScheduler.schedule(new Runnable() {
@Override
public void run() {
if (taskTracker == null) {
return;
}

LOG.info("Checking to see if TaskTracker is idle");

// If the task tracker is idle, all tasks have finished and task output
// has been cleaned up.
if (taskTracker.isIdle()) {
LOG.warn("TaskTracker is idle, terminating");

try {
taskTracker.shutdown();
} catch (IOException e) {
LOG.error("Failed to shutdown TaskTracker", e);
} catch (InterruptedException e) {
LOG.error("Failed to shutdown TaskTracker", e);
}
}
else {
try {
Field field = taskTracker.getClass().getDeclaredField("tasksToCleanup");
field.setAccessible(true);
BlockingQueue<TaskTrackerAction> tasksToCleanup = ((BlockingQueue<TaskTrackerAction>) field.get(taskTracker));
LOG.info("TaskTracker has " + taskTracker.tasks.size() +
" running tasks and " + tasksToCleanup +
" tasks to clean up.");
} catch (ReflectiveOperationException e) {
LOG.fatal("Failed to get task counts from TaskTracker", e);
}

scheduleSuicideTimer();
}
}
}, 1000, TimeUnit.MILLISECONDS);
}
}
21 changes: 19 additions & 2 deletions src/main/java/org/apache/hadoop/mapred/MesosScheduler.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,19 @@ public class MesosScheduler extends TaskScheduler implements Scheduler {
public static final double SLOT_CPUS_DEFAULT = 1; // 1 cores.
public static final int SLOT_DISK_DEFAULT = 1024; // 1 GB.
public static final int SLOT_JVM_HEAP_DEFAULT = 1024; // 1024MB.
public static final double TASKTRACKER_CPUS = 1.0; // 1 core.
public static final double TASKTRACKER_CPUS_DEFAULT = 1.0; // 1 core.
public static final int TASKTRACKER_MEM_DEFAULT = 1024; // 1 GB.
public static final int TASKTRACKER_DISK_DEFAULT = 1024; // 1 GB.
// The default behavior in Hadoop is to use 4 slots per TaskTracker:
public static final int MAP_SLOTS_DEFAULT = 2;
public static final int REDUCE_SLOTS_DEFAULT = 2;
// The amount of time to wait for task trackers to launch before
// giving up.
public static final long LAUNCH_TIMEOUT_MS = 300000; // 5 minutes
public static final long PERIODIC_MS = 300000; // 5 minutes
public static final long DEFAULT_IDLE_CHECK_INTERVAL = 5; // 5 seconds
// Destroy task trackers after being idle for N idle checks
public static final long DEFAULT_IDLE_REVOCATION_CHECKS = 5;
private SchedulerDriver driver;

protected TaskScheduler taskScheduler;
Expand Down Expand Up @@ -245,6 +249,19 @@ public List<Task> assignTasks(TaskTracker taskTracker)
LOG.info("Unknown/exited TaskTracker: " + tracker + ". ");
return null;
}

MesosTracker mesosTracker = mesosTrackers.get(tracker);

// Make sure we're not asked to assign tasks to any task trackers that have
// been stopped. This could happen while the task tracker has not been
// removed from the cluster e.g still in the heartbeat timeout period.
synchronized (this) {
if (mesosTracker.stopped) {
LOG.info("Asked to assign tasks to stopped tracker " + tracker + ".");
return null;
}
}

// Let the underlying task scheduler do the actual task scheduling.
List<Task> tasks = taskScheduler.assignTasks(taskTracker);

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

// Keep track of which TaskTracker contains which tasks.
for (Task task : tasks) {
mesosTrackers.get(tracker).jobs.add(task.getJobID());
mesosTracker.jobs.add(task.getJobID());
}

return tasks;
Expand Down
Loading

0 comments on commit 4fd5bf1

Please sign in to comment.