You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by rs...@apache.org on 2017/11/03 09:38:04 UTC

[1/4] kafka git commit: KAFKA-6060; Add workload generation capabilities to Trogdor

Repository: kafka
Updated Branches:
  refs/heads/trunk e4208b1d5 -> 4fac83ba1


http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java
new file mode 100644
index 0000000..3d8323d
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state for a task which is being run by the agent.
+ */
+public class WorkerRunning extends WorkerState {
+    /**
+     * The time on the agent when the task was started.
+     */
+    private final long startedMs;
+
+    /**
+     * The task status.  The format will depend on the type of task that is
+     * being run.
+     */
+    private final String status;
+
+    @JsonCreator
+    public WorkerRunning(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs,
+            @JsonProperty("status") String status) {
+        super(spec);
+        this.startedMs = startedMs;
+        this.status = status;
+    }
+
+    @JsonProperty
+    @Override
+    public long startedMs() {
+        return startedMs;
+    }
+
+    @JsonProperty
+    @Override
+    public String status() {
+        return status;
+    }
+
+    @Override
+    public boolean running() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java
new file mode 100644
index 0000000..3a766ea
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * When we have just started a worker.
+ */
+public final class WorkerStarting extends WorkerState {
+    @JsonCreator
+    public WorkerStarting(@JsonProperty("spec") TaskSpec spec) {
+        super(spec);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java
new file mode 100644
index 0000000..6d7c687
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state which a worker is in on the Agent.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME,
+    include = JsonTypeInfo.As.PROPERTY,
+    property = "state")
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(value = WorkerReceiving.class, name = "RECEIVING"),
+    @JsonSubTypes.Type(value = WorkerStarting.class, name = "STARTING"),
+    @JsonSubTypes.Type(value = WorkerRunning.class, name = "RUNNING"),
+    @JsonSubTypes.Type(value = WorkerStopping.class, name = "STOPPING"),
+    @JsonSubTypes.Type(value = WorkerDone.class, name = "DONE")
+    })
+public abstract class WorkerState extends Message {
+    private final TaskSpec spec;
+
+    public WorkerState(TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+
+    public boolean stopping() {
+        return false;
+    }
+
+    public boolean done() {
+        return false;
+    }
+
+    public long startedMs() {
+        throw new KafkaException("invalid state");
+    }
+
+    public String status() {
+        throw new KafkaException("invalid state");
+    }
+
+    public boolean running() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java
new file mode 100644
index 0000000..fa2d546
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state for a worker which is being stopped on the agent.
+ */
+public class WorkerStopping extends WorkerState {
+    /**
+     * The time on the agent when the task was received.
+     */
+    private final long startedMs;
+
+    /**
+     * The task status.  The format will depend on the type of task that is
+     * being run.
+     */
+    private final String status;
+
+    @JsonCreator
+    public WorkerStopping(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs,
+            @JsonProperty("status") String status) {
+        super(spec);
+        this.startedMs = startedMs;
+        this.status = status;
+    }
+
+    @JsonProperty
+    @Override
+    public long startedMs() {
+        return startedMs;
+    }
+
+    @JsonProperty
+    @Override
+    public String status() {
+        return status;
+    }
+
+    @Override
+    public boolean stopping() {
+        return true;
+    }
+
+    @Override
+    public boolean running() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java
new file mode 100644
index 0000000..b5906c3
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.trogdor.common.Topology;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+
+public class NoOpTaskController implements TaskController {
+    private static final Logger log = LoggerFactory.getLogger(NoOpTaskController.class);
+
+    public NoOpTaskController() {
+    }
+
+    @Override
+    public Set<String> targetNodes(Topology topology) {
+        return Topology.Util.agentNodeNames(topology);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java
new file mode 100644
index 0000000..63e6023
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * The specification for a task that does nothing.
+ *
+ * This task type exists to test trogodor itself.
+ */
+public class NoOpTaskSpec extends TaskSpec {
+    @JsonCreator
+    public NoOpTaskSpec(@JsonProperty("startMs") long startMs,
+                         @JsonProperty("durationMs") long durationMs) {
+        super(startMs, durationMs);
+    }
+
+    @Override
+    public TaskController newController(String id) {
+        return new NoOpTaskController();
+    }
+
+    @Override
+    public TaskWorker newTaskWorker(String id) {
+        return new NoOpTaskWorker(id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java
new file mode 100644
index 0000000..dfa8084
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.trogdor.common.Platform;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+public class NoOpTaskWorker implements TaskWorker {
+    private static final Logger log = LoggerFactory.getLogger(NoOpTaskWorker.class);
+
+    private final String id;
+
+    public NoOpTaskWorker(String id) {
+        this.id = id;
+    }
+
+    @Override
+    public void start(Platform platform, AtomicReference<String> status,
+                      KafkaFutureImpl<String> errorFuture) throws Exception {
+        log.info("{}: Activating NoOpTask.", id);
+    }
+
+    @Override
+    public void stop(Platform platform) throws Exception {
+        log.info("{}: Deactivating NoOpTask.", id);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java
new file mode 100644
index 0000000..dbd0b09
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.trogdor.common.Topology;
+
+import java.util.Set;
+
+/**
+ * Controls a Trogdor task.
+ */
+public interface TaskController {
+    /**
+     * Get the agent nodes which this task is targetting.
+     *
+     * @param topology      The topology to use.
+     *
+     * @return              A set of target node names.
+     */
+    Set<String> targetNodes(Topology topology);
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java
new file mode 100644
index 0000000..84ed75a
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.kafka.trogdor.common.JsonUtil;
+
+import java.util.Objects;
+
+
+/**
+ * The specification for a task. This should be immutable and suitable for serializing and sending over the wire.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS,
+              include = JsonTypeInfo.As.PROPERTY,
+              property = "class")
+public abstract class TaskSpec {
+    /**
+     * The maximum task duration.
+     *
+     * We cap the task duration at this value to avoid worrying about 64-bit overflow or floating
+     * point rounding.  (Objects serialized as JSON canonically contain only floating point numbers,
+     * because JavaScript did not support integers.)
+     */
+    public final static long MAX_TASK_DURATION_MS = 1000000000000000L;
+
+    /**
+     * When the time should start in milliseconds.
+     */
+    private final long startMs;
+
+    /**
+     * How long the task should run in milliseconds.
+     */
+    private final long durationMs;
+
+    protected TaskSpec(@JsonProperty("startMs") long startMs,
+            @JsonProperty("durationMs") long durationMs) {
+        this.startMs = startMs;
+        this.durationMs = Math.max(0, Math.min(durationMs, MAX_TASK_DURATION_MS));
+    }
+
+    /**
+     * Get the target start time of this task in ms.
+     */
+    @JsonProperty
+    public final long startMs() {
+        return startMs;
+    }
+
+    /**
+     * Get the duration of this task in ms.
+     */
+    @JsonProperty
+    public final long durationMs() {
+        return durationMs;
+    }
+
+    /**
+     * Hydrate this task on the coordinator.
+     *
+     * @param id        The task id.
+     */
+    public abstract TaskController newController(String id);
+
+    /**
+     * Hydrate this task on the agent.
+     *
+     * @param id        The worker id.
+     */
+    public abstract TaskWorker newTaskWorker(String id);
+
+    @Override
+    public final boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        return toString().equals(o.toString());
+    }
+
+    @Override
+    public final int hashCode() {
+        return Objects.hashCode(toString());
+    }
+
+    @Override
+    public String toString() {
+        return JsonUtil.toJsonString(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java
new file mode 100644
index 0000000..288eb9c
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.trogdor.common.Platform;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * The agent-side interface for implementing tasks.
+ */
+public interface TaskWorker {
+    /**
+     * Starts the TaskWorker.
+     *
+     * We do not hold any locks or block the WorkerManager state machine on this call.
+     * However, createTask requests to the agent call this function directly.
+     * Therefore, your start() implementation may take a little while, but not too long.
+     * While you can perform short blocking tasks in this function, it is better to
+     * start a background thread to do something time-consuming.
+     *
+     * If the start() function throws an exception, the Agent will assume that the TaskWorker
+     * never started.  Therefore, stop() will never be invoked.  On the other hand, if the
+     * errorFuture is completed, either by a background task or by the start function itself,
+     * the Agent will invoke the stop() method to clean up the worker.
+     *
+     *
+     * @param platform          The platform to use.
+     * @param status            The current status string.  The TaskWorker can update
+     *                          this at any time to provide an updated status.
+     * @param haltFuture        A future which the worker should complete if it halts.
+     *                          If it is completed with an empty string, that means the task
+     *                          halted with no error.  Otherwise, the string is treated as the error.
+     *                          If you start a background thread, you may pass haltFuture
+     *                          to that thread.  Then, the thread can use this future to indicate
+     *                          that the worker should be stopped.
+     *
+     * @throws Exception        If the TaskWorker failed to start.  stop() will not be invoked.
+     */
+    void start(Platform platform, AtomicReference<String> status, KafkaFutureImpl<String> haltFuture)
+        throws Exception;
+
+    /**
+     * Stops the TaskWorker.
+     *
+     * A TaskWorker may be stopped because it has run for its assigned duration, or because a
+     * request arrived instructing the Agent to stop the worker.  The TaskWorker will
+     * also be stopped if errorFuture was completed to indicate that there was an error.
+     *
+     * Regardless of why the TaskWorker was stopped, the stop() function should release all
+     * resources and stop all threads before returning.  The stop() function can block for
+     * as long as it wants.  It is run in a background thread which will not block other
+     * agent operations.  All tasks will be stopped when the Agent cleanly shuts down.
+     *
+     * @param platform          The platform to use.
+     *
+     * @throws Exception        If there was an error cleaning up the TaskWorker.
+     *                          If there is no existing TaskWorker error, the worker will be
+     *                          treated as having failed with the given error.
+     */
+    void stop(Platform platform) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java b/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java
index 53ef849..342fefc 100644
--- a/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java
+++ b/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java
@@ -17,21 +17,25 @@
 
 package org.apache.kafka.trogdor.agent;
 
+import org.apache.kafka.common.utils.MockScheduler;
 import org.apache.kafka.common.utils.MockTime;
-import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.trogdor.basic.BasicNode;
 import org.apache.kafka.trogdor.basic.BasicPlatform;
 import org.apache.kafka.trogdor.basic.BasicTopology;
-import org.apache.kafka.trogdor.common.ExpectedFaults;
+import org.apache.kafka.trogdor.common.ExpectedTasks;
+import org.apache.kafka.trogdor.common.ExpectedTasks.ExpectedTaskBuilder;
 import org.apache.kafka.trogdor.common.Node;
-import org.apache.kafka.trogdor.fault.DoneState;
-import org.apache.kafka.trogdor.fault.NoOpFaultSpec;
-import org.apache.kafka.trogdor.fault.RunningState;
-import org.apache.kafka.trogdor.rest.AgentFaultsResponse;
 import org.apache.kafka.trogdor.rest.AgentStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateAgentFaultRequest;
 
+import org.apache.kafka.trogdor.rest.CreateWorkerRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerResponse;
 import org.apache.kafka.trogdor.rest.JsonRestServer;
+import org.apache.kafka.trogdor.rest.StopWorkerRequest;
+import org.apache.kafka.trogdor.rest.WorkerDone;
+import org.apache.kafka.trogdor.rest.WorkerRunning;
+import org.apache.kafka.trogdor.task.NoOpTaskSpec;
+import org.apache.kafka.trogdor.task.SampleTaskSpec;
 import org.junit.Rule;
 import org.junit.rules.Timeout;
 import org.junit.Test;
@@ -46,73 +50,91 @@ public class AgentTest {
     @Rule
     final public Timeout globalTimeout = Timeout.millis(120000);
 
-    private static BasicPlatform createBasicPlatform() {
+    private static BasicPlatform createBasicPlatform(Scheduler scheduler) {
         TreeMap<String, Node> nodes = new TreeMap<>();
         HashMap<String, String> config = new HashMap<>();
         nodes.put("node01", new BasicNode("node01", "localhost",
             config, Collections.<String>emptySet()));
         BasicTopology topology = new BasicTopology(nodes);
-        return new BasicPlatform("node01", topology, new BasicPlatform.ShellCommandRunner());
+        return new BasicPlatform("node01", topology,
+            scheduler, new BasicPlatform.ShellCommandRunner());
     }
 
-    private Agent createAgent(Time time) {
+    private Agent createAgent(Scheduler scheduler) {
         JsonRestServer restServer = new JsonRestServer(0);
         AgentRestResource resource = new AgentRestResource();
         restServer.start(resource);
-        return new Agent(createBasicPlatform(), time, restServer, resource);
+        return new Agent(createBasicPlatform(scheduler), scheduler,
+                restServer, resource);
     }
 
     @Test
     public void testAgentStartShutdown() throws Exception {
-        Agent agent = createAgent(Time.SYSTEM);
+        Agent agent = createAgent(Scheduler.SYSTEM);
         agent.beginShutdown();
         agent.waitForShutdown();
     }
 
     @Test
     public void testAgentProgrammaticShutdown() throws Exception {
-        Agent agent = createAgent(Time.SYSTEM);
-        AgentClient client = new AgentClient("localhost", agent.port());
+        Agent agent = createAgent(Scheduler.SYSTEM);
+        AgentClient client = new AgentClient(10, "localhost", agent.port());
         client.invokeShutdown();
         agent.waitForShutdown();
     }
 
     @Test
     public void testAgentGetStatus() throws Exception {
-        Agent agent = createAgent(Time.SYSTEM);
-        AgentClient client = new AgentClient("localhost", agent.port());
-        AgentStatusResponse status = client.getStatus();
-        assertEquals(agent.startTimeMs(), status.startTimeMs());
+        Agent agent = createAgent(Scheduler.SYSTEM);
+        AgentClient client = new AgentClient(10, "localhost", agent.port());
+        AgentStatusResponse status = client.status();
+        assertEquals(agent.status(), status);
         agent.beginShutdown();
         agent.waitForShutdown();
     }
 
     @Test
-    public void testAgentCreateFaults() throws Exception {
-        Time time = new MockTime(0, 0, 0);
-        Agent agent = createAgent(time);
-        AgentClient client = new AgentClient("localhost", agent.port());
-        AgentFaultsResponse faults = client.getFaults();
-        assertEquals(Collections.emptyMap(), faults.faults());
-        new ExpectedFaults().waitFor(client);
-
-        final NoOpFaultSpec fooSpec = new NoOpFaultSpec(1000, 600000);
-        client.putFault(new CreateAgentFaultRequest("foo", fooSpec));
-        new ExpectedFaults().addFault("foo", fooSpec).waitFor(client);
-
-        final NoOpFaultSpec barSpec = new NoOpFaultSpec(2000, 900000);
-        client.putFault(new CreateAgentFaultRequest("bar", barSpec));
-        new ExpectedFaults().
-            addFault("foo", fooSpec).
-            addFault("bar", barSpec).
+    public void testAgentCreateWorkers() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        MockScheduler scheduler = new MockScheduler(time);
+        Agent agent = createAgent(scheduler);
+        AgentClient client = new AgentClient(10, "localhost", agent.port());
+        AgentStatusResponse status = client.status();
+        assertEquals(Collections.emptyMap(), status.workers());
+        new ExpectedTasks().waitFor(client);
+
+        final NoOpTaskSpec fooSpec = new NoOpTaskSpec(1000, 600000);
+        CreateWorkerResponse response = client.createWorker(new CreateWorkerRequest("foo", fooSpec));
+        assertEquals(fooSpec.toString(), response.spec().toString());
+        new ExpectedTasks().addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
             waitFor(client);
 
-        final NoOpFaultSpec bazSpec = new NoOpFaultSpec(1, 450000);
-        client.putFault(new CreateAgentFaultRequest("baz", bazSpec));
-        new ExpectedFaults().
-            addFault("foo", fooSpec).
-            addFault("bar", barSpec).
-            addFault("baz", bazSpec).
+        final NoOpTaskSpec barSpec = new NoOpTaskSpec(2000, 900000);
+        client.createWorker(new CreateWorkerRequest("bar", barSpec));
+        client.createWorker(new CreateWorkerRequest("bar", barSpec));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerRunning(barSpec, 0, "")).
+                build()).
+            waitFor(client);
+
+        final NoOpTaskSpec bazSpec = new NoOpTaskSpec(1, 450000);
+        client.createWorker(new CreateWorkerRequest("baz", bazSpec));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerRunning(barSpec, 0, "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("baz").
+                workerState(new WorkerRunning(bazSpec, 0, "")).
+                build()).
             waitFor(client);
 
         agent.beginShutdown();
@@ -120,45 +142,97 @@ public class AgentTest {
     }
 
     @Test
-    public void testAgentActivatesFaults() throws Exception {
-        Time time = new MockTime(0, 0, 0);
-        Agent agent = createAgent(time);
-        AgentClient client = new AgentClient("localhost", agent.port());
-        AgentFaultsResponse faults = client.getFaults();
-        assertEquals(Collections.emptyMap(), faults.faults());
-        new ExpectedFaults().waitFor(client);
-
-        final NoOpFaultSpec fooSpec = new NoOpFaultSpec(10, 2);
-        client.putFault(new CreateAgentFaultRequest("foo", fooSpec));
-        new ExpectedFaults().addFault("foo", new RunningState(0)).waitFor(client);
-
-        time.sleep(3);
-        new ExpectedFaults().addFault("foo", new DoneState(3, "")).waitFor(client);
-
-        final NoOpFaultSpec barSpec = new NoOpFaultSpec(20, 3);
-        client.putFault(new CreateAgentFaultRequest("bar", barSpec));
-        new ExpectedFaults().
-            addFault("foo", new DoneState(3, "")).
-            addFault("bar", new RunningState(3)).
+    public void testAgentFinishesTasks() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        MockScheduler scheduler = new MockScheduler(time);
+        Agent agent = createAgent(scheduler);
+        AgentClient client = new AgentClient(10, "localhost", agent.port());
+        new ExpectedTasks().waitFor(client);
+
+        final NoOpTaskSpec fooSpec = new NoOpTaskSpec(10, 2);
+        client.createWorker(new CreateWorkerRequest("foo", fooSpec));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
             waitFor(client);
 
-        time.sleep(4);
-        final NoOpFaultSpec bazSpec = new NoOpFaultSpec(1, 2);
-        client.putFault(new CreateAgentFaultRequest("baz", bazSpec));
-        new ExpectedFaults().
-            addFault("foo", new DoneState(3, "")).
-            addFault("bar", new DoneState(7, "")).
-            addFault("baz", new RunningState(7)).
+        time.sleep(1);
+
+        final NoOpTaskSpec barSpec = new NoOpTaskSpec(2000, 900000);
+        client.createWorker(new CreateWorkerRequest("bar", barSpec));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerRunning(barSpec, 1, "")).
+                build()).
             waitFor(client);
 
-        time.sleep(3);
-        new ExpectedFaults().
-            addFault("foo", new DoneState(3, "")).
-            addFault("bar", new DoneState(7, "")).
-            addFault("baz", new DoneState(10, "")).
+        time.sleep(1);
+
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerDone(fooSpec, 0, 2, "", "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerRunning(barSpec, 1, "")).
+                build()).
+            waitFor(client);
+
+        time.sleep(5);
+        client.stopWorker(new StopWorkerRequest("bar"));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerDone(fooSpec, 0, 2, "", "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerDone(barSpec, 1, 7, "", "")).
+                build()).
             waitFor(client);
 
         agent.beginShutdown();
         agent.waitForShutdown();
     }
+
+    @Test
+    public void testWorkerCompletions() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        MockScheduler scheduler = new MockScheduler(time);
+        Agent agent = createAgent(scheduler);
+        AgentClient client = new AgentClient(10, "localhost", agent.port());
+        new ExpectedTasks().waitFor(client);
+
+        SampleTaskSpec fooSpec = new SampleTaskSpec(0, 900000, 1, "");
+        client.createWorker(new CreateWorkerRequest("foo", fooSpec));
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerRunning(fooSpec, 0, "")).
+                build()).
+            waitFor(client);
+
+        SampleTaskSpec barSpec = new SampleTaskSpec(0, 900000, 2, "baz");
+        client.createWorker(new CreateWorkerRequest("bar", barSpec));
+
+        time.sleep(1);
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerDone(fooSpec, 0, 1, "", "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerRunning(barSpec, 0, "")).
+                build()).
+            waitFor(client);
+
+        time.sleep(1);
+        new ExpectedTasks().
+            addTask(new ExpectedTaskBuilder("foo").
+                workerState(new WorkerDone(fooSpec, 0, 1, "", "")).
+                build()).
+            addTask(new ExpectedTaskBuilder("bar").
+                workerState(new WorkerDone(barSpec, 0, 2, "", "baz")).
+                build()).
+            waitFor(client);
+    }
 };

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedFaults.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedFaults.java b/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedFaults.java
deleted file mode 100644
index 1fab903..0000000
--- a/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedFaults.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.common;
-
-import org.apache.kafka.test.TestCondition;
-import org.apache.kafka.test.TestUtils;
-import org.apache.kafka.trogdor.agent.AgentClient;
-import org.apache.kafka.trogdor.coordinator.CoordinatorClient;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-import org.apache.kafka.trogdor.fault.FaultState;
-import org.apache.kafka.trogdor.rest.AgentFaultsResponse;
-import org.apache.kafka.trogdor.rest.CoordinatorFaultsResponse;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-public class ExpectedFaults {
-    private static final Logger log = LoggerFactory.getLogger(ExpectedFaults.class);
-
-    private static class FaultData {
-        final FaultSpec spec;
-        final FaultState state;
-
-        FaultData(FaultSpec spec, FaultState state) {
-            this.spec = spec;
-            this.state = state;
-        }
-    }
-
-    private interface FaultFetcher {
-        TreeMap<String, FaultData> fetch() throws Exception;
-    }
-
-    private static class AgentFaultFetcher implements FaultFetcher {
-        private final AgentClient client;
-
-        AgentFaultFetcher(AgentClient client) {
-            this.client = client;
-        }
-
-        @Override
-        public TreeMap<String, FaultData> fetch() throws Exception {
-            TreeMap<String, FaultData> results = new TreeMap<>();
-            AgentFaultsResponse response = client.getFaults();
-            for (Map.Entry<String, AgentFaultsResponse.FaultData> entry :
-                    response.faults().entrySet()) {
-                results.put(entry.getKey(),
-                    new FaultData(entry.getValue().spec(), entry.getValue().state()));
-            }
-            return results;
-        }
-    }
-
-    private static class CoordinatorFaultFetcher implements FaultFetcher {
-        private final CoordinatorClient client;
-
-        CoordinatorFaultFetcher(CoordinatorClient client) {
-            this.client = client;
-        }
-
-        @Override
-        public TreeMap<String, FaultData> fetch() throws Exception {
-            TreeMap<String, FaultData> results = new TreeMap<>();
-            CoordinatorFaultsResponse response = client.getFaults();
-            for (Map.Entry<String, CoordinatorFaultsResponse.FaultData> entry :
-                response.faults().entrySet()) {
-                results.put(entry.getKey(),
-                    new FaultData(entry.getValue().spec(), entry.getValue().state()));
-            }
-            return results;
-        }
-    }
-
-    private final TreeMap<String, FaultData> expected = new TreeMap<String, FaultData>();
-
-    public ExpectedFaults addFault(String id, FaultSpec spec) {
-        expected.put(id, new FaultData(spec, null));
-        return this;
-    }
-
-    public ExpectedFaults addFault(String id, FaultState state) {
-        expected.put(id, new FaultData(null, state));
-        return this;
-    }
-
-    public ExpectedFaults addFault(String id, FaultSpec spec, FaultState state) {
-        expected.put(id, new FaultData(spec, state));
-        return this;
-    }
-
-    public ExpectedFaults waitFor(AgentClient agentClient) throws InterruptedException {
-        waitFor(new AgentFaultFetcher(agentClient));
-        return this;
-    }
-
-    public ExpectedFaults waitFor(CoordinatorClient client) throws InterruptedException {
-        waitFor(new CoordinatorFaultFetcher(client));
-        return this;
-    }
-
-    private void waitFor(final FaultFetcher faultFetcher) throws InterruptedException {
-        TestUtils.waitForCondition(new TestCondition() {
-            @Override
-            public boolean conditionMet() {
-                TreeMap<String, FaultData> curData = null;
-                try {
-                    curData = faultFetcher.fetch();
-                } catch (Exception e) {
-                    log.info("Got error fetching faults", e);
-                    throw new RuntimeException(e);
-                }
-                StringBuilder errors = new StringBuilder();
-                for (Map.Entry<String, FaultData> entry : expected.entrySet()) {
-                    String id = entry.getKey();
-                    FaultData expectedFaultData = entry.getValue();
-                    FaultData curFaultData = curData.get(id);
-                    if (curFaultData == null) {
-                        errors.append("Did not find fault id " + id + "\n");
-                    } else {
-                        if (expectedFaultData.spec != null) {
-                            if (!expectedFaultData.spec.equals(curFaultData.spec)) {
-                                errors.append("For fault id " + id + ", expected fault " +
-                                    "spec " + expectedFaultData.spec + ", but got " +
-                                    curFaultData.spec + "\n");
-                            }
-                        }
-                        if (expectedFaultData.state != null) {
-                            if (!expectedFaultData.state.equals(curFaultData.state)) {
-                                errors.append("For fault id " + id + ", expected fault " +
-                                    "state " + expectedFaultData.state + ", but got " +
-                                    curFaultData.state + "\n");
-                            }
-                        }
-                    }
-                }
-                for (String id : curData.keySet()) {
-                    if (expected.get(id) == null) {
-                        errors.append("Got unexpected fault id " + id + "\n");
-                    }
-                }
-                String errorString = errors.toString();
-                if (!errorString.isEmpty()) {
-                    log.info("EXPECTED FAULTS: {}", faultsToString(expected));
-                    log.info("ACTUAL FAULTS  : {}", faultsToString(curData));
-                    log.info(errorString);
-                    return false;
-                }
-                return true;
-            }
-        }, "Timed out waiting for expected fault specs " + faultsToString(expected));
-    }
-
-    private static String faultsToString(TreeMap<String, FaultData> faults) {
-        StringBuilder bld = new StringBuilder();
-        bld.append("{");
-        String faultsPrefix = "";
-        for (Map.Entry<String, FaultData> entry : faults.entrySet()) {
-            String id = entry.getKey();
-            bld.append(faultsPrefix).append(id).append(": {");
-            faultsPrefix = ", ";
-            String faultValuesPrefix = "";
-            FaultData faultData = entry.getValue();
-            if (faultData.spec != null) {
-                bld.append(faultValuesPrefix).append("spec: ").append(faultData.spec);
-                faultValuesPrefix = ", ";
-            }
-            if (faultData.state != null) {
-                bld.append(faultValuesPrefix).append("state: ").append(faultData.state);
-                faultValuesPrefix = ", ";
-            }
-            bld.append("}");
-        }
-        bld.append("}");
-        return bld.toString();
-    }
-};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java b/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java
new file mode 100644
index 0000000..f72779f
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.common;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.test.TestCondition;
+import org.apache.kafka.test.TestUtils;
+import org.apache.kafka.trogdor.agent.AgentClient;
+import org.apache.kafka.trogdor.coordinator.CoordinatorClient;
+import org.apache.kafka.trogdor.rest.AgentStatusResponse;
+import org.apache.kafka.trogdor.rest.TaskState;
+import org.apache.kafka.trogdor.rest.TasksResponse;
+import org.apache.kafka.trogdor.rest.WorkerState;
+import org.apache.kafka.trogdor.task.TaskSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+public class ExpectedTasks {
+    private static final Logger log = LoggerFactory.getLogger(ExpectedTasks.class);
+
+    private final TreeMap<String, ExpectedTask> expected = new TreeMap<>();
+
+    public static class ExpectedTaskBuilder {
+        private final String id;
+        private TaskSpec taskSpec = null;
+        private TaskState taskState = null;
+        private WorkerState workerState = null;
+
+        public ExpectedTaskBuilder(String id) {
+            this.id = id;
+        }
+
+        public ExpectedTaskBuilder taskSpec(TaskSpec taskSpec) {
+            this.taskSpec = taskSpec;
+            return this;
+        }
+
+        public ExpectedTaskBuilder taskState(TaskState taskState) {
+            this.taskState = taskState;
+            return this;
+        }
+
+        public ExpectedTaskBuilder workerState(WorkerState workerState) {
+            this.workerState = workerState;
+            return this;
+        }
+
+        public ExpectedTask build() {
+            return new ExpectedTask(id, taskSpec, taskState, workerState);
+        }
+    }
+
+    static class ExpectedTask {
+        private final String id;
+        private final TaskSpec taskSpec;
+        private final TaskState taskState;
+        private final WorkerState workerState;
+
+        @JsonCreator
+        private ExpectedTask(@JsonProperty("id") String id,
+                     @JsonProperty("taskSpec") TaskSpec taskSpec,
+                     @JsonProperty("taskState") TaskState taskState,
+                     @JsonProperty("workerState") WorkerState workerState) {
+            this.id = id;
+            this.taskSpec = taskSpec;
+            this.taskState = taskState;
+            this.workerState = workerState;
+        }
+
+        String compare(TaskState actual) {
+            if (actual == null) {
+                return "Did not find task " + id + "\n";
+            }
+            if ((taskSpec != null) && (!actual.spec().equals(taskSpec))) {
+                return "Invalid spec for task " + id + ": expected " + taskSpec +
+                    ", got " + actual.spec();
+            }
+            if ((taskState != null) && (!actual.equals(taskState))) {
+                return "Invalid state for task " + id + ": expected " + taskState +
+                    ", got " + actual;
+            }
+            return null;
+        }
+
+        String compare(WorkerState actual) {
+            if ((workerState != null) && (!workerState.equals(actual))) {
+                if (actual == null) {
+                    return "Did not find worker " + id + "\n";
+                }
+                return "Invalid state for task " + id + ": expected " + workerState +
+                    ", got " + actual;
+            }
+            return null;
+        }
+
+        @JsonProperty
+        public String id() {
+            return id;
+        }
+
+        @JsonProperty
+        public TaskSpec taskSpec() {
+            return taskSpec;
+        }
+
+        @JsonProperty
+        public TaskState taskState() {
+            return taskState;
+        }
+
+        @JsonProperty
+        public WorkerState workerState() {
+            return workerState;
+        }
+    }
+
+    public ExpectedTasks addTask(ExpectedTask task) {
+        expected.put(task.id, task);
+        return this;
+    }
+
+    public ExpectedTasks waitFor(final CoordinatorClient client) throws InterruptedException {
+        TestUtils.waitForCondition(new TestCondition() {
+            @Override
+            public boolean conditionMet() {
+                TasksResponse tasks = null;
+                try {
+                    tasks = client.tasks();
+                } catch (Exception e) {
+                    log.info("Unable to get coordinator tasks", e);
+                    throw new RuntimeException(e);
+                }
+                StringBuilder errors = new StringBuilder();
+                for (Map.Entry<String, ExpectedTask> entry : expected.entrySet()) {
+                    String id = entry.getKey();
+                    ExpectedTask task = entry.getValue();
+                    String differences = task.compare(tasks.tasks().get(id));
+                    if (differences != null) {
+                        errors.append(differences);
+                    }
+                }
+                String errorString = errors.toString();
+                if (!errorString.isEmpty()) {
+                    log.info("EXPECTED TASKS: {}", JsonUtil.toJsonString(expected));
+                    log.info("ACTUAL TASKS  : {}", JsonUtil.toJsonString(tasks.tasks()));
+                    log.info(errorString);
+                    return false;
+                }
+                return true;
+            }
+        }, "Timed out waiting for expected tasks " + JsonUtil.toJsonString(expected));
+        return this;
+    }
+
+    public ExpectedTasks waitFor(final AgentClient client) throws InterruptedException {
+        TestUtils.waitForCondition(new TestCondition() {
+            @Override
+            public boolean conditionMet() {
+                AgentStatusResponse status = null;
+                try {
+                    status = client.status();
+                } catch (Exception e) {
+                    log.info("Unable to get agent status", e);
+                    throw new RuntimeException(e);
+                }
+                StringBuilder errors = new StringBuilder();
+                for (Map.Entry<String, ExpectedTask> entry : expected.entrySet()) {
+                    String id = entry.getKey();
+                    ExpectedTask worker = entry.getValue();
+                    String differences = worker.compare(status.workers().get(id));
+                    if (differences != null) {
+                        errors.append(differences);
+                    }
+                }
+                String errorString = errors.toString();
+                if (!errorString.isEmpty()) {
+                    log.info("EXPECTED WORKERS: {}", JsonUtil.toJsonString(expected));
+                    log.info("ACTUAL WORKERS  : {}", JsonUtil.toJsonString(status.workers()));
+                    log.info(errorString);
+                    return false;
+                }
+                return true;
+            }
+        }, "Timed out waiting for expected workers " + JsonUtil.toJsonString(expected));
+        return this;
+    }
+};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java b/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java
index 1947b79..b180c02 100644
--- a/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java
+++ b/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java
@@ -17,7 +17,7 @@
 
 package org.apache.kafka.trogdor.common;
 
-import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.trogdor.agent.Agent;
 import org.apache.kafka.trogdor.agent.AgentClient;
@@ -38,6 +38,11 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * MiniTrogdorCluster sets up a local cluster of Trogdor Agents and Coordinators.
@@ -53,7 +58,7 @@ public class MiniTrogdorCluster implements AutoCloseable {
 
         private String coordinatorName = null;
 
-        private Time time = Time.SYSTEM;
+        private Scheduler scheduler = Scheduler.SYSTEM;
 
         private BasicPlatform.CommandRunner commandRunner =
                 new BasicPlatform.ShellCommandRunner();
@@ -81,8 +86,8 @@ public class MiniTrogdorCluster implements AutoCloseable {
         /**
          * Set the timekeeper used by this MiniTrogdorCluster.
          */
-        public Builder time(Time time) {
-            this.time = time;
+        public Builder scheduler(Scheduler scheduler) {
+            this.scheduler = scheduler;
             return this;
         }
 
@@ -126,7 +131,7 @@ public class MiniTrogdorCluster implements AutoCloseable {
         /**
          * Create the MiniTrogdorCluster.
          */
-        public MiniTrogdorCluster build() {
+        public MiniTrogdorCluster build() throws Exception {
             log.info("Creating MiniTrogdorCluster with agents: {} and coordinator: {}",
                 Utils.join(agentNames, ", "), coordinatorName);
             TreeMap<String, NodeData> nodes = new TreeMap<>();
@@ -158,20 +163,41 @@ public class MiniTrogdorCluster implements AutoCloseable {
             for (Map.Entry<String, NodeData> entry : nodes.entrySet()) {
                 topologyNodes.put(entry.getKey(), entry.getValue().node);
             }
-            BasicTopology topology = new BasicTopology(topologyNodes);
-            for (Map.Entry<String, NodeData> entry : nodes.entrySet()) {
-                String nodeName = entry.getKey();
-                NodeData node = entry.getValue();
-                node.platform = new BasicPlatform(nodeName, topology, commandRunner);
-                if (node.agentRestResource != null) {
-                    node.agent = new Agent(node.platform, time, node.agentRestServer,
-                        node.agentRestResource);
-                }
-                if (node.coordinatorRestResource != null) {
-                    node.coordinator = new Coordinator(node.platform, time,
-                        node.coordinatorRestServer, node.coordinatorRestResource);
-                }
+            final BasicTopology topology = new BasicTopology(topologyNodes);
+            ScheduledExecutorService executor = Executors.newScheduledThreadPool(1,
+                ThreadUtils.createThreadFactory("MiniTrogdorClusterStartupThread%d", false));
+            final AtomicReference<Exception> failure = new AtomicReference<Exception>(null);
+            for (final Map.Entry<String, NodeData> entry : nodes.entrySet()) {
+                executor.submit(new Callable<Void>() {
+                    @Override
+                    public Void call() throws Exception {
+                        String nodeName = entry.getKey();
+                        try {
+                            NodeData node = entry.getValue();
+                            node.platform = new BasicPlatform(nodeName, topology, scheduler, commandRunner);
+                            if (node.agentRestResource != null) {
+                                node.agent = new Agent(node.platform, scheduler, node.agentRestServer,
+                                    node.agentRestResource);
+                            }
+                            if (node.coordinatorRestResource != null) {
+                                node.coordinator = new Coordinator(node.platform, scheduler,
+                                    node.coordinatorRestServer, node.coordinatorRestResource);
+                            }
+                        } catch (Exception e) {
+                            log.error("Unable to initialize {}", nodeName, e);
+                            failure.compareAndSet(null, e);
+                        }
+                        return null;
+                    }
+                });
+            }
+            executor.shutdown();
+            executor.awaitTermination(1, TimeUnit.DAYS);
+            Exception failureException = failure.get();
+            if (failureException != null) {
+                throw failureException;
             }
+
             TreeMap<String, Agent> agents = new TreeMap<>();
             Coordinator coordinator = null;
             for (Map.Entry<String, NodeData> entry : nodes.entrySet()) {
@@ -209,7 +235,7 @@ public class MiniTrogdorCluster implements AutoCloseable {
         if (coordinator == null) {
             throw new RuntimeException("No coordinator configured.");
         }
-        return new CoordinatorClient("localhost", coordinator.port());
+        return new CoordinatorClient(10, "localhost", coordinator.port());
     }
 
     public AgentClient agentClient(String nodeName) {
@@ -217,17 +243,18 @@ public class MiniTrogdorCluster implements AutoCloseable {
         if (agent == null) {
             throw new RuntimeException("No agent configured on node " + nodeName);
         }
-        return new AgentClient("localhost", agent.port());
+        return new AgentClient(10, "localhost", agent.port());
     }
 
     @Override
     public void close() throws Exception {
+        log.info("Closing MiniTrogdorCluster.");
+        if (coordinator != null) {
+            coordinator.beginShutdown(false);
+        }
         for (Agent agent : agents.values()) {
             agent.beginShutdown();
         }
-        if (coordinator != null) {
-            coordinator.beginShutdown();
-        }
         for (Agent agent : agents.values()) {
             agent.waitForShutdown();
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java b/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java
index 75109d2..4973823 100644
--- a/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java
+++ b/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java
@@ -17,23 +17,28 @@
 
 package org.apache.kafka.trogdor.coordinator;
 
+import org.apache.kafka.common.utils.MockScheduler;
 import org.apache.kafka.common.utils.MockTime;
-import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.test.TestCondition;
 import org.apache.kafka.test.TestUtils;
 import org.apache.kafka.trogdor.agent.AgentClient;
 import org.apache.kafka.trogdor.common.CapturingCommandRunner;
-import org.apache.kafka.trogdor.common.ExpectedFaults;
+import org.apache.kafka.trogdor.common.ExpectedTasks;
+import org.apache.kafka.trogdor.common.ExpectedTasks.ExpectedTaskBuilder;
 import org.apache.kafka.trogdor.common.MiniTrogdorCluster;
 
-import org.apache.kafka.trogdor.fault.DoneState;
 import org.apache.kafka.trogdor.fault.NetworkPartitionFaultSpec;
-import org.apache.kafka.trogdor.fault.NoOpFaultSpec;
-import org.apache.kafka.trogdor.fault.PendingState;
-import org.apache.kafka.trogdor.fault.RunningState;
 import org.apache.kafka.trogdor.rest.CoordinatorStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateCoordinatorFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskRequest;
+import org.apache.kafka.trogdor.rest.StopTaskRequest;
+import org.apache.kafka.trogdor.rest.TaskDone;
+import org.apache.kafka.trogdor.rest.TaskPending;
+import org.apache.kafka.trogdor.rest.TaskRunning;
+import org.apache.kafka.trogdor.rest.WorkerDone;
+import org.apache.kafka.trogdor.rest.WorkerRunning;
+import org.apache.kafka.trogdor.task.NoOpTaskSpec;
 import org.junit.Rule;
 import org.junit.rules.Timeout;
 import org.slf4j.Logger;
@@ -57,62 +62,143 @@ public class CoordinatorTest {
         try (MiniTrogdorCluster cluster = new MiniTrogdorCluster.Builder().
                 addCoordinator("node01").
                 build()) {
-            CoordinatorStatusResponse status = cluster.coordinatorClient().getStatus();
-            assertEquals(cluster.coordinator().startTimeMs(), status.startTimeMs());
+            CoordinatorStatusResponse status = cluster.coordinatorClient().status();
+            assertEquals(cluster.coordinator().status(), status);
         }
     }
 
     @Test
-    public void testCreateFault() throws Exception {
-        Time time = new MockTime(0, 0, 0);
+    public void testCreateTask() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        Scheduler scheduler = new MockScheduler(time);
         try (MiniTrogdorCluster cluster = new MiniTrogdorCluster.Builder().
                 addCoordinator("node01").
-                time(time).
+                addAgent("node02").
+                scheduler(scheduler).
                 build()) {
-            new ExpectedFaults().waitFor(cluster.coordinatorClient());
+            new ExpectedTasks().waitFor(cluster.coordinatorClient());
 
-            NoOpFaultSpec noOpFaultSpec = new NoOpFaultSpec(1, 2);
-            cluster.coordinatorClient().putFault(
-                new CreateCoordinatorFaultRequest("fault1", noOpFaultSpec));
-            new ExpectedFaults().
-                addFault("fault1", noOpFaultSpec, new PendingState()).
+            NoOpTaskSpec fooSpec = new NoOpTaskSpec(1, 2);
+            cluster.coordinatorClient().createTask(
+                new CreateTaskRequest("foo", fooSpec));
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskPending(fooSpec)).
+                    build()).
                 waitFor(cluster.coordinatorClient());
 
             time.sleep(2);
-            new ExpectedFaults().
-                addFault("fault1", noOpFaultSpec, new DoneState(2, "")).
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskRunning(fooSpec, 2)).
+                    workerState(new WorkerRunning(fooSpec, 2, "")).
+                    build()).
+                waitFor(cluster.coordinatorClient()).
+                waitFor(cluster.agentClient("node02"));
+
+            time.sleep(3);
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskDone(fooSpec, 2, 5, "", false)).
+                    build()).
                 waitFor(cluster.coordinatorClient());
         }
     }
 
     @Test
-    public void testFaultDistribution() throws Exception {
-        Time time = new MockTime(0, 0, 0);
+    public void testTaskDistribution() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        Scheduler scheduler = new MockScheduler(time);
         try (MiniTrogdorCluster cluster = new MiniTrogdorCluster.Builder().
                 addCoordinator("node01").
                 addAgent("node01").
                 addAgent("node02").
-                time(time).
+                scheduler(scheduler).
                 build()) {
             CoordinatorClient coordinatorClient = cluster.coordinatorClient();
             AgentClient agentClient1 = cluster.agentClient("node01");
             AgentClient agentClient2 = cluster.agentClient("node02");
 
-            NoOpFaultSpec noOpFaultSpec = new NoOpFaultSpec(10, 2);
-            coordinatorClient.putFault(new CreateCoordinatorFaultRequest("fault1", noOpFaultSpec));
-            new ExpectedFaults().
-                addFault("fault1", noOpFaultSpec, new PendingState()).
-                waitFor(coordinatorClient);
-            new ExpectedFaults().
+            new ExpectedTasks().
+                waitFor(coordinatorClient).
                 waitFor(agentClient1).
                 waitFor(agentClient2);
 
-            time.sleep(10);
-            new ExpectedFaults().
-                addFault("fault1", noOpFaultSpec, new DoneState(10, "")).
-                waitFor(coordinatorClient);
-            new ExpectedFaults().
-                addFault("fault1", noOpFaultSpec, new RunningState(10)).
+            NoOpTaskSpec fooSpec = new NoOpTaskSpec(5, 2);
+            coordinatorClient.createTask(new CreateTaskRequest("foo", fooSpec));
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").taskState(new TaskPending(fooSpec)).build()).
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+
+            time.sleep(11);
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskRunning(fooSpec, 11)).
+                    workerState(new WorkerRunning(fooSpec, 11, "")).
+                    build()).
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+
+            time.sleep(2);
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskDone(fooSpec, 11, 13, "", false)).
+                    workerState(new WorkerDone(fooSpec, 11, 13, "", "")).
+                    build()).
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+        }
+    }
+
+    @Test
+    public void testTaskCancellation() throws Exception {
+        MockTime time = new MockTime(0, 0, 0);
+        Scheduler scheduler = new MockScheduler(time);
+        try (MiniTrogdorCluster cluster = new MiniTrogdorCluster.Builder().
+            addCoordinator("node01").
+            addAgent("node01").
+            addAgent("node02").
+            scheduler(scheduler).
+            build()) {
+            CoordinatorClient coordinatorClient = cluster.coordinatorClient();
+            AgentClient agentClient1 = cluster.agentClient("node01");
+            AgentClient agentClient2 = cluster.agentClient("node02");
+
+            new ExpectedTasks().
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+
+            NoOpTaskSpec fooSpec = new NoOpTaskSpec(5, 2);
+            coordinatorClient.createTask(new CreateTaskRequest("foo", fooSpec));
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").taskState(new TaskPending(fooSpec)).build()).
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+
+            time.sleep(11);
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskRunning(fooSpec, 11)).
+                    workerState(new WorkerRunning(fooSpec, 11, "")).
+                    build()).
+                waitFor(coordinatorClient).
+                waitFor(agentClient1).
+                waitFor(agentClient2);
+
+            time.sleep(1);
+            coordinatorClient.stopTask(new StopTaskRequest("foo"));
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("foo").
+                    taskState(new TaskDone(fooSpec, 11, 12, "", true)).
+                    workerState(new WorkerDone(fooSpec, 11, 12, "", "")).
+                    build()).
+                waitFor(coordinatorClient).
                 waitFor(agentClient1).
                 waitFor(agentClient2);
         }
@@ -191,24 +277,29 @@ public class CoordinatorTest {
                     new String[] {"node01", "node02"},
                     new String[] {"node03"},
                 }));
-            coordinatorClient.putFault(new CreateCoordinatorFaultRequest("netpart", spec));
-            new ExpectedFaults().
-                addFault("netpart", spec).
+            coordinatorClient.createTask(new CreateTaskRequest("netpart", spec));
+            new ExpectedTasks().
+                addTask(new ExpectedTaskBuilder("netpart").taskSpec(spec).build()).
                 waitFor(coordinatorClient);
-            new ExpectedLines().
-                addLine("sudo iptables -A INPUT -p tcp -s 127.0.0.1 -j DROP " +
-                        "-m comment --comment node03").
-                waitFor("node01", runner);
-            new ExpectedLines().
-                addLine("sudo iptables -A INPUT -p tcp -s 127.0.0.1 -j DROP " +
-                        "-m comment --comment node03").
-                waitFor("node02", runner);
-            new ExpectedLines().
-                addLine("sudo iptables -A INPUT -p tcp -s 127.0.0.1 -j DROP " +
-                        "-m comment --comment node01").
-                addLine("sudo iptables -A INPUT -p tcp -s 127.0.0.1 -j DROP " +
-                        "-m comment --comment node02").
-                waitFor("node03", runner);
+            checkLines("-A", runner);
         }
+        checkLines("-D", runner);
+    }
+
+    private void checkLines(String prefix, CapturingCommandRunner runner) throws InterruptedException {
+        new ExpectedLines().
+            addLine("sudo iptables " + prefix + " INPUT -p tcp -s 127.0.0.1 -j DROP " +
+                "-m comment --comment node03").
+            waitFor("node01", runner);
+        new ExpectedLines().
+            addLine("sudo iptables " + prefix + " INPUT -p tcp -s 127.0.0.1 -j DROP " +
+                "-m comment --comment node03").
+            waitFor("node02", runner);
+        new ExpectedLines().
+            addLine("sudo iptables " + prefix + " INPUT -p tcp -s 127.0.0.1 -j DROP " +
+                "-m comment --comment node01").
+            addLine("sudo iptables " + prefix + " INPUT -p tcp -s 127.0.0.1 -j DROP " +
+                "-m comment --comment node02").
+            waitFor("node03", runner);
     }
 };

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/fault/FaultSetTest.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/fault/FaultSetTest.java b/tools/src/test/java/org/apache/kafka/trogdor/fault/FaultSetTest.java
deleted file mode 100644
index 5f097b6..0000000
--- a/tools/src/test/java/org/apache/kafka/trogdor/fault/FaultSetTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-public class FaultSetTest {
-    private static final NoOpFault FAULT_A =
-        new NoOpFault("faultA", new NoOpFaultSpec(0, 100));
-
-    private static final NoOpFault FAULT_B =
-        new NoOpFault("faultB", new NoOpFaultSpec(20, 60));
-
-    private static final NoOpFault FAULT_C =
-        new NoOpFault("faultC", new NoOpFaultSpec(40, 50));
-
-    private static final NoOpFault FAULT_D =
-        new NoOpFault("faultD", new NoOpFaultSpec(50, 10));
-
-    private static final List<Fault> FAULTS_IN_START_ORDER =
-        Arrays.<Fault>asList(FAULT_A, FAULT_B, FAULT_C, FAULT_D);
-
-    private static final List<Fault> FAULTS_IN_END_ORDER =
-        Arrays.<Fault>asList(FAULT_D, FAULT_B, FAULT_C, FAULT_A);
-
-    @Test
-    public void testIterateByStart() throws Exception {
-        FaultSet faultSet = new FaultSet();
-        for (Fault fault: FAULTS_IN_END_ORDER) {
-            faultSet.add(fault);
-        }
-        int i = 0;
-        for (Iterator<Fault> iter = faultSet.iterateByStart(); iter.hasNext(); ) {
-            Fault fault = iter.next();
-            assertEquals(FAULTS_IN_START_ORDER.get(i), fault);
-            i++;
-        }
-    }
-
-    @Test
-    public void testIterateByEnd() throws Exception {
-        FaultSet faultSet = new FaultSet();
-        for (Fault fault: FAULTS_IN_START_ORDER) {
-            faultSet.add(fault);
-        }
-        int i = 0;
-        for (Iterator<Fault> iter = faultSet.iterateByEnd(); iter.hasNext(); ) {
-            Fault fault = iter.next();
-            assertEquals(FAULTS_IN_END_ORDER.get(i), fault);
-            i++;
-        }
-    }
-
-    @Test
-    public void testDeletes() throws Exception {
-        FaultSet faultSet = new FaultSet();
-        for (Fault fault: FAULTS_IN_START_ORDER) {
-            faultSet.add(fault);
-        }
-        Iterator<Fault> iter = faultSet.iterateByEnd();
-        iter.next();
-        iter.next();
-        iter.remove();
-        iter.next();
-        iter.next();
-        iter.remove();
-        assertFalse(iter.hasNext());
-        try {
-            iter.next();
-            fail("expected NoSuchElementException");
-        } catch (NoSuchElementException e) {
-        }
-        iter = faultSet.iterateByEnd();
-        assertEquals(FAULT_D, iter.next());
-        assertEquals(FAULT_C, iter.next());
-        assertFalse(iter.hasNext());
-        iter = faultSet.iterateByStart();
-        faultSet.remove(FAULT_C);
-        assertEquals(FAULT_D, iter.next());
-        assertFalse(iter.hasNext());
-    }
-
-    @Test
-    public void testEqualRanges() throws Exception {
-        FaultSet faultSet = new FaultSet();
-        faultSet.add(new NoOpFault("fault1", new NoOpFaultSpec(10, 20)));
-        faultSet.add(new NoOpFault("fault2", new NoOpFaultSpec(10, 20)));
-        faultSet.add(new NoOpFault("fault3", new NoOpFaultSpec(10, 20)));
-        faultSet.add(new NoOpFault("fault4", new NoOpFaultSpec(10, 20)));
-        for (Iterator<Fault> iter = faultSet.iterateByStart(); iter.hasNext(); ) {
-            Fault fault = iter.next();
-            if (fault.id().equals("fault3")) {
-                iter.remove();
-            }
-        }
-        Iterator<Fault> iter = faultSet.iterateByStart();
-        assertEquals("fault1", iter.next().id());
-        assertEquals("fault2", iter.next().id());
-        assertEquals("fault4", iter.next().id());
-        assertFalse(iter.hasNext());
-    }
-};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java
new file mode 100644
index 0000000..2640c39
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.trogdor.common.Topology;
+
+import java.util.Set;
+
+public class SampleTaskController implements TaskController {
+    @Override
+    public Set<String> targetNodes(Topology topology) {
+        return Topology.Util.agentNodeNames(topology);
+    }
+};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java
new file mode 100644
index 0000000..2bbbb20
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class SampleTaskSpec extends TaskSpec {
+    private final long exitMs;
+    private final String error;
+
+    @JsonCreator
+    public SampleTaskSpec(@JsonProperty("startMs") long startMs,
+                        @JsonProperty("durationMs") long durationMs,
+                        @JsonProperty("exitMs") long exitMs,
+                        @JsonProperty("error") String error) {
+        super(startMs, durationMs);
+        this.exitMs = exitMs;
+        this.error = error;
+    }
+
+    @JsonProperty
+    public long exitMs() {
+        return exitMs;
+    }
+
+    @JsonProperty
+    public String error() {
+        return error;
+    }
+
+    @Override
+    public TaskController newController(String id) {
+        return new SampleTaskController();
+    }
+
+    @Override
+    public TaskWorker newTaskWorker(String id) {
+        return new SampleTaskWorker(this);
+    }
+};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java
----------------------------------------------------------------------
diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java
new file mode 100644
index 0000000..ebac27e
--- /dev/null
+++ b/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.task;
+
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.trogdor.common.Platform;
+import org.apache.kafka.trogdor.common.ThreadUtils;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class SampleTaskWorker implements TaskWorker {
+    private final SampleTaskSpec spec;
+    private final ScheduledExecutorService executor;
+    private Future<Void> future;
+
+    SampleTaskWorker(SampleTaskSpec spec) {
+        this.spec = spec;
+        this.executor = Executors.newSingleThreadScheduledExecutor(
+            ThreadUtils.createThreadFactory("SampleTaskWorker", false));
+        this.future = null;
+    }
+
+    @Override
+    public synchronized void start(Platform platform, AtomicReference<String> status,
+                      final KafkaFutureImpl<String> haltFuture) throws Exception {
+        if (this.future != null)
+            return;
+        this.future = platform.scheduler().schedule(executor, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                haltFuture.complete(spec.error());
+                return null;
+            }
+        }, spec.exitMs());
+    }
+
+    @Override
+    public void stop(Platform platform) throws Exception {
+        this.future.cancel(false);
+        this.executor.shutdown();
+        this.executor.awaitTermination(1, TimeUnit.DAYS);
+    }
+};


[2/4] kafka git commit: KAFKA-6060; Add workload generation capabilities to Trogdor

Posted by rs...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSpec.java
deleted file mode 100644
index e15c4e9..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSpec.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.kafka.common.utils.Utils;
-
-
-/**
- * The specification for a fault.
- */
-@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS,
-              include = JsonTypeInfo.As.PROPERTY,
-              property = "class")
-public interface FaultSpec {
-    class Util {
-        private static final String SPEC_STRING = "Spec";
-
-        public static Fault createFault(String faultId, FaultSpec faultSpec) throws ClassNotFoundException {
-            String faultSpecClassName = faultSpec.getClass().getName();
-            if (!faultSpecClassName.endsWith(SPEC_STRING)) {
-                throw new RuntimeException("FaultSpec class name must end with " + SPEC_STRING);
-            }
-            String faultClassName = faultSpecClassName.substring(0,
-                    faultSpecClassName.length() - SPEC_STRING.length());
-            return Utils.newParameterizedInstance(faultClassName,
-                String.class, faultId,
-                FaultSpec.class, faultSpec);
-        }
-    }
-
-    /**
-     * Get the start time of this fault in ms.
-     */
-    @JsonProperty
-    long startMs();
-
-    /**
-     * Get the duration of this fault in ms.
-     */
-    @JsonProperty
-    long durationMs();
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultState.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultState.java
deleted file mode 100644
index cba8419..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultState.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.kafka.trogdor.common.JsonUtil;
-import java.util.Objects;
-
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME,
-    include = JsonTypeInfo.As.PROPERTY,
-    property = "stateName")
-@JsonSubTypes({
-        @JsonSubTypes.Type(value = DoneState.class, name = "done"),
-        @JsonSubTypes.Type(value = PendingState.class, name = "pending"),
-        @JsonSubTypes.Type(value = RunningState.class, name = "running"),
-        @JsonSubTypes.Type(value = SendingState.class, name = "sending")
-    })
-public abstract class FaultState {
-    @Override
-    public final boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        return toString().equals(o.toString());
-    }
-
-    @Override
-    public final int hashCode() {
-        return Objects.hashCode(toString());
-    }
-
-    @Override
-    public final String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFault.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFault.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFault.java
deleted file mode 100644
index cf3270a..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFault.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import org.apache.kafka.trogdor.common.Node;
-import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.common.Topology;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-public class NetworkPartitionFault extends AbstractFault {
-    private static final Logger log = LoggerFactory.getLogger(NetworkPartitionFault.class);
-
-    private final List<Set<String>> partitions;
-
-    public NetworkPartitionFault(String id, FaultSpec spec) {
-        super(id, spec);
-        NetworkPartitionFaultSpec faultSpec = (NetworkPartitionFaultSpec) spec;
-        this.partitions = new ArrayList<>();
-        HashSet<String> prevNodes = new HashSet<>();
-        for (List<String> partition : faultSpec.partitions()) {
-            for (String nodeName : partition) {
-                if (prevNodes.contains(nodeName)) {
-                    throw new RuntimeException("Node " + nodeName +
-                        " appears in more than one partition.");
-                }
-                prevNodes.add(nodeName);
-                this.partitions.add(new HashSet<String>(partition));
-            }
-        }
-    }
-
-    @Override
-    protected void handleActivation(long now, Platform platform) throws Exception {
-        log.info("Activating NetworkPartitionFault...");
-        runIptablesCommands(platform, "-A");
-    }
-
-    @Override
-    protected void handleDeactivation(long now, Platform platform) throws Exception {
-        log.info("Deactivating NetworkPartitionFault...");
-        runIptablesCommands(platform, "-D");
-    }
-
-    private void runIptablesCommands(Platform platform, String iptablesAction) throws Exception {
-        Node curNode = platform.curNode();
-        Topology topology = platform.topology();
-        TreeSet<String> toBlock = new TreeSet<>();
-        for (Set<String> partition : partitions) {
-            if (!partition.contains(curNode.name())) {
-                for (String nodeName : partition) {
-                    toBlock.add(nodeName);
-                }
-            }
-        }
-        for (String nodeName : toBlock) {
-            Node node = topology.node(nodeName);
-            InetAddress addr = InetAddress.getByName(node.hostname());
-            platform.runCommand(new String[] {
-                "sudo", "iptables", iptablesAction, "INPUT", "-p", "tcp", "-s",
-                addr.getHostAddress(), "-j", "DROP", "-m", "comment", "--comment", nodeName
-            });
-        }
-    }
-
-    @Override
-    public Set<String> targetNodes(Topology topology) {
-        Set<String> targetNodes = new HashSet<>();
-        for (Set<String> partition : partitions) {
-            targetNodes.addAll(partition);
-        }
-        return targetNodes;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java
new file mode 100644
index 0000000..d90534f
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.fault;
+
+import org.apache.kafka.trogdor.common.Topology;
+import org.apache.kafka.trogdor.task.TaskController;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class NetworkPartitionFaultController implements TaskController {
+    private final List<Set<String>> partitionSets;
+
+    public NetworkPartitionFaultController(List<Set<String>> partitionSets) {
+        this.partitionSets = partitionSets;
+    }
+
+    @Override
+    public Set<String> targetNodes(Topology topology) {
+        Set<String> targetNodes = new HashSet<>();
+        for (Set<String> partitionSet : partitionSets) {
+            targetNodes.addAll(partitionSet);
+        }
+        return targetNodes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java
index d734dce..7b9ccc4 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java
@@ -19,15 +19,19 @@ package org.apache.kafka.trogdor.fault;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
+import org.apache.kafka.trogdor.task.TaskController;
+import org.apache.kafka.trogdor.task.TaskSpec;
+import org.apache.kafka.trogdor.task.TaskWorker;
 
+import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Objects;
+import java.util.Set;
 
 /**
  * The specification for a fault that creates a network partition.
  */
-public class NetworkPartitionFaultSpec extends AbstractFaultSpec {
+public class NetworkPartitionFaultSpec extends TaskSpec {
     private final List<List<String>> partitions;
 
     @JsonCreator
@@ -44,22 +48,28 @@ public class NetworkPartitionFaultSpec extends AbstractFaultSpec {
     }
 
     @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        NetworkPartitionFaultSpec that = (NetworkPartitionFaultSpec) o;
-        return Objects.equals(startMs(), that.startMs()) &&
-            Objects.equals(durationMs(), that.durationMs()) &&
-            Objects.equals(partitions, that.partitions);
+    public TaskController newController(String id) {
+        return new NetworkPartitionFaultController(partitionSets());
     }
 
     @Override
-    public int hashCode() {
-        return Objects.hash(startMs(), durationMs(), partitions);
+    public TaskWorker newTaskWorker(String id) {
+        return new NetworkPartitionFaultWorker(id, partitionSets());
     }
 
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
+    private List<Set<String>> partitionSets() {
+        List<Set<String>> partitionSets = new ArrayList<>();
+        HashSet<String> prevNodes = new HashSet<>();
+        for (List<String> partition : this.partitions()) {
+            for (String nodeName : partition) {
+                if (prevNodes.contains(nodeName)) {
+                    throw new RuntimeException("Node " + nodeName +
+                        " appears in more than one partition.");
+                }
+                prevNodes.add(nodeName);
+                partitionSets.add(new HashSet<>(partition));
+            }
+        }
+        return partitionSets;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java
new file mode 100644
index 0000000..787c5e0
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.fault;
+
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.trogdor.common.Node;
+import org.apache.kafka.trogdor.common.Platform;
+import org.apache.kafka.trogdor.common.Topology;
+import org.apache.kafka.trogdor.task.TaskWorker;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class NetworkPartitionFaultWorker implements TaskWorker {
+    private static final Logger log = LoggerFactory.getLogger(NetworkPartitionFaultWorker.class);
+
+    private final String id;
+
+    private final List<Set<String>> partitionSets;
+
+    public NetworkPartitionFaultWorker(String id, List<Set<String>> partitionSets) {
+        this.id = id;
+        this.partitionSets = partitionSets;
+    }
+
+    @Override
+    public void start(Platform platform, AtomicReference<String> status,
+                      KafkaFutureImpl<String> errorFuture) throws Exception {
+        log.info("Activating NetworkPartitionFault {}.", id);
+        runIptablesCommands(platform, "-A");
+    }
+
+    @Override
+    public void stop(Platform platform) throws Exception {
+        log.info("Deactivating NetworkPartitionFault {}.", id);
+        runIptablesCommands(platform, "-D");
+    }
+
+    private void runIptablesCommands(Platform platform, String iptablesAction) throws Exception {
+        Node curNode = platform.curNode();
+        Topology topology = platform.topology();
+        TreeSet<String> toBlock = new TreeSet<>();
+        for (Set<String> partitionSet : partitionSets) {
+            if (!partitionSet.contains(curNode.name())) {
+                for (String nodeName : partitionSet) {
+                    toBlock.add(nodeName);
+                }
+            }
+        }
+        for (String nodeName : toBlock) {
+            Node node = topology.node(nodeName);
+            InetAddress addr = InetAddress.getByName(node.hostname());
+            platform.runCommand(new String[] {
+                "sudo", "iptables", iptablesAction, "INPUT", "-p", "tcp", "-s",
+                addr.getHostAddress(), "-j", "DROP", "-m", "comment", "--comment", nodeName
+            });
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFault.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFault.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFault.java
deleted file mode 100644
index 70b4965..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFault.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import org.apache.kafka.trogdor.common.Node;
-import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.common.Topology;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-public class NoOpFault extends AbstractFault {
-    private static final Logger log = LoggerFactory.getLogger(NoOpFault.class);
-
-    public NoOpFault(String id, FaultSpec spec) {
-        super(id, spec);
-    }
-
-    @Override
-    protected void handleActivation(long now, Platform platform) throws Exception {
-        log.info("Activating NoOpFault...");
-    }
-
-    @Override
-    protected void handleDeactivation(long now, Platform platform) throws Exception {
-        log.info("Deactivating NoOpFault...");
-    }
-
-    @Override
-    public Set<String> targetNodes(Topology topology) {
-        Set<String> set = new HashSet<>();
-        for (Map.Entry<String, Node> entry : topology.nodes().entrySet()) {
-            if (Node.Util.getTrogdorAgentPort(entry.getValue()) > 0) {
-                set.add(entry.getKey());
-            }
-        }
-        return set;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFaultSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFaultSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFaultSpec.java
deleted file mode 100644
index 1d4b94d..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/NoOpFaultSpec.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import java.util.Objects;
-
-/**
- * The specification for a fault that does nothing.
- *
- * This fault type exists mainly to test the fault injection system.
- */
-public class NoOpFaultSpec extends AbstractFaultSpec {
-    @JsonCreator
-    public NoOpFaultSpec(@JsonProperty("startMs") long startMs,
-                         @JsonProperty("durationMs") long durationMs) {
-        super(startMs, durationMs);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        NoOpFaultSpec that = (NoOpFaultSpec) o;
-        return Objects.equals(startMs(), that.startMs()) &&
-            Objects.equals(durationMs(), that.durationMs());
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(startMs(), durationMs());
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/PendingState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/PendingState.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/PendingState.java
deleted file mode 100644
index 57c8e88..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/PendingState.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-
-/**
- * The state a fault is in on the agent or controller when we haven't yet done
- * anything with it.
- */
-public class PendingState extends FaultState {
-    @JsonCreator
-    public PendingState() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/RunningState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/RunningState.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/RunningState.java
deleted file mode 100644
index 1b81bf5..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/RunningState.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-/**
- * The state a fault is in on the agent when it is running.
- */
-public class RunningState extends FaultState {
-    private final long startedMs;
-
-    @JsonCreator
-    public RunningState(@JsonProperty("startedMs") long startedMs) {
-        this.startedMs = startedMs;
-    }
-
-    @JsonProperty
-    public long startedMs() {
-        return startedMs;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/SendingState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/SendingState.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/SendingState.java
deleted file mode 100644
index edfbed2..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/SendingState.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.common.utils.Utils;
-
-import java.util.TreeMap;
-import java.util.Set;
-
-/**
- * The state a fault is in on the controller when it is scheduled to be sent to several agents.
- */
-public class SendingState extends FaultState {
-    private final TreeMap<String, Boolean> nodes;
-    private int remainingNodes;
-
-    public SendingState(@JsonProperty("nodeNames") Set<String> nodeNames) {
-        this.nodes = new TreeMap<>();
-        for (String nodeName : nodeNames) {
-            nodes.put(nodeName, false);
-        }
-        remainingNodes = nodeNames.size();
-    }
-
-    @JsonProperty
-    public synchronized Set<String> nodeNames() {
-        return nodes.keySet();
-    }
-
-    /**
-     * Complete a send operation.
-     *
-     * @param nodeName      The name of the node we sent to.
-     * @return              True if there are no more send operations left.
-     */
-    public synchronized boolean completeSend(String nodeName) {
-        if (!nodes.containsKey(nodeName)) {
-            throw new RuntimeException("Node " + nodeName + " was not to supposed to " +
-                "receive this fault.  The fault was scheduled on nodes: " +
-                Utils.join(nodes.keySet(), ", "));
-        }
-        if (nodes.put(nodeName, true)) {
-            throw new RuntimeException("Node " + nodeName + " already received this fault.");
-        }
-        remainingNodes--;
-        return remainingNodes == 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentFaultsResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentFaultsResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentFaultsResponse.java
deleted file mode 100644
index a1b5246..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentFaultsResponse.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.rest;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-
-import java.util.Map;
-
-/**
- * Response to GET /faults
- */
-public class AgentFaultsResponse extends FaultDataMap {
-    @JsonCreator
-    public AgentFaultsResponse(@JsonProperty("faults") Map<String, FaultData> faults) {
-        super(faults);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        AgentFaultsResponse that = (AgentFaultsResponse) o;
-        return super.equals(that);
-    }
-
-    @Override
-    public int hashCode() {
-        return super.hashCode();
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java
index 8e32f87..77b4bfb 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java
@@ -19,41 +19,30 @@ package org.apache.kafka.trogdor.rest;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
 
-import java.util.Objects;
+import java.util.TreeMap;
 
 /**
- * The status of the Trogdor agent.
+ * A response from the Trogdor agent about the worker states and specifications.
  */
-public class AgentStatusResponse {
-    private final long startTimeMs;
+public class AgentStatusResponse extends Message {
+    private final long serverStartMs;
+    private final TreeMap<String, WorkerState> workers;
 
     @JsonCreator
-    public AgentStatusResponse(@JsonProperty("startTimeMs") long startTimeMs) {
-        this.startTimeMs = startTimeMs;
+    public AgentStatusResponse(@JsonProperty("serverStartMs") long serverStartMs,
+            @JsonProperty("workers") TreeMap<String, WorkerState> workers) {
+        this.serverStartMs = serverStartMs;
+        this.workers = workers;
     }
 
     @JsonProperty
-    public long startTimeMs() {
-        return startTimeMs;
+    public long serverStartMs() {
+        return serverStartMs;
     }
 
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        AgentStatusResponse that = (AgentStatusResponse) o;
-        return Objects.equals(startTimeMs, that.startTimeMs);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(startTimeMs);
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
+    @JsonProperty
+    public TreeMap<String, WorkerState> workers() {
+        return workers;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorFaultsResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorFaultsResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorFaultsResponse.java
deleted file mode 100644
index df26274..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorFaultsResponse.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.rest;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-
-import java.util.Map;
-
-/**
- * Response to GET /faults
- */
-public class CoordinatorFaultsResponse extends FaultDataMap {
-    @JsonCreator
-    public CoordinatorFaultsResponse(@JsonProperty("faults") Map<String, FaultData> faults) {
-        super(faults);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        CoordinatorFaultsResponse that = (CoordinatorFaultsResponse) o;
-        return super.equals(that);
-    }
-
-    @Override
-    public int hashCode() {
-        return super.hashCode();
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java
new file mode 100644
index 0000000..1aacaaf
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A request to the Trogdor coordinator to shut down.
+ */
+public class CoordinatorShutdownRequest extends Message {
+    private final boolean stopAgents;
+
+    @JsonCreator
+    public CoordinatorShutdownRequest(@JsonProperty("stopAgents") boolean stopAgents) {
+        this.stopAgents = stopAgents;
+    }
+
+    @JsonProperty
+    public boolean stopAgents() {
+        return stopAgents;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java
index 348e310..8840d29 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java
@@ -19,41 +19,20 @@ package org.apache.kafka.trogdor.rest;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-
-import java.util.Objects;
 
 /**
- * The status of the Trogdor coordinator.
+ * A status response from the Trogdor coordinator.
  */
-public class CoordinatorStatusResponse {
-    private final long startTimeMs;
+public class CoordinatorStatusResponse extends Message {
+    private final long serverStartMs;
 
     @JsonCreator
-    public CoordinatorStatusResponse(@JsonProperty("startTimeMs") long startTimeMs) {
-        this.startTimeMs = startTimeMs;
+    public CoordinatorStatusResponse(@JsonProperty("serverStartMs") long serverStartMs) {
+        this.serverStartMs = serverStartMs;
     }
 
     @JsonProperty
-    public long startTimeMs() {
-        return startTimeMs;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        CoordinatorStatusResponse that = (CoordinatorStatusResponse) o;
-        return Objects.equals(startTimeMs, that.startTimeMs);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(startTimeMs);
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
+    public long serverStartMs() {
+        return serverStartMs;
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateAgentFaultRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateAgentFaultRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateAgentFaultRequest.java
deleted file mode 100644
index 6e772d9..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateAgentFaultRequest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.rest;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-
-import java.util.Objects;
-
-/**
- * A request to the Trogdor agent to create a fault.
- */
-public class CreateAgentFaultRequest {
-    private final String id;
-    private final FaultSpec spec;
-
-    @JsonCreator
-    public CreateAgentFaultRequest(@JsonProperty("id") String id,
-            @JsonProperty("spec") FaultSpec spec) {
-        this.id = id;
-        this.spec = spec;
-    }
-
-    @JsonProperty
-    public String id() {
-        return id;
-    }
-
-    @JsonProperty
-    public FaultSpec spec() {
-        return spec;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        CreateAgentFaultRequest that = (CreateAgentFaultRequest) o;
-        return Objects.equals(id, that.id) &&
-               Objects.equals(spec, that.spec);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(id, spec);
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateCoordinatorFaultRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateCoordinatorFaultRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateCoordinatorFaultRequest.java
deleted file mode 100644
index ec00cf3..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateCoordinatorFaultRequest.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.rest;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-
-import java.util.Objects;
-
-/**
- * A request to the Trogdor coordinator to create a fault.
- */
-public class CreateCoordinatorFaultRequest {
-    private final String id;
-    private final FaultSpec spec;
-
-    @JsonCreator
-    public CreateCoordinatorFaultRequest(@JsonProperty("id") String id,
-            @JsonProperty("spec") FaultSpec spec) {
-        this.id = id;
-        this.spec = spec;
-    }
-
-    @JsonProperty
-    public String id() {
-        return id;
-    }
-
-    @JsonProperty
-    public FaultSpec spec() {
-        return spec;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        CreateCoordinatorFaultRequest that = (CreateCoordinatorFaultRequest) o;
-        return Objects.equals(id, that.id) &&
-               Objects.equals(spec, that.spec);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(id, spec);
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java
new file mode 100644
index 0000000..d463e36
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A request to the Trogdor coorinator to create a task.
+ */
+public class CreateTaskRequest extends Message {
+    private final String id;
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public CreateTaskRequest(@JsonProperty("id") String id,
+            @JsonProperty("spec") TaskSpec spec) {
+        this.id = id;
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public String id() {
+        return id;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskResponse.java
new file mode 100644
index 0000000..54ea0f2
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskResponse.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A response from the Trogdor coordinator about creating a task.
+ */
+public class CreateTaskResponse extends Message {
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public CreateTaskResponse(@JsonProperty("spec") TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java
new file mode 100644
index 0000000..9f6e8dc
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A request to the Trogdor agent to create a worker.
+ */
+public class CreateWorkerRequest extends Message {
+    private final String id;
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public CreateWorkerRequest(@JsonProperty("id") String id,
+            @JsonProperty("spec") TaskSpec spec) {
+        this.id = id;
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public String id() {
+        return id;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerResponse.java
new file mode 100644
index 0000000..9e068ec
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerResponse.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A response from the Trogdor agent about creating a worker.
+ */
+public class CreateWorkerResponse extends Message {
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public CreateWorkerResponse(@JsonProperty("spec") TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/FaultDataMap.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/FaultDataMap.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/FaultDataMap.java
deleted file mode 100644
index b2f7c91..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/FaultDataMap.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.rest;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-import org.apache.kafka.trogdor.fault.FaultState;
-
-import java.util.Map;
-import java.util.Objects;
-
-/**
- * Response to GET /faults
- */
-public class FaultDataMap {
-    private final Map<String, FaultData> faults;
-
-    public static class FaultData  {
-        private final FaultSpec spec;
-        private final FaultState state;
-
-        @JsonCreator
-        public FaultData(@JsonProperty("spec") FaultSpec spec,
-                @JsonProperty("state") FaultState state) {
-            this.spec = spec;
-            this.state = state;
-        }
-
-        @JsonProperty
-        public FaultSpec spec() {
-            return spec;
-        }
-
-        @JsonProperty
-        public FaultState state() {
-            return state;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            FaultData that = (FaultData) o;
-            return Objects.equals(spec, that.spec) &&
-                Objects.equals(state, that.state);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(spec, state);
-        }
-    }
-
-    @JsonCreator
-    public FaultDataMap(@JsonProperty("faults") Map<String, FaultData> faults) {
-        this.faults = faults;
-    }
-
-    @JsonProperty
-    public Map<String, FaultData> faults() {
-        return faults;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        FaultDataMap that = (FaultDataMap) o;
-        return Objects.equals(faults, that.faults);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(faults);
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java
index 1b23a9e..e61b7fe 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
 
 import org.apache.kafka.trogdor.common.JsonUtil;
+import org.apache.kafka.trogdor.common.ThreadUtils;
 import org.eclipse.jetty.server.Connector;
 import org.eclipse.jetty.server.Handler;
 import org.eclipse.jetty.server.Server;
@@ -43,6 +44,10 @@ import java.io.OutputStream;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.nio.charset.StandardCharsets;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Embedded server for the REST API that provides the control plane for Trogdor.
@@ -50,7 +55,9 @@ import java.nio.charset.StandardCharsets;
 public class JsonRestServer {
     private static final Logger log = LoggerFactory.getLogger(JsonRestServer.class);
 
-    private static final long GRACEFUL_SHUTDOWN_TIMEOUT_MS = 2 * 1000;
+    private static final long GRACEFUL_SHUTDOWN_TIMEOUT_MS = 100;
+
+    private final ScheduledExecutorService shutdownExecutor;
 
     private final Server jettyServer;
 
@@ -63,6 +70,8 @@ public class JsonRestServer {
      *                          0 to use a random port.
      */
     public JsonRestServer(int port) {
+        this.shutdownExecutor = Executors.newSingleThreadScheduledExecutor(
+            ThreadUtils.createThreadFactory("JsonRestServerCleanupExecutor", false));
         this.jettyServer = new Server();
         this.connector = new ServerConnector(jettyServer);
         if (port > 0) {
@@ -78,7 +87,6 @@ public class JsonRestServer {
      */
     public void start(Object... resources) {
         log.info("Starting REST server");
-
         ResourceConfig resourceConfig = new ResourceConfig();
         resourceConfig.register(new JacksonJsonProvider(JsonUtil.JSON_SERDE));
         for (Object resource : resources) {
@@ -119,17 +127,37 @@ public class JsonRestServer {
         return connector.getLocalPort();
     }
 
-    public void stop() {
-        log.info("Stopping REST server");
+    /**
+     * Initiate shutdown, but do not wait for it to complete.
+     */
+    public void beginShutdown() {
+        if (!shutdownExecutor.isShutdown()) {
+            shutdownExecutor.submit(new Callable<Void>() {
+                @Override
+                public Void call() throws Exception {
+                    try {
+                        log.info("Stopping REST server");
+                        jettyServer.stop();
+                        jettyServer.join();
+                        log.info("REST server stopped");
+                    } catch (Exception e) {
+                        log.error("Unable to stop REST server", e);
+                    } finally {
+                        jettyServer.destroy();
+                    }
+                    shutdownExecutor.shutdown();
+                    return null;
+                }
+            });
+        }
+    }
 
-        try {
-            jettyServer.stop();
-            jettyServer.join();
-            log.info("REST server stopped");
-        } catch (Exception e) {
-            log.error("Unable to stop REST server", e);
-        } finally {
-            jettyServer.destroy();
+    /**
+     * Wait for shutdown to complete.  May be called prior to beginShutdown.
+     */
+    public void waitForShutdown() throws InterruptedException {
+        while (!shutdownExecutor.isShutdown()) {
+            shutdownExecutor.awaitTermination(1, TimeUnit.DAYS);
         }
     }
 
@@ -197,6 +225,24 @@ public class JsonRestServer {
         }
     }
 
+    public static <T> HttpResponse<T> httpRequest(String url, String method,
+            Object requestBodyData, TypeReference<T> responseFormat, int maxTries)
+            throws IOException, InterruptedException {
+        IOException exc = null;
+        for (int tries = 0; tries < maxTries; tries++) {
+            if (tries > 0) {
+                Thread.sleep(tries > 1 ? 10 : 2);
+            }
+            try {
+                return httpRequest(url, method, requestBodyData, responseFormat);
+            } catch (IOException e) {
+                log.info("{} {}: error: {}", method, url, e.getMessage());
+                exc = e;
+            }
+        }
+        throw exc;
+    }
+
     public static class HttpResponse<T> {
         private final T body;
         private final ErrorResponse error;

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java
new file mode 100644
index 0000000..c2ee840
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import org.apache.kafka.trogdor.common.JsonUtil;
+
+import java.util.Objects;
+
+public abstract class Message {
+    @Override
+    public final boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        return Objects.equals(toString(), o.toString());
+    }
+
+    @Override
+    public final int hashCode() {
+        return toString().hashCode();
+    }
+
+    @Override
+    public final String toString() {
+        return JsonUtil.toJsonString(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java
new file mode 100644
index 0000000..3287801
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A request to the Trogdor agent to stop a task.
+ */
+public class StopTaskRequest extends Message {
+    private final String id;
+
+    @JsonCreator
+    public StopTaskRequest(@JsonProperty("id") String id) {
+        this.id = id;
+    }
+
+    @JsonProperty
+    public String id() {
+        return id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskResponse.java
new file mode 100644
index 0000000..f344dc9
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskResponse.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A response from the Trogdor coordinator about stopping a task.
+ */
+public class StopTaskResponse extends Message {
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public StopTaskResponse(@JsonProperty("spec") TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java
new file mode 100644
index 0000000..54c689a
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A request to the Trogdor agent to stop a worker.
+ */
+public class StopWorkerRequest extends Message {
+    private final String id;
+
+    @JsonCreator
+    public StopWorkerRequest(@JsonProperty("id") String id) {
+        this.id = id;
+    }
+
+    @JsonProperty
+    public String id() {
+        return id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerResponse.java
new file mode 100644
index 0000000..7d5b468
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerResponse.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * A response from the Trogdor agent about stopping a worker.
+ */
+public class StopWorkerResponse extends Message {
+    private final TaskSpec spec;
+
+    @JsonCreator
+    public StopWorkerResponse(@JsonProperty("spec") TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java
new file mode 100644
index 0000000..536d3f2
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state a task is in once it's done.
+ */
+public class TaskDone extends TaskState {
+    /**
+     * The time on the coordinator when the task was started.
+     */
+    private final long startedMs;
+
+    /**
+     * The time on the coordinator when the task was completed.
+     */
+    private final long doneMs;
+
+    /**
+     * Empty if the task completed without error; the error message otherwise.
+     */
+    private final String error;
+
+    /**
+     * True if the task was manually cancelled, rather than terminating itself.
+     */
+    private final boolean cancelled;
+
+    @JsonCreator
+    public TaskDone(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs,
+            @JsonProperty("doneMs") long doneMs,
+            @JsonProperty("error") String error,
+            @JsonProperty("cancelled") boolean cancelled) {
+        super(spec);
+        this.startedMs = startedMs;
+        this.doneMs = doneMs;
+        this.error = error;
+        this.cancelled = cancelled;
+    }
+
+    @JsonProperty
+    public long startedMs() {
+        return startedMs;
+    }
+
+    @JsonProperty
+    public long doneMs() {
+        return doneMs;
+    }
+
+    @JsonProperty
+    public String error() {
+        return error;
+    }
+
+    @JsonProperty
+    public boolean cancelled() {
+        return cancelled;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java
new file mode 100644
index 0000000..b0162d3
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state for a task which is still pending.
+ */
+public class TaskPending extends TaskState {
+    @JsonCreator
+    public TaskPending(@JsonProperty("spec") TaskSpec spec) {
+        super(spec);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java
new file mode 100644
index 0000000..bff3676
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state for a task which is being run by the agent.
+ */
+public class TaskRunning extends TaskState {
+    /**
+     * The time on the agent when the task was started.
+     */
+    private final long startedMs;
+
+    @JsonCreator
+    public TaskRunning(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs) {
+        super(spec);
+        this.startedMs = startedMs;
+    }
+
+    @JsonProperty
+    public long startedMs() {
+        return startedMs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java
new file mode 100644
index 0000000..28b6108
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state which a task is in on the Coordinator.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME,
+    include = JsonTypeInfo.As.PROPERTY,
+    property = "state")
+@JsonSubTypes({
+        @JsonSubTypes.Type(value = TaskPending.class, name = "PENDING"),
+        @JsonSubTypes.Type(value = TaskRunning.class, name = "RUNNING"),
+        @JsonSubTypes.Type(value = TaskStopping.class, name = "STOPPING"),
+        @JsonSubTypes.Type(value = TaskDone.class, name = "DONE")
+    })
+public abstract class TaskState extends Message {
+    private final TaskSpec spec;
+
+    public TaskState(TaskSpec spec) {
+        this.spec = spec;
+    }
+
+    @JsonProperty
+    public TaskSpec spec() {
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java
new file mode 100644
index 0000000..4446b75
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state for a task which is being stopped on the coordinator.
+ */
+public class TaskStopping extends TaskState {
+    /**
+     * The time on the agent when the task was received.
+     */
+    private final long startedMs;
+
+    @JsonCreator
+    public TaskStopping(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs) {
+        super(spec);
+        this.startedMs = startedMs;
+    }
+
+    @JsonProperty
+    public long startedMs() {
+        return startedMs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java
new file mode 100644
index 0000000..d3b415b
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.TreeMap;
+
+/**
+ * The response to /coordinator/tasks
+ */
+public class TasksResponse extends Message {
+    private final TreeMap<String, TaskState> tasks;
+
+    @JsonCreator
+    public TasksResponse(@JsonProperty("tasks") TreeMap<String, TaskState> tasks) {
+        this.tasks = tasks;
+    }
+
+    @JsonProperty
+    public TreeMap<String, TaskState> tasks() {
+        return tasks;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java
new file mode 100644
index 0000000..0f46b25
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * The state a worker is in once it's done.
+ */
+public class WorkerDone extends WorkerState {
+    /**
+     * The time on the agent when the task was started.
+     */
+    private final long startedMs;
+
+    /**
+     * The time on the agent when the task was completed.
+     */
+    private final long doneMs;
+
+    /**
+     * The task status.  The format will depend on the type of task that is
+     * being run.
+     */
+    private final String status;
+
+    /**
+     * Empty if the task completed without error; the error message otherwise.
+     */
+    private final String error;
+
+    @JsonCreator
+    public WorkerDone(@JsonProperty("spec") TaskSpec spec,
+            @JsonProperty("startedMs") long startedMs,
+            @JsonProperty("doneMs") long doneMs,
+            @JsonProperty("status") String status,
+            @JsonProperty("error") String error) {
+        super(spec);
+        this.startedMs = startedMs;
+        this.doneMs = doneMs;
+        this.status = status;
+        this.error = error;
+    }
+
+    @JsonProperty
+    @Override
+    public long startedMs() {
+        return startedMs;
+    }
+
+    @JsonProperty
+    public long doneMs() {
+        return doneMs;
+    }
+
+    @JsonProperty
+    @Override
+    public String status() {
+        return status;
+    }
+
+    @JsonProperty
+    public String error() {
+        return error;
+    }
+
+    @Override
+    public boolean done() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java
new file mode 100644
index 0000000..d3e3565
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.rest;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kafka.trogdor.task.TaskSpec;
+
+/**
+ * When we're in the process of sending a TaskSpec to the Agent, the Worker is regarded
+ * as being in WorkerReceiving state.
+ */
+public final class WorkerReceiving extends WorkerState {
+    @JsonCreator
+    public WorkerReceiving(@JsonProperty("spec") TaskSpec spec) {
+        super(spec);
+    }
+}


[3/4] kafka git commit: KAFKA-6060; Add workload generation capabilities to Trogdor

Posted by rs...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java b/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java
index 85270cd..6922c2e 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java
@@ -18,6 +18,7 @@
 package org.apache.kafka.trogdor.basic;
 
 import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.common.utils.Shell;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.trogdor.common.Node;
@@ -36,6 +37,7 @@ public class BasicPlatform implements Platform {
 
     private final Node curNode;
     private final BasicTopology topology;
+    private final Scheduler scheduler;
     private final CommandRunner commandRunner;
 
     public interface CommandRunner {
@@ -57,7 +59,7 @@ public class BasicPlatform implements Platform {
     }
 
     public BasicPlatform(String curNodeName, BasicTopology topology,
-                         CommandRunner commandRunner) {
+                         Scheduler scheduler, CommandRunner commandRunner) {
         this.curNode = topology.node(curNodeName);
         if (this.curNode == null) {
             throw new RuntimeException(String.format("No node named %s found " +
@@ -65,16 +67,18 @@ public class BasicPlatform implements Platform {
                 Utils.join(topology.nodes().keySet(), ",")));
         }
         this.topology = topology;
+        this.scheduler = scheduler;
         this.commandRunner = commandRunner;
     }
 
-    public BasicPlatform(String curNodeName, JsonNode configRoot) {
+    public BasicPlatform(String curNodeName, Scheduler scheduler, JsonNode configRoot) {
         JsonNode nodes = configRoot.get("nodes");
         if (nodes == null) {
             throw new RuntimeException("Expected to find a 'nodes' field " +
                 "in the root JSON configuration object");
         }
         this.topology = new BasicTopology(nodes);
+        this.scheduler = scheduler;
         this.curNode = topology.node(curNodeName);
         if (this.curNode == null) {
             throw new RuntimeException(String.format("No node named %s found " +
@@ -100,6 +104,11 @@ public class BasicPlatform implements Platform {
     }
 
     @Override
+    public Scheduler scheduler() {
+        return scheduler;
+    }
+
+    @Override
     public String runCommand(String[] command) throws IOException {
         return commandRunner.run(curNode, command);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java b/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java
index 1177ace..cb20620 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java
@@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.JsonNode;
 
 import java.io.File;
 import java.io.IOException;
+
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.common.utils.Utils;
 
 /**
@@ -47,6 +49,7 @@ public interface Platform {
             String platformName = platformNode.textValue();
             return Utils.newParameterizedInstance(platformName,
                 String.class, curNodeName,
+                Scheduler.class, Scheduler.SYSTEM,
                 JsonNode.class, root);
         }
     }
@@ -67,6 +70,11 @@ public interface Platform {
     Topology topology();
 
     /**
+     * Get the scheduler to use.
+     */
+    Scheduler scheduler();
+
+    /**
      * Run a command on this local node.
      *
      * Throws an exception if the command could not be run, or if the

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/common/ThreadUtils.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/ThreadUtils.java b/tools/src/main/java/org/apache/kafka/trogdor/common/ThreadUtils.java
new file mode 100644
index 0000000..7e02856
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/common/ThreadUtils.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.common;
+
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Utilities for working with threads.
+ */
+public class ThreadUtils {
+    /**
+     * Create a new ThreadFactory.
+     *
+     * @param pattern       The pattern to use.  If this contains %d, it will be
+     *                      replaced with a thread number.  It should not contain more
+     *                      than one %d.
+     * @param daemon        True if we want daemon threads.
+     * @return              The new ThreadFactory.
+     */
+    public static ThreadFactory createThreadFactory(final String pattern,
+                                                    final boolean daemon) {
+        return new ThreadFactory() {
+            private final AtomicLong threadEpoch = new AtomicLong(0);
+
+            @Override
+            public Thread newThread(Runnable r) {
+                String threadName;
+                if (pattern.contains("%d")) {
+                    threadName = String.format(pattern, threadEpoch.addAndGet(1));
+                } else {
+                    threadName = pattern;
+                }
+                Thread thread = new Thread(r, threadName);
+                thread.setDaemon(daemon);
+                return thread;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java b/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java
index 94a5474..d48bbde 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java
@@ -17,12 +17,30 @@
 
 package org.apache.kafka.trogdor.common;
 
+import java.util.HashSet;
+import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Set;
 
 /**
  * Defines a cluster topology
  */
 public interface Topology {
+    class Util {
+        /**
+         * Get the names of agent nodes in the topology.
+         */
+        public static Set<String> agentNodeNames(Topology topology) {
+            Set<String> set = new HashSet<>();
+            for (Map.Entry<String, Node> entry : topology.nodes().entrySet()) {
+                if (Node.Util.getTrogdorAgentPort(entry.getValue()) > 0) {
+                    set.add(entry.getKey());
+                }
+            }
+            return set;
+        }
+    }
+
     /**
      * Get the node with the given name.
      */

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java
index 8f3563b..8c26d8d 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java
@@ -22,34 +22,19 @@ import net.sourceforge.argparse4j.inf.ArgumentParser;
 import net.sourceforge.argparse4j.inf.ArgumentParserException;
 import net.sourceforge.argparse4j.inf.Namespace;
 import org.apache.kafka.common.utils.Exit;
-import org.apache.kafka.common.utils.KafkaThread;
-import org.apache.kafka.common.utils.Time;
-import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.trogdor.common.Node;
 import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.fault.DoneState;
-import org.apache.kafka.trogdor.fault.Fault;
-import org.apache.kafka.trogdor.fault.FaultSet;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-import org.apache.kafka.trogdor.fault.SendingState;
-import org.apache.kafka.trogdor.rest.CoordinatorFaultsResponse;
-import org.apache.kafka.trogdor.rest.CreateCoordinatorFaultRequest;
+import org.apache.kafka.trogdor.rest.CoordinatorStatusResponse;
+import org.apache.kafka.trogdor.rest.CreateTaskRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskResponse;
 import org.apache.kafka.trogdor.rest.JsonRestServer;
+import org.apache.kafka.trogdor.rest.StopTaskRequest;
+import org.apache.kafka.trogdor.rest.StopTaskResponse;
+import org.apache.kafka.trogdor.rest.TasksResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
 import static net.sourceforge.argparse4j.impl.Arguments.store;
 
 /**
@@ -61,39 +46,14 @@ public final class Coordinator {
     private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
 
     /**
-     * The clock to use for this coordinator.
-     */
-    private final Time time;
-
-    /**
-     * The start time in milliseconds.
+     * The start time of the Coordinator in milliseconds.
      */
     private final long startTimeMs;
 
     /**
-     * The platform.
+     * The task manager.
      */
-    private final Platform platform;
-
-    /**
-     * NodeManager objects for each node in the cluster.
-     */
-    private final Map<String, NodeManager> nodeManagers;
-
-    /**
-     * The lock protecting shutdown and faultQueue.
-     */
-    private final ReentrantLock lock = new ReentrantLock();
-
-    /**
-     * The condition variable which the coordinator thread waits on.
-     */
-    private final Condition cond = lock.newCondition();
-
-    /**
-     * The coordinator runnable.
-     */
-    private final CoordinatorRunnable runnable;
+    private final TaskManager taskManager;
 
     /**
      * The REST server.
@@ -101,83 +61,6 @@ public final class Coordinator {
     private final JsonRestServer restServer;
 
     /**
-     * The coordinator thread.
-     */
-    private final KafkaThread thread;
-
-    /**
-     * True if the server is shutting down.
-     */
-    private boolean shutdown = false;
-
-    /**
-     * The set of faults which have been scheduled.
-     */
-    private final FaultSet pendingFaults = new FaultSet();
-
-    /**
-     * The set of faults which have been sent to the NodeManagers.
-     */
-    private final FaultSet processedFaults = new FaultSet();
-
-    class CoordinatorRunnable implements Runnable {
-        @Override
-        public void run() {
-            log.info("Starting main service thread.");
-            try {
-                long nextWakeMs = 0;
-                while (true) {
-                    long now = time.milliseconds();
-                    List<Fault> toStart = new ArrayList<>();
-                    lock.lock();
-                    try {
-                        if (shutdown) {
-                            break;
-                        }
-                        if (nextWakeMs > now) {
-                            if (cond.await(nextWakeMs - now, TimeUnit.MILLISECONDS)) {
-                                log.trace("CoordinatorRunnable woke up early.");
-                            }
-                            now = time.milliseconds();
-                            if (shutdown) {
-                                break;
-                            }
-                        }
-                        nextWakeMs = now + (60L * 60L * 1000L);
-                        Iterator<Fault> iter = pendingFaults.iterateByStart();
-                        while (iter.hasNext()) {
-                            Fault fault = iter.next();
-                            if (now < fault.spec().startMs()) {
-                                nextWakeMs = Math.min(nextWakeMs, fault.spec().startMs());
-                                break;
-                            }
-                            toStart.add(fault);
-                            iter.remove();
-                            processedFaults.add(fault);
-                        }
-                    } finally {
-                        lock.unlock();
-                    }
-                    for (Fault fault: toStart) {
-                        startFault(now, fault);
-                    }
-                }
-            } catch (Throwable t) {
-                log.error("CoordinatorRunnable shutting down with exception", t);
-            } finally {
-                log.info("CoordinatorRunnable shutting down.");
-                restServer.stop();
-                for (NodeManager nodeManager : nodeManagers.values()) {
-                    nodeManager.beginShutdown();
-                }
-                for (NodeManager nodeManager : nodeManagers.values()) {
-                    nodeManager.waitForShutdown();
-                }
-            }
-        }
-    }
-
-    /**
      * Create a new Coordinator.
      *
      * @param platform      The platform object to use.
@@ -185,24 +68,11 @@ public final class Coordinator {
      * @param restServer    The REST server to use.
      * @param resource      The AgentRestResoure to use.
      */
-    public Coordinator(Platform platform, Time time, JsonRestServer restServer,
+    public Coordinator(Platform platform, Scheduler scheduler, JsonRestServer restServer,
                        CoordinatorRestResource resource) {
-        this.platform = platform;
-        this.time = time;
-        this.startTimeMs = time.milliseconds();
-        this.runnable = new CoordinatorRunnable();
+        this.startTimeMs = scheduler.time().milliseconds();
+        this.taskManager = new TaskManager(platform, scheduler);
         this.restServer = restServer;
-        this.nodeManagers = new HashMap<>();
-        for (Node node : platform.topology().nodes().values()) {
-            if (Node.Util.getTrogdorAgentPort(node) > 0) {
-                this.nodeManagers.put(node.name(), new NodeManager(time, node));
-            }
-        }
-        if (this.nodeManagers.isEmpty()) {
-            log.warn("No agent nodes configured.");
-        }
-        this.thread = new KafkaThread("TrogdorCoordinatorThread", runnable, false);
-        this.thread.start();
         resource.setCoordinator(this);
     }
 
@@ -210,94 +80,32 @@ public final class Coordinator {
         return this.restServer.port();
     }
 
-    private void startFault(long now, Fault fault) {
-        Set<String> affectedNodes = fault.targetNodes(platform.topology());
-        Set<NodeManager> affectedManagers = new HashSet<>();
-        Set<String> nonexistentNodes = new HashSet<>();
-        Set<String> nodeNames = new HashSet<>();
-        for (String affectedNode : affectedNodes) {
-            NodeManager nodeManager = nodeManagers.get(affectedNode);
-            if (nodeManager == null) {
-                nonexistentNodes.add(affectedNode);
-            } else {
-                affectedManagers.add(nodeManager);
-                nodeNames.add(affectedNode);
-            }
-        }
-        if (!nonexistentNodes.isEmpty()) {
-            log.warn("Fault {} refers to {} non-existent node(s): {}", fault.id(),
-                    nonexistentNodes.size(), Utils.join(nonexistentNodes, ", "));
-        }
-        log.info("Applying fault {} on {} node(s): {}", fault.id(),
-                nodeNames.size(), Utils.join(nodeNames, ", "));
-        if (nodeNames.isEmpty()) {
-            fault.setState(new DoneState(now, ""));
-        } else {
-            fault.setState(new SendingState(nodeNames));
-        }
-        for (NodeManager nodeManager : affectedManagers) {
-            nodeManager.enqueueFault(fault);
-        }
-    }
-
-    public void beginShutdown() {
-        lock.lock();
-        try {
-            this.shutdown = true;
-            cond.signalAll();
-        } finally {
-            lock.unlock();
-        }
+    public CoordinatorStatusResponse status() throws Exception {
+        return new CoordinatorStatusResponse(startTimeMs);
     }
 
-    public void waitForShutdown() {
-        try {
-            this.thread.join();
-        } catch (InterruptedException e) {
-            log.error("Interrupted while waiting for thread shutdown", e);
-            Thread.currentThread().interrupt();
-        }
+    public CreateTaskResponse createTask(CreateTaskRequest request) throws Exception {
+        return new CreateTaskResponse(taskManager.createTask(request.id(), request.spec()));
     }
 
-    public long startTimeMs() {
-        return startTimeMs;
+    public StopTaskResponse stopTask(StopTaskRequest request) throws Exception {
+        return new StopTaskResponse(taskManager.stopTask(request.id()));
     }
 
-    public CoordinatorFaultsResponse getFaults() {
-        Map<String, CoordinatorFaultsResponse.FaultData> faultData = new TreeMap<>();
-        lock.lock();
-        try {
-            getFaultsImpl(faultData, pendingFaults);
-            getFaultsImpl(faultData, processedFaults);
-        } finally {
-            lock.unlock();
-        }
-        return new CoordinatorFaultsResponse(faultData);
+    public TasksResponse tasks() throws Exception {
+        return taskManager.tasks();
     }
 
-    private void getFaultsImpl(Map<String, CoordinatorFaultsResponse.FaultData> faultData,
-                               FaultSet faultSet) {
-        for (Iterator<Fault> iter = faultSet.iterateByStart();
-             iter.hasNext(); ) {
-            Fault fault = iter.next();
-            CoordinatorFaultsResponse.FaultData data =
-                new CoordinatorFaultsResponse.FaultData(fault.spec(), fault.state());
-            faultData.put(fault.id(), data);
-        }
+    public void beginShutdown(boolean stopAgents) throws Exception {
+        restServer.beginShutdown();
+        taskManager.beginShutdown(stopAgents);
     }
 
-    public void createFault(CreateCoordinatorFaultRequest request) throws ClassNotFoundException {
-        lock.lock();
-        try {
-            Fault fault = FaultSpec.Util.createFault(request.id(), request.spec());
-            pendingFaults.add(fault);
-            cond.signalAll();
-        } finally {
-            lock.unlock();
-        }
+    public void waitForShutdown() throws Exception {
+        restServer.waitForShutdown();
+        taskManager.waitForShutdown();
     }
 
-
     public static void main(String[] args) throws Exception {
         ArgumentParser parser = ArgumentParsers
             .newArgumentParser("trogdor-coordinator")
@@ -336,15 +144,20 @@ public final class Coordinator {
         JsonRestServer restServer = new JsonRestServer(
             Node.Util.getTrogdorCoordinatorPort(platform.curNode()));
         CoordinatorRestResource resource = new CoordinatorRestResource();
-        final Coordinator coordinator = new Coordinator(platform, Time.SYSTEM,
+        log.info("Starting coordinator process.");
+        final Coordinator coordinator = new Coordinator(platform, Scheduler.SYSTEM,
             restServer, resource);
         restServer.start(resource);
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
-                log.error("Running shutdown hook...");
-                coordinator.beginShutdown();
-                coordinator.waitForShutdown();
+                log.warn("Running coordinator shutdown hook.");
+                try {
+                    coordinator.beginShutdown(false);
+                    coordinator.waitForShutdown();
+                } catch (Exception e) {
+                    log.error("Got exception while running coordinator shutdown hook.", e);
+                }
             }
         });
         coordinator.waitForShutdown();

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java
index 1137d08..821a76b 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java
@@ -25,12 +25,15 @@ import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup;
 import net.sourceforge.argparse4j.inf.Namespace;
 import org.apache.kafka.common.utils.Exit;
 import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.rest.CoordinatorFaultsResponse;
 import org.apache.kafka.trogdor.rest.CoordinatorStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateCoordinatorFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskResponse;
 import org.apache.kafka.trogdor.rest.Empty;
 import org.apache.kafka.trogdor.rest.JsonRestServer;
 import org.apache.kafka.trogdor.rest.JsonRestServer.HttpResponse;
+import org.apache.kafka.trogdor.rest.StopTaskRequest;
+import org.apache.kafka.trogdor.rest.StopTaskResponse;
+import org.apache.kafka.trogdor.rest.TasksResponse;
 
 import static net.sourceforge.argparse4j.impl.Arguments.store;
 import static net.sourceforge.argparse4j.impl.Arguments.storeTrue;
@@ -40,47 +43,64 @@ import static net.sourceforge.argparse4j.impl.Arguments.storeTrue;
  */
 public class CoordinatorClient {
     /**
+     * The maximum number of tries to make.
+     */
+    private final int maxTries;
+
+    /**
      * The URL target.
      */
     private final String target;
 
-    public CoordinatorClient(String host, int port) {
-        this(String.format("%s:%d", host, port));
+    public CoordinatorClient(int maxTries, String host, int port) {
+        this(maxTries, String.format("%s:%d", host, port));
     }
 
-    public CoordinatorClient(String target) {
+    public CoordinatorClient(int maxTries, String target) {
+        this.maxTries = maxTries;
         this.target = target;
     }
 
+    public int maxTries() {
+        return maxTries;
+    }
+
     private String url(String suffix) {
         return String.format("http://%s%s", target, suffix);
     }
 
-    public CoordinatorStatusResponse getStatus() throws Exception {
+    public CoordinatorStatusResponse status() throws Exception {
         HttpResponse<CoordinatorStatusResponse> resp =
             JsonRestServer.<CoordinatorStatusResponse>httpRequest(url("/coordinator/status"), "GET",
-                null, new TypeReference<CoordinatorStatusResponse>() { });
+                null, new TypeReference<CoordinatorStatusResponse>() { }, maxTries);
         return resp.body();
     }
 
-    public CoordinatorFaultsResponse getFaults() throws Exception {
-        HttpResponse<CoordinatorFaultsResponse> resp =
-            JsonRestServer.<CoordinatorFaultsResponse>httpRequest(url("/coordinator/faults"), "GET",
-                null, new TypeReference<CoordinatorFaultsResponse>() { });
+    public CreateTaskResponse createTask(CreateTaskRequest request) throws Exception {
+        HttpResponse<CreateTaskResponse> resp =
+            JsonRestServer.<CreateTaskResponse>httpRequest(url("/coordinator/task/create"), "POST",
+                request, new TypeReference<CreateTaskResponse>() { }, maxTries);
         return resp.body();
     }
 
-    public void putFault(CreateCoordinatorFaultRequest request) throws Exception {
-        HttpResponse<CreateCoordinatorFaultRequest> resp =
-            JsonRestServer.<CreateCoordinatorFaultRequest>httpRequest(url("/coordinator/fault"), "PUT",
-                request, new TypeReference<CreateCoordinatorFaultRequest>() { });
-        resp.body();
+    public StopTaskResponse stopTask(StopTaskRequest request) throws Exception {
+        HttpResponse<StopTaskResponse> resp =
+            JsonRestServer.<StopTaskResponse>httpRequest(url("/coordinator/task/stop"), "PUT",
+                request, new TypeReference<StopTaskResponse>() { }, maxTries);
+        return resp.body();
+    }
+
+    public TasksResponse tasks() throws Exception {
+        HttpResponse<TasksResponse> resp =
+            JsonRestServer.<TasksResponse>httpRequest(url("/coordinator/tasks"), "GET",
+                null, new TypeReference<TasksResponse>() { }, maxTries);
+        return resp.body();
     }
 
     public void shutdown() throws Exception {
         HttpResponse<Empty> resp =
             JsonRestServer.<Empty>httpRequest(url("/coordinator/shutdown"), "PUT",
-                null, new TypeReference<Empty>() { });
+                null, new TypeReference<Empty>() { }, maxTries);
         resp.body();
     }
 
@@ -102,17 +122,23 @@ public class CoordinatorClient {
             .type(Boolean.class)
             .dest("status")
             .help("Get coordinator status.");
-        actions.addArgument("--get-faults")
+        actions.addArgument("--show-tasks")
             .action(storeTrue())
             .type(Boolean.class)
-            .dest("get_faults")
-            .help("Get coordinator faults.");
-        actions.addArgument("--create-fault")
+            .dest("show_tasks")
+            .help("Show coordinator tasks.");
+        actions.addArgument("--create-task")
+            .action(store())
+            .type(String.class)
+            .dest("create_task")
+            .metavar("TASK_SPEC_JSON")
+            .help("Create a new task from a task spec.");
+        actions.addArgument("--stop-task")
             .action(store())
             .type(String.class)
-            .dest("create_fault")
-            .metavar("FAULT_JSON")
-            .help("Create a new fault.");
+            .dest("stop_task")
+            .metavar("TASK_ID")
+            .help("Stop a task.");
         actions.addArgument("--shutdown")
             .action(storeTrue())
             .type(Boolean.class)
@@ -132,17 +158,20 @@ public class CoordinatorClient {
             }
         }
         String target = res.getString("target");
-        CoordinatorClient client = new CoordinatorClient(target);
+        CoordinatorClient client = new CoordinatorClient(3, target);
         if (res.getBoolean("status")) {
             System.out.println("Got coordinator status: " +
-                JsonUtil.toPrettyJsonString(client.getStatus()));
-        } else if (res.getBoolean("get_faults")) {
-            System.out.println("Got coordinator faults: " +
-                JsonUtil.toPrettyJsonString(client.getFaults()));
-        } else if (res.getString("create_fault") != null) {
-            client.putFault(JsonUtil.JSON_SERDE.readValue(res.getString("create_fault"),
-                CreateCoordinatorFaultRequest.class));
-            System.out.println("Created fault.");
+                JsonUtil.toPrettyJsonString(client.status()));
+        } else if (res.getBoolean("show_tasks")) {
+            System.out.println("Got coordinator tasks: " +
+                JsonUtil.toPrettyJsonString(client.tasks()));
+        } else if (res.getString("create_task") != null) {
+            client.createTask(JsonUtil.JSON_SERDE.readValue(res.getString("create_task"),
+                CreateTaskRequest.class));
+            System.out.println("Created task.");
+        } else if (res.getString("stop_task") != null) {
+            client.stopTask(new StopTaskRequest(res.getString("stop_task")));
+            System.out.println("Created task.");
         } else if (res.getBoolean("shutdown")) {
             client.shutdown();
             System.out.println("Sent shutdown request.");

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java
index 60357b8..7775dd0 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java
@@ -16,14 +16,19 @@
  */
 package org.apache.kafka.trogdor.coordinator;
 
-import org.apache.kafka.trogdor.rest.CoordinatorFaultsResponse;
+import org.apache.kafka.trogdor.rest.CoordinatorShutdownRequest;
 import org.apache.kafka.trogdor.rest.CoordinatorStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateCoordinatorFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskRequest;
+import org.apache.kafka.trogdor.rest.CreateTaskResponse;
 import org.apache.kafka.trogdor.rest.Empty;
+import org.apache.kafka.trogdor.rest.StopTaskRequest;
+import org.apache.kafka.trogdor.rest.StopTaskResponse;
+import org.apache.kafka.trogdor.rest.TasksResponse;
 
 import javax.servlet.ServletContext;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -46,27 +51,32 @@ public class CoordinatorRestResource {
 
     @GET
     @Path("/status")
-    public CoordinatorStatusResponse getStatus() throws Throwable {
-        return new CoordinatorStatusResponse(coordinator().startTimeMs());
+    public CoordinatorStatusResponse status() throws Throwable {
+        return coordinator().status();
     }
 
-    @GET
-    @Path("/faults")
-    public CoordinatorFaultsResponse getCoordinatorFaults() throws Throwable {
-        return coordinator().getFaults();
+    @POST
+    @Path("/task/create")
+    public CreateTaskResponse createTask(CreateTaskRequest request) throws Throwable {
+        return coordinator().createTask(request);
     }
 
     @PUT
-    @Path("/fault")
-    public Empty putCoordinatorFault(CreateCoordinatorFaultRequest request) throws Throwable {
-        coordinator().createFault(request);
-        return Empty.INSTANCE;
+    @Path("/task/stop")
+    public StopTaskResponse stopTask(StopTaskRequest request) throws Throwable {
+        return coordinator().stopTask(request);
+    }
+
+    @GET
+    @Path("/tasks")
+    public TasksResponse tasks() throws Throwable {
+        return coordinator().tasks();
     }
 
     @PUT
     @Path("/shutdown")
-    public Empty shutdown() throws Throwable {
-        coordinator().beginShutdown();
+    public Empty beginShutdown(CoordinatorShutdownRequest request) throws Throwable {
+        coordinator().beginShutdown(request.stopAgents());
         return Empty.INSTANCE;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java
index ee71190..aea9617 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java
@@ -15,254 +15,315 @@
  * limitations under the License.
  */
 
+/*
+ * So, when a task comes in, it happens via createTask (the RPC backend).
+ * This starts a CreateTask on the main state change thread, and waits for it.
+ * That task checks the main task hash map, and returns back the existing task spec
+ * if there is something there.  If there is nothing there, it creates
+ * something new, and returns null.
+ * It also schedules a RunTask some time in the future on the main state change thread.
+ * We save the future from this in case we need to cancel it later, in a StopTask.
+ * If we can't create the TaskController for the task, we transition to DONE with an
+ * appropriate error message.
+ *
+ * RunTask actually starts the task which was created earlier.  This could
+ * happen an arbitrary amount of time after task creation (it is based on the
+ * task spec).  RunTask must operate only on PENDING tasks... if the task has been
+ * stopped, then we have nothing to do here.
+ * RunTask asks the TaskController for a list of all the names of nodes
+ * affected by this task.
+ * If this list contains nodes we don't know about, or zero nodes, we
+ * transition directly to DONE state with an appropriate error set.
+ * RunTask schedules CreateWorker Callables on all the affected worker nodes.
+ * These callables run in the context of the relevant NodeManager.
+ *
+ * CreateWorker calls the RPC of the same name for the agent.
+ * There is some complexity here due to retries.
+ */
+
 package org.apache.kafka.trogdor.coordinator;
 
-import org.apache.kafka.common.utils.KafkaThread;
-import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.trogdor.agent.AgentClient;
 import org.apache.kafka.trogdor.common.Node;
-import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.fault.DoneState;
-import org.apache.kafka.trogdor.fault.Fault;
-import org.apache.kafka.trogdor.fault.SendingState;
+import org.apache.kafka.trogdor.common.ThreadUtils;
 import org.apache.kafka.trogdor.rest.AgentStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateAgentFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerRequest;
+import org.apache.kafka.trogdor.rest.StopWorkerRequest;
+import org.apache.kafka.trogdor.rest.WorkerDone;
+import org.apache.kafka.trogdor.rest.WorkerReceiving;
+import org.apache.kafka.trogdor.rest.WorkerRunning;
+import org.apache.kafka.trogdor.rest.WorkerStarting;
+import org.apache.kafka.trogdor.rest.WorkerState;
+import org.apache.kafka.trogdor.task.TaskSpec;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.net.ConnectException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 
-class NodeManager {
+/**
+ * The NodeManager handles communicating with a specific agent node.
+ * Each NodeManager has its own ExecutorService which runs in a dedicated thread.
+ */
+public final class NodeManager {
     private static final Logger log = LoggerFactory.getLogger(NodeManager.class);
 
     /**
-     * The Time object used to fetch the current time.
+     * The normal amount of seconds between heartbeats sent to the agent.
      */
-    private final Time time;
+    private static final long HEARTBEAT_DELAY_MS = 1000L;
 
-    /**
-     * The node which is being managed.
-     */
-    private final Node node;
+    class ManagedWorker {
+        private final String id;
+        private final TaskSpec spec;
+        private boolean shouldRun;
+        private WorkerState state;
 
-    /**
-     * The client for the node being managed.
-     */
-    private final AgentClient client;
+        ManagedWorker(String id, TaskSpec spec, boolean shouldRun, WorkerState state) {
+            this.id = id;
+            this.spec = spec;
+            this.shouldRun = shouldRun;
+            this.state = state;
+        }
 
-    /**
-     * The maximum amount of time to go without contacting the node.
-     */
-    private final long heartbeatMs;
+        void tryCreate() {
+            try {
+                client.createWorker(new CreateWorkerRequest(id, spec));
+            } catch (Throwable e) {
+                log.error("{}: error creating worker {}.", node.name(), id, e);
+            }
+        }
 
-    /**
-     * True if the NodeManager is shutting down.  Protected by the queueLock.
-     */
-    private boolean shutdown = false;
+        void tryStop() {
+            try {
+                client.stopWorker(new StopWorkerRequest(id));
+            } catch (Throwable e) {
+                log.error("{}: error stopping worker {}.", node.name(), id, e);
+            }
+        }
+    }
 
     /**
-     * The Node Manager runnable.
+     * The node which we are managing.
      */
-    private final NodeManagerRunnable runnable;
+    private final Node node;
 
     /**
-     * The Node Manager thread.
+     * The task manager.
      */
-    private final KafkaThread thread;
+    private final TaskManager taskManager;
 
     /**
-     * The lock protecting the NodeManager fields.
+     * A client for the Node's Agent.
      */
-    private final Lock lock = new ReentrantLock();
+    private final AgentClient client;
 
     /**
-     * The condition variable used to wake the thread when it is waiting for a
-     * queue or shutdown change.
+     * Maps task IDs to worker structures.
      */
-    private final Condition cond = lock.newCondition();
+    private final Map<String, ManagedWorker> workers;
 
     /**
-     * A queue of faults which should be sent to this node.  Protected by the lock.
+     * An executor service which manages the thread dedicated to this node.
      */
-    private final List<Fault> faultQueue = new ArrayList<>();
+    private final ScheduledExecutorService executor;
 
     /**
-     * The last time we successfully contacted the node.  Protected by the lock.
+     * The heartbeat runnable.
      */
-    private long lastContactMs = 0;
+    private final NodeHeartbeat heartbeat;
 
     /**
-     * The current status of this node.
+     * A future which can be used to cancel the periodic hearbeat task.
      */
-    public static class NodeStatus {
-        private final String nodeName;
-        private final long lastContactMs;
+    private ScheduledFuture<?> heartbeatFuture;
 
-        NodeStatus(String nodeName, long lastContactMs) {
-            this.nodeName = nodeName;
-            this.lastContactMs = lastContactMs;
-        }
-
-        public String nodeName() {
-            return nodeName;
-        }
+    NodeManager(Node node, TaskManager taskManager) {
+        this.node = node;
+        this.taskManager = taskManager;
+        this.client = new AgentClient(1, node.hostname(), Node.Util.getTrogdorAgentPort(node));
+        this.workers = new HashMap<>();
+        this.executor = Executors.newSingleThreadScheduledExecutor(
+            ThreadUtils.createThreadFactory("NodeManager(" + node.name() + ")",
+                false));
+        this.heartbeat = new NodeHeartbeat();
+        rescheduleNextHeartbeat(HEARTBEAT_DELAY_MS);
+    }
 
-        public long lastContactMs() {
-            return lastContactMs;
+    /**
+     * Reschedule the heartbeat runnable.
+     *
+     * @param initialDelayMs        The initial delay to use.
+     */
+    void rescheduleNextHeartbeat(long initialDelayMs) {
+        if (this.heartbeatFuture != null) {
+            this.heartbeatFuture.cancel(false);
         }
+        this.heartbeatFuture = this.executor.scheduleAtFixedRate(heartbeat,
+            initialDelayMs, HEARTBEAT_DELAY_MS, TimeUnit.MILLISECONDS);
     }
 
-    class NodeManagerRunnable implements Runnable {
+    /**
+     * The heartbeat runnable.
+     */
+    class NodeHeartbeat implements Runnable {
         @Override
         public void run() {
+            rescheduleNextHeartbeat(HEARTBEAT_DELAY_MS);
             try {
-                Fault fault = null;
-                long lastCommAttemptMs = 0;
-                while (true) {
-                    long now = time.milliseconds();
-                    if (fault != null) {
-                        lastCommAttemptMs = now;
-                        if (sendFault(now, fault)) {
-                            fault = null;
+                AgentStatusResponse agentStatus = null;
+                try {
+                    agentStatus = client.status();
+                } catch (ConnectException e) {
+                    log.error("{}: failed to get agent status: ConnectException {}", node.name(), e.getMessage());
+                    return;
+                } catch (Exception e) {
+                    log.error("{}: failed to get agent status", node.name(), e);
+                    // TODO: eventually think about putting tasks into a bad state as a result of
+                    // agents going down?
+                    return;
+                }
+                // Identify workers which we think should be running, but which do not appear
+                // in the agent's response.  We need to send startWorker requests for these.
+                for (Map.Entry<String, ManagedWorker> entry : workers.entrySet()) {
+                    String id = entry.getKey();
+                    if (!agentStatus.workers().containsKey(id)) {
+                        ManagedWorker worker = entry.getValue();
+                        if (worker.shouldRun) {
+                            worker.tryCreate();
                         }
                     }
-                    long nextCommAttemptMs = lastCommAttemptMs + heartbeatMs;
-                    if (now < nextCommAttemptMs) {
-                        lastCommAttemptMs = now;
-                        sendHeartbeat(now);
+                }
+                // Identify tasks which are running, but which we don't know about.
+                // Add these to the NodeManager as tasks that should not be running.
+                for (Map.Entry<String, WorkerState> entry : agentStatus.workers().entrySet()) {
+                    String id = entry.getKey();
+                    WorkerState state = entry.getValue();
+                    if (!workers.containsKey(id)) {
+                        log.warn("{}: scheduling unknown worker {} for stopping.", node.name(), id);
+                        workers.put(id, new ManagedWorker(id, state.spec(), false, state));
                     }
-                    long waitMs = Math.max(0L, nextCommAttemptMs - now);
-                    lock.lock();
-                    try {
-                        if (shutdown) {
-                            return;
-                        }
-                        try {
-                            cond.await(waitMs, TimeUnit.MILLISECONDS);
-                        } catch (InterruptedException e) {
-                            log.info("{}: NodeManagerRunnable got InterruptedException", node.name());
-                            Thread.currentThread().interrupt();
+                }
+                // Handle workers which need to be stopped.  Handle workers which have newly completed.
+                for (Map.Entry<String, WorkerState> entry : agentStatus.workers().entrySet()) {
+                    String id = entry.getKey();
+                    WorkerState state = entry.getValue();
+                    ManagedWorker worker = workers.get(id);
+                    if (state instanceof WorkerStarting || state instanceof WorkerRunning) {
+                        if (!worker.shouldRun) {
+                            worker.tryStop();
                         }
-                        if (fault == null) {
-                            if (!faultQueue.isEmpty()) {
-                                fault = faultQueue.remove(0);
+                    } else if (state instanceof WorkerDone) {
+                        if (!(worker.state instanceof WorkerDone)) {
+                            String error = ((WorkerDone) state).error();
+                            if (error.isEmpty()) {
+                                log.warn("{}: Worker {} finished with no error.", node.name(), id);
+                            } else {
+                                log.warn("{}: Worker {} finished with error '{}'", node.name(), id, error);
                             }
+                            taskManager.handleWorkerCompletion(node.name(), worker.id, error);
                         }
-                    } finally {
-                        lock.unlock();
                     }
+                    worker.state = state;
                 }
             } catch (Throwable e) {
-                log.warn("{}: exiting NodeManagerRunnable with exception", node.name(), e);
-            } finally {
+                log.error("{}: Unhandled exception in NodeHeartbeatRunnable", node.name(), e);
             }
         }
     }
 
-    NodeManager(Time time, Node node) {
-        this.time = time;
-        this.node = node;
-        this.client = new AgentClient(node.hostname(), Node.Util.getTrogdorAgentPort(node));
-        this.heartbeatMs = Node.Util.getIntConfig(node,
-                Platform.Config.TROGDOR_COORDINATOR_HEARTBEAT_MS,
-                Platform.Config.TROGDOR_COORDINATOR_HEARTBEAT_MS_DEFAULT);
-        this.runnable = new NodeManagerRunnable();
-        this.thread = new KafkaThread("NodeManagerThread(" + node.name() + ")", runnable, false);
-        this.thread.start();
+    /**
+     * Create a new worker.
+     *
+     * @param id                    The new worker id.
+     * @param spec                  The task specification to use with the new worker.
+     */
+    public void createWorker(String id, TaskSpec spec) {
+        executor.submit(new CreateWorker(id, spec));
     }
 
-    private boolean sendFault(long now, Fault fault) {
-        try {
-            client.putFault(new CreateAgentFaultRequest(fault.id(), fault.spec()));
-        } catch (Exception e) {
-            log.warn("{}: error sending fault to {}.", node.name(), client.target(), e);
-            return false;
-        }
-        lock.lock();
-        try {
-            lastContactMs = now;
-        } finally {
-            lock.unlock();
-        }
-        SendingState state = (SendingState) fault.state();
-        if (state.completeSend(node.name())) {
-            fault.setState(new DoneState(now, ""));
-        }
-        return true;
-    }
+    /**
+     * Starts a worker.
+     */
+    class CreateWorker implements Callable<Void> {
+        private final String id;
+        private final TaskSpec spec;
 
-    private void sendHeartbeat(long now) {
-        AgentStatusResponse status = null;
-        try {
-            status = client.getStatus();
-        } catch (Exception e) {
-            log.warn("{}: error sending heartbeat to {}.", node.name(), client.target(), e);
-            return;
+        CreateWorker(String id, TaskSpec spec) {
+            this.id = id;
+            this.spec = spec;
         }
-        lock.lock();
-        try {
-            lastContactMs = now;
-        } finally {
-            lock.unlock();
-        }
-        log.debug("{}: got heartbeat status {}.", node.name(), status);
-    }
 
-    public void beginShutdown() {
-        lock.lock();
-        try {
-            if (shutdown)
-                return;
-            log.trace("{}: beginning shutdown.", node.name());
-            shutdown = true;
-            cond.signalAll();
-        } finally {
-            lock.unlock();
-        }
-    }
-
-    public void waitForShutdown() {
-        log.trace("waiting for NodeManager({}) shutdown.", node.name());
-        try {
-            thread.join();
-        } catch (InterruptedException e) {
-            log.error("{}: Interrupted while waiting for thread shutdown", node.name(), e);
-            Thread.currentThread().interrupt();
+        @Override
+        public Void call() throws Exception {
+            ManagedWorker worker = workers.get(id);
+            if (worker != null) {
+                log.error("{}: there is already a worker for task {}.", node.name(), id);
+                return null;
+            }
+            log.info("{}: scheduling worker {} to start.", node.name(), id);
+            workers.put(id, new ManagedWorker(id, spec, true, new WorkerReceiving(spec)));
+            rescheduleNextHeartbeat(0);
+            return null;
         }
     }
 
     /**
-     * Get the current status of this node.
+     * Stop a worker.
      *
-     * @return                  The node status.
+     * @param id                    The id of the worker to stop.
      */
-    public NodeStatus status() {
-        lock.lock();
-        try {
-            return new NodeStatus(node.name(), lastContactMs);
-        } finally {
-            lock.unlock();
-        }
+    public void stopWorker(String id) {
+        executor.submit(new StopWorker(id));
     }
 
     /**
-     * Enqueue a new fault.
-     *
-     * @param fault             The fault to enqueue.
+     * Stops a worker.
      */
-    public void enqueueFault(Fault fault) {
-        lock.lock();
-        try {
-            log.trace("{}: added {} to fault queue.", node.name(), fault);
-            faultQueue.add(fault);
-            cond.signalAll();
-        } finally {
-            lock.unlock();
+    class StopWorker implements Callable<Void> {
+        private final String id;
+
+        StopWorker(String id) {
+            this.id = id;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            ManagedWorker worker = workers.get(id);
+            if (worker == null) {
+                log.error("{}: can't stop non-existent worker {}.", node.name(), id);
+                return null;
+            }
+            if (!worker.shouldRun) {
+                log.error("{}: The worker for task {} is already scheduled to stop.",
+                    node.name(), id);
+                return null;
+            }
+            log.info("{}: scheduling worker {} on {} to stop.", node.name(), id);
+            worker.shouldRun = false;
+            rescheduleNextHeartbeat(0);
+            return null;
         }
     }
+
+    public void beginShutdown(boolean stopNode) {
+        executor.shutdownNow();
+        if (stopNode) {
+            try {
+                client.invokeShutdown();
+            } catch (Exception e) {
+                log.error("{}: Failed to send shutdown request", node.name(), e);
+            }
+        }
+    }
+
+    public void waitForShutdown() throws InterruptedException {
+        executor.awaitTermination(1, TimeUnit.DAYS);
+    }
 };

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java
new file mode 100644
index 0000000..547c9da
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java
@@ -0,0 +1,535 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.coordinator;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.utils.Scheduler;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.trogdor.common.Node;
+import org.apache.kafka.trogdor.common.Platform;
+import org.apache.kafka.trogdor.common.ThreadUtils;
+import org.apache.kafka.trogdor.rest.TaskDone;
+import org.apache.kafka.trogdor.rest.TaskPending;
+import org.apache.kafka.trogdor.rest.TaskRunning;
+import org.apache.kafka.trogdor.rest.TaskState;
+import org.apache.kafka.trogdor.rest.TaskStopping;
+import org.apache.kafka.trogdor.rest.TasksResponse;
+import org.apache.kafka.trogdor.task.TaskController;
+import org.apache.kafka.trogdor.task.TaskSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * The TaskManager is responsible for managing tasks inside the Trogdor coordinator.
+ *
+ * The task manager has a single thread, managed by the executor.  We start, stop,
+ * and handle state changes to tasks by adding requests to the executor queue.
+ * Because the executor is single threaded, no locks are needed when accessing
+ * TaskManager data structures.
+ *
+ * The TaskManager maintains a state machine for each task.  Tasks begin in the
+ * PENDING state, waiting for their designated start time to arrive.
+ * When their time arrives, they transition to the RUNNING state.  In this state,
+ * the NodeManager will start them, and monitor them.
+ *
+ * The TaskManager does not handle communication with the agents.  This is handled
+ * by the NodeManagers.  There is one NodeManager per node being managed.
+ * See {org.apache.kafka.trogdor.coordinator.NodeManager} for details.
+ */
+public final class TaskManager {
+    private static final Logger log = LoggerFactory.getLogger(TaskManager.class);
+
+    /**
+     * The platform.
+     */
+    private final Platform platform;
+
+    /**
+     * The scheduler to use for this coordinator.
+     */
+    private final Scheduler scheduler;
+
+    /**
+     * The clock to use for this coordinator.
+     */
+    private final Time time;
+
+    /**
+     * A map of task IDs to Task objects.
+     */
+    private final Map<String, ManagedTask> tasks;
+
+    /**
+     * The executor used for handling Task state changes.
+     */
+    private final ScheduledExecutorService executor;
+
+    /**
+     * Maps node names to node managers.
+     */
+    private final Map<String, NodeManager> nodeManagers;
+
+    /**
+     * True if the TaskManager is shut down.
+     */
+    private AtomicBoolean shutdown = new AtomicBoolean(false);
+
+    TaskManager(Platform platform, Scheduler scheduler) {
+        this.platform = platform;
+        this.scheduler = scheduler;
+        this.time = scheduler.time();
+        this.tasks = new HashMap<>();
+        this.executor = Executors.newSingleThreadScheduledExecutor(
+            ThreadUtils.createThreadFactory("TaskManagerStateThread", false));
+        this.nodeManagers = new HashMap<>();
+        for (Node node : platform.topology().nodes().values()) {
+            if (Node.Util.getTrogdorAgentPort(node) > 0) {
+                this.nodeManagers.put(node.name(), new NodeManager(node, this));
+            }
+        }
+    }
+
+    enum ManagedTaskState {
+        PENDING,
+        RUNNING,
+        STOPPING,
+        DONE;
+    }
+
+    class ManagedTask {
+        /**
+         * The task id.
+         */
+        final private String id;
+
+        /**
+         * The task specification.
+         */
+        final private TaskSpec spec;
+
+        /**
+         * The task controller.
+         */
+        final private TaskController controller;
+
+        /**
+         * The task state.
+         */
+        private ManagedTaskState state;
+
+        /**
+         * The time when the task was started, or -1 if the task has not been started.
+         */
+        private long startedMs = -1;
+
+        /**
+         * The time when the task was finished, or -1 if the task has not been finished.
+         */
+        private long doneMs = -1;
+
+        /**
+         * True if the task was cancelled by a stop request.
+         */
+        boolean cancelled = false;
+
+        /**
+         * If there is a task start scheduled, this is a future which can
+         * be used to cancel it.
+         */
+        private Future<?> startFuture = null;
+
+        /**
+         * The name of the worker nodes involved with this task.
+         * Null if the task is not running.
+         */
+        private Set<String> workers = null;
+
+        /**
+         * The names of the worker nodes which are still running this task.
+         * Null if the task is not running.
+         */
+        private Set<String> activeWorkers = null;
+
+        /**
+         * If this is non-empty, a message describing how this task failed.
+         */
+        private String error = "";
+
+        ManagedTask(String id, TaskSpec spec, TaskController controller, ManagedTaskState state) {
+            this.id = id;
+            this.spec = spec;
+            this.controller = controller;
+            this.state = state;
+        }
+
+        void clearStartFuture() {
+            if (startFuture != null) {
+                startFuture.cancel(false);
+                startFuture = null;
+            }
+        }
+
+        long startDelayMs(long now) {
+            if (now > spec.startMs()) {
+                return 0;
+            }
+            return spec.startMs() - now;
+        }
+
+        TreeSet<String> findNodeNames() {
+            Set<String> nodeNames = controller.targetNodes(platform.topology());
+            TreeSet<String> validNodeNames = new TreeSet<>();
+            TreeSet<String> nonExistentNodeNames = new TreeSet<>();
+            for (String nodeName : nodeNames) {
+                if (nodeManagers.containsKey(nodeName)) {
+                    validNodeNames.add(nodeName);
+                } else {
+                    nonExistentNodeNames.add(nodeName);
+                }
+            }
+            if (!nonExistentNodeNames.isEmpty()) {
+                throw new KafkaException("Unknown node names: " +
+                        Utils.join(nonExistentNodeNames, ", "));
+            }
+            if (validNodeNames.isEmpty()) {
+                throw new KafkaException("No node names specified.");
+            }
+            return validNodeNames;
+        }
+
+        void maybeSetError(String newError) {
+            if (error.isEmpty()) {
+                error = newError;
+            }
+        }
+
+        TaskState taskState() {
+            switch (state) {
+                case PENDING:
+                    return new TaskPending(spec);
+                case RUNNING:
+                    return new TaskRunning(spec, startedMs);
+                case STOPPING:
+                    return new TaskStopping(spec, startedMs);
+                case DONE:
+                    return new TaskDone(spec, startedMs, doneMs, error, cancelled);
+            }
+            throw new RuntimeException("unreachable");
+        }
+    }
+
+    /**
+     * Create a task.
+     *
+     * @param id                    The ID of the task to create.
+     * @param spec                  The specification of the task to create.
+     *
+     * @return                      The specification of the task with the given ID.
+     *                              Note that if there was already a task with the given ID,
+     *                              this may be different from the specification that was
+     *                              requested.
+     */
+    public TaskSpec createTask(final String id, TaskSpec spec)
+            throws ExecutionException, InterruptedException {
+        final TaskSpec existingSpec = executor.submit(new CreateTask(id, spec)).get();
+        if (existingSpec != null) {
+            log.info("Ignoring request to create task {}, because there is already " +
+                "a task with that id.", id);
+            return existingSpec;
+        }
+        return spec;
+    }
+
+    /**
+     * Handles a request to create a new task.  Processed by the state change thread.
+     */
+    class CreateTask implements Callable<TaskSpec> {
+        private final String id;
+        private final TaskSpec spec;
+
+        CreateTask(String id, TaskSpec spec) {
+            this.id = id;
+            this.spec = spec;
+        }
+
+        @Override
+        public TaskSpec call() throws Exception {
+            ManagedTask task = tasks.get(id);
+            if (task != null) {
+                log.info("Task ID {} is already in use.", id);
+                return task.spec;
+            }
+            TaskController controller = null;
+            String failure = null;
+            try {
+                controller = spec.newController(id);
+            } catch (Throwable t) {
+                failure = "Failed to create TaskController: " + t.getMessage();
+            }
+            if (failure != null) {
+                log.info("Failed to create a new task {} with spec {}: {}",
+                    id, spec, failure);
+                task = new ManagedTask(id, spec, null, ManagedTaskState.DONE);
+                task.doneMs = time.milliseconds();
+                task.maybeSetError(failure);
+                tasks.put(id, task);
+                return null;
+            }
+            task = new ManagedTask(id, spec, controller, ManagedTaskState.PENDING);
+            tasks.put(id, task);
+            long delayMs = task.startDelayMs(time.milliseconds());
+            task.startFuture = scheduler.schedule(executor, new RunTask(task), delayMs);
+            log.info("Created a new task {} with spec {}, scheduled to start {} ms from now.",
+                    id, spec, delayMs);
+            return null;
+        }
+    }
+
+    /**
+     * Handles starting a task.  Processed by the state change thread.
+     */
+    class RunTask implements Callable<Void> {
+        private final ManagedTask task;
+
+        RunTask(ManagedTask task) {
+            this.task = task;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            task.clearStartFuture();
+            if (task.state != ManagedTaskState.PENDING) {
+                log.info("Can't start task {}, because it is already in state {}.",
+                    task.id, task.state);
+                return null;
+            }
+            TreeSet<String> nodeNames;
+            try {
+                nodeNames = task.findNodeNames();
+            } catch (Exception e) {
+                log.error("Unable to find nodes for task {}", task.id, e);
+                task.doneMs = time.milliseconds();
+                task.state = ManagedTaskState.DONE;
+                task.maybeSetError("Unable to find nodes for task: " + e.getMessage());
+                return null;
+            }
+            log.info("Running task {} on node(s): {}", task.id, Utils.join(nodeNames, ", "));
+            task.state = ManagedTaskState.RUNNING;
+            task.startedMs = time.milliseconds();
+            task.workers = nodeNames;
+            task.activeWorkers = new HashSet<>();
+            for (String workerName : task.workers) {
+                task.activeWorkers.add(workerName);
+                nodeManagers.get(workerName).createWorker(task.id, task.spec);
+            }
+            return null;
+        }
+    }
+
+    /**
+     * Stop a task.
+     *
+     * @param id                    The ID of the task to stop.
+     * @return                      The specification of the task which was stopped, or null if there
+     *                              was no task found with the given ID.
+     */
+    public TaskSpec stopTask(final String id) throws ExecutionException, InterruptedException {
+        final TaskSpec spec = executor.submit(new CancelTask(id)).get();
+        return spec;
+    }
+
+    /**
+     * Handles cancelling a task.  Processed by the state change thread.
+     */
+    class CancelTask implements Callable<TaskSpec> {
+        private final String id;
+
+        CancelTask(String id) {
+            this.id = id;
+        }
+
+        @Override
+        public TaskSpec call() throws Exception {
+            ManagedTask task = tasks.get(id);
+            if (task == null) {
+                log.info("Can't cancel non-existent task {}.", id);
+                return null;
+            }
+            switch (task.state) {
+                case PENDING:
+                    task.cancelled = true;
+                    task.clearStartFuture();
+                    task.doneMs = time.milliseconds();
+                    task.state = ManagedTaskState.DONE;
+                    log.info("Stopped pending task {}.", id);
+                    break;
+                case RUNNING:
+                    task.cancelled = true;
+                    if (task.activeWorkers.size() == 0) {
+                        log.info("Task {} is now complete with error: {}", id, task.error);
+                        task.doneMs = time.milliseconds();
+                        task.state = ManagedTaskState.DONE;
+                    } else {
+                        for (String workerName : task.activeWorkers) {
+                            nodeManagers.get(workerName).stopWorker(id);
+                        }
+                        log.info("Cancelling task {} on worker(s): {}", id, Utils.join(task.activeWorkers, ", "));
+                        task.state = ManagedTaskState.STOPPING;
+                    }
+                    break;
+                case STOPPING:
+                    log.info("Can't cancel task {} because it is already stopping.", id);
+                    break;
+                case DONE:
+                    log.info("Can't cancel task {} because it is already done.", id);
+                    break;
+            }
+            return task.spec;
+        }
+    }
+
+    /**
+     * A callback NodeManager makes to indicate that a worker has completed.
+     * The task will transition to DONE once all workers are done.
+     *
+     * @param nodeName      The node name.
+     * @param id            The worker name.
+     * @param error         An empty string if there is no error, or an error string.
+     */
+    public void handleWorkerCompletion(String nodeName, String id, String error) {
+        executor.submit(new HandleWorkerCompletion(nodeName, id, error));
+    }
+
+    class HandleWorkerCompletion implements Callable<Void> {
+        private final String nodeName;
+        private final String id;
+        private final String error;
+
+        HandleWorkerCompletion(String nodeName, String id, String error) {
+            this.nodeName = nodeName;
+            this.id = id;
+            this.error = error;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            ManagedTask task = tasks.get(id);
+            if (task == null) {
+                log.error("Can't handle completion of unknown worker {} on node {}",
+                    id, nodeName);
+                return null;
+            }
+            if ((task.state == ManagedTaskState.PENDING) || (task.state == ManagedTaskState.DONE)) {
+                log.error("Task {} got unexpected worker completion from {} while " +
+                    "in {} state.", id, nodeName, task.state);
+                return null;
+            }
+            boolean broadcastStop = false;
+            if (task.state == ManagedTaskState.RUNNING) {
+                task.state = ManagedTaskState.STOPPING;
+                broadcastStop = true;
+            }
+            task.maybeSetError(error);
+            task.activeWorkers.remove(nodeName);
+            if (task.activeWorkers.size() == 0) {
+                task.doneMs = time.milliseconds();
+                task.state = ManagedTaskState.DONE;
+                log.info("Task {} is now complete on {} with error: {}", id,
+                    Utils.join(task.workers, ", "),
+                    task.error.isEmpty() ? "(none)" : task.error);
+            } else if (broadcastStop) {
+                log.info("Node {} stopped.  Stopping task {} on worker(s): {}",
+                    id, Utils.join(task.activeWorkers, ", "));
+                for (String workerName : task.activeWorkers) {
+                    nodeManagers.get(workerName).stopWorker(id);
+                }
+            }
+            return null;
+        }
+    }
+
+    /**
+     * Get information about the tasks being managed.
+     */
+    public TasksResponse tasks() throws ExecutionException, InterruptedException {
+        return executor.submit(new GetTasksResponse()).get();
+    }
+
+    class GetTasksResponse implements Callable<TasksResponse> {
+        @Override
+        public TasksResponse call() throws Exception {
+            TreeMap<String, TaskState> states = new TreeMap<>();
+            for (ManagedTask task : tasks.values()) {
+                states.put(task.id, task.taskState());
+            }
+            return new TasksResponse(states);
+        }
+    }
+
+    /**
+     * Initiate shutdown, but do not wait for it to complete.
+     */
+    public void beginShutdown(boolean stopAgents) throws ExecutionException, InterruptedException {
+        if (shutdown.compareAndSet(false, true)) {
+            executor.submit(new Shutdown(stopAgents));
+        }
+    }
+
+    /**
+     * Wait for shutdown to complete.  May be called prior to beginShutdown.
+     */
+    public void waitForShutdown() throws ExecutionException, InterruptedException {
+        while (!executor.awaitTermination(1, TimeUnit.DAYS)) { }
+    }
+
+    class Shutdown implements Callable<Void> {
+        private final boolean stopAgents;
+
+        Shutdown(boolean stopAgents) {
+            this.stopAgents = stopAgents;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            log.info("Shutting down TaskManager{}.", stopAgents ? " and agents" : "");
+            for (NodeManager nodeManager : nodeManagers.values()) {
+                nodeManager.beginShutdown(stopAgents);
+            }
+            for (NodeManager nodeManager : nodeManagers.values()) {
+                nodeManager.waitForShutdown();
+            }
+            executor.shutdown();
+            return null;
+        }
+    }
+};

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFault.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFault.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFault.java
deleted file mode 100644
index 2d63b82..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFault.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.common.Topology;
-
-import java.util.Objects;
-import java.util.Set;
-
-public abstract class AbstractFault implements Fault {
-    private final String id;
-    private final FaultSpec spec;
-    private FaultState state;
-
-    public AbstractFault(String id, FaultSpec spec) {
-        this.id = id;
-        this.spec = spec;
-        this.state = new PendingState();
-    }
-
-    @Override
-    public final String id() {
-        return id;
-    }
-
-    @Override
-    public final FaultSpec spec() {
-        return spec;
-    }
-
-    @Override
-    public synchronized FaultState state() {
-        return state;
-    }
-
-    @Override
-    public synchronized void setState(FaultState state) {
-        this.state = state;
-    }
-
-    @Override
-    public final void activate(long now, Platform platform) throws Exception {
-        try {
-            handleActivation(now, platform);
-            setState(new RunningState(now));
-        } catch (Exception e) {
-            setState(new DoneState(now, e.getMessage()));
-            throw e;
-        }
-    }
-
-    protected abstract void handleActivation(long now, Platform platform) throws Exception;
-
-    @Override
-    public final void deactivate(long now, Platform platform) throws Exception {
-        try {
-            handleDeactivation(now, platform);
-            setState(new DoneState(now, ""));
-        } catch (Exception e) {
-            setState(new DoneState(now, e.getMessage()));
-            throw e;
-        }
-    }
-
-    protected abstract void handleDeactivation(long now, Platform platform) throws Exception;
-
-    @Override
-    public abstract Set<String> targetNodes(Topology topology);
-
-    @Override
-    public final boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        return toString().equals(o.toString());
-    }
-
-    @Override
-    public final int hashCode() {
-        return Objects.hashCode(toString());
-    }
-
-    @Override
-    public final String toString() {
-        return getClass().getSimpleName() + "(id=" + id +
-            ", spec=" + JsonUtil.toJsonString(spec) +
-            ", state=" + JsonUtil.toJsonString(state()) +
-            ")";
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFaultSpec.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFaultSpec.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFaultSpec.java
deleted file mode 100644
index 5f551b5..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/AbstractFaultSpec.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.kafka.trogdor.common.JsonUtil;
-
-
-/**
- * A base class that can be used for FaultSpecs. 
- */
-public abstract class AbstractFaultSpec implements FaultSpec {
-    private final long startMs;
-    private final long durationMs;
-
-    protected AbstractFaultSpec(@JsonProperty("startMs") long startMs,
-                             @JsonProperty("durationMs") long durationMs) {
-        this.startMs = startMs;
-        this.durationMs = durationMs;
-    }
-
-    @JsonProperty
-    @Override
-    public long startMs() {
-        return startMs;
-    }
-
-    @JsonProperty
-    @Override
-    public long durationMs() {
-        return durationMs;
-    }
-
-    @Override
-    public String toString() {
-        return JsonUtil.toJsonString(this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/DoneState.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/DoneState.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/DoneState.java
deleted file mode 100644
index 222caf0..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/DoneState.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-/**
- * The state a fault is in on the agent or controller when it is completed,
- * either normally or with an error.
- */
-public class DoneState extends FaultState {
-    private final long doneMs;
-    private final String errorStr;
-
-    @JsonCreator
-    public DoneState(@JsonProperty("doneMs") long doneMs,
-                     @JsonProperty("errorStr") String errorStr) {
-        this.doneMs = doneMs;
-        this.errorStr = errorStr;
-    }
-
-    @JsonProperty
-    public long doneMs() {
-        return doneMs;
-    }
-
-    @JsonProperty
-    public String errorStr() {
-        return errorStr;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/Fault.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/Fault.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/Fault.java
deleted file mode 100644
index e44d56a..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/Fault.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.common.Topology;
-
-import java.util.Set;
-
-public interface Fault {
-    /**
-     * Get the ID of this fault.
-     */
-    String id();
-
-    /**
-     * Get the specification for this Fault.
-     */
-    FaultSpec spec();
-
-    /**
-     * Get the current fault state.  Thread-safe.
-     */
-    FaultState state();
-
-    /**
-     * Set the current fault state.  Thread-safe.
-     */
-    void setState(FaultState state);
-
-    /**
-     * Activate the fault.  Will transition into RunningState or DoneState.
-     *
-     * @param now           The current time in ms.
-     * @param platform      The platform to use.
-     */
-    void activate(long now, Platform platform) throws Exception;
-
-    /**
-     * Deactivate the fault.  Will transition into DoneState.
-     *
-     * @param now           The current time in ms.
-     * @param platform      The platform to use.
-     */
-    void deactivate(long now, Platform platform) throws Exception;
-
-    /**
-     * Get the nodes which this fault is targetting.
-     *
-     * @param topology      The topology to use.
-     *
-     * @return              A set of target node names.
-     */
-    Set<String> targetNodes(Topology topology);
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSet.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSet.java b/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSet.java
deleted file mode 100644
index 63e5ff4..0000000
--- a/tools/src/main/java/org/apache/kafka/trogdor/fault/FaultSet.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kafka.trogdor.fault;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.TreeMap;
-
-public class FaultSet {
-    private final static long NS_PER_MS = 1000000L;
-
-    /**
-     * Maps fault start times in nanoseconds to faults.
-     */
-    private final TreeMap<Long, Fault> byStart = new TreeMap<Long, Fault>();
-
-    /**
-     * Maps fault end times in nanoseconds to faults.
-     */
-    private final TreeMap<Long, Fault> byEnd = new TreeMap<Long, Fault>();
-
-    /**
-     * Return an iterator that iterates over the fault set in start time order.
-     */
-    public FaultSetIterator iterateByStart() {
-        return new FaultSetIterator(byStart);
-    }
-
-    /**
-     * Return an iterator that iterates over the fault set in end time order.
-     */
-    public FaultSetIterator iterateByEnd() {
-        return new FaultSetIterator(byEnd);
-    }
-
-    /**
-     * Add a new fault to the FaultSet.
-     */
-    public void add(Fault fault) {
-        insertUnique(byStart, fault.spec().startMs() * NS_PER_MS, fault);
-        long endMs = fault.spec().startMs() + fault.spec().durationMs();
-        insertUnique(byEnd, endMs * NS_PER_MS, fault);
-    }
-
-    /**
-     * Insert a new fault to a TreeMap.
-     *
-     * If there is already a fault with the given key, the fault will be stored
-     * with the next available key.
-     */
-    private void insertUnique(TreeMap<Long, Fault> map, long key, Fault fault) {
-        while (true) {
-            Fault existing = map.get(key);
-            if (existing == null) {
-                map.put(key, fault);
-                return;
-            } else if (existing == fault) {
-                return;
-            } else {
-                key++;
-            }
-        }
-    }
-
-    /**
-     * Remove a fault from the TreeMap.  The fault is removed by object equality.
-     */
-    public void remove(Fault fault) {
-        removeUnique(byStart, fault.spec().startMs() * NS_PER_MS, fault);
-        long endMs = fault.spec().startMs() + fault.spec().durationMs();
-        removeUnique(byEnd, endMs * NS_PER_MS, fault);
-    }
-
-    /**
-     * Helper function to remove a fault from a map.  We will search every
-     * element of the map equal to or higher than the given key.
-     */
-    private void removeUnique(TreeMap<Long, Fault> map, long key, Fault fault) {
-        while (true) {
-            Map.Entry<Long, Fault> existing = map.ceilingEntry(key);
-            if (existing == null) {
-                throw new NoSuchElementException("No such element as " + fault);
-            } else if (existing.getValue() == fault) {
-                map.remove(existing.getKey());
-                return;
-            } else {
-                key = existing.getKey() + 1;
-            }
-        }
-    }
-
-    /**
-     * An iterator over the FaultSet.
-     */
-    class FaultSetIterator implements Iterator<Fault> {
-        private final TreeMap<Long, Fault> map;
-        private Fault cur = null;
-        private long prevKey = -1;
-
-        FaultSetIterator(TreeMap<Long, Fault> map) {
-            this.map = map;
-        }
-
-        @Override
-        public boolean hasNext() {
-            Map.Entry<Long, Fault> entry = map.higherEntry(prevKey);
-            return entry != null;
-        }
-
-        @Override
-        public Fault next() {
-            Map.Entry<Long, Fault> entry = map.higherEntry(prevKey);
-            if (entry == null) {
-                throw new NoSuchElementException();
-            }
-            prevKey = entry.getKey();
-            cur = entry.getValue();
-            return cur;
-        }
-
-        @Override
-        public void remove() {
-            if (cur == null) {
-                throw new IllegalStateException();
-            }
-            FaultSet.this.remove(cur);
-            cur = null;
-        }
-    }
-};


[4/4] kafka git commit: KAFKA-6060; Add workload generation capabilities to Trogdor

Posted by rs...@apache.org.
KAFKA-6060; Add workload generation capabilities to Trogdor

Previously, Trogdor only handled "Faults."  Now, Trogdor can handle
"Tasks" which may be either faults, or workloads to execute in the
background.

The Agent and Coordinator have been refactored from a
mutexes-and-condition-variables paradigm into a message passing
paradigm.  No locks are necessary, because only one thread can access
the task state or worker state.  This makes them a lot easier to reason
about.

The MockTime class can now handle mocking deferred message passing
(adding a message to an ExecutorService with a delay).  I added a
MockTimeTest.

MiniTrogdorCluster now starts up Agent and Coordinator classes in
paralle in order to minimize junit test time.

RPC messages now inherit from a common Message.java class.  This class
handles implementing serialization, equals, hashCode, etc.

Remove FaultSet, since it is no longer necessary.

Previously, if CoordinatorClient or AgentClient hit a networking
problem, they would throw an exception.  They now retry several times
before giving up.  Additionally, the REST RPCs to the Coordinator and
Agent have been changed to be idempotent.  If a response is lost, and
the request is resent, no harm will be done.

Author: Colin P. Mccabe <cm...@confluent.io>

Reviewers: Rajini Sivaram <ra...@googlemail.com>, Ismael Juma <is...@juma.me.uk>

Closes #4073 from cmccabe/KAFKA-6060


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4fac83ba
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4fac83ba
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4fac83ba

Branch: refs/heads/trunk
Commit: 4fac83ba1f80353e9544b15b95b8da9dc557041d
Parents: e4208b1
Author: Colin P. Mccabe <cm...@confluent.io>
Authored: Fri Nov 3 09:37:29 2017 +0000
Committer: Rajini Sivaram <ra...@googlemail.com>
Committed: Fri Nov 3 09:37:29 2017 +0000

----------------------------------------------------------------------
 checkstyle/import-control.xml                   |   4 +
 .../apache/kafka/common/utils/Scheduler.java    |  49 ++
 .../kafka/common/utils/SystemScheduler.java     |  43 ++
 .../kafka/common/utils/MockScheduler.java       | 121 ++++
 .../org/apache/kafka/common/utils/MockTime.java |  22 +
 .../apache/kafka/common/utils/MockTimeTest.java |  50 ++
 gradle/findbugs-exclude.xml                     |   8 +
 tests/kafkatest/services/trogdor/fault_spec.py  |  45 --
 .../trogdor/network_partition_fault_spec.py     |   6 +-
 .../services/trogdor/no_op_fault_spec.py        |  41 --
 .../services/trogdor/no_op_task_spec.py         |  41 ++
 tests/kafkatest/services/trogdor/task_spec.py   |  45 ++
 tests/kafkatest/services/trogdor/trogdor.py     | 115 +++-
 tests/kafkatest/tests/tools/trogdor_test.py     |  21 +-
 .../org/apache/kafka/trogdor/agent/Agent.java   | 281 ++-------
 .../apache/kafka/trogdor/agent/AgentClient.java |  82 +--
 .../kafka/trogdor/agent/AgentRestResource.java  |  24 +-
 .../kafka/trogdor/agent/WorkerManager.java      | 574 +++++++++++++++++++
 .../kafka/trogdor/basic/BasicPlatform.java      |  13 +-
 .../apache/kafka/trogdor/common/Platform.java   |   8 +
 .../kafka/trogdor/common/ThreadUtils.java       |  55 ++
 .../apache/kafka/trogdor/common/Topology.java   |  18 +
 .../kafka/trogdor/coordinator/Coordinator.java  | 259 ++-------
 .../trogdor/coordinator/CoordinatorClient.java  |  95 +--
 .../coordinator/CoordinatorRestResource.java    |  38 +-
 .../kafka/trogdor/coordinator/NodeManager.java  | 407 +++++++------
 .../kafka/trogdor/coordinator/TaskManager.java  | 535 +++++++++++++++++
 .../kafka/trogdor/fault/AbstractFault.java      | 106 ----
 .../kafka/trogdor/fault/AbstractFaultSpec.java  |  53 --
 .../apache/kafka/trogdor/fault/DoneState.java   |  47 --
 .../org/apache/kafka/trogdor/fault/Fault.java   |  70 ---
 .../apache/kafka/trogdor/fault/FaultSet.java    | 146 -----
 .../apache/kafka/trogdor/fault/FaultSpec.java   |  59 --
 .../apache/kafka/trogdor/fault/FaultState.java  |  51 --
 .../trogdor/fault/NetworkPartitionFault.java    |  96 ----
 .../fault/NetworkPartitionFaultController.java  |  42 ++
 .../fault/NetworkPartitionFaultSpec.java        |  40 +-
 .../fault/NetworkPartitionFaultWorker.java      |  79 +++
 .../apache/kafka/trogdor/fault/NoOpFault.java   |  57 --
 .../kafka/trogdor/fault/NoOpFaultSpec.java      |  50 --
 .../kafka/trogdor/fault/PendingState.java       |  30 -
 .../kafka/trogdor/fault/RunningState.java       |  38 --
 .../kafka/trogdor/fault/SendingState.java       |  64 ---
 .../kafka/trogdor/rest/AgentFaultsResponse.java |  52 --
 .../kafka/trogdor/rest/AgentStatusResponse.java |  39 +-
 .../trogdor/rest/CoordinatorFaultsResponse.java |  52 --
 .../rest/CoordinatorShutdownRequest.java        |  38 ++
 .../trogdor/rest/CoordinatorStatusResponse.java |  35 +-
 .../trogdor/rest/CreateAgentFaultRequest.java   |  69 ---
 .../rest/CreateCoordinatorFaultRequest.java     |  69 ---
 .../kafka/trogdor/rest/CreateTaskRequest.java   |  47 ++
 .../kafka/trogdor/rest/CreateTaskResponse.java  |  39 ++
 .../kafka/trogdor/rest/CreateWorkerRequest.java |  47 ++
 .../trogdor/rest/CreateWorkerResponse.java      |  39 ++
 .../apache/kafka/trogdor/rest/FaultDataMap.java |  98 ----
 .../kafka/trogdor/rest/JsonRestServer.java      |  70 ++-
 .../org/apache/kafka/trogdor/rest/Message.java  |  41 ++
 .../kafka/trogdor/rest/StopTaskRequest.java     |  38 ++
 .../kafka/trogdor/rest/StopTaskResponse.java    |  39 ++
 .../kafka/trogdor/rest/StopWorkerRequest.java   |  38 ++
 .../kafka/trogdor/rest/StopWorkerResponse.java  |  39 ++
 .../org/apache/kafka/trogdor/rest/TaskDone.java |  80 +++
 .../apache/kafka/trogdor/rest/TaskPending.java  |  32 ++
 .../apache/kafka/trogdor/rest/TaskRunning.java  |  44 ++
 .../apache/kafka/trogdor/rest/TaskState.java    |  48 ++
 .../apache/kafka/trogdor/rest/TaskStopping.java |  44 ++
 .../kafka/trogdor/rest/TasksResponse.java       |  40 ++
 .../apache/kafka/trogdor/rest/WorkerDone.java   |  88 +++
 .../kafka/trogdor/rest/WorkerReceiving.java     |  33 ++
 .../kafka/trogdor/rest/WorkerRunning.java       |  64 +++
 .../kafka/trogdor/rest/WorkerStarting.java      |  32 ++
 .../apache/kafka/trogdor/rest/WorkerState.java  |  70 +++
 .../kafka/trogdor/rest/WorkerStopping.java      |  69 +++
 .../kafka/trogdor/task/NoOpTaskController.java  |  36 ++
 .../apache/kafka/trogdor/task/NoOpTaskSpec.java |  44 ++
 .../kafka/trogdor/task/NoOpTaskWorker.java      |  46 ++
 .../kafka/trogdor/task/TaskController.java      |  36 ++
 .../org/apache/kafka/trogdor/task/TaskSpec.java | 105 ++++
 .../apache/kafka/trogdor/task/TaskWorker.java   |  78 +++
 .../apache/kafka/trogdor/agent/AgentTest.java   | 220 ++++---
 .../kafka/trogdor/common/ExpectedFaults.java    | 193 -------
 .../kafka/trogdor/common/ExpectedTasks.java     | 206 +++++++
 .../trogdor/common/MiniTrogdorCluster.java      |  73 ++-
 .../trogdor/coordinator/CoordinatorTest.java    | 195 +++++--
 .../kafka/trogdor/fault/FaultSetTest.java       | 126 ----
 .../trogdor/task/SampleTaskController.java      |  29 +
 .../kafka/trogdor/task/SampleTaskSpec.java      |  56 ++
 .../kafka/trogdor/task/SampleTaskWorker.java    |  63 ++
 88 files changed, 4461 insertions(+), 2606 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index ddb13bc..403cae2 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -187,6 +187,10 @@
     <allow pkg="javax.servlet" />
     <allow pkg="javax.ws.rs" />
     <allow pkg="net.sourceforge.argparse4j" />
+    <allow pkg="org.apache.kafka.clients.admin" />
+    <allow pkg="org.apache.kafka.clients.consumer" exact-match="true"/>
+    <allow pkg="org.apache.kafka.clients.producer" exact-match="true"/>
+    <allow pkg="org.apache.kafka.common" />
     <allow pkg="org.apache.kafka.test"/>
     <allow pkg="org.apache.kafka.trogdor" />
     <allow pkg="org.apache.log4j" />

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/clients/src/main/java/org/apache/kafka/common/utils/Scheduler.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Scheduler.java b/clients/src/main/java/org/apache/kafka/common/utils/Scheduler.java
new file mode 100644
index 0000000..a8ada65
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Scheduler.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * An interface for scheduling tasks for the future.
+ *
+ * Implementations of this class should be thread-safe.
+ */
+public interface Scheduler {
+    Scheduler SYSTEM = new SystemScheduler();
+
+    /**
+     * Get the timekeeper associated with this scheduler.
+     */
+    Time time();
+
+    /**
+     * Schedule a callable to be executed in the future on a
+     * ScheduledExecutorService.  Note that the Callable may not be queued on
+     * the executor until the designated time arrives.
+     *
+     * @param executor      The executor to use.
+     * @param callable      The callable to execute.
+     * @param delayMs       The delay to use, in milliseconds.
+     * @param <T>           The return type of the callable.
+     * @return              A future which will complete when the callable is finished.
+     */
+    <T> Future<T> schedule(final ScheduledExecutorService executor,
+                           final Callable<T> callable, long delayMs);
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/clients/src/main/java/org/apache/kafka/common/utils/SystemScheduler.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SystemScheduler.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemScheduler.java
new file mode 100644
index 0000000..c8c1148
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemScheduler.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A scheduler implementation that uses the system clock.
+ *
+ * Use Scheduler.SYSTEM instead of constructing an instance of this class.
+ */
+public class SystemScheduler implements Scheduler {
+    SystemScheduler() {
+    }
+
+    @Override
+    public Time time() {
+        return Time.SYSTEM;
+    }
+
+    @Override
+    public <T> Future<T> schedule(final ScheduledExecutorService executor,
+                                  final Callable<T> callable, long delayMs) {
+        return executor.schedule(callable, delayMs, TimeUnit.MILLISECONDS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/clients/src/test/java/org/apache/kafka/common/utils/MockScheduler.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockScheduler.java b/clients/src/test/java/org/apache/kafka/common/utils/MockScheduler.java
new file mode 100644
index 0000000..ba5e1ed
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/utils/MockScheduler.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+
+public class MockScheduler implements Scheduler, MockTime.MockTimeListener {
+    private static final Logger log = LoggerFactory.getLogger(MockScheduler.class);
+
+    /**
+     * The MockTime object.
+     */
+    private final MockTime time;
+
+    /**
+     * Futures which are waiting for a specified wall-clock time to arrive.
+     */
+    private final TreeMap<Long, List<KafkaFutureImpl<Long>>> waiters = new TreeMap<>();
+
+    public MockScheduler(MockTime time) {
+        this.time = time;
+        time.addListener(this);
+    }
+
+    @Override
+    public Time time() {
+        return time;
+    }
+
+    @Override
+    public synchronized void tick() {
+        long timeMs = time.milliseconds();
+        while (true) {
+            Map.Entry<Long, List<KafkaFutureImpl<Long>>> entry = waiters.firstEntry();
+            if ((entry == null) || (entry.getKey() > timeMs)) {
+                break;
+            }
+            for (KafkaFutureImpl<Long> future : entry.getValue()) {
+                future.complete(timeMs);
+            }
+            waiters.remove(entry.getKey());
+        }
+    }
+
+    public synchronized void addWaiter(long delayMs, KafkaFutureImpl<Long> waiter) {
+        long timeMs = time.milliseconds();
+        if (delayMs <= 0) {
+            waiter.complete(timeMs);
+        } else {
+            long triggerTimeMs = timeMs + delayMs;
+            List<KafkaFutureImpl<Long>> futures = waiters.get(triggerTimeMs);
+            if (futures == null) {
+                futures = new ArrayList<>();
+                waiters.put(triggerTimeMs, futures);
+            }
+            futures.add(waiter);
+        }
+    }
+
+    @Override
+    public <T> Future<T> schedule(final ScheduledExecutorService executor,
+                                  final Callable<T> callable, long delayMs) {
+        final KafkaFutureImpl<T> future = new KafkaFutureImpl<>();
+        KafkaFutureImpl<Long> waiter = new KafkaFutureImpl<>();
+        waiter.thenApply(new KafkaFuture.Function<Long, Void>() {
+            @Override
+            public Void apply(final Long now) {
+                executor.submit(new Callable<Void>() {
+                    @Override
+                    public Void call() throws Exception {
+                        // Note: it is possible that we'll execute Callable#call right after
+                        // the future is cancelled.  This is a valid sequence of events
+                        // that the author of the Callable needs to be able to handle.
+                        //
+                        // Note 2: If the future is cancelled, we will not remove the waiter
+                        // from this MockTime object.  This small bit of inefficiency is acceptable
+                        // in testing code (at least we aren't polling!)
+                        if (!future.isCancelled()) {
+                            try {
+                                log.trace("Invoking {} at {}", callable, now);
+                                future.complete(callable.call());
+                            } catch (Throwable throwable) {
+                                future.completeExceptionally(throwable);
+                            }
+                        }
+                        return null;
+                    }
+                });
+                return null;
+            }
+        });
+        log.trace("Scheduling {} for {} ms from now.", callable, delayMs);
+        addWaiter(delayMs, waiter);
+        return future;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
index be04aed..011eba2 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.common.utils;
 
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -24,6 +25,15 @@ import java.util.concurrent.atomic.AtomicLong;
  */
 public class MockTime implements Time {
 
+    interface MockTimeListener {
+        void tick();
+    }
+
+    /**
+     * Listeners which are waiting for time changes.
+     */
+    private final CopyOnWriteArrayList<MockTimeListener> listeners = new CopyOnWriteArrayList<>();
+
     private final long autoTickMs;
 
     // Values from `nanoTime` and `currentTimeMillis` are not comparable, so we store them separately to allow tests
@@ -45,6 +55,10 @@ public class MockTime implements Time {
         this.autoTickMs = autoTickMs;
     }
 
+    public void addListener(MockTimeListener listener) {
+        listeners.add(listener);
+    }
+
     @Override
     public long milliseconds() {
         maybeSleep(autoTickMs);
@@ -71,6 +85,7 @@ public class MockTime implements Time {
     public void sleep(long ms) {
         timeMs.addAndGet(ms);
         highResTimeNs.addAndGet(TimeUnit.MILLISECONDS.toNanos(ms));
+        tick();
     }
 
     public void setCurrentTimeMs(long newMs) {
@@ -81,5 +96,12 @@ public class MockTime implements Time {
             throw new IllegalArgumentException("Setting the time to " + newMs + " while current time " + oldMs + " is newer; this is not allowed");
 
         highResTimeNs.set(TimeUnit.MILLISECONDS.toNanos(newMs));
+        tick();
+    }
+
+    private void tick() {
+        for (MockTimeListener listener : listeners) {
+            listener.tick();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/clients/src/test/java/org/apache/kafka/common/utils/MockTimeTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTimeTest.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTimeTest.java
new file mode 100644
index 0000000..58bcb19
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTimeTest.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.junit.Test;
+
+public class MockTimeTest {
+    private static final Logger log = LoggerFactory.getLogger(MockTimeTest.class);
+
+    @Rule
+    final public Timeout globalTimeout = Timeout.millis(120000);
+
+    @Test
+    public void testAdvanceClock() throws Exception {
+        MockTime time = new MockTime(0, 100, 200);
+        Assert.assertEquals(100, time.milliseconds());
+        Assert.assertEquals(200, time.nanoseconds());
+        time.sleep(1);
+        Assert.assertEquals(101, time.milliseconds());
+        Assert.assertEquals(1000200, time.nanoseconds());
+    }
+
+    @Test
+    public void testAutoTickMs() throws Exception {
+        MockTime time = new MockTime(1, 100, 200);
+        Assert.assertEquals(101, time.milliseconds());
+        Assert.assertEquals(2000200, time.nanoseconds());
+        Assert.assertEquals(103, time.milliseconds());
+        Assert.assertEquals(104, time.milliseconds());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/gradle/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/gradle/findbugs-exclude.xml b/gradle/findbugs-exclude.xml
index 9cb1db8..3bc3d39 100644
--- a/gradle/findbugs-exclude.xml
+++ b/gradle/findbugs-exclude.xml
@@ -319,4 +319,12 @@ For a detailed description of findbugs bug categories, see http://findbugs.sourc
         <Method name="main"/>
         <Bug pattern="IL_INFINITE_LOOP"/>
     </Match>
+
+    <Match>
+        <!-- Suppress a spurious warning about calling notify without modifying
+             other state under the monitor. -->
+        <Package name="org.apache.kafka.trogdor.workload"/>
+        <Source name="RoundTripWorker.java"/>
+        <Bug pattern="NN_NAKED_NOTIFY"/>
+    </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/fault_spec.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/fault_spec.py b/tests/kafkatest/services/trogdor/fault_spec.py
deleted file mode 100644
index 9768765..0000000
--- a/tests/kafkatest/services/trogdor/fault_spec.py
+++ /dev/null
@@ -1,45 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-import json
-
-
-class FaultSpec(object):
-    """
-    The base class for a fault specification.
-
-    MAX_DURATION_MS         The longest duration we should use for a fault specification.
-    """
-
-    MAX_DURATION_MS=10000000
-
-    def __init__(self, start_ms, duration_ms):
-        """
-        Create a new fault specification.
-
-        :param start_ms:        The start time in milliseconds since the epoch.
-        :param duration_ms:     The duration in milliseconds.
-        """
-        self.start_ms = start_ms
-        self.duration_ms = duration_ms
-
-    def message(self):
-        """
-        Return a message suitable for sending to the Trogdor daemon.
-        """
-        raise NotImplemented
-
-    def __str__(self):
-        return json.dumps(self.message())

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/network_partition_fault_spec.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/network_partition_fault_spec.py b/tests/kafkatest/services/trogdor/network_partition_fault_spec.py
index deb5c56..91c731e 100644
--- a/tests/kafkatest/services/trogdor/network_partition_fault_spec.py
+++ b/tests/kafkatest/services/trogdor/network_partition_fault_spec.py
@@ -13,10 +13,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-from kafkatest.services.trogdor.fault_spec import FaultSpec
+from kafkatest.services.trogdor.task_spec import TaskSpec
 
 
-class NetworkPartitionFaultSpec(FaultSpec):
+class NetworkPartitionFaultSpec(TaskSpec):
     """
     The specification for a network partition fault.
 
@@ -28,7 +28,7 @@ class NetworkPartitionFaultSpec(FaultSpec):
         """
         Create a new NetworkPartitionFaultSpec.
 
-        :param start_ms:        The start time, as described in fault_spec.py
+        :param start_ms:        The start time, as described in task_spec.py
         :param duration_ms:     The duration in milliseconds.
         :param partitions:      An array of arrays describing the partitions.
                                 The inner arrays may contain either node names,

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/no_op_fault_spec.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/no_op_fault_spec.py b/tests/kafkatest/services/trogdor/no_op_fault_spec.py
deleted file mode 100644
index 82e9713..0000000
--- a/tests/kafkatest/services/trogdor/no_op_fault_spec.py
+++ /dev/null
@@ -1,41 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#    http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-from kafkatest.services.trogdor.fault_spec import FaultSpec
-
-
-class NoOpFaultSpec(FaultSpec):
-    """
-    The specification for a nop-op fault.
-
-    No-op faults are used to test the fault injector.  They don't do anything,
-    but must be propagated to all fault injector daemons.
-    """
-
-    def __init__(self, start_ms, duration_ms):
-        """
-        Create a new NoOpFault.
-
-        :param start_ms:        The start time, as described in fault_spec.py
-        :param duration_ms:     The duration in milliseconds.
-        """
-        super(NoOpFaultSpec, self).__init__(start_ms, duration_ms)
-
-    def message(self):
-        return {
-            "class": "org.apache.kafka.trogdor.fault.NoOpFaultSpec",
-            "startMs": self.start_ms,
-            "durationMs": self.duration_ms,
-        }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/no_op_task_spec.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/no_op_task_spec.py b/tests/kafkatest/services/trogdor/no_op_task_spec.py
new file mode 100644
index 0000000..eb75264
--- /dev/null
+++ b/tests/kafkatest/services/trogdor/no_op_task_spec.py
@@ -0,0 +1,41 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+from kafkatest.services.trogdor.task_spec import TaskSpec
+
+
+class NoOpTaskSpec(TaskSpec):
+    """
+    The specification for a nop-op task.
+
+    No-op faults are used to test Trogdor.  They don't do anything,
+    but must be propagated to all Trogdor agents.
+    """
+
+    def __init__(self, start_ms, duration_ms):
+        """
+        Create a new NoOpFault.
+
+        :param start_ms:        The start time, as described in task_spec.py
+        :param duration_ms:     The duration in milliseconds.
+        """
+        super(NoOpTaskSpec, self).__init__(start_ms, duration_ms)
+
+    def message(self):
+        return {
+            "class": "org.apache.kafka.trogdor.task.NoOpTaskSpec",
+            "startMs": self.start_ms,
+            "durationMs": self.duration_ms,
+        }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/task_spec.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/task_spec.py b/tests/kafkatest/services/trogdor/task_spec.py
new file mode 100644
index 0000000..61a080a
--- /dev/null
+++ b/tests/kafkatest/services/trogdor/task_spec.py
@@ -0,0 +1,45 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import json
+
+
+class TaskSpec(object):
+    """
+    The base class for a task specification.
+
+    MAX_DURATION_MS         The longest duration we should use for a task specification.
+    """
+
+    MAX_DURATION_MS=10000000
+
+    def __init__(self, start_ms, duration_ms):
+        """
+        Create a new task specification.
+
+        :param start_ms:        The target start time in milliseconds since the epoch.
+        :param duration_ms:     The duration in milliseconds.
+        """
+        self.start_ms = start_ms
+        self.duration_ms = duration_ms
+
+    def message(self):
+        """
+        Return a message suitable for sending to the Trogdor daemon.
+        """
+        raise NotImplemented
+
+    def __str__(self):
+        return json.dumps(self.message())

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/services/trogdor/trogdor.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/services/trogdor/trogdor.py b/tests/kafkatest/services/trogdor/trogdor.py
index 8b05e99..a4fcfb5 100644
--- a/tests/kafkatest/services/trogdor/trogdor.py
+++ b/tests/kafkatest/services/trogdor/trogdor.py
@@ -20,6 +20,7 @@ from requests.adapters import HTTPAdapter
 from requests.packages.urllib3 import Retry
 
 from ducktape.services.service import Service
+from ducktape.utils.util import wait_until
 from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
 
 
@@ -70,22 +71,29 @@ class TrogdorService(KafkaPathResolverMixin, Service):
             "collect_default": True},
     }
 
-    def __init__(self, context, agent_nodes, agent_port=DEFAULT_AGENT_PORT,
-                 coordinator_port=DEFAULT_COORDINATOR_PORT):
+
+    def __init__(self, context, agent_nodes=None, client_services=None,
+                 agent_port=DEFAULT_AGENT_PORT, coordinator_port=DEFAULT_COORDINATOR_PORT):
         """
         Create a Trogdor service.
 
         :param context:             The test context.
         :param agent_nodes:         The nodes to run the agents on.
+        :param client_services:     Services whose nodes we should run agents on.
         :param agent_port:          The port to use for the trogdor_agent daemons.
         :param coordinator_port:    The port to use for the trogdor_coordinator daemons.
         """
         Service.__init__(self, context, num_nodes=1)
         self.coordinator_node = self.nodes[0]
-        if (len(agent_nodes) == 0):
-            raise RuntimeError("You must supply at least one node to run the service on.")
-        for agent_node in agent_nodes:
-            self.nodes.append(agent_node)
+        if client_services is not None:
+            for client_service in client_services:
+                for node in client_service.nodes:
+                    self.nodes.append(node)
+        if agent_nodes is not None:
+            for agent_node in agent_nodes:
+                self.nodes.append(agent_node)
+        if (len(self.nodes) == 1):
+            raise RuntimeError("You must supply at least one agent node to run the service on.")
         self.agent_port = agent_port
         self.coordinator_port = coordinator_port
 
@@ -108,9 +116,12 @@ class TrogdorService(KafkaPathResolverMixin, Service):
         for node in self.nodes:
             dict_nodes[node.name] = {
                 "hostname": node.account.ssh_hostname,
-                "trogdor.agent.port": self.agent_port,
             }
-        dict_nodes[self.coordinator_node.name]["trogdor.coordinator.port"] = self.coordinator_port
+            if node.name == self.coordinator_node.name:
+                dict_nodes[node.name]["trogdor.coordinator.port"] = self.coordinator_port
+            else:
+                dict_nodes[node.name]["trogdor.agent.port"] = self.agent_port
+
         return {
             "platform": "org.apache.kafka.trogdor.basic.BasicPlatform",
             "nodes": dict_nodes,
@@ -160,7 +171,7 @@ class TrogdorService(KafkaPathResolverMixin, Service):
                 stdout_stderr_capture_path)
         node.account.ssh(cmd)
         with node.account.monitor_log(log_path) as monitor:
-            monitor.wait_until("Starting main service thread.", timeout_sec=30, backoff_sec=.25,
+            monitor.wait_until("Starting %s process." % daemon_name, timeout_sec=60, backoff_sec=.25,
                                err_msg=("%s on %s didn't finish startup" % (daemon_name, node.name)))
 
     def wait_node(self, node, timeout_sec=None):
@@ -194,6 +205,22 @@ class TrogdorService(KafkaPathResolverMixin, Service):
                       HTTPAdapter(max_retries=Retry(total=4, backoff_factor=0.3)))
         return session
 
+    def _coordinator_post(self, path, message):
+        """
+        Make a POST request to the Trogdor coordinator.
+
+        :param path:            The URL path to use.
+        :param message:         The message object to send.
+        :return:                The response as an object.
+        """
+        url = self._coordinator_url(path)
+        self.logger.info("POST %s %s" % (url, message))
+        response = self.request_session().post(url, json=message,
+                                               timeout=TrogdorService.REQUEST_TIMEOUT,
+                                               headers=TrogdorService.REQUEST_HEADERS)
+        response.raise_for_status()
+        return response.json()
+
     def _coordinator_put(self, path, message):
         """
         Make a PUT request to the Trogdor coordinator.
@@ -226,24 +253,33 @@ class TrogdorService(KafkaPathResolverMixin, Service):
         response.raise_for_status()
         return response.json()
 
-    def create_fault(self, id, spec):
+    def create_task(self, id, spec):
         """
-        Create a new fault.
+        Create a new task.
 
-        :param id:          The fault id.
-        :param spec:        The fault spec.
+        :param id:          The task id.
+        :param spec:        The task spec.
         """
-        self._coordinator_put("fault", { "id": id, "spec": spec.message()})
+        self._coordinator_post("task/create", { "id": id, "spec": spec.message()})
+        return TrogdorTask(id, self)
 
-    def get_faults(self):
+    def stop_task(self, id):
         """
-        Get the faults which are on the coordinator.
+        Stop a task.
 
-        :returns:           A map of fault id strings to fault data objects.
-                            Fault data objects contain a 'spec' field with the spec
+        :param id:          The task id.
+        """
+        self._coordinator_put("task/stop", { "id": id })
+
+    def tasks(self):
+        """
+        Get the tasks which are on the coordinator.
+
+        :returns:           A map of task id strings to task state objects.
+                            Task state objects contain a 'spec' field with the spec
                             and a 'state' field with the state.
         """
-        return self._coordinator_get("faults", {})
+        return self._coordinator_get("tasks", {})
 
     def is_coordinator(self, node):
         return node == self.coordinator_node
@@ -253,3 +289,44 @@ class TrogdorService(KafkaPathResolverMixin, Service):
 
     def coordinator_class_name(self):
         return "org.apache.kafka.trogdor.coordinator.Coordinator"
+
+class TrogdorTask(object):
+    PENDING_STATE = "PENDING"
+    RUNNING_STATE = "RUNNING"
+    STOPPING_STATE = "STOPPING"
+    DONE_STATE = "DONE"
+
+    def __init__(self, id, trogdor):
+        self.id = id
+        self.trogdor = trogdor
+
+    def done(self):
+        """
+        Check if this task is done.
+
+        :raises RuntimeError:       If the task encountered an error.
+        :returns:                   True if the task is in DONE_STATE;
+                                    False if it is in a different state.
+        """
+        task_state = self.trogdor.tasks()["tasks"][self.id]
+        if task_state is None:
+            raise RuntimeError("Coordinator did not know about %s." % self.id)
+        error = task_state.get("error")
+        if error is None or error == "":
+            return task_state["state"] == TrogdorTask.DONE_STATE
+        raise RuntimeError("Failed to gracefully stop %s: got task error: %s" % (self.id, error))
+
+    def stop(self):
+        """
+        Stop this task.
+
+        :raises RuntimeError:       If the task encountered an error.
+        """
+        if self.done():
+            return
+        self.trogdor.stop_task(self.id)
+
+    def wait_for_done(self, timeout_sec=360):
+        wait_until(lambda: self.done(),
+                   timeout_sec=timeout_sec,
+                   err_msg="%s failed to finish in the expected amount of time." % self.id)

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tests/kafkatest/tests/tools/trogdor_test.py
----------------------------------------------------------------------
diff --git a/tests/kafkatest/tests/tools/trogdor_test.py b/tests/kafkatest/tests/tools/trogdor_test.py
index 026ecaf..44d00b2 100644
--- a/tests/kafkatest/tests/tools/trogdor_test.py
+++ b/tests/kafkatest/tests/tools/trogdor_test.py
@@ -19,8 +19,8 @@ from ducktape.cluster.cluster_spec import ClusterSpec
 from ducktape.mark.resource import cluster
 from ducktape.tests.test import Test
 from ducktape.utils.util import wait_until
-from kafkatest.services.trogdor.fault_spec import FaultSpec
-from kafkatest.services.trogdor.no_op_fault_spec import NoOpFaultSpec
+from kafkatest.services.trogdor.task_spec import TaskSpec
+from kafkatest.services.trogdor.no_op_task_spec import NoOpTaskSpec
 from kafkatest.services.trogdor.trogdor import TrogdorService
 from kafkatest.utils import node_is_reachable
 
@@ -58,14 +58,15 @@ class TrogdorTest(Test):
         Test that we can bring up Trogdor and create a no-op fault.
         """
         self.set_up_trogdor(3)
-        spec = NoOpFaultSpec(0, FaultSpec.MAX_DURATION_MS)
-        self.trogdor.create_fault("myfault", spec)
-        def check_for_faults():
-            faults = self.trogdor.get_faults()
-            self.logger.info("faults = %s" % faults)
+        spec = NoOpTaskSpec(0, TaskSpec.MAX_DURATION_MS)
+        self.trogdor.create_task("myfault", spec)
+        def check_for_myfault():
+            faults = self.trogdor.tasks()["tasks"]
+            self.logger.info("tasks = %s" % faults)
             return "myfault" in faults
-        wait_until(lambda: check_for_faults,
+        wait_until(lambda: check_for_myfault,
                    timeout_sec=10, backoff_sec=.2, err_msg="Failed to read back myfault.")
+        self.trogdor.stop_task("myfault")
 
     @cluster(num_nodes=4)
     def test_network_partition_fault(self):
@@ -73,12 +74,12 @@ class TrogdorTest(Test):
         Test that the network partition fault results in a true network partition between nodes.
         """
         self.set_up_trogdor(3)
-        spec = NetworkPartitionFaultSpec(0, FaultSpec.MAX_DURATION_MS,
+        spec = NetworkPartitionFaultSpec(0, TaskSpec.MAX_DURATION_MS,
                                             [[self.agent_nodes[0]], self.agent_nodes[1:]])
         assert 2 == len(spec.partitions)
         assert [self.agent_nodes[0].name] == spec.partitions[0]
         assert [self.agent_nodes[1].name, self.agent_nodes[2].name] == spec.partitions[1]
-        self.trogdor.create_fault("partition0", spec)
+        self.trogdor.create_task("partition0", spec)
         def verify_nodes_partitioned():
             if node_is_reachable(self.agent_nodes[0], self.agent_nodes[1]):
                 return False

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java
index 0ddf4c1..43334a1 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java
@@ -22,68 +22,37 @@ import net.sourceforge.argparse4j.inf.ArgumentParser;
 import net.sourceforge.argparse4j.inf.ArgumentParserException;
 import net.sourceforge.argparse4j.inf.Namespace;
 import org.apache.kafka.common.utils.Exit;
-import org.apache.kafka.common.utils.KafkaThread;
-import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Scheduler;
 import org.apache.kafka.trogdor.common.Node;
 import org.apache.kafka.trogdor.common.Platform;
-import org.apache.kafka.trogdor.fault.Fault;
-import org.apache.kafka.trogdor.fault.FaultSet;
-import org.apache.kafka.trogdor.fault.FaultSpec;
-import org.apache.kafka.trogdor.fault.RunningState;
-import org.apache.kafka.trogdor.rest.AgentFaultsResponse;
-import org.apache.kafka.trogdor.rest.CreateAgentFaultRequest;
+import org.apache.kafka.trogdor.rest.AgentStatusResponse;
+import org.apache.kafka.trogdor.rest.CreateWorkerRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerResponse;
 import org.apache.kafka.trogdor.rest.JsonRestServer;
+import org.apache.kafka.trogdor.rest.StopWorkerRequest;
+import org.apache.kafka.trogdor.rest.StopWorkerResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
 import static net.sourceforge.argparse4j.impl.Arguments.store;
 
 /**
  * The Trogdor agent.
  *
- * The agent process implements faults directly.
+ * The agent process runs tasks.
  */
 public final class Agent {
     private static final Logger log = LoggerFactory.getLogger(Agent.class);
 
     /**
-     * The clock to use for this agent.
-     */
-    private final Time time;
-
-    /**
      * The time at which this server was started.
      */
-    private final long startTimeMs;
-
-    /**
-     * The platform.
-     */
-    private final Platform platform;
+    private final long serverStartMs;
 
     /**
-     * The lock protecting shutdown and faultSet.
+     * The WorkerManager.
      */
-    private final ReentrantLock lock = new ReentrantLock();
-
-    /**
-     * The condition variable which the agent thread waits on.
-     */
-    private final Condition cond = lock.newCondition();
-
-    /**
-     * The agent runnable.
-     */
-    private final AgentRunnable runnable;
+    private final WorkerManager workerManager;
 
     /**
      * The REST server.
@@ -91,162 +60,18 @@ public final class Agent {
     private final JsonRestServer restServer;
 
     /**
-     * The agent thread.
-     */
-    private final KafkaThread thread;
-
-    /**
-     * The set of pending faults.
-     */
-    private final FaultSet pendingFaults = new FaultSet();
-
-    /**
-     * The set of faults which are running.
-     */
-    private final FaultSet runningFaults = new FaultSet();
-
-    /**
-     * The set of faults which are done.
-     */
-    private final FaultSet doneFaults = new FaultSet();
-
-    /**
-     * True if the server is shutting down.
-     */
-    private boolean shutdown = false;
-
-    class AgentRunnable implements Runnable {
-        @Override
-        public void run() {
-            log.info("Starting main service thread.");
-            try {
-                while (true) {
-                    List<Fault> toStart = new ArrayList<>();
-                    List<Fault> started = new ArrayList<>();
-                    List<Fault> toEnd = new ArrayList<>();
-                    List<Fault> ended = new ArrayList<>();
-                    long now = time.milliseconds();
-                    long nextWakeMs = now + (60L * 60L * 1000L);
-                    lock.lock();
-                    try {
-                        Iterator<Fault> pending = pendingFaults.iterateByStart();
-                        while (pending.hasNext()) {
-                            Fault fault = pending.next();
-                            toStart.add(fault);
-                            long endMs = fault.spec().startMs() + fault.spec().durationMs();
-                            nextWakeMs = Math.min(nextWakeMs, endMs);
-                            pending.remove();
-                        }
-                        Iterator<Fault> running = runningFaults.iterateByEnd();
-                        while (running.hasNext()) {
-                            Fault fault = running.next();
-                            RunningState state = (RunningState) fault.state();
-                            long endMs = state.startedMs() + fault.spec().durationMs();
-                            if (now < endMs) {
-                                nextWakeMs = Math.min(nextWakeMs, endMs);
-                                break;
-                            }
-                            toEnd.add(fault);
-                            running.remove();
-                        }
-                    } finally {
-                        lock.unlock();
-                    }
-                    for (Fault fault: toStart) {
-                        try {
-                            log.debug("Activating fault " + fault);
-                            fault.activate(now, platform);
-                            started.add(fault);
-                        } catch (Throwable e) {
-                            log.error("Error activating fault " + fault.id(), e);
-                            ended.add(fault);
-                        }
-                    }
-                    for (Fault fault: toEnd) {
-                        try {
-                            log.debug("Deactivating fault " + fault);
-                            fault.deactivate(now, platform);
-                        } catch (Throwable e) {
-                            log.error("Error deactivating fault " + fault.id(), e);
-                        } finally {
-                            ended.add(fault);
-                        }
-                    }
-                    lock.lock();
-                    try {
-                        for (Fault fault : started) {
-                            runningFaults.add(fault);
-                        }
-                        for (Fault fault : ended) {
-                            doneFaults.add(fault);
-                        }
-                        if (shutdown) {
-                            return;
-                        }
-                        if (nextWakeMs > now) {
-                            log.trace("Sleeping for {} ms", nextWakeMs - now);
-                            if (cond.await(nextWakeMs - now, TimeUnit.MILLISECONDS)) {
-                                log.trace("AgentRunnable woke up early");
-                            }
-                        }
-                        if (shutdown) {
-                            return;
-                        }
-                    } finally {
-                        lock.unlock();
-                    }
-                }
-            } catch (Throwable t) {
-                log.error("Unhandled exception in AgentRunnable", t);
-            } finally {
-                log.info("AgentRunnable shutting down.");
-                restServer.stop();
-                int numDeactivated = deactivateRunningFaults();
-                log.info("AgentRunnable deactivated {} fault(s).", numDeactivated);
-            }
-        }
-    }
-
-    private int deactivateRunningFaults() {
-        long now = time.milliseconds();
-        int numDeactivated = 0;
-        lock.lock();
-        try {
-            for (Iterator<Fault> iter = runningFaults.iterateByStart(); iter.hasNext(); ) {
-                Fault fault = iter.next();
-                try {
-                    numDeactivated++;
-                    iter.remove();
-                    fault.deactivate(now, platform);
-                } catch (Exception e) {
-                    log.error("Got exception while deactivating {}", fault, e);
-                } finally {
-                    doneFaults.add(fault);
-                }
-            }
-        } finally {
-            lock.unlock();
-        }
-        return numDeactivated;
-    }
-
-    /**
      * Create a new Agent.
      *
      * @param platform      The platform object to use.
-     * @param time          The timekeeper to use for this Agent.
+     * @param scheduler     The scheduler to use for this Agent.
      * @param restServer    The REST server to use.
      * @param resource      The AgentRestResoure to use.
      */
-    public Agent(Platform platform, Time time, JsonRestServer restServer,
-                 AgentRestResource resource) {
-        this.platform = platform;
-        this.time = time;
+    public Agent(Platform platform, Scheduler scheduler,
+                 JsonRestServer restServer, AgentRestResource resource) {
+        this.serverStartMs = scheduler.time().milliseconds();
+        this.workerManager = new WorkerManager(platform, scheduler);
         this.restServer = restServer;
-        this.startTimeMs = time.milliseconds();
-        this.runnable = new AgentRunnable();
-        this.thread = new KafkaThread("TrogdorAgentThread", runnable, false);
-        this.thread.start();
         resource.setAgent(this);
     }
 
@@ -254,64 +79,27 @@ public final class Agent {
         return this.restServer.port();
     }
 
-    public void beginShutdown() {
-        lock.lock();
-        try {
-            if (shutdown)
-                return;
-            this.shutdown = true;
-            cond.signalAll();
-        } finally {
-            lock.unlock();
-        }
+    public void beginShutdown() throws Exception {
+        restServer.beginShutdown();
+        workerManager.beginShutdown();
     }
 
-    public void waitForShutdown() {
-        try {
-            this.thread.join();
-        } catch (InterruptedException e) {
-            log.error("Interrupted while waiting for thread shutdown", e);
-            Thread.currentThread().interrupt();
-        }
+    public void waitForShutdown() throws Exception {
+        restServer.waitForShutdown();
+        workerManager.waitForShutdown();
     }
 
-    public long startTimeMs() {
-        return startTimeMs;
+    public AgentStatusResponse status() throws Exception {
+        return new AgentStatusResponse(serverStartMs, workerManager.workerStates());
     }
 
-    public AgentFaultsResponse faults() {
-        Map<String, AgentFaultsResponse.FaultData> faultData = new TreeMap<>();
-        lock.lock();
-        try {
-            updateFaultsResponse(faultData, pendingFaults);
-            updateFaultsResponse(faultData, runningFaults);
-            updateFaultsResponse(faultData, doneFaults);
-        } finally {
-            lock.unlock();
-        }
-        return new AgentFaultsResponse(faultData);
+    public CreateWorkerResponse createWorker(CreateWorkerRequest req) throws Exception {
+        workerManager.createWorker(req.id(), req.spec());
+        return new CreateWorkerResponse(req.spec());
     }
 
-    private void updateFaultsResponse(Map<String, AgentFaultsResponse.FaultData> faultData,
-                                      FaultSet faultSet) {
-        for (Iterator<Fault> iter = faultSet.iterateByStart();
-                iter.hasNext(); ) {
-            Fault fault = iter.next();
-            AgentFaultsResponse.FaultData data =
-                new AgentFaultsResponse.FaultData(fault.spec(), fault.state());
-            faultData.put(fault.id(), data);
-        }
-    }
-
-    public void createFault(CreateAgentFaultRequest request) throws ClassNotFoundException {
-        lock.lock();
-        try {
-            Fault fault = FaultSpec.Util.createFault(request.id(), request.spec());
-            pendingFaults.add(fault);
-            cond.signalAll();
-        } finally {
-            lock.unlock();
-        }
+    public StopWorkerResponse stopWorker(StopWorkerRequest req) throws Exception {
+        return new StopWorkerResponse(workerManager.stopWorker(req.id()));
     }
 
     public static void main(String[] args) throws Exception {
@@ -352,14 +140,19 @@ public final class Agent {
         JsonRestServer restServer =
             new JsonRestServer(Node.Util.getTrogdorAgentPort(platform.curNode()));
         AgentRestResource resource = new AgentRestResource();
-        final Agent agent = new Agent(platform, Time.SYSTEM, restServer, resource);
+        log.info("Starting agent process.");
+        final Agent agent = new Agent(platform, Scheduler.SYSTEM, restServer, resource);
         restServer.start(resource);
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
-                log.error("Running shutdown hook...");
-                agent.beginShutdown();
-                agent.waitForShutdown();
+                log.warn("Running agent shutdown hook.");
+                try {
+                    agent.beginShutdown();
+                    agent.waitForShutdown();
+                } catch (Exception e) {
+                    log.error("Got exception while running agent shutdown hook.", e);
+                }
             }
         });
         agent.waitForShutdown();

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java
index 928820b..9c06591 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java
@@ -25,12 +25,14 @@ import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup;
 import net.sourceforge.argparse4j.inf.Namespace;
 import org.apache.kafka.common.utils.Exit;
 import org.apache.kafka.trogdor.common.JsonUtil;
-import org.apache.kafka.trogdor.rest.AgentFaultsResponse;
 import org.apache.kafka.trogdor.rest.AgentStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateAgentFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerResponse;
 import org.apache.kafka.trogdor.rest.Empty;
 import org.apache.kafka.trogdor.rest.JsonRestServer;
 import org.apache.kafka.trogdor.rest.JsonRestServer.HttpResponse;
+import org.apache.kafka.trogdor.rest.StopWorkerRequest;
+import org.apache.kafka.trogdor.rest.StopWorkerResponse;
 
 import static net.sourceforge.argparse4j.impl.Arguments.store;
 import static net.sourceforge.argparse4j.impl.Arguments.storeTrue;
@@ -40,51 +42,64 @@ import static net.sourceforge.argparse4j.impl.Arguments.storeTrue;
  */
 public class AgentClient {
     /**
+     * The maximum number of tries to make.
+     */
+    private final int maxTries;
+
+    /**
      * The URL target.
      */
     private final String target;
 
-    public AgentClient(String host, int port) {
-        this(String.format("%s:%d", host, port));
+    public AgentClient(int maxTries, String host, int port) {
+        this(maxTries, String.format("%s:%d", host, port));
     }
 
-    public AgentClient(String target) {
+    public AgentClient(int maxTries, String target) {
         this.target = target;
+        this.maxTries = maxTries;
     }
 
     public String target() {
         return target;
     }
 
+    public int maxTries() {
+        return maxTries;
+    }
+
     private String url(String suffix) {
         return String.format("http://%s%s", target, suffix);
     }
 
-    public AgentStatusResponse getStatus() throws Exception {
+    public AgentStatusResponse status() throws Exception {
         HttpResponse<AgentStatusResponse> resp =
             JsonRestServer.<AgentStatusResponse>httpRequest(url("/agent/status"), "GET",
-                null, new TypeReference<AgentStatusResponse>() { });
+                null, new TypeReference<AgentStatusResponse>() { }, maxTries);
         return resp.body();
     }
 
-    public AgentFaultsResponse getFaults() throws Exception {
-        HttpResponse<AgentFaultsResponse> resp =
-            JsonRestServer.<AgentFaultsResponse>httpRequest(url("/agent/faults"), "GET",
-                null, new TypeReference<AgentFaultsResponse>() { });
+    public CreateWorkerResponse createWorker(CreateWorkerRequest request) throws Exception {
+        HttpResponse<CreateWorkerResponse> resp =
+            JsonRestServer.<CreateWorkerResponse>httpRequest(
+                url("/agent/worker/create"), "POST",
+                request, new TypeReference<CreateWorkerResponse>() { }, maxTries);
         return resp.body();
     }
 
-    public void putFault(CreateAgentFaultRequest request) throws Exception {
-        HttpResponse<AgentFaultsResponse> resp =
-            JsonRestServer.<AgentFaultsResponse>httpRequest(url("/agent/fault"), "PUT",
-                request, new TypeReference<AgentFaultsResponse>() { });
-        resp.body();
+    public StopWorkerResponse stopWorker(StopWorkerRequest request) throws Exception {
+        HttpResponse<StopWorkerResponse> resp =
+            JsonRestServer.<StopWorkerResponse>httpRequest(url(
+                "/agent/worker/stop"), "PUT",
+                request, new TypeReference<StopWorkerResponse>() { }, maxTries);
+        return resp.body();
     }
 
     public void invokeShutdown() throws Exception {
         HttpResponse<Empty> resp =
-            JsonRestServer.<Empty>httpRequest(url("/agent/shutdown"), "PUT",
-                null, new TypeReference<Empty>() { });
+            JsonRestServer.<Empty>httpRequest(url(
+                "/agent/shutdown"), "PUT",
+                null, new TypeReference<Empty>() { }, maxTries);
         resp.body();
     }
 
@@ -106,16 +121,17 @@ public class AgentClient {
             .type(Boolean.class)
             .dest("status")
             .help("Get agent status.");
-        actions.addArgument("--get-faults")
-            .action(storeTrue())
-            .type(Boolean.class)
-            .dest("get_faults")
-            .help("Get agent faults.");
-        actions.addArgument("--create-fault")
+        actions.addArgument("--create-worker")
+            .action(store())
+            .type(String.class)
+            .dest("create_worker")
+            .metavar("SPEC_JSON")
+            .help("Create a new fault.");
+        actions.addArgument("--stop-worker")
             .action(store())
             .type(String.class)
-            .dest("create_fault")
-            .metavar("FAULT_JSON")
+            .dest("stop_worker")
+            .metavar("SPEC_JSON")
             .help("Create a new fault.");
         actions.addArgument("--shutdown")
             .action(storeTrue())
@@ -136,16 +152,14 @@ public class AgentClient {
             }
         }
         String target = res.getString("target");
-        AgentClient client = new AgentClient(target);
+        AgentClient client = new AgentClient(3, target);
         if (res.getBoolean("status")) {
             System.out.println("Got agent status: " +
-                JsonUtil.toPrettyJsonString(client.getStatus()));
-        } else if (res.getBoolean("get_faults")) {
-            System.out.println("Got agent faults: " +
-                JsonUtil.toPrettyJsonString(client.getFaults()));
-        } else if (res.getString("create_fault") != null) {
-            client.putFault(JsonUtil.JSON_SERDE.readValue(res.getString("create_fault"),
-                CreateAgentFaultRequest.class));
+                JsonUtil.toPrettyJsonString(client.status()));
+        } else if (res.getString("create_worker") != null) {
+            client.createWorker(JsonUtil.JSON_SERDE.
+                readValue(res.getString("create_worker"),
+                    CreateWorkerRequest.class));
             System.out.println("Created fault.");
         } else if (res.getBoolean("shutdown")) {
             client.invokeShutdown();

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java
index 690fa68..773c580 100644
--- a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java
+++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java
@@ -16,14 +16,17 @@
  */
 package org.apache.kafka.trogdor.agent;
 
-import org.apache.kafka.trogdor.rest.AgentFaultsResponse;
 import org.apache.kafka.trogdor.rest.AgentStatusResponse;
-import org.apache.kafka.trogdor.rest.CreateAgentFaultRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerRequest;
+import org.apache.kafka.trogdor.rest.CreateWorkerResponse;
 import org.apache.kafka.trogdor.rest.Empty;
+import org.apache.kafka.trogdor.rest.StopWorkerRequest;
+import org.apache.kafka.trogdor.rest.StopWorkerResponse;
 
 import javax.servlet.ServletContext;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
@@ -47,20 +50,19 @@ public class AgentRestResource {
     @GET
     @Path("/status")
     public AgentStatusResponse getStatus() throws Throwable {
-        return new AgentStatusResponse(agent().startTimeMs());
+        return agent().status();
     }
 
-    @GET
-    @Path("/faults")
-    public AgentFaultsResponse getAgentFaults() throws Throwable {
-        return agent().faults();
+    @POST
+    @Path("/worker/create")
+    public CreateWorkerResponse createWorker(CreateWorkerRequest req) throws Throwable {
+        return agent().createWorker(req);
     }
 
     @PUT
-    @Path("/fault")
-    public Empty putAgentFault(CreateAgentFaultRequest request) throws Throwable {
-        agent().createFault(request);
-        return Empty.INSTANCE;
+    @Path("/worker/stop")
+    public StopWorkerResponse stopWorker(StopWorkerRequest req) throws Throwable {
+        return agent().stopWorker(req);
     }
 
     @PUT

http://git-wip-us.apache.org/repos/asf/kafka/blob/4fac83ba/tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java
----------------------------------------------------------------------
diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java b/tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java
new file mode 100644
index 0000000..3c03e1e
--- /dev/null
+++ b/tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java
@@ -0,0 +1,574 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.trogdor.agent;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaFuture;
+import org.apache.kafka.common.internals.KafkaFutureImpl;
+import org.apache.kafka.common.utils.Scheduler;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.trogdor.common.Platform;
+import org.apache.kafka.trogdor.common.ThreadUtils;
+import org.apache.kafka.trogdor.rest.WorkerDone;
+import org.apache.kafka.trogdor.rest.WorkerRunning;
+import org.apache.kafka.trogdor.rest.WorkerStarting;
+import org.apache.kafka.trogdor.rest.WorkerStopping;
+import org.apache.kafka.trogdor.rest.WorkerState;
+import org.apache.kafka.trogdor.task.TaskSpec;
+import org.apache.kafka.trogdor.task.TaskWorker;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+public final class WorkerManager {
+    private static final Logger log = LoggerFactory.getLogger(WorkerManager.class);
+
+    /**
+     * The platform to use.
+     */
+    private final Platform platform;
+
+    /**
+     * The name of this node.
+     */
+    private final String nodeName;
+
+    /**
+     * The scheduler to use.
+     */
+    private final Scheduler scheduler;
+
+    /**
+     * The clock to use.
+     */
+    private final Time time;
+
+    /**
+     * A map of task IDs to Work objects.
+     */
+    private final Map<String, Worker> workers;
+
+    /**
+     * An ExecutorService used to schedule events in the future.
+     */
+    private final ScheduledExecutorService stateChangeExecutor;
+
+    /**
+     * An ExecutorService used to clean up TaskWorkers.
+     */
+    private final ScheduledExecutorService workerCleanupExecutor;
+
+    /**
+     * An ExecutorService to help with shutting down.
+     */
+    private final ScheduledExecutorService shutdownExecutor;
+
+    /**
+     * The shutdown manager.
+     */
+    private final ShutdownManager shutdownManager = new ShutdownManager();
+
+    /**
+     * The shutdown manager handles shutting down gracefully.
+     *
+     * We can shut down gracefully only when all the references handed out
+     * by the ShutdownManager has been closed, and the shutdown bit has
+     * been set.  RPC operations hold a reference for the duration of their
+     * execution, and so do Workers which have not been shut down.
+     * This prevents us from shutting down in the middle of an RPC, or with
+     * workers which are still running.
+     */
+    static class ShutdownManager {
+        private boolean shutdown = false;
+        private long refCount = 0;
+
+        class Reference implements AutoCloseable {
+            AtomicBoolean closed = new AtomicBoolean(false);
+
+            @Override
+            public void close() {
+                if (closed.compareAndSet(false, true)) {
+                    synchronized (ShutdownManager.this) {
+                        refCount--;
+                        if (shutdown && (refCount == 0)) {
+                            ShutdownManager.this.notifyAll();
+                        }
+                    }
+                }
+            }
+        }
+
+        synchronized Reference takeReference() {
+            if (shutdown) {
+                throw new KafkaException("WorkerManager is shut down.");
+            }
+            refCount++;
+            return new Reference();
+        }
+
+        synchronized boolean shutdown() {
+            if (shutdown) {
+                return false;
+            }
+            shutdown = true;
+            return true;
+        }
+
+        synchronized void waitForQuiescence() throws InterruptedException {
+            while ((!shutdown) || (refCount > 0)) {
+                wait();
+            }
+        }
+    }
+
+    WorkerManager(Platform platform, Scheduler scheduler) {
+        this.platform = platform;
+        this.nodeName = platform.curNode().name();
+        this.scheduler = scheduler;
+        this.time = scheduler.time();
+        this.workers = new HashMap<>();
+        this.stateChangeExecutor = Executors.newSingleThreadScheduledExecutor(
+                ThreadUtils.createThreadFactory("WorkerManagerStateThread", false));
+        this.workerCleanupExecutor = Executors.newScheduledThreadPool(1,
+            ThreadUtils.createThreadFactory("WorkerCleanupThread%d", false));
+        this.shutdownExecutor = Executors.newScheduledThreadPool(0,
+            ThreadUtils.createThreadFactory("WorkerManagerShutdownThread%d", false));
+    }
+
+    enum State {
+        STARTING,
+        CANCELLING,
+        RUNNING,
+        STOPPING,
+        DONE,
+    }
+
+    /**
+     * A worker which is being tracked.
+     */
+    class Worker {
+        /**
+         * The task ID.
+         */
+        private final String id;
+
+        /**
+         * The task specification.
+         */
+        private final TaskSpec spec;
+
+        /**
+         * The work which this worker is performing.
+         */
+        private final TaskWorker taskWorker;
+
+        /**
+         * The worker status.
+         */
+        private final AtomicReference<String> status = new AtomicReference<>("");
+
+        /**
+         * The time when this task was started.
+         */
+        private final long startedMs;
+
+        /**
+         * The work state.
+         */
+        private State state = State.STARTING;
+
+        /**
+         * The time when this task was completed, or -1 if it has not been.
+         */
+        private long doneMs = -1;
+
+        /**
+         * The worker error.
+         */
+        private String error = "";
+
+        /**
+         * If there is a task timeout scheduled, this is a future which can
+         * be used to cancel it.
+         */
+        private Future<TaskSpec> timeoutFuture = null;
+
+        /**
+         * A shutdown manager reference which will keep the WorkerManager
+         * alive for as long as this worker is alive.
+         */
+        private ShutdownManager.Reference reference;
+
+        Worker(String id, TaskSpec spec, long now) {
+            this.id = id;
+            this.spec = spec;
+            this.taskWorker = spec.newTaskWorker(id);
+            this.startedMs = now;
+            this.reference = shutdownManager.takeReference();
+        }
+
+        String id() {
+            return id;
+        }
+
+        TaskSpec spec() {
+            return spec;
+        }
+
+        WorkerState state() {
+            switch (state) {
+                case STARTING:
+                    return new WorkerStarting(spec);
+                case RUNNING:
+                    return new WorkerRunning(spec, startedMs, status.get());
+                case CANCELLING:
+                case STOPPING:
+                    return new WorkerStopping(spec, startedMs, status.get());
+                case DONE:
+                    return new WorkerDone(spec, startedMs, doneMs, status.get(), error);
+            }
+            throw new RuntimeException("unreachable");
+        }
+
+        void transitionToRunning() {
+            state = State.RUNNING;
+            timeoutFuture = scheduler.schedule(stateChangeExecutor,
+                new StopWorker(id), spec.durationMs());
+        }
+
+        void transitionToStopping() {
+            state = State.STOPPING;
+            if (timeoutFuture != null) {
+                timeoutFuture.cancel(false);
+                timeoutFuture = null;
+            }
+            workerCleanupExecutor.submit(new CleanupWorker(this));
+        }
+
+        void transitionToDone() {
+            state = State.DONE;
+            doneMs = time.milliseconds();
+            if (reference != null) {
+                reference.close();
+                reference = null;
+            }
+        }
+    }
+
+    public void createWorker(final String id, TaskSpec spec) throws Exception {
+        try (ShutdownManager.Reference ref = shutdownManager.takeReference()) {
+            final Worker worker = stateChangeExecutor.
+                submit(new CreateWorker(id, spec, time.milliseconds())).get();
+            if (worker == null) {
+                log.info("{}: Ignoring request to create worker {}, because there is already " +
+                    "a worker with that id.", nodeName, id);
+                return;
+            }
+            KafkaFutureImpl<String> haltFuture = new KafkaFutureImpl<>();
+            haltFuture.thenApply(new KafkaFuture.Function<String, Void>() {
+                @Override
+                public Void apply(String errorString) {
+                    if (errorString.isEmpty()) {
+                        log.info("{}: Worker {} is halting.", nodeName, id);
+                    } else {
+                        log.info("{}: Worker {} is halting with error {}", nodeName, id, errorString);
+                    }
+                    stateChangeExecutor.submit(
+                        new HandleWorkerHalting(worker, errorString, false));
+                    return null;
+                }
+            });
+            try {
+                worker.taskWorker.start(platform, worker.status, haltFuture);
+            } catch (Exception e) {
+                stateChangeExecutor.submit(new HandleWorkerHalting(worker,
+                    "worker.start() exception: " + e.getMessage(), true));
+            }
+            stateChangeExecutor.submit(new FinishCreatingWorker(worker));
+        }
+    }
+
+    /**
+     * Handles a request to create a new worker.  Processed by the state change thread.
+     */
+    class CreateWorker implements Callable<Worker> {
+        private final String id;
+        private final TaskSpec spec;
+        private final long now;
+
+        CreateWorker(String id, TaskSpec spec, long now) {
+            this.id = id;
+            this.spec = spec;
+            this.now = now;
+        }
+
+        @Override
+        public Worker call() throws Exception {
+            Worker worker = workers.get(id);
+            if (worker != null) {
+                log.info("{}: Task ID {} is already in use.", nodeName, id);
+                return null;
+            }
+            worker = new Worker(id, spec, now);
+            workers.put(id, worker);
+            log.info("{}: Created a new worker for task {} with spec {}", nodeName, id, spec);
+            return worker;
+        }
+    }
+
+    /**
+     * Finish creating a Worker.  Processed by the state change thread.
+     */
+    class FinishCreatingWorker implements Callable<Void> {
+        private final Worker worker;
+
+        FinishCreatingWorker(Worker worker) {
+            this.worker = worker;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            switch (worker.state) {
+                case CANCELLING:
+                    log.info("{}: Worker {} was cancelled while it was starting up.  " +
+                        "Transitioning to STOPPING.", nodeName, worker.id);
+                    worker.transitionToStopping();
+                    break;
+                case STARTING:
+                    log.info("{}: Worker {} is now RUNNING.  Scheduled to stop in {} ms.",
+                        nodeName, worker.id, worker.spec.durationMs());
+                    worker.transitionToRunning();
+                    break;
+                default:
+                    break;
+            }
+            return null;
+        }
+    }
+
+    /**
+     * Handles a worker halting.  Processed by the state change thread.
+     */
+    class HandleWorkerHalting implements Callable<Void> {
+        private final Worker worker;
+        private final String failure;
+        private final boolean startupHalt;
+
+        HandleWorkerHalting(Worker worker, String failure, boolean startupHalt) {
+            this.worker = worker;
+            this.failure = failure;
+            this.startupHalt = startupHalt;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            if (worker.error.isEmpty()) {
+                worker.error = failure;
+            }
+            String verb = (worker.error.isEmpty()) ? "halting" :
+                "halting with error [" + worker.error + "]";
+            switch (worker.state) {
+                case STARTING:
+                    if (startupHalt) {
+                        log.info("{}: Worker {} {} during startup.  Transitioning to DONE.",
+                            nodeName, worker.id, verb);
+                        worker.transitionToDone();
+                    } else {
+                        log.info("{}: Worker {} {} during startup.  Transitioning to CANCELLING.",
+                            nodeName, worker.id, verb);
+                        worker.state = State.CANCELLING;
+                    }
+                    break;
+                case CANCELLING:
+                    log.info("{}: Cancelling worker {} {}.  ",
+                            nodeName, worker.id, verb);
+                    break;
+                case RUNNING:
+                    log.info("{}: Running worker {} {}.  Transitioning to STOPPING.",
+                        nodeName, worker.id, verb);
+                    worker.transitionToStopping();
+                    break;
+                case STOPPING:
+                    log.info("{}: Stopping worker {} {}.", nodeName, worker.id, verb);
+                    break;
+                case DONE:
+                    log.info("{}: Can't halt worker {} because it is already DONE.",
+                        nodeName, worker.id);
+                    break;
+            }
+            return null;
+        }
+    }
+
+    /**
+     * Transitions a worker to WorkerDone.  Processed by the state change thread.
+     */
+    static class CompleteWorker implements Callable<Void> {
+        private final Worker worker;
+
+        private final String failure;
+
+        CompleteWorker(Worker worker, String failure) {
+            this.worker = worker;
+            this.failure = failure;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            if (worker.error.isEmpty() && !failure.isEmpty()) {
+                worker.error = failure;
+            }
+            worker.transitionToDone();
+            return null;
+        }
+    }
+
+    public TaskSpec stopWorker(String id) throws Exception {
+        try (ShutdownManager.Reference ref = shutdownManager.takeReference()) {
+            TaskSpec taskSpec = stateChangeExecutor.submit(new StopWorker(id)).get();
+            if (taskSpec == null) {
+                throw new KafkaException("No task found with id " + id);
+            }
+            return taskSpec;
+        }
+    }
+
+    /**
+     * Stops a worker.  Processed by the state change thread.
+     */
+    class StopWorker implements Callable<TaskSpec> {
+        private final String id;
+
+        StopWorker(String id) {
+            this.id = id;
+        }
+
+        @Override
+        public TaskSpec call() throws Exception {
+            Worker worker = workers.get(id);
+            if (worker == null) {
+                return null;
+            }
+            switch (worker.state) {
+                case STARTING:
+                    log.info("{}: Cancelling worker {} during its startup process.",
+                        nodeName, id);
+                    worker.state = State.CANCELLING;
+                    break;
+                case CANCELLING:
+                    log.info("{}: Can't stop worker {}, because it is already being " +
+                        "cancelled.", nodeName, id);
+                    break;
+                case RUNNING:
+                    log.info("{}: Stopping running worker {}.", nodeName, id);
+                    worker.transitionToStopping();
+                    break;
+                case STOPPING:
+                    log.info("{}: Can't stop worker {}, because it is already " +
+                            "stopping.", nodeName, id);
+                    break;
+                case DONE:
+                    log.debug("{}: Can't stop worker {}, because it is already done.",
+                        nodeName, id);
+                    break;
+            }
+            return worker.spec();
+        }
+    }
+
+    /**
+     * Cleans up the resources associated with a worker.  Processed by the worker
+     * cleanup thread pool.
+     */
+    class CleanupWorker implements Callable<Void> {
+        private final Worker worker;
+
+        CleanupWorker(Worker worker) {
+            this.worker = worker;
+        }
+
+        @Override
+        public Void call() throws Exception {
+            String failure = "";
+            try {
+                worker.taskWorker.stop(platform);
+            } catch (Exception exception) {
+                log.error("{}: worker.stop() exception", nodeName, exception);
+                failure = exception.getMessage();
+            }
+            stateChangeExecutor.submit(new CompleteWorker(worker, failure));
+            return null;
+        }
+    }
+
+    public TreeMap<String, WorkerState> workerStates() throws Exception {
+        try (ShutdownManager.Reference ref = shutdownManager.takeReference()) {
+            return stateChangeExecutor.submit(new GetWorkerStates()).get();
+        }
+    }
+
+    class GetWorkerStates implements Callable<TreeMap<String, WorkerState>> {
+        @Override
+        public TreeMap<String, WorkerState> call() throws Exception {
+            TreeMap<String, WorkerState> workerMap = new TreeMap<>();
+            for (Worker worker : workers.values()) {
+                workerMap.put(worker.id(), worker.state());
+            }
+            return workerMap;
+        }
+    }
+
+    public void beginShutdown() throws Exception {
+        if (shutdownManager.shutdown()) {
+            shutdownExecutor.submit(new Shutdown());
+        }
+    }
+
+    public void waitForShutdown() throws Exception {
+        while (!shutdownExecutor.isShutdown()) {
+            shutdownExecutor.awaitTermination(1, TimeUnit.DAYS);
+        }
+    }
+
+    class Shutdown implements Callable<Void> {
+        @Override
+        public Void call() throws Exception {
+            log.info("{}: Shutting down WorkerManager.", platform.curNode().name());
+            for (Worker worker : workers.values()) {
+                stateChangeExecutor.submit(new StopWorker(worker.id));
+            }
+            shutdownManager.waitForQuiescence();
+            workerCleanupExecutor.shutdownNow();
+            stateChangeExecutor.shutdownNow();
+            workerCleanupExecutor.awaitTermination(1, TimeUnit.DAYS);
+            stateChangeExecutor.awaitTermination(1, TimeUnit.DAYS);
+            shutdownExecutor.shutdown();
+            return null;
+        }
+    }
+}