diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala
index 6018d251d19e9404dff10dbe5d6b32e10eacbacf..86e2061b9f0cb47c6bce071f87ece0806a8af62f 100644
--- a/core/src/main/scala/spark/Accumulators.scala
+++ b/core/src/main/scala/spark/Accumulators.scala
@@ -6,8 +6,8 @@ import scala.collection.mutable.Map
 
 class Accumulator[T] (
     @transient initialValue: T,
-    param: AccumulatorParam[T]
-    ) extends Serializable {
+    param: AccumulatorParam[T])
+  extends Serializable {
   
   val id = Accumulators.newId
   @transient
diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala
index 36e70f74032d56bf8bf23318fde4173344def56b..a721a136b253cb333463ea5eb281526dc6220d96 100644
--- a/core/src/main/scala/spark/Aggregator.scala
+++ b/core/src/main/scala/spark/Aggregator.scala
@@ -1,7 +1,7 @@
 package spark
 
 class Aggregator[K, V, C] (
-  val createCombiner: V => C,
-  val mergeValue: (C, V) => C,
-  val mergeCombiners: (C, C) => C
-) extends Serializable
\ No newline at end of file
+    val createCombiner: V => C,
+    val mergeValue: (C, V) => C,
+    val mergeCombiners: (C, C) => C
+  ) extends Serializable
\ No newline at end of file
diff --git a/core/src/main/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala
index 10143d3dd22bb24ab2a05044facd84f5636cea70..9735b9b1a40b199e0b66f47b289f35592e8abdf8 100644
--- a/core/src/main/scala/spark/BoundedMemoryCache.scala
+++ b/core/src/main/scala/spark/BoundedMemoryCache.scala
@@ -3,13 +3,11 @@ package spark
 import java.util.LinkedHashMap
 
 /**
- * An implementation of Cache that estimates the sizes of its entries and
- * attempts to limit its total memory usage to a fraction of the JVM heap.
- * Objects' sizes are estimated using SizeEstimator, which has limitations;
- * most notably, we will overestimate total memory used if some cache
- * entries have pointers to a shared object. Nonetheless, this Cache should
- * work well when most of the space is used by arrays of primitives or of
- * simple classes.
+ * An implementation of Cache that estimates the sizes of its entries and attempts to limit its
+ * total memory usage to a fraction of the JVM heap. Objects' sizes are estimated using
+ * SizeEstimator, which has limitations; most notably, we will overestimate total memory used if
+ * some cache entries have pointers to a shared object. Nonetheless, this Cache should work well
+ * when most of the space is used by arrays of primitives or of simple classes.
  */
 class BoundedMemoryCache extends Cache with Logging {
   private val maxBytes: Long = getMaxBytes()
@@ -24,7 +22,11 @@ class BoundedMemoryCache extends Cache with Logging {
   override def get(key: Any): Any = {
     synchronized {
       val entry = map.get(key)
-      if (entry != null) entry.value else null
+      if (entry != null) {
+        entry.value
+      } else {
+        null
+      }
     }
   }
 
@@ -51,8 +53,8 @@ class BoundedMemoryCache extends Cache with Logging {
   }
 
   /**
-   * Remove least recently used entries from the map until at least space
-   * bytes are free. Assumes that a lock is held on the BoundedMemoryCache.
+   * Remove least recently used entries from the map until at least space bytes are free. Assumes
+   * that a lock is held on the BoundedMemoryCache.
    */
   private def ensureFreeSpace(space: Long) {
     logInfo("ensureFreeSpace(%d) called with curBytes=%d, maxBytes=%d".format(
@@ -67,7 +69,6 @@ class BoundedMemoryCache extends Cache with Logging {
   }
 
   protected def dropEntry(key: Any, entry: Entry) {
-    logInfo("Dropping key %s of size %d to make space".format(
-      key, entry.size))
+    logInfo("Dropping key %s of size %d to make space".format(key, entry.size))
   }
 }
diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala
index 24ac88c14f9c2f410bf8214882ef74705e0566aa..696fff4e5e712def8c425377eadf260e5a285853 100644
--- a/core/src/main/scala/spark/Cache.scala
+++ b/core/src/main/scala/spark/Cache.scala
@@ -3,20 +3,18 @@ package spark
 import java.util.concurrent.atomic.AtomicLong
 
 /**
- * An interface for caches in Spark, to allow for multiple implementations.
- * Caches are used to store both partitions of cached RDDs and broadcast
- * variables on Spark executors.
+ * An interface for caches in Spark, to allow for multiple implementations. Caches are used to store
+ * both partitions of cached RDDs and broadcast variables on Spark executors.
  *
- * A single Cache instance gets created on each machine and is shared by all
- * caches (i.e. both the RDD split cache and the broadcast variable cache),
- * to enable global replacement policies. However, because these several
- * independent modules all perform caching, it is important to give them
- * separate key namespaces, so that an RDD and a broadcast variable (for
- * example) do not use the same key. For this purpose, Cache has the
- * notion of KeySpaces. Each client module must first ask for a KeySpace,
- * and then call get() and put() on that space using its own keys.
- * This abstract class handles the creation of key spaces, so that subclasses
- * need only deal with keys that are unique across modules.
+ * A single Cache instance gets created on each machine and is shared by all caches (i.e. both the 
+ * RDD split cache and the broadcast variable cache), to enable global replacement policies. 
+ * However, because these several independent modules all perform caching, it is important to give
+ * them separate key namespaces, so that an RDD and a broadcast variable (for example) do not use
+ * the same key. For this purpose, Cache has the notion of KeySpaces. Each client module must first
+ * ask for a KeySpace, and then call get() and put() on that space using its own keys.
+ * 
+ * This abstract class handles the creation of key spaces, so that subclasses need only deal with
+ * keys that are unique across modules.
  */
 abstract class Cache {
   private val nextKeySpaceId = new AtomicLong(0)
diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala
index 223c5dc5f7d555efc9966190ea24b8dce88627e8..5e9a70cc7e03813cd31fd4d24043c77add1aceb7 100644
--- a/core/src/main/scala/spark/CacheTracker.scala
+++ b/core/src/main/scala/spark/CacheTracker.scala
@@ -96,15 +96,16 @@ class CacheTracker(isMaster: Boolean, theCache: Cache) extends Logging {
   // Get a snapshot of the currently known locations
   def getLocationsSnapshot(): HashMap[Int, Array[List[String]]] = {
     (trackerActor !? GetCacheLocations) match {
-      case h: HashMap[_, _] => h.asInstanceOf[HashMap[Int, Array[List[String]]]]
-      case _ => throw new SparkException(
-          "Internal error: CacheTrackerActor did not reply with a HashMap")
+      case h: HashMap[_, _] =>
+        h.asInstanceOf[HashMap[Int, Array[List[String]]]]
+        
+      case _ => 
+        throw new SparkException("Internal error: CacheTrackerActor did not reply with a HashMap")
     }
   }
   
   // Gets or computes an RDD split
-  def getOrCompute[T](rdd: RDD[T], split: Split)(implicit m: ClassManifest[T])
-      : Iterator[T] = {
+  def getOrCompute[T](rdd: RDD[T], split: Split)(implicit m: ClassManifest[T]): Iterator[T] = {
     val key = (rdd.id, split.index)
     logInfo("CachedRDD partition key is " + key)
     val cachedVal = cache.get(key)
diff --git a/core/src/main/scala/spark/CartesianRDD.scala b/core/src/main/scala/spark/CartesianRDD.scala
index df822b3552a44a13600b5e8b33954948f47895b4..38afa59b29d1d379648774658f699e99c074a149 100644
--- a/core/src/main/scala/spark/CartesianRDD.scala
+++ b/core/src/main/scala/spark/CartesianRDD.scala
@@ -1,16 +1,20 @@
 package spark
 
-class CartesianSplit(idx: Int, val s1: Split, val s2: Split)
-extends Split with Serializable {
+class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
   override val index = idx
 }
 
 class CartesianRDD[T: ClassManifest, U:ClassManifest](
-  sc: SparkContext, rdd1: RDD[T], rdd2: RDD[U])
-extends RDD[Pair[T, U]](sc) with Serializable {
+    sc: SparkContext,
+    rdd1: RDD[T],
+    rdd2: RDD[U])
+  extends RDD[Pair[T, U]](sc)
+  with Serializable {
+  
   val numSplitsInRdd2 = rdd2.splits.size
   
-  @transient val splits_ = {
+  @transient
+  val splits_ = {
     // create the cross product split
     val array = new Array[Split](rdd1.splits.size * rdd2.splits.size)
     for (s1 <- rdd1.splits; s2 <- rdd2.splits) {
diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala
index 62d2c4cb129e26441182cc2ae709b79455cfdc70..699fdc29820ef21f61860c10544cd6841e7139e0 100644
--- a/core/src/main/scala/spark/ClosureCleaner.scala
+++ b/core/src/main/scala/spark/ClosureCleaner.scala
@@ -69,10 +69,11 @@ object ClosureCleaner extends Logging {
   }
   
   private def createNullValue(cls: Class[_]): AnyRef = {
-    if (cls.isPrimitive)
+    if (cls.isPrimitive) {
       new java.lang.Byte(0: Byte) // Should be convertible to any primitive type
-    else
+    } else {
       null
+    }
   }
   
   def clean(func: AnyRef): Unit = {
@@ -157,26 +158,28 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor
   override def visitMethod(access: Int, name: String, desc: String,
       sig: String, exceptions: Array[String]): MethodVisitor = {
     return new EmptyVisitor {
-      override def visitFieldInsn(op: Int, owner: String, name: String,
-          desc: String) {
-        if (op == GETFIELD)
-          for (cl <- output.keys if cl.getName == owner.replace('/', '.'))
+      override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) {
+        if (op == GETFIELD) {
+          for (cl <- output.keys if cl.getName == owner.replace('/', '.')) {
             output(cl) += name
+          }
+        }
       }
       
       override def visitMethodInsn(op: Int, owner: String, name: String,
           desc: String) {
         // Check for calls a getter method for a variable in an interpreter wrapper object.
         // This means that the corresponding field will be accessed, so we should save it.
-        if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer"))
-          for (cl <- output.keys if cl.getName == owner.replace('/', '.'))
+        if (op == INVOKEVIRTUAL && owner.endsWith("$iwC") && !name.endsWith("$outer")) {
+          for (cl <- output.keys if cl.getName == owner.replace('/', '.')) {
             output(cl) += name
+          }
+        }
       }
     }
   }
 }
 
-
 class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor {
   var myName: String = null
   
@@ -194,8 +197,10 @@ class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor {
         if (op == INVOKESPECIAL && name == "<init>" && argTypes.length > 0
             && argTypes(0).toString.startsWith("L") // is it an object?
             && argTypes(0).getInternalName == myName)
-          output += Class.forName(owner.replace('/', '.'), false,
-                                  Thread.currentThread.getContextClassLoader)
+          output += Class.forName(
+              owner.replace('/', '.'),
+              false,
+              Thread.currentThread.getContextClassLoader)
       }
     }
   }
diff --git a/core/src/main/scala/spark/CoGroupedRDD.scala b/core/src/main/scala/spark/CoGroupedRDD.scala
index 4a8fa6d3fc6e23dd30b62e24bc0325c7ee23c529..ed51f5ae47046ea7672f5c47eacf090d3adf84f3 100644
--- a/core/src/main/scala/spark/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/CoGroupedRDD.scala
@@ -10,20 +10,20 @@ sealed trait CoGroupSplitDep extends Serializable
 case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
 case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
 
-class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep])
-extends Split with Serializable {
+class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable {
   override val index = idx
   override def hashCode(): Int = idx
 }
 
 class CoGroupAggregator extends Aggregator[Any, Any, ArrayBuffer[Any]] (
-  { x => ArrayBuffer(x) },
-  { (b, x) => b += x },
-  { (b1, b2) => b1 ++ b2 }
-) with Serializable
+    { x => ArrayBuffer(x) },
+    { (b, x) => b += x },
+    { (b1, b2) => b1 ++ b2 }
+  ) with Serializable
 
 class CoGroupedRDD[K](rdds: Seq[RDD[(_, _)]], part: Partitioner)
-extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
+  extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
+  
   val aggr = new CoGroupAggregator
   
   override val dependencies = {
@@ -41,7 +41,8 @@ extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
     deps.toList
   }
   
-  @transient val splits_ : Array[Split] = {
+  @transient
+  val splits_ : Array[Split] = {
     val firstRdd = rdds.head
     val array = new Array[Split](part.numPartitions)
     for (i <- 0 until array.size) {
diff --git a/core/src/main/scala/spark/DAGScheduler.scala b/core/src/main/scala/spark/DAGScheduler.scala
index be6756aa482aab8b52f48ffc202d8cc259149d8a..b048b246a380db9bb89590ced98f5aee5e3ed8df 100644
--- a/core/src/main/scala/spark/DAGScheduler.scala
+++ b/core/src/main/scala/spark/DAGScheduler.scala
@@ -144,7 +144,8 @@ private trait DAGScheduler extends Scheduler with Logging {
                   if (!stage.isAvailable) {
                     missing += stage
                   }
-                case narrowDep: NarrowDependency[_] => visit(narrowDep.rdd)
+                case narrowDep: NarrowDependency[_] =>
+                  visit(narrowDep.rdd)
               }
             }
           }
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala
index bd20634fb9ac4ff583a4820d51b82d31e45af3f5..d93c84924a5038fb202157b907092591b1343ac8 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/spark/Dependency.scala
@@ -2,28 +2,29 @@ package spark
 
 abstract class Dependency[T](val rdd: RDD[T], val isShuffle: Boolean) extends Serializable
 
-abstract class NarrowDependency[T](rdd: RDD[T])
-extends Dependency(rdd, false) {
+abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd, false) {
   def getParents(outputPartition: Int): Seq[Int]
 }
 
 class ShuffleDependency[K, V, C](
-  val shuffleId: Int,
-  rdd: RDD[(K, V)],
-  val aggregator: Aggregator[K, V, C],
-  val partitioner: Partitioner
-) extends Dependency(rdd, true)
+    val shuffleId: Int,
+    rdd: RDD[(K, V)],
+    val aggregator: Aggregator[K, V, C],
+    val partitioner: Partitioner)
+  extends Dependency(rdd, true)
 
 class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
   override def getParents(partitionId: Int) = List(partitionId)
 }
 
 class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int)
-extends NarrowDependency[T](rdd) {
+  extends NarrowDependency[T](rdd) {
+  
   override def getParents(partitionId: Int) = {
-    if (partitionId >= outStart && partitionId < outStart + length)
+    if (partitionId >= outStart && partitionId < outStart + length) {
       List(partitionId - outStart + inStart)
-    else
+    } else {
       Nil
+    }
   }
 }
diff --git a/core/src/main/scala/spark/DiskSpillingCache.scala b/core/src/main/scala/spark/DiskSpillingCache.scala
index 80e13a25196b79d2695d06688b7a3da1c345d080..157e071c7f91f146aeaeeb043fdf314105c7c970 100644
--- a/core/src/main/scala/spark/DiskSpillingCache.scala
+++ b/core/src/main/scala/spark/DiskSpillingCache.scala
@@ -8,7 +8,6 @@ import java.util.UUID
 
 // TODO: cache into a separate directory using Utils.createTempDir
 // TODO: clean up disk cache afterwards
-
 class DiskSpillingCache extends BoundedMemoryCache {
   private val diskMap = new LinkedHashMap[Any, File](32, 0.75f, true)
 
diff --git a/core/src/main/scala/spark/Executor.scala b/core/src/main/scala/spark/Executor.scala
index 83d2df4f94bb515e74aabb866bba76d428b4cb65..71a2ded7e7ffcdff37b55c18ff07a47e053596cd 100644
--- a/core/src/main/scala/spark/Executor.scala
+++ b/core/src/main/scala/spark/Executor.scala
@@ -67,8 +67,9 @@ class Executor extends org.apache.mesos.Executor with Logging {
         Thread.currentThread.setContextClassLoader(classLoader)
         Accumulators.clear
         val task = Utils.deserialize[Task[Any]](desc.getData.toByteArray, classLoader)
-        for (gen <- task.generation) // Update generation if any is set
+        for (gen <- task.generation) {// Update generation if any is set
           env.mapOutputTracker.updateGeneration(gen)
+        }
         val value = task.run(tid.toInt)
         val accumUpdates = Accumulators.values
         val result = new TaskResult(value, accumUpdates)
diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala
index 6b6aea1ae42bda957123eba3dfe2b2261293ea04..db711e099c355d641417dc35eb1241096a021b76 100644
--- a/core/src/main/scala/spark/FetchFailedException.scala
+++ b/core/src/main/scala/spark/FetchFailedException.scala
@@ -1,13 +1,17 @@
 package spark
 
-class FetchFailedException(val serverUri: String, val shuffleId: Int,
-  val mapId: Int, val reduceId: Int, cause: Throwable)
-extends Exception {
+class FetchFailedException(
+    val serverUri: String,
+    val shuffleId: Int,
+    val mapId: Int,
+    val reduceId: Int,
+    cause: Throwable)
+  extends Exception {
+  
   override def getMessage(): String = 
     "Fetch failed: %s %d %d %d".format(serverUri, shuffleId, mapId, reduceId)
 
   override def getCause(): Throwable = cause
 
-  def toTaskEndReason: TaskEndReason =
-    FetchFailed(serverUri, shuffleId, mapId, reduceId)
+  def toTaskEndReason: TaskEndReason = FetchFailed(serverUri, shuffleId, mapId, reduceId)
 }
diff --git a/core/src/main/scala/spark/HadoopRDD.scala b/core/src/main/scala/spark/HadoopRDD.scala
index 62468d04d8cd564cd5804cc46e2e377e9f3c2a11..41608e5a4e98f4576fcae31f4384062fb267fa16 100644
--- a/core/src/main/scala/spark/HadoopRDD.scala
+++ b/core/src/main/scala/spark/HadoopRDD.scala
@@ -18,8 +18,8 @@ import org.apache.hadoop.util.ReflectionUtils
 class HadoopSplit(
     rddId: Int,
     idx: Int,
-    @transient s: InputSplit
-    ) extends Split with Serializable {
+    @transient s: InputSplit)
+  extends Split with Serializable {
   
   val inputSplit = new SerializableWritable[InputSplit](s)
 
@@ -33,13 +33,13 @@ class HadoopSplit(
  * system, or S3, tables in HBase, etc).
  */
 class HadoopRDD[K, V](
-  sc: SparkContext,
-  @transient conf: JobConf,
-  inputFormatClass: Class[_ <: InputFormat[K, V]],
-  keyClass: Class[K],
-  valueClass: Class[V],
-  minSplits: Int
-  ) extends RDD[(K, V)](sc) {
+    sc: SparkContext,
+    @transient conf: JobConf,
+    inputFormatClass: Class[_ <: InputFormat[K, V]],
+    keyClass: Class[K],
+    valueClass: Class[V],
+    minSplits: Int)
+  extends RDD[(K, V)](sc) {
   
   val serializableConf = new SerializableWritable(conf)
   
diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala
index 5574ffc28f30f2e1b6726ff920316e3d5b50c1ef..84b37218b52f5078100489c386d9b4ca1bf3e08e 100644
--- a/core/src/main/scala/spark/HadoopWriter.scala
+++ b/core/src/main/scala/spark/HadoopWriter.scala
@@ -16,13 +16,11 @@ import spark.SerializableWritable
 import spark.Logging
 
 /**
- * Saves an RDD using a Hadoop OutputFormat as specified by a JobConf. The
- * JobConf should also contain an output key class, an output value class, a
- * filename to write to, etc exactly like in a Hadoop job.
+ * Saves an RDD using a Hadoop OutputFormat as specified by a JobConf. The JobConf should also 
+ * contain an output key class, an output value class, a filename to write to, etc exactly like in 
+ * a Hadoop job.
  */
-class HadoopWriter(
-    @transient jobConf: JobConf
-    ) extends Logging with Serializable {
+class HadoopWriter(@transient jobConf: JobConf) extends Logging with Serializable {
   
   private val now = new Date()
   private val conf = new SerializableWritable(jobConf)
@@ -149,7 +147,8 @@ class HadoopWriter(
     attemptID = attemptid
 
     jID = new SerializableWritable[JobID](HadoopWriter.createJobID(now, jobid))
-    taID = new SerializableWritable[TaskAttemptID] (new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
+    taID = new SerializableWritable[TaskAttemptID](
+        new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
   }
 
   private def setConfParams() {
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala
index c9408bbcb63694be883813c36a031185a09bfea5..0d11ab9cbd836a5495f5392b942cb39ffd60e385 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/spark/Logging.scala
@@ -4,22 +4,24 @@ import org.slf4j.Logger
 import org.slf4j.LoggerFactory
 
 /**
- * Utility trait for classes that want to log data. Creates a SLF4J logger
- * for the class and allows logging messages at different levels using
- * methods that only evaluate parameters lazily if the log level is enabled.
+ * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
+ * logging messages at different levels using methods that only evaluate parameters lazily if the
+ * log level is enabled.
  */
 trait Logging {
   // Make the log field transient so that objects with Logging can
   // be serialized and used on another machine
-  @transient private var log_ : Logger = null
+  @transient
+  private var log_ : Logger = null
 
   // Method to get or create the logger for this object
   def log: Logger = {
     if (log_ == null) {
       var className = this.getClass().getName()
       // Ignore trailing $'s in the class names for Scala objects
-      if (className.endsWith("$"))
+      if (className.endsWith("$")) {
         className = className.substring(0, className.length - 1)
+      }
       log_ = LoggerFactory.getLogger(className)
     }
     return log_
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index a183bf80faf457d728bef95f689bce001412cdbe..a934c5a02fe30706ddb9d6ce7194743c91c40ca1 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -24,6 +24,7 @@ extends DaemonActor with Logging {
         case GetMapOutputLocations(shuffleId: Int) =>
           logInfo("Asked to get map output locations for shuffle " + shuffleId)
           reply(serverUris.get(shuffleId))
+          
         case StopMapOutputTracker =>
           reply('OK)
           exit()
@@ -74,8 +75,9 @@ class MapOutputTracker(isMaster: Boolean) extends Logging {
     var array = serverUris.get(shuffleId)
     if (array != null) {
       array.synchronized {
-        if (array(mapId) == serverUri)
+        if (array(mapId) == serverUri) {
           array(mapId) = null
+        }
       }
       incrementGeneration()
     } else {
@@ -95,7 +97,11 @@ class MapOutputTracker(isMaster: Boolean) extends Logging {
         if (fetching.contains(shuffleId)) {
           // Someone else is fetching it; wait for them to be done
           while (fetching.contains(shuffleId)) {
-            try {fetching.wait()} catch {case _ =>}
+            try {
+              fetching.wait()
+            } catch {
+              case _ =>
+            }
           }
           return serverUris.get(shuffleId)
         } else {
diff --git a/core/src/main/scala/spark/MesosScheduler.scala b/core/src/main/scala/spark/MesosScheduler.scala
index 5de6b10155f071385374e79ba4dc67c0beef7f46..618ee724f9beeaa5bd0fbab09750ebd766bee801 100644
--- a/core/src/main/scala/spark/MesosScheduler.scala
+++ b/core/src/main/scala/spark/MesosScheduler.scala
@@ -25,8 +25,10 @@ import org.apache.mesos.Protos._
 private class MesosScheduler(
     sc: SparkContext,
     master: String,
-    frameworkName: String
-    )extends MScheduler with DAGScheduler with Logging {
+    frameworkName: String)
+  extends MScheduler
+  with DAGScheduler
+  with Logging {
   
   // Environment variables to pass to our executors
   val ENV_VARS_TO_SEND_TO_EXECUTORS = Array(
@@ -172,8 +174,9 @@ private class MesosScheduler(
   
   override def waitForRegister() {
     registeredLock.synchronized {
-      while (!isRegistered)
+      while (!isRegistered) {
         registeredLock.wait()
+      }
     }
   }
 
@@ -197,7 +200,7 @@ private class MesosScheduler(
           launchedTask = false
           for (i <- 0 until offers.size if enoughMem(i)) {
             job.slaveOffer(offers(i), availableCpus(i)) match {
-              case Some(task) =>
+              case Some(task) => 
                 tasks(i).add(task)
                 val tid = task.getTaskId.getValue
                 val sid = offers(i).getSlaveId.getValue
@@ -207,6 +210,7 @@ private class MesosScheduler(
                 slavesWithExecutors += sid
                 availableCpus(i) -= getResource(task.getResourcesList(), "cpus")
                 launchedTask = true
+                
               case None => {}
             }
           }
@@ -221,8 +225,10 @@ private class MesosScheduler(
 
   // Helper function to pull out a resource from a Mesos Resources protobuf
   def getResource(res: JList[Resource], name: String): Double = {
-    for (r <- res if r.getName == name)
+    for (r <- res if r.getName == name) {
       return r.getScalar.getValue
+    }
+    
     throw new IllegalArgumentException("No resource called " + name + " in " + res)
   }
 
@@ -348,7 +354,8 @@ private class MesosScheduler(
     return Utils.serialize(props.toArray)
   }
 
-  override def frameworkMessage(d: SchedulerDriver, 
+  override def frameworkMessage(
+      d: SchedulerDriver, 
       s: SlaveID,
       e: ExecutorID,
       b: Array[Byte]) {}
diff --git a/core/src/main/scala/spark/NewHadoopRDD.scala b/core/src/main/scala/spark/NewHadoopRDD.scala
index c40a39cbe0824c36290a0ee9b91dfeffba3f2742..cd42586aa6b63e69f5c9a7bcaf2ef1ffaa29075c 100644
--- a/core/src/main/scala/spark/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/NewHadoopRDD.scala
@@ -14,18 +14,20 @@ import java.util.Date
 import java.text.SimpleDateFormat
 
 class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
-extends Split {
+  extends Split {
+  
   val serializableHadoopSplit = new SerializableWritable(rawSplit)
 
   override def hashCode(): Int = (41 * (41 + rddId) + index).toInt
 }
 
 class NewHadoopRDD[K, V](
-  sc: SparkContext,
-  inputFormatClass: Class[_ <: InputFormat[K, V]],
-  keyClass: Class[K], valueClass: Class[V],
-  @transient conf: Configuration)
-extends RDD[(K, V)](sc) {
+    sc: SparkContext,
+    inputFormatClass: Class[_ <: InputFormat[K, V]],
+    keyClass: Class[K], valueClass: Class[V],
+    @transient conf: Configuration)
+  extends RDD[(K, V)](sc) {
+  
   private val serializableConf = new SerializableWritable(conf)
 
   private val jobtrackerId: String = {
@@ -33,15 +35,18 @@ extends RDD[(K, V)](sc) {
     formatter.format(new Date())
   }
 
-  @transient private val jobId = new JobID(jobtrackerId, id)
+  @transient
+  private val jobId = new JobID(jobtrackerId, id)
 
-  @transient private val splits_ : Array[Split] = {
+  @transient
+  private val splits_ : Array[Split] = {
     val inputFormat = inputFormatClass.newInstance
     val jobContext = new JobContext(serializableConf.value, jobId)
     val rawSplits = inputFormat.getSplits(jobContext).toArray
     val result = new Array[Split](rawSplits.size)
-    for (i <- 0 until rawSplits.size)
+    for (i <- 0 until rawSplits.size) {
       result(i) = new NewHadoopSplit(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
+    }
     result
   }
 
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index 074d5abb38af7c77a2dd5534f7fb6282fecd867a..5b4de97e4b36c8b4c094a37d9434d59aae31da87 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -39,7 +39,11 @@ import SparkContext._
 /**
  * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
  */
-class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) extends Logging with Serializable {
+class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
+    self: RDD[(K, V)])
+  extends Logging
+  with Serializable {
+  
   def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = {
     def mergeMaps(m1: HashMap[K, V], m2: HashMap[K, V]): HashMap[K, V] = {
       for ((k, v) <- m2) {
@@ -54,23 +58,20 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
   }
 
   def combineByKey[C](createCombiner: V => C,
-                      mergeValue: (C, V) => C,
-                      mergeCombiners: (C, C) => C,
-                      numSplits: Int,
-                      partitioner: Partitioner)
-  : RDD[(K, C)] =
-  {
+      mergeValue: (C, V) => C,
+      mergeCombiners: (C, C) => C,
+      numSplits: Int,
+      partitioner: Partitioner): RDD[(K, C)] = {
     val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
     new ShuffledRDD(self, aggregator, partitioner)
   }
 
   def combineByKey[C](createCombiner: V => C,
-                      mergeValue: (C, V) => C,
-                      mergeCombiners: (C, C) => C,
-                      numSplits: Int)
-  : RDD[(K, C)] = {
+      mergeValue: (C, V) => C,
+      mergeCombiners: (C, C) => C,
+      numSplits: Int): RDD[(K, C)] = {
     combineByKey(createCombiner, mergeValue, mergeCombiners, numSplits,
-                 new HashPartitioner(numSplits))
+        new HashPartitioner(numSplits))
   }
 
   def reduceByKey(func: (V, V) => V, numSplits: Int): RDD[(K, V)] = {
@@ -159,9 +160,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
   }
 
   def combineByKey[C](createCombiner: V => C,
-                      mergeValue: (C, V) => C,
-                      mergeCombiners: (C, C) => C)
-  : RDD[(K, C)] = {
+      mergeValue: (C, V) => C,
+      mergeCombiners: (C, C) => C) : RDD[(K, C)] = {
     combineByKey(createCombiner, mergeValue, mergeCombiners, defaultParallelism)
   }
 
@@ -204,8 +204,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
       case Some(p) => p
       case None => new HashPartitioner(defaultParallelism)
     }
-    val cg = new CoGroupedRDD[K](Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]), part)
-    val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
+    val cg = new CoGroupedRDD[K](
+        Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]),
+        part)
+    val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(
+        classManifest[K],
+        Manifests.seqSeqManifest)
     prfs.mapValues {
       case Seq(vs, ws) =>
         (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
@@ -219,7 +223,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
       case None => new HashPartitioner(defaultParallelism)
     }
     new CoGroupedRDD[K](
-        Seq(self.asInstanceOf[RDD[(_, _)]], 
+        Seq(self.asInstanceOf[RDD[(_, _)]],
             other1.asInstanceOf[RDD[(_, _)]], 
             other2.asInstanceOf[RDD[(_, _)]]),
         part).map {
@@ -234,8 +238,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
         val index = p.getPartition(key)
         def process(it: Iterator[(K, V)]): Seq[V] = {
           val buf = new ArrayBuffer[V]
-          for ((k, v) <- it if k == key)
+          for ((k, v) <- it if k == key) {
             buf += v
+          }
           buf
         }
         val res = self.context.runJob(self, process _, Array(index), false)
@@ -253,10 +258,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
     saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
   }
 
-  def saveAsNewAPIHadoopFile(path: String,
-                             keyClass: Class[_],
-                             valueClass: Class[_],
-                             outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
+  def saveAsNewAPIHadoopFile(
+      path: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
     val job = new NewAPIHadoopJob
     job.setOutputKeyClass(keyClass)
     job.setOutputValueClass(valueClass)
@@ -295,11 +301,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
     jobCommitter.cleanupJob(jobTaskContext)
   }
 
-  def saveAsHadoopFile(path: String,
-                       keyClass: Class[_],
-                       valueClass: Class[_],
-                       outputFormatClass: Class[_ <: OutputFormat[_, _]],
-                       conf: JobConf = new JobConf) {
+  def saveAsHadoopFile(
+      path: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: OutputFormat[_, _]],
+      conf: JobConf = new JobConf) {
     conf.setOutputKeyClass(keyClass)
     conf.setOutputValueClass(valueClass)
     // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug
@@ -313,12 +320,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
     val outputFormatClass = conf.getOutputFormat
     val keyClass = conf.getOutputKeyClass
     val valueClass = conf.getOutputValueClass
-    if (outputFormatClass == null)
+    if (outputFormatClass == null) {
       throw new SparkException("Output format class not set")
-    if (keyClass == null)
+    }
+    if (keyClass == null) {
       throw new SparkException("Output key class not set")
-    if (valueClass == null)
+    }
+    if (valueClass == null) {
       throw new SparkException("Output value class not set")
+    }
     
     logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")")
 
@@ -349,19 +359,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) ex
   def getValueClass() = implicitly[ClassManifest[V]].erasure
 }
 
-class MappedValuesRDD[K, V, U](
-  prev: RDD[(K, V)], f: V => U)
-extends RDD[(K, U)](prev.context) {
+class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override val partitioner = prev.partitioner
-  override def compute(split: Split) =
-    prev.iterator(split).map{case (k, v) => (k, f(v))}
+  override def compute(split: Split) = prev.iterator(split).map{case (k, v) => (k, f(v))}
 }
 
-class FlatMappedValuesRDD[K, V, U](
-  prev: RDD[(K, V)], f: V => Traversable[U])
-extends RDD[(K, U)](prev.context) {
+class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => Traversable[U])
+  extends RDD[(K, U)](prev.context) {
+  
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override val partitioner = prev.partitioner
diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala
index 4bcb9e0d54eb9a7a520ff30f8d2882abc20df0a7..21f68f21c2c408ab2c22d994ca647fd5d3469988 100644
--- a/core/src/main/scala/spark/ParallelCollection.scala
+++ b/core/src/main/scala/spark/ParallelCollection.scala
@@ -6,8 +6,8 @@ import scala.collection.mutable.ArrayBuffer
 class ParallelCollectionSplit[T: ClassManifest](
     val rddId: Long,
     val slice: Int,
-    values: Seq[T]
-    ) extends Split with Serializable {
+    values: Seq[T])
+  extends Split with Serializable {
   
   def iterator(): Iterator[T] = values.iterator
 
@@ -24,8 +24,8 @@ class ParallelCollectionSplit[T: ClassManifest](
 class ParallelCollection[T: ClassManifest](
     sc: SparkContext, 
     @transient data: Seq[T],
-    numSlices: Int
-    ) extends RDD[T](sc) {
+    numSlices: Int)
+  extends RDD[T](sc) {
   // TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets
   // cached. It might be worthwhile to write the data to a file in the DFS and read it in the split
   // instead.
diff --git a/core/src/main/scala/spark/ParallelShuffleFetcher.scala b/core/src/main/scala/spark/ParallelShuffleFetcher.scala
index 95dfb01aac956bcaec361cbde542478ae1c71e5b..60125180c862c12e50ff019c05c857c5fc070f89 100644
--- a/core/src/main/scala/spark/ParallelShuffleFetcher.scala
+++ b/core/src/main/scala/spark/ParallelShuffleFetcher.scala
@@ -29,8 +29,9 @@ class ParallelShuffleFetcher extends ShuffleFetcher with Logging {
     
     // Randomize them and put them in a LinkedBlockingQueue
     val serverQueue = new LinkedBlockingQueue[(String, ArrayBuffer[Int])]
-    for (pair <- Utils.randomize(inputsByUri))
+    for (pair <- Utils.randomize(inputsByUri)) {
       serverQueue.put(pair)
+    }
 
     // Create a queue to hold the fetched data
     val resultQueue = new LinkedBlockingQueue[Array[Byte]]
@@ -57,17 +58,19 @@ class ParallelShuffleFetcher extends ShuffleFetcher with Logging {
                 val conn = new URL(url).openConnection()
                 conn.connect()
                 val len = conn.getContentLength()
-                if (len == -1)
+                if (len == -1) {
                   throw new SparkException("Content length was not specified by server")
+                }
                 val buf = new Array[Byte](len)
                 val in = conn.getInputStream()
                 var pos = 0
                 while (pos < len) {
                   val n = in.read(buf, pos, len-pos)
-                  if (n == -1)
+                  if (n == -1) {
                     throw new SparkException("EOF before reading the expected " + len + " bytes")
-                  else
+                  } else {
                     pos += n
+                  }
                 }
                 // Done reading everything
                 resultQueue.put(buf)
diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala
index 4491de1734f3ed44a7218cce408012c9bd3a5eae..7b3c7b0b3729968dc60b28f20a3bb7f8934e9c9d 100644
--- a/core/src/main/scala/spark/Partitioner.scala
+++ b/core/src/main/scala/spark/Partitioner.scala
@@ -10,12 +10,17 @@ class HashPartitioner(partitions: Int) extends Partitioner {
 
   def getPartition(key: Any) = {
     val mod = key.hashCode % partitions
-    if (mod < 0) mod + partitions else mod // Guard against negative hash codes
+    if (mod < 0) {
+      mod + partitions
+    } else {
+      mod // Guard against negative hash codes
+    }
   }
   
   override def equals(other: Any): Boolean = other match {
     case h: HashPartitioner =>
       h.numPartitions == numPartitions
-    case _ => false
+    case _ =>
+      false
   }
 }
\ No newline at end of file
diff --git a/core/src/main/scala/spark/PipedRDD.scala b/core/src/main/scala/spark/PipedRDD.scala
index 7c0049298655584faaa88185eca22db6e951e0dc..3f993d895a356ddceba5ff6e663256f118ed09cd 100644
--- a/core/src/main/scala/spark/PipedRDD.scala
+++ b/core/src/main/scala/spark/PipedRDD.scala
@@ -6,13 +6,12 @@ import java.util.StringTokenizer
 import scala.collection.mutable.ArrayBuffer
 import scala.io.Source
 
-
 /**
  * An RDD that pipes the contents of each parent partition through an external command
  * (printing them one per line) and returns the output as a collection of strings.
  */
 class PipedRDD[T: ClassManifest](parent: RDD[T], command: Seq[String])
-extends RDD[String](parent.context) {
+  extends RDD[String](parent.context) {
   // Similar to Runtime.exec(), if we are given a single string, split it into words
   // using a standard StringTokenizer (i.e. by spaces)
   def this(parent: RDD[T], command: String) = this(parent, PipedRDD.tokenize(command))
@@ -28,8 +27,9 @@ extends RDD[String](parent.context) {
     // Start a thread to print the process's stderr to ours
     new Thread("stderr reader for " + command) {
       override def run() {
-        for(line <- Source.fromInputStream(proc.getErrorStream).getLines)
+        for(line <- Source.fromInputStream(proc.getErrorStream).getLines) {
           System.err.println(line)
+        }
       }
     }.start()
 
@@ -38,8 +38,9 @@ extends RDD[String](parent.context) {
       override def run() {
         SparkEnv.set(env)
         val out = new PrintWriter(proc.getOutputStream)
-        for(elem <- parent.iterator(split))
+        for(elem <- parent.iterator(split)) {
           out.println(elem)
+        }
         out.close()
       }
     }.start()
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index c5e0b1585b573f08f470f1a7f5beebe67be66080..c85973fc0c021526c0221fc8b2c47c7c1d1e783e 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -264,8 +264,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
 
 class MappedRDD[U: ClassManifest, T: ClassManifest](
     prev: RDD[T],
-    f: T => U
-    ) extends RDD[U](prev.context) {
+    f: T => U)
+  extends RDD[U](prev.context) {
+  
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override def compute(split: Split) = prev.iterator(split).map(f)
@@ -273,25 +274,21 @@ class MappedRDD[U: ClassManifest, T: ClassManifest](
 
 class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
     prev: RDD[T],
-    f: T => Traversable[U]
-    ) extends RDD[U](prev.context) {
+    f: T => Traversable[U])
+  extends RDD[U](prev.context) {
+  
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override def compute(split: Split) = prev.iterator(split).flatMap(f)
 }
 
-class FilteredRDD[T: ClassManifest](
-    prev: RDD[T],
-    f: T => Boolean
-    ) extends RDD[T](prev.context) {
+class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override def compute(split: Split) = prev.iterator(split).filter(f)
 }
 
-class GlommedRDD[T: ClassManifest](
-    prev: RDD[T]
-    ) extends RDD[Array[T]](prev.context) {
+class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override def compute(split: Split) = Array(prev.iterator(split).toArray).iterator
@@ -299,8 +296,9 @@ class GlommedRDD[T: ClassManifest](
 
 class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
     prev: RDD[T],
-    f: Iterator[T] => Iterator[U]
-    ) extends RDD[U](prev.context) {
+    f: Iterator[T] => Iterator[U])
+  extends RDD[U](prev.context) {
+  
   override def splits = prev.splits
   override val dependencies = List(new OneToOneDependency(prev))
   override def compute(split: Split) = f(prev.iterator(split))
diff --git a/core/src/main/scala/spark/ResultTask.scala b/core/src/main/scala/spark/ResultTask.scala
index 8bbe31444f6b6eedd91187e59c2da22e2f6f6d2b..25d85b7e0ced19366b5a2172220382ce5edeef7f 100644
--- a/core/src/main/scala/spark/ResultTask.scala
+++ b/core/src/main/scala/spark/ResultTask.scala
@@ -1,8 +1,14 @@
 package spark
 
-class ResultTask[T, U](stageId: Int, rdd: RDD[T], func: (TaskContext, Iterator[T]) => U,
-                       val partition: Int, locs: Seq[String], val outputId: Int)
-extends DAGTask[U](stageId) {
+class ResultTask[T, U](
+    stageId: Int, 
+    rdd: RDD[T], 
+    func: (TaskContext, Iterator[T]) => U,
+    val partition: Int, 
+    locs: Seq[String],
+    val outputId: Int)
+    extends DAGTask[U](stageId) {
+  
   val split = rdd.splits(partition)
 
   override def run(attemptId: Int): U = {
diff --git a/core/src/main/scala/spark/SampledRDD.scala b/core/src/main/scala/spark/SampledRDD.scala
index 89d91e5603810c3211f374dd6e6f8a95fd2936b1..c9a9e53d18475e3be1a86577de984dd948695a2a 100644
--- a/core/src/main/scala/spark/SampledRDD.scala
+++ b/core/src/main/scala/spark/SampledRDD.scala
@@ -7,11 +7,11 @@ class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Seriali
 }
 
 class SampledRDD[T: ClassManifest](
-  prev: RDD[T],
-  withReplacement: Boolean, 
-  frac: Double,
-  seed: Int
-  ) extends RDD[T](prev.context) {
+    prev: RDD[T],
+    withReplacement: Boolean, 
+    frac: Double,
+    seed: Int)
+  extends RDD[T](prev.context) {
 
   @transient
   val splits_ = {
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index bd4a526b89ad7525cc971ee5875477ab8cfe1266..b213ca9dcbde6c70ad6ef03ca4c2150a84a1390f 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -25,26 +25,29 @@ import org.apache.hadoop.io.Text
 
 import SparkContext._
 
-
 /**
  * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile,
  * through an implicit conversion. Note that this can't be part of PairRDDFunctions because
  * we need more implicit parameters to convert our keys and values to Writable.
  */
-class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest](self: RDD[(K,V)]) extends Logging with Serializable {
+class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest](
+    self: RDD[(K,V)])
+  extends Logging
+  with Serializable {
+  
   def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
     val c = {
-      if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) 
+      if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { 
         classManifest[T].erasure
-      else
+      } else {
         implicitly[T => Writable].getClass.getMethods()(0).getReturnType
+      }
        // TODO: use something like WritableConverter to avoid reflection
     }
     c.asInstanceOf[Class[ _ <: Writable]]
   }
 
   def saveAsSequenceFile(path: String) {
-    
     def anyToWritable[U <% Writable](u: U): Writable = u
 
     val keyClass = getWritableClass[K]
diff --git a/core/src/main/scala/spark/Serializer.scala b/core/src/main/scala/spark/Serializer.scala
index cfc6d978bce818d467fd851e8dbe35152c3c99dc..15fca9fcda1f479a065fa12fbb0a9f90f351fce9 100644
--- a/core/src/main/scala/spark/Serializer.scala
+++ b/core/src/main/scala/spark/Serializer.scala
@@ -3,9 +3,8 @@ package spark
 import java.io.{InputStream, OutputStream}
 
 /**
- * A serializer. Because some serialization libraries are not thread safe,
- * this class is used to create SerializerInstances that do the actual
- * serialization.
+ * A serializer. Because some serialization libraries are not thread safe, this class is used to 
+ * create SerializerInstances that do the actual serialization.
  */
 trait Serializer {
   def newInstance(): SerializerInstance
diff --git a/core/src/main/scala/spark/SerializingCache.scala b/core/src/main/scala/spark/SerializingCache.scala
index 2c1f96a7001dd49108a0669554f96eb1fc445f07..a74922ec4ce13fa251589ad36780e7aa0610c8f5 100644
--- a/core/src/main/scala/spark/SerializingCache.scala
+++ b/core/src/main/scala/spark/SerializingCache.scala
@@ -3,8 +3,8 @@ package spark
 import java.io._
 
 /**
- * Wrapper around a BoundedMemoryCache that stores serialized objects as
- * byte arrays in order to reduce storage cost and GC overhead
+ * Wrapper around a BoundedMemoryCache that stores serialized objects as byte arrays in order to 
+ * reduce storage cost and GC overhead
  */
 class SerializingCache extends Cache with Logging {
   val bmc = new BoundedMemoryCache
diff --git a/core/src/main/scala/spark/ShuffleMapTask.scala b/core/src/main/scala/spark/ShuffleMapTask.scala
index eb6a5e2df39f8b2a313d6e8ba1e397df8c83ed78..1c9dfa3f18a019bdeccc2d73948e640593d13aa0 100644
--- a/core/src/main/scala/spark/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/ShuffleMapTask.scala
@@ -5,9 +5,15 @@ import java.io.FileOutputStream
 import java.io.ObjectOutputStream
 import scala.collection.mutable.HashMap
 
-
-class ShuffleMapTask(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_,_], val partition: Int, locs: Seq[String])
-extends DAGTask[String](stageId) with Logging {
+class ShuffleMapTask(
+    stageId: Int,
+    rdd: RDD[_], 
+    dep: ShuffleDependency[_,_,_],
+    val partition: Int, 
+    locs: Seq[String])
+  extends DAGTask[String](stageId)
+  with Logging {
+  
   val split = rdd.splits(partition)
 
   override def run (attemptId: Int): String = {
diff --git a/core/src/main/scala/spark/ShuffledRDD.scala b/core/src/main/scala/spark/ShuffledRDD.scala
index 4ab1958ea113d7901f61fff8c6b37662015bb4d4..4225a9551646e781aa1da3b77b415d3108f94735 100644
--- a/core/src/main/scala/spark/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/ShuffledRDD.scala
@@ -2,22 +2,21 @@ package spark
 
 import scala.collection.mutable.HashMap
 
-
 class ShuffledRDDSplit(val idx: Int) extends Split {
   override val index = idx
   override def hashCode(): Int = idx
 }
 
 class ShuffledRDD[K, V, C](
-  parent: RDD[(K, V)],
-  aggregator: Aggregator[K, V, C],
-  part : Partitioner)
-extends RDD[(K, C)](parent.context) {
+    parent: RDD[(K, V)],
+    aggregator: Aggregator[K, V, C],
+    part : Partitioner) 
+  extends RDD[(K, C)](parent.context) {
   //override val partitioner = Some(part)
   override val partitioner = Some(part)
   
-  @transient val splits_ =
-    Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
+  @transient
+  val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
 
   override def splits = splits_
   
diff --git a/core/src/main/scala/spark/SimpleJob.scala b/core/src/main/scala/spark/SimpleJob.scala
index 636e18eb4c73633efd51de3259a105a7c3cb2aa5..6eee8b45cea8d741f1299a82ee1a68ada9dc9206 100644
--- a/core/src/main/scala/spark/SimpleJob.scala
+++ b/core/src/main/scala/spark/SimpleJob.scala
@@ -10,14 +10,16 @@ import com.google.protobuf.ByteString
 import org.apache.mesos._
 import org.apache.mesos.Protos._
 
-
 /**
  * A Job that runs a set of tasks with no interdependencies.
  */
 class SimpleJob(
-  sched: MesosScheduler, tasksSeq: Seq[Task[_]], val jobId: Int)
-extends Job(jobId) with Logging
-{
+    sched: MesosScheduler, 
+    tasksSeq: Seq[Task[_]], 
+    val jobId: Int) 
+  extends Job(jobId)
+  with Logging {
+  
   // Maximum time to wait to run a task in a preferred location (in ms)
   val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "5000").toLong
 
@@ -163,11 +165,10 @@ extends Job(jobId) with Logging
             lastPreferredLaunchTime = time
           // Create and return the Mesos task object
           val cpuRes = Resource.newBuilder()
-                         .setName("cpus")
-                         .setType(Resource.Type.SCALAR)
-                         .setScalar(Resource.Scalar.newBuilder()
-                                      .setValue(CPUS_PER_TASK).build())
-                         .build()
+            .setName("cpus")
+            .setType(Resource.Type.SCALAR)
+            .setScalar(Resource.Scalar.newBuilder().setValue(CPUS_PER_TASK).build())
+            .build()
           val serializedTask = Utils.serialize(task)
           logDebug("Serialized size: " + serializedTask.size)
           val taskName = "task %d:%d".format(jobId, index)
@@ -204,8 +205,7 @@ extends Job(jobId) with Logging
     val index = tidToIndex(tid)
     if (!finished(index)) {
       tasksFinished += 1
-      logInfo("Finished TID %s (progress: %d/%d)".format(
-        tid, tasksFinished, numTasks))
+      logInfo("Finished TID %s (progress: %d/%d)".format(tid, tasksFinished, numTasks))
       // Deserialize task result
       val result = Utils.deserialize[TaskResult[_]](status.getData.toByteArray)
       sched.taskEnded(tasks(index), Success, result.value, result.accumUpdates)
@@ -236,8 +236,9 @@ extends Job(jobId) with Logging
             sched.taskEnded(tasks(index), fetchFailed, null, null)
             finished(index) = true
             tasksFinished += 1
-            if (tasksFinished == numTasks)
+            if (tasksFinished == numTasks) {
               sched.jobFinished(this)
+            }
             return
           case ef: ExceptionFailure =>
             val key = ef.exception.toString
@@ -279,8 +280,7 @@ extends Job(jobId) with Logging
         if (numFailures(index) > MAX_TASK_FAILURES) {
           logError("Task %d:%d failed more than %d times; aborting job".format(
             jobId, index, MAX_TASK_FAILURES))
-          abort("Task %d failed more than %d times".format(
-            index, MAX_TASK_FAILURES))
+          abort("Task %d failed more than %d times".format(index, MAX_TASK_FAILURES))
         }
       }
     } else {
diff --git a/core/src/main/scala/spark/SimpleShuffleFetcher.scala b/core/src/main/scala/spark/SimpleShuffleFetcher.scala
index 1cc0cfc3318d2e2d542c514e2d3a3748371a1c94..7b4a65e8bfc4cccd852be81071285cdd21499c8a 100644
--- a/core/src/main/scala/spark/SimpleShuffleFetcher.scala
+++ b/core/src/main/scala/spark/SimpleShuffleFetcher.scala
@@ -6,7 +6,6 @@ import java.net.URL
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-
 class SimpleShuffleFetcher extends ShuffleFetcher with Logging {
   def fetch[K, V](shuffleId: Int, reduceId: Int, func: (K, V) => Unit) {
     logInfo("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId))
diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala
index a3774fb0551274738fed0b1ce7dd9e47ac4f88a9..4b89503e84ab0bd51eaaf4e547c1f2085b3452d8 100644
--- a/core/src/main/scala/spark/SizeEstimator.scala
+++ b/core/src/main/scala/spark/SizeEstimator.scala
@@ -9,10 +9,9 @@ import java.util.Random
 
 import scala.collection.mutable.ArrayBuffer
 
-
 /**
- * Estimates the sizes of Java objects (number of bytes of memory they occupy),
- * for use in memory-aware caches.
+ * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in 
+ * memory-aware caches.
  *
  * Based on the following JavaWorld article:
  * http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html
@@ -36,9 +35,9 @@ object SizeEstimator {
   classInfos.put(classOf[Object], new ClassInfo(OBJECT_SIZE, Nil))
 
   /**
-   * The state of an ongoing size estimation. Contains a stack of objects
-   * to visit as well as an IdentityHashMap of visited objects, and provides
-   * utility methods for enqueueing new objects to visit.
+   * The state of an ongoing size estimation. Contains a stack of objects to visit as well as an
+   * IdentityHashMap of visited objects, and provides utility methods for enqueueing new objects
+   * to visit.
    */
   private class SearchState {
     val visited = new IdentityHashMap[AnyRef, AnyRef]
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 4a0732bd5a0bda7d6508b1a0fc16400d55c96d3f..ef3dbe9b81968af08a3e3c9e0163c6a7b8889c63 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -34,8 +34,9 @@ class SparkContext(
     master: String,
     frameworkName: String,
     val sparkHome: String = null,
-    val jars: Seq[String] = Nil
-    ) extends Logging {
+    val jars: Seq[String] = Nil)
+  extends Logging {
+  
   // Ensure logging is initialized before we spawn any threads
   initLogging()
 
@@ -64,8 +65,10 @@ class SparkContext(
     // Regular expression for local[N, maxRetries], used in tests with failing tasks
     val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+),([0-9]+)\]""".r
     master match {
-      case "local" => new LocalScheduler(1, 0)
-      case LOCAL_N_REGEX(threads) => new LocalScheduler(threads.toInt, 0)
+      case "local" => 
+        new LocalScheduler(1, 0)
+      case LOCAL_N_REGEX(threads) => 
+        new LocalScheduler(threads.toInt, 0)
       case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
         new LocalScheduler(threads.toInt, maxFailures.toInt)
       case _ =>
@@ -79,17 +82,11 @@ class SparkContext(
 
   // Methods for creating RDDs
 
-  def parallelize[T: ClassManifest](
-      seq: Seq[T],
-      numSlices: Int = defaultParallelism
-      ): RDD[T] = {
+  def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = {
     new ParallelCollection[T](this, seq, numSlices)
   }
     
-  def makeRDD[T: ClassManifest](
-      seq: Seq[T],
-      numSlices: Int = defaultParallelism
-      ): RDD[T] = {
+  def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = {
     parallelize(seq, numSlices)
   }
 
@@ -152,7 +149,8 @@ class SparkContext(
     val job = new NewHadoopJob
     NewFileInputFormat.addInputPath(job, new Path(path))
     val conf = job.getConfiguration
-    newAPIHadoopFile(path,
+    newAPIHadoopFile(
+        path,
         fm.erasure.asInstanceOf[Class[F]],
         km.erasure.asInstanceOf[Class[K]],
         vm.erasure.asInstanceOf[Class[V]],
@@ -185,18 +183,16 @@ class SparkContext(
     sequenceFile(path, keyClass, valueClass, defaultMinSplits)
 
   /**
-   * Version of sequenceFile() for types implicitly convertible to Writables 
-   * through a WritableConverter.
+   * Version of sequenceFile() for types implicitly convertible to Writables through a 
+   * WritableConverter.
    *
-   * WritableConverters are provided in a somewhat strange way (by an implicit
-   * function) to support both subclasses of Writable and types for which we 
-   * define a converter (e.g. Int to IntWritable). The most natural thing 
-   * would've been to have implicit objects for the converters, but then we
-   * couldn't have an object for every subclass of Writable (you can't have a
-   * parameterized singleton object). We use functions instead to create a new 
-   * converter for the appropriate type. In addition, we pass the converter a
-   * ClassManifest of its type to allow it to figure out the Writable class to
-   * use in the subclass case.
+   * WritableConverters are provided in a somewhat strange way (by an implicit function) to support
+   * both subclasses of Writable and types for which we define a converter (e.g. Int to 
+   * IntWritable). The most natural thing would've been to have implicit objects for the
+   * converters, but then we couldn't have an object for every subclass of Writable (you can't
+   * have a parameterized singleton object). We use functions instead to create a new converter 
+   * for the appropriate type. In addition, we pass the converter a ClassManifest of its type to
+   * allow it to figure out the Writable class to use in the subclass case.
    */
    def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits)
       (implicit km: ClassManifest[K], vm: ClassManifest[V],
@@ -443,4 +439,7 @@ object SparkContext {
  * that doesn't know the type of T when it is created. This sounds strange but is necessary to
  * support converting subclasses of Writable to themselves (writableWritableConverter).
  */
-class WritableConverter[T](val writableClass: ClassManifest[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable
+class WritableConverter[T](
+    val writableClass: ClassManifest[T] => Class[_ <: Writable],
+    val convert: Writable => T)
+  extends Serializable
diff --git a/core/src/main/scala/spark/Stage.scala b/core/src/main/scala/spark/Stage.scala
index 401b33bd1629927a68c52ee5fb8ff7b5ab24dc38..9452ea3a8e57db93c4cc31744a80bef8b3dfbd15 100644
--- a/core/src/main/scala/spark/Stage.scala
+++ b/core/src/main/scala/spark/Stage.scala
@@ -1,16 +1,22 @@
 package spark
 
-class Stage(val id: Int, val rdd: RDD[_], val shuffleDep: Option[ShuffleDependency[_,_,_]], val parents: List[Stage]) {
+class Stage(
+    val id: Int,
+    val rdd: RDD[_],
+    val shuffleDep: Option[ShuffleDependency[_,_,_]],
+    val parents: List[Stage]) {
+  
   val isShuffleMap = shuffleDep != None
   val numPartitions = rdd.splits.size
   val outputLocs = Array.fill[List[String]](numPartitions)(Nil)
   var numAvailableOutputs = 0
 
   def isAvailable: Boolean = {
-    if (parents.size == 0 && !isShuffleMap)
+    if (parents.size == 0 && !isShuffleMap) {
       true
-    else
+    } else {
       numAvailableOutputs == numPartitions
+    }
   }
 
   def addOutputLoc(partition: Int, host: String) {
@@ -24,8 +30,9 @@ class Stage(val id: Int, val rdd: RDD[_], val shuffleDep: Option[ShuffleDependen
     val prevList = outputLocs(partition)
     val newList = prevList.filterNot(_ == host)
     outputLocs(partition) = newList
-    if (prevList != Nil && newList == Nil)
+    if (prevList != Nil && newList == Nil) {
       numAvailableOutputs -= 1
+    }
   }
 
   override def toString = "Stage " + id
diff --git a/core/src/main/scala/spark/UnionRDD.scala b/core/src/main/scala/spark/UnionRDD.scala
index dadfd94eefdb5c9c931076152f5f817055c1f5d0..6fded339ee885ba7c372690d4e52963e8fbf4bbb 100644
--- a/core/src/main/scala/spark/UnionRDD.scala
+++ b/core/src/main/scala/spark/UnionRDD.scala
@@ -2,16 +2,26 @@ package spark
 
 import scala.collection.mutable.ArrayBuffer
 
-class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], split: Split)
-extends Split with Serializable {
+class UnionSplit[T: ClassManifest](
+    idx: Int, 
+    rdd: RDD[T],
+    split: Split)
+  extends Split
+  with Serializable {
+  
   def iterator() = rdd.iterator(split)
   def preferredLocations() = rdd.preferredLocations(split)
   override val index = idx
 }
 
-class UnionRDD[T: ClassManifest](sc: SparkContext, rdds: Seq[RDD[T]])
-extends RDD[T](sc) with Serializable {
-  @transient val splits_ : Array[Split] = {
+class UnionRDD[T: ClassManifest](
+    sc: SparkContext,
+    rdds: Seq[RDD[T]])
+  extends RDD[T](sc)
+  with Serializable {
+  
+  @transient
+  val splits_ : Array[Split] = {
     val array = new Array[Split](rdds.map(_.splits.size).sum)
     var pos = 0
     for (rdd <- rdds; split <- rdd.splits) {
@@ -33,8 +43,7 @@ extends RDD[T](sc) with Serializable {
     deps.toList
   }
   
-  override def compute(s: Split): Iterator[T] =
-    s.asInstanceOf[UnionSplit[T]].iterator()
+  override def compute(s: Split): Iterator[T] = s.asInstanceOf[UnionSplit[T]].iterator()
 
   override def preferredLocations(s: Split): Seq[String] =
     s.asInstanceOf[UnionSplit[T]].preferredLocations()
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index 89a3b1c1f9a9ce911b981da5a4248bcc4093189a..d0735f74d3cbc107e50f53f776f3c6ecc8373e30 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -59,16 +59,15 @@ object Utils {
   }
 
   // Create a temporary directory inside the given parent directory
-  def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File =
-  {
+  def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = {
     var attempts = 0
     val maxAttempts = 10
     var dir: File = null
     while (dir == null) {
       attempts += 1
       if (attempts > maxAttempts) {
-        throw new IOException("Failed to create a temp directory " +
-                              "after " + maxAttempts + " attempts!")
+        throw new IOException("Failed to create a temp directory after " + maxAttempts + 
+            " attempts!")
       }
       try {
         dir = new File(root, "spark-" + UUID.randomUUID.toString)
@@ -137,8 +136,7 @@ object Utils {
    * Wrapper over newCachedThreadPool
    */
   def newDaemonCachedThreadPool(): ThreadPoolExecutor = {
-    var threadPool =
-      Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor]
+    var threadPool = Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor]
 
     threadPool.setThreadFactory (newDaemonThreadFactory)
 
@@ -149,8 +147,7 @@ object Utils {
    * Wrapper over newFixedThreadPool
    */
   def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = {
-    var threadPool =
-      Executors.newFixedThreadPool(nThreads).asInstanceOf[ThreadPoolExecutor]
+    var threadPool = Executors.newFixedThreadPool(nThreads).asInstanceOf[ThreadPoolExecutor]
 
     threadPool.setThreadFactory(newDaemonThreadFactory)
 
diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala
index f492ca762c4092f63bc72e6c9dd90cb4b12d1936..cdf05fe5de8ba40cee9a522cb055aae9798f1ff2 100644
--- a/core/src/main/scala/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/spark/broadcast/Broadcast.scala
@@ -18,8 +18,7 @@ trait Broadcast[T] extends Serializable {
   override def toString = "spark.Broadcast(" + uuid + ")"
 }
 
-object Broadcast
-extends Logging with Serializable {
+object Broadcast extends Logging with Serializable {
   // Messages
   val REGISTER_BROADCAST_TRACKER = 0
   val UNREGISTER_BROADCAST_TRACKER = 1
@@ -90,18 +89,14 @@ extends Logging with Serializable {
   private var MaxPeersInGuideResponse_ = System.getProperty(
     "spark.broadcast.maxPeersInGuideResponse", "4").toInt
 
-  private var MaxRxSlots_ = System.getProperty(
-    "spark.broadcast.maxRxSlots", "4").toInt
-  private var MaxTxSlots_ = System.getProperty(
-    "spark.broadcast.maxTxSlots", "4").toInt
+  private var MaxRxSlots_ = System.getProperty("spark.broadcast.maxRxSlots", "4").toInt
+  private var MaxTxSlots_ = System.getProperty("spark.broadcast.maxTxSlots", "4").toInt
 
-  private var MaxChatTime_ = System.getProperty(
-    "spark.broadcast.maxChatTime", "500").toInt
-  private var MaxChatBlocks_ = System.getProperty(
-    "spark.broadcast.maxChatBlocks", "1024").toInt
+  private var MaxChatTime_ = System.getProperty("spark.broadcast.maxChatTime", "500").toInt
+  private var MaxChatBlocks_ = System.getProperty("spark.broadcast.maxChatBlocks", "1024").toInt
 
   private var EndGameFraction_ = System.getProperty(
-    "spark.broadcast.endGameFraction", "0.95").toDouble
+      "spark.broadcast.endGameFraction", "0.95").toDouble
 
   def isMaster = isMaster_
 
@@ -167,9 +162,9 @@ extends Logging with Serializable {
   }
 
   // Helper function to convert Array[BroadcastBlock] to object
-  def unBlockifyObject[OUT](arrayOfBlocks: Array[BroadcastBlock], 
-                            totalBytes: Int, 
-                            totalBlocks: Int): OUT = {
+  def unBlockifyObject[OUT](arrayOfBlocks: Array[BroadcastBlock],
+      totalBytes: Int, 
+      totalBlocks: Int): OUT = {
 
     var retByteArray = new Array[Byte](totalBytes)
     for (i <- 0 until totalBlocks) {
@@ -193,9 +188,12 @@ extends Logging with Serializable {
 case class BroadcastBlock (val blockID: Int, val byteArray: Array[Byte]) extends Serializable
 
 case class VariableInfo (@transient val arrayOfBlocks : Array[BroadcastBlock],
-                                    val totalBlocks: Int, 
-                                    val totalBytes: Int) extends Serializable {
-  @transient var hasBlocks = 0
+    val totalBlocks: Int, 
+    val totalBytes: Int)
+  extends Serializable {
+  
+  @transient
+  var hasBlocks = 0
 }
 
 class SpeedTracker extends Serializable {