diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 89173540d400923c435d6502381cb0a036c609a4..c31e2e7064715cfdcf44df7ff934defc52e3f1a7 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -30,9 +30,22 @@ import spark.TaskEndReason
 
 import spark.{FetchFailed, Success}
 
+/**
+ * Tests for DAGScheduler. These tests directly call the event processing functinos in DAGScheduler
+ * rather than spawning an event loop thread as happens in the real code. They use EasyMock
+ * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are
+ * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead
+ * host notifications are sent). In addition, tests may check for side effects on a non-mocked
+ * MapOutputTracker instance.
+ *
+ * Tests primarily consist of running DAGScheduler#processEvent and
+ * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet)
+ * and capturing the resulting TaskSets from the mock TaskScheduler.
+ */
 class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar with TimeLimitedTests {
 
-  // impose a time limit on this test in case we don't let the job finish.
+  // impose a time limit on this test in case we don't let the job finish, in which case
+  // JobWaiter#getResult will hang.
   override val timeLimit = Span(5, Seconds)
 
   val sc: SparkContext = new SparkContext("local", "DAGSchedulerSuite")