From 4ce2782a61e23ed0326faac2ee97a9bd36ec8963 Mon Sep 17 00:00:00 2001
From: Reynold Xin <rxin@databricks.com>
Date: Mon, 23 Mar 2015 23:41:06 -0700
Subject: [PATCH] [SPARK-6428] Added explicit types for all public methods in
 core.

Author: Reynold Xin <rxin@databricks.com>

Closes #5125 from rxin/core-explicit-type and squashes the following commits:

f471415 [Reynold Xin] Revert style checker changes.
81b66e4 [Reynold Xin] Code review feedback.
a7533e3 [Reynold Xin] Mima excludes.
1d795f5 [Reynold Xin] [SPARK-6428] Added explicit types for all public methods in core.
---
 .../scala/org/apache/spark/Accumulators.scala | 23 +++--
 .../scala/org/apache/spark/Dependency.scala   |  6 +-
 .../scala/org/apache/spark/FutureAction.scala |  4 +-
 .../org/apache/spark/HeartbeatReceiver.scala  |  2 +-
 .../org/apache/spark/MapOutputTracker.scala   |  2 +-
 .../scala/org/apache/spark/Partitioner.scala  |  4 +-
 .../apache/spark/SerializableWritable.scala   |  6 +-
 .../scala/org/apache/spark/SparkConf.scala    |  2 +-
 .../scala/org/apache/spark/SparkContext.scala | 48 +++++-----
 .../scala/org/apache/spark/TaskState.scala    |  4 +-
 .../scala/org/apache/spark/TestUtils.scala    |  2 +-
 .../org/apache/spark/api/java/JavaRDD.scala   |  2 +-
 .../spark/api/java/JavaSparkContext.scala     |  2 +-
 .../org/apache/spark/api/java/JavaUtils.scala | 35 ++++----
 .../apache/spark/api/python/PythonRDD.scala   | 39 ++++----
 .../apache/spark/api/python/SerDeUtil.scala   |  2 +-
 .../WriteInputFormatTestDataGenerator.scala   | 27 +++---
 .../apache/spark/broadcast/Broadcast.scala    |  2 +-
 .../spark/broadcast/BroadcastManager.scala    |  2 +-
 .../spark/broadcast/HttpBroadcast.scala       |  4 +-
 .../broadcast/HttpBroadcastFactory.scala      |  2 +-
 .../spark/broadcast/TorrentBroadcast.scala    |  2 +-
 .../broadcast/TorrentBroadcastFactory.scala   |  2 +-
 .../org/apache/spark/deploy/Client.scala      |  4 +-
 .../apache/spark/deploy/ClientArguments.scala |  2 +-
 .../apache/spark/deploy/DeployMessage.scala   |  2 +-
 .../spark/deploy/FaultToleranceTest.scala     |  2 +-
 .../apache/spark/deploy/JsonProtocol.scala    | 15 ++--
 .../apache/spark/deploy/SparkHadoopUtil.scala |  2 +-
 .../org/apache/spark/deploy/SparkSubmit.scala |  2 +-
 .../spark/deploy/SparkSubmitArguments.scala   |  2 +-
 .../spark/deploy/client/AppClient.scala       |  2 +-
 .../deploy/history/FsHistoryProvider.scala    |  4 +-
 .../spark/deploy/history/HistoryServer.scala  | 10 +--
 .../master/FileSystemPersistenceEngine.scala  |  2 +-
 .../apache/spark/deploy/master/Master.scala   |  2 +-
 .../deploy/master/RecoveryModeFactory.scala   | 15 +++-
 .../spark/deploy/master/WorkerInfo.scala      |  2 +-
 .../master/ZooKeeperPersistenceEngine.scala   |  2 +-
 .../spark/deploy/master/ui/MasterPage.scala   |  2 +-
 .../spark/deploy/worker/DriverRunner.scala    | 22 +++--
 .../apache/spark/deploy/worker/Worker.scala   |  2 +-
 .../spark/deploy/worker/WorkerWatcher.scala   |  2 +-
 .../CoarseGrainedExecutorBackend.scala        |  2 +-
 .../apache/spark/executor/ExecutorActor.scala |  2 +-
 .../apache/spark/executor/TaskMetrics.scala   | 90 +++++++++----------
 .../spark/input/PortableDataStream.scala      | 17 ++--
 .../spark/mapred/SparkHadoopMapRedUtil.scala  |  2 +-
 .../mapreduce/SparkHadoopMapReduceUtil.scala  |  2 +-
 .../apache/spark/metrics/MetricsSystem.scala  |  3 +-
 .../spark/metrics/sink/MetricsServlet.scala   | 18 ++--
 .../org/apache/spark/metrics/sink/Sink.scala  |  4 +-
 .../spark/network/nio/BlockMessageArray.scala |  6 +-
 .../spark/network/nio/BufferMessage.scala     | 10 +--
 .../apache/spark/network/nio/Connection.scala |  8 +-
 .../spark/network/nio/ConnectionId.scala      |  4 +-
 .../spark/network/nio/ConnectionManager.scala |  2 +-
 .../network/nio/ConnectionManagerId.scala     |  2 +-
 .../apache/spark/network/nio/Message.scala    |  6 +-
 .../spark/network/nio/MessageChunk.scala      |  6 +-
 .../network/nio/MessageChunkHeader.scala      |  4 +-
 .../apache/spark/partial/PartialResult.scala  |  2 +-
 .../org/apache/spark/rdd/CartesianRDD.scala   |  2 +-
 .../org/apache/spark/rdd/CoalescedRDD.scala   | 10 +--
 .../org/apache/spark/rdd/HadoopRDD.scala      | 13 ++-
 .../scala/org/apache/spark/rdd/JdbcRDD.scala  | 10 ++-
 .../apache/spark/rdd/MapPartitionsRDD.scala   |  2 +-
 .../org/apache/spark/rdd/NewHadoopRDD.scala   |  2 +-
 .../spark/rdd/ParallelCollectionRDD.scala     |  2 +-
 .../spark/rdd/PartitionPruningRDD.scala       | 10 ++-
 .../scala/org/apache/spark/rdd/PipedRDD.scala |  8 +-
 .../main/scala/org/apache/spark/rdd/RDD.scala | 12 +--
 .../org/apache/spark/rdd/ShuffledRDD.scala    |  4 +-
 .../org/apache/spark/rdd/SubtractedRDD.scala  |  2 +-
 .../scala/org/apache/spark/rdd/UnionRDD.scala |  2 +-
 .../spark/rdd/ZippedPartitionsRDD.scala       |  2 +-
 .../spark/scheduler/AccumulableInfo.scala     |  7 +-
 .../apache/spark/scheduler/DAGScheduler.scala |  2 +-
 .../scheduler/EventLoggingListener.scala      | 61 ++++++++-----
 .../apache/spark/scheduler/JobLogger.scala    |  2 +-
 .../apache/spark/scheduler/JobWaiter.scala    |  2 +-
 .../scheduler/OutputCommitCoordinator.scala   |  2 +-
 .../apache/spark/scheduler/ResultTask.scala   |  2 +-
 .../spark/scheduler/ShuffleMapTask.scala      |  4 +-
 .../spark/scheduler/SparkListener.scala       |  4 +-
 .../org/apache/spark/scheduler/Stage.scala    |  2 +-
 .../apache/spark/scheduler/TaskLocation.scala | 19 ++--
 .../spark/scheduler/TaskSchedulerImpl.scala   | 16 ++--
 .../spark/scheduler/TaskSetManager.scala      | 13 +--
 .../CoarseGrainedSchedulerBackend.scala       |  2 +-
 .../cluster/YarnSchedulerBackend.scala        |  2 +-
 .../scheduler/cluster/mesos/MemoryUtils.scala |  2 +-
 .../cluster/mesos/MesosSchedulerBackend.scala |  2 +-
 .../spark/scheduler/local/LocalBackend.scala  |  4 +-
 .../spark/serializer/JavaSerializer.scala     |  5 +-
 .../spark/serializer/KryoSerializer.scala     |  2 +-
 .../shuffle/FileShuffleBlockManager.scala     |  4 +-
 .../shuffle/IndexShuffleBlockManager.scala    |  4 +-
 .../org/apache/spark/storage/BlockId.scala    | 34 +++----
 .../apache/spark/storage/BlockManagerId.scala |  8 +-
 .../spark/storage/BlockManagerMaster.scala    |  2 +-
 .../storage/BlockManagerMasterActor.scala     |  4 +-
 .../storage/BlockManagerSlaveActor.scala      |  2 +-
 .../spark/storage/BlockObjectWriter.scala     | 17 ++--
 .../apache/spark/storage/FileSegment.scala    |  4 +-
 .../org/apache/spark/storage/RDDInfo.scala    |  4 +-
 .../apache/spark/storage/StorageLevel.scala   | 16 ++--
 .../spark/storage/StorageStatusListener.scala |  7 +-
 .../spark/storage/TachyonFileSegment.scala    |  4 +-
 .../scala/org/apache/spark/ui/SparkUI.scala   |  2 +-
 .../scala/org/apache/spark/ui/UIUtils.scala   |  6 +-
 .../apache/spark/ui/UIWorkloadGenerator.scala |  4 +-
 .../apache/spark/ui/exec/ExecutorsTab.scala   | 10 +--
 .../spark/ui/jobs/JobProgressListener.scala   | 14 +--
 .../org/apache/spark/ui/jobs/JobsTab.scala    |  2 +-
 .../org/apache/spark/ui/jobs/StagePage.scala  | 10 ++-
 .../org/apache/spark/ui/jobs/StagesTab.scala  |  6 +-
 .../org/apache/spark/ui/jobs/UIData.scala     | 10 +--
 .../apache/spark/ui/storage/StorageTab.scala  | 12 +--
 .../spark/util/CompletionIterator.scala       | 10 +--
 .../org/apache/spark/util/Distribution.scala  |  2 +-
 .../org/apache/spark/util/ManualClock.scala   | 30 +++----
 .../apache/spark/util/MetadataCleaner.scala   |  7 +-
 .../org/apache/spark/util/MutablePair.scala   |  2 +-
 .../apache/spark/util/ParentClassLoader.scala |  2 +-
 .../spark/util/SerializableBuffer.scala       |  2 +-
 .../org/apache/spark/util/StatCounter.scala   |  4 +-
 .../util/TimeStampedWeakValueHashMap.scala    |  6 +-
 .../scala/org/apache/spark/util/Utils.scala   | 30 +++----
 .../apache/spark/util/collection/BitSet.scala |  4 +-
 .../collection/ExternalAppendOnlyMap.scala    |  4 +-
 .../util/collection/ExternalSorter.scala      |  2 +-
 .../spark/util/collection/OpenHashMap.scala   |  6 +-
 .../spark/util/collection/OpenHashSet.scala   |  4 +-
 .../collection/PrimitiveKeyOpenHashMap.scala  |  8 +-
 .../apache/spark/util/collection/Utils.scala  |  2 +-
 .../spark/util/logging/FileAppender.scala     |  4 +-
 .../spark/util/random/RandomSampler.scala     | 44 +++++----
 .../util/random/StratifiedSamplingUtils.scala |  2 +-
 .../spark/util/random/XORShiftRandom.scala    |  2 +-
 project/MimaExcludes.scala                    |  8 +-
 scalastyle-config.xml                         |  4 +-
 142 files changed, 597 insertions(+), 526 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index bcf832467f..330df1d59a 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -18,8 +18,6 @@
 package org.apache.spark
 
 import java.io.{ObjectInputStream, Serializable}
-import java.util.concurrent.atomic.AtomicLong
-import java.lang.ThreadLocal
 
 import scala.collection.generic.Growable
 import scala.collection.mutable.Map
@@ -109,7 +107,7 @@ class Accumulable[R, T] (
    * The typical use of this method is to directly mutate the local value, eg., to add
    * an element to a Set.
    */
-  def localValue = value_
+  def localValue: R = value_
 
   /**
    * Set the accumulator's value; only allowed on master.
@@ -137,7 +135,7 @@ class Accumulable[R, T] (
     Accumulators.register(this, false)
   }
 
-  override def toString = if (value_ == null) "null" else value_.toString
+  override def toString: String = if (value_ == null) "null" else value_.toString
 }
 
 /**
@@ -257,22 +255,22 @@ object AccumulatorParam {
 
   implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
     def addInPlace(t1: Double, t2: Double): Double = t1 + t2
-    def zero(initialValue: Double) = 0.0
+    def zero(initialValue: Double): Double = 0.0
   }
 
   implicit object IntAccumulatorParam extends AccumulatorParam[Int] {
     def addInPlace(t1: Int, t2: Int): Int = t1 + t2
-    def zero(initialValue: Int) = 0
+    def zero(initialValue: Int): Int = 0
   }
 
   implicit object LongAccumulatorParam extends AccumulatorParam[Long] {
-    def addInPlace(t1: Long, t2: Long) = t1 + t2
-    def zero(initialValue: Long) = 0L
+    def addInPlace(t1: Long, t2: Long): Long = t1 + t2
+    def zero(initialValue: Long): Long = 0L
   }
 
   implicit object FloatAccumulatorParam extends AccumulatorParam[Float] {
-    def addInPlace(t1: Float, t2: Float) = t1 + t2
-    def zero(initialValue: Float) = 0f
+    def addInPlace(t1: Float, t2: Float): Float = t1 + t2
+    def zero(initialValue: Float): Float = 0f
   }
 
   // TODO: Add AccumulatorParams for other types, e.g. lists and strings
@@ -351,6 +349,7 @@ private[spark] object Accumulators extends Logging {
     }
   }
 
-  def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue)
-  def stringifyValue(value: Any) = "%s".format(value)
+  def stringifyPartialValue(partialValue: Any): String = "%s".format(partialValue)
+
+  def stringifyValue(value: Any): String = "%s".format(value)
 }
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index 9a7cd4523e..fc8cdde934 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -74,7 +74,7 @@ class ShuffleDependency[K, V, C](
     val mapSideCombine: Boolean = false)
   extends Dependency[Product2[K, V]] {
 
-  override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]]
+  override def rdd: RDD[Product2[K, V]] = _rdd.asInstanceOf[RDD[Product2[K, V]]]
 
   val shuffleId: Int = _rdd.context.newShuffleId()
 
@@ -91,7 +91,7 @@ class ShuffleDependency[K, V, C](
  */
 @DeveloperApi
 class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
-  override def getParents(partitionId: Int) = List(partitionId)
+  override def getParents(partitionId: Int): List[Int] = List(partitionId)
 }
 
 
@@ -107,7 +107,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
 class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int)
   extends NarrowDependency[T](rdd) {
 
-  override def getParents(partitionId: Int) = {
+  override def getParents(partitionId: Int): List[Int] = {
     if (partitionId >= outStart && partitionId < outStart + length) {
       List(partitionId - outStart + inStart)
     } else {
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index e97a7375a2..91f9ef8ce7 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -168,7 +168,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
     }
   }
 
-  def jobIds = Seq(jobWaiter.jobId)
+  def jobIds: Seq[Int] = Seq(jobWaiter.jobId)
 }
 
 
@@ -276,7 +276,7 @@ class ComplexFutureAction[T] extends FutureAction[T] {
 
   override def value: Option[Try[T]] = p.future.value
 
-  def jobIds = jobs
+  def jobIds: Seq[Int] = jobs
 
 }
 
diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
index 69178da1a7..715f292f03 100644
--- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
+++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
@@ -65,7 +65,7 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler: TaskSchedule
     super.preStart()
   }
   
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case Heartbeat(executorId, taskMetrics, blockManagerId) =>
       val unknownExecutor = !scheduler.executorHeartbeatReceived(
         executorId, taskMetrics, blockManagerId)
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 6e4edc7c80..c9426c5de2 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -43,7 +43,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
   extends Actor with ActorLogReceive with Logging {
   val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf)
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case GetMapOutputStatuses(shuffleId: Int) =>
       val hostPort = sender.path.address.hostPort
       logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort)
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index e53a78ead2..b8d244408b 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -76,7 +76,7 @@ object Partitioner {
  * produce an unexpected or incorrect result.
  */
 class HashPartitioner(partitions: Int) extends Partitioner {
-  def numPartitions = partitions
+  def numPartitions: Int = partitions
 
   def getPartition(key: Any): Int = key match {
     case null => 0
@@ -154,7 +154,7 @@ class RangePartitioner[K : Ordering : ClassTag, V](
     }
   }
 
-  def numPartitions = rangeBounds.length + 1
+  def numPartitions: Int = rangeBounds.length + 1
 
   private var binarySearch: ((Array[K], K) => Int) = CollectionsUtils.makeBinarySearch[K]
 
diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala
index 55cb25946c..cb2cae1852 100644
--- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala
+++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala
@@ -28,8 +28,10 @@ import org.apache.spark.util.Utils
 
 @DeveloperApi
 class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable {
-  def value = t
-  override def toString = t.toString
+
+  def value: T = t
+
+  override def toString: String = t.toString
 
   private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException {
     out.defaultWriteObject()
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 2ca19f53d2..0c123c96b8 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -133,7 +133,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
   }
 
   /** Set multiple parameters together */
-  def setAll(settings: Traversable[(String, String)]) = {
+  def setAll(settings: Traversable[(String, String)]): SparkConf = {
     this.settings.putAll(settings.toMap.asJava)
     this
   }
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 228ff715fe..a70be16f77 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -986,7 +986,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
     union(Seq(first) ++ rest)
 
   /** Get an RDD that has no partitions or elements. */
-  def emptyRDD[T: ClassTag] = new EmptyRDD[T](this)
+  def emptyRDD[T: ClassTag]: EmptyRDD[T] = new EmptyRDD[T](this)
 
   // Methods for creating shared variables
 
@@ -994,7 +994,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
    * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add"
    * values to using the `+=` method. Only the driver can access the accumulator's `value`.
    */
-  def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
+  def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] =
   {
     val acc = new Accumulator(initialValue, param)
     cleaner.foreach(_.registerAccumulatorForCleanup(acc))
@@ -1006,7 +1006,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
    * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the
    * driver can access the accumulator's `value`.
    */
-  def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = {
+  def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T])
+    : Accumulator[T] = {
     val acc = new Accumulator(initialValue, param, Some(name))
     cleaner.foreach(_.registerAccumulatorForCleanup(acc))
     acc
@@ -1018,7 +1019,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
    * @tparam R accumulator result type
    * @tparam T type that can be added to the accumulator
    */
-  def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = {
+  def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T])
+    : Accumulable[R, T] = {
     val acc = new Accumulable(initialValue, param)
     cleaner.foreach(_.registerAccumulatorForCleanup(acc))
     acc
@@ -1031,7 +1033,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
    * @tparam R accumulator result type
    * @tparam T type that can be added to the accumulator
    */
-  def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = {
+  def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T])
+    : Accumulable[R, T] = {
     val acc = new Accumulable(initialValue, param, Some(name))
     cleaner.foreach(_.registerAccumulatorForCleanup(acc))
     acc
@@ -1209,7 +1212,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
   override def killExecutor(executorId: String): Boolean = super.killExecutor(executorId)
 
   /** The version of Spark on which this application is running. */
-  def version = SPARK_VERSION
+  def version: String = SPARK_VERSION
 
   /**
    * Return a map from the slave to the max memory available for caching and the remaining
@@ -1659,7 +1662,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
     }
   }
 
-  def getCheckpointDir = checkpointDir
+  def getCheckpointDir: Option[String] = checkpointDir
 
   /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
   def defaultParallelism: Int = {
@@ -1900,28 +1903,28 @@ object SparkContext extends Logging {
     "backward compatibility.", "1.3.0")
   object DoubleAccumulatorParam extends AccumulatorParam[Double] {
     def addInPlace(t1: Double, t2: Double): Double = t1 + t2
-    def zero(initialValue: Double) = 0.0
+    def zero(initialValue: Double): Double = 0.0
   }
 
   @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
     "backward compatibility.", "1.3.0")
   object IntAccumulatorParam extends AccumulatorParam[Int] {
     def addInPlace(t1: Int, t2: Int): Int = t1 + t2
-    def zero(initialValue: Int) = 0
+    def zero(initialValue: Int): Int = 0
   }
 
   @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
     "backward compatibility.", "1.3.0")
   object LongAccumulatorParam extends AccumulatorParam[Long] {
-    def addInPlace(t1: Long, t2: Long) = t1 + t2
-    def zero(initialValue: Long) = 0L
+    def addInPlace(t1: Long, t2: Long): Long = t1 + t2
+    def zero(initialValue: Long): Long = 0L
   }
 
   @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " +
     "backward compatibility.", "1.3.0")
   object FloatAccumulatorParam extends AccumulatorParam[Float] {
-    def addInPlace(t1: Float, t2: Float) = t1 + t2
-    def zero(initialValue: Float) = 0f
+    def addInPlace(t1: Float, t2: Float): Float = t1 + t2
+    def zero(initialValue: Float): Float = 0f
   }
 
   // The following deprecated functions have already been moved to `object RDD` to
@@ -1931,18 +1934,18 @@ object SparkContext extends Logging {
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
   def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)])
-      (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = {
+      (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): PairRDDFunctions[K, V] =
     RDD.rddToPairRDDFunctions(rdd)
-  }
 
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
-  def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd)
+  def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]): AsyncRDDActions[T] =
+    RDD.rddToAsyncRDDActions(rdd)
 
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
   def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
-      rdd: RDD[(K, V)]) = {
+      rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = {
     val kf = implicitly[K => Writable]
     val vf = implicitly[V => Writable]
     // Set the Writable class to null and `SequenceFileRDDFunctions` will use Reflection to get it
@@ -1954,16 +1957,17 @@ object SparkContext extends Logging {
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
   def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](
-      rdd: RDD[(K, V)]) =
+      rdd: RDD[(K, V)]): OrderedRDDFunctions[K, V, (K, V)] =
     RDD.rddToOrderedRDDFunctions(rdd)
 
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
-  def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd)
+  def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions =
+    RDD.doubleRDDToDoubleRDDFunctions(rdd)
 
   @deprecated("Replaced by implicit functions in the RDD companion object. This is " +
     "kept here only for backward compatibility.", "1.3.0")
-  def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) =
+  def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]): DoubleRDDFunctions =
     RDD.numericRDDToDoubleRDDFunctions(rdd)
 
   // The following deprecated functions have already been moved to `object WritableFactory` to
@@ -2134,7 +2138,7 @@ object SparkContext extends Logging {
         (backend, scheduler)
 
       case LOCAL_N_REGEX(threads) =>
-        def localCpuCount = Runtime.getRuntime.availableProcessors()
+        def localCpuCount: Int = Runtime.getRuntime.availableProcessors()
         // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads.
         val threadCount = if (threads == "*") localCpuCount else threads.toInt
         if (threadCount <= 0) {
@@ -2146,7 +2150,7 @@ object SparkContext extends Logging {
         (backend, scheduler)
 
       case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
-        def localCpuCount = Runtime.getRuntime.availableProcessors()
+        def localCpuCount: Int = Runtime.getRuntime.availableProcessors()
         // local[*, M] means the number of cores on the computer with M failures
         // local[N, M] means exactly N threads with M failures
         val threadCount = if (threads == "*") localCpuCount else threads.toInt
diff --git a/core/src/main/scala/org/apache/spark/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala
index c415fe99b1..fe19f07e32 100644
--- a/core/src/main/scala/org/apache/spark/TaskState.scala
+++ b/core/src/main/scala/org/apache/spark/TaskState.scala
@@ -27,9 +27,9 @@ private[spark] object TaskState extends Enumeration {
 
   type TaskState = Value
 
-  def isFailed(state: TaskState) = (LOST == state) || (FAILED == state)
+  def isFailed(state: TaskState): Boolean = (LOST == state) || (FAILED == state)
 
-  def isFinished(state: TaskState) = FINISHED_STATES.contains(state)
+  def isFinished(state: TaskState): Boolean = FINISHED_STATES.contains(state)
 
   def toMesos(state: TaskState): MesosTaskState = state match {
     case LAUNCHING => MesosTaskState.TASK_STARTING
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index 35b324ba6f..398ca41e16 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -107,7 +107,7 @@ private[spark] object TestUtils {
 
   private class JavaSourceFromString(val name: String, val code: String)
     extends SimpleJavaFileObject(createURI(name), SOURCE) {
-    override def getCharContent(ignoreEncodingErrors: Boolean) = code
+    override def getCharContent(ignoreEncodingErrors: Boolean): String = code
   }
 
   /** Creates a compiled class with the given name. Class file will be placed in destDir. */
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index 3e9beb670f..18ccd625fc 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -179,7 +179,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T])
   def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] =
     wrapRDD(rdd.subtract(other, p))
 
-  override def toString = rdd.toString
+  override def toString: String = rdd.toString
 
   /** Assign a name to this RDD */
   def setName(name: String): JavaRDD[T] = {
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 6d6ed693be..3be6783bba 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -108,7 +108,7 @@ class JavaSparkContext(val sc: SparkContext)
 
   private[spark] val env = sc.env
 
-  def statusTracker = new JavaSparkStatusTracker(sc)
+  def statusTracker: JavaSparkStatusTracker = new JavaSparkStatusTracker(sc)
 
   def isLocal: java.lang.Boolean = sc.isLocal
 
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
index 71b26737b8..8f9647eea9 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.api.java
 
+import java.util.Map.Entry
+
 import com.google.common.base.Optional
 
 import java.{util => ju}
@@ -30,8 +32,8 @@ private[spark] object JavaUtils {
     }
 
   // Workaround for SPARK-3926 / SI-8911
-  def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) =
-    new SerializableMapWrapper(underlying)
+  def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]): SerializableMapWrapper[A, B]
+    = new SerializableMapWrapper(underlying)
 
   // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper,
   // but implements java.io.Serializable. It can't just be subclassed to make it
@@ -40,36 +42,33 @@ private[spark] object JavaUtils {
   class SerializableMapWrapper[A, B](underlying: collection.Map[A, B])
     extends ju.AbstractMap[A, B] with java.io.Serializable { self =>
 
-    override def size = underlying.size
+    override def size: Int = underlying.size
 
     override def get(key: AnyRef): B = try {
-      underlying get key.asInstanceOf[A] match {
-        case None => null.asInstanceOf[B]
-        case Some(v) => v
-      }
+      underlying.getOrElse(key.asInstanceOf[A], null.asInstanceOf[B])
     } catch {
       case ex: ClassCastException => null.asInstanceOf[B]
     }
 
     override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] {
-      def size = self.size
+      override def size: Int = self.size
 
-      def iterator = new ju.Iterator[ju.Map.Entry[A, B]] {
+      override def iterator: ju.Iterator[ju.Map.Entry[A, B]] = new ju.Iterator[ju.Map.Entry[A, B]] {
         val ui = underlying.iterator
         var prev : Option[A] = None
 
-        def hasNext = ui.hasNext
+        def hasNext: Boolean = ui.hasNext
 
-        def next() = {
-          val (k, v) = ui.next
+        def next(): Entry[A, B] = {
+          val (k, v) = ui.next()
           prev = Some(k)
           new ju.Map.Entry[A, B] {
             import scala.util.hashing.byteswap32
-            def getKey = k
-            def getValue = v
-            def setValue(v1 : B) = self.put(k, v1)
-            override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16)
-            override def equals(other: Any) = other match {
+            override def getKey: A = k
+            override def getValue: B = v
+            override def setValue(v1 : B): B = self.put(k, v1)
+            override def hashCode: Int = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16)
+            override def equals(other: Any): Boolean = other match {
               case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue
               case _ => false
             }
@@ -81,7 +80,7 @@ private[spark] object JavaUtils {
             case Some(k) =>
               underlying match {
                 case mm: mutable.Map[A, _] =>
-                  mm remove k
+                  mm.remove(k)
                   prev = None
                 case _ =>
                   throw new UnsupportedOperationException("remove")
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 4c71b69069..19f4c95fca 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -54,9 +54,11 @@ private[spark] class PythonRDD(
   val bufferSize = conf.getInt("spark.buffer.size", 65536)
   val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true)
 
-  override def getPartitions = firstParent.partitions
+  override def getPartitions: Array[Partition] = firstParent.partitions
 
-  override val partitioner = if (preservePartitoning) firstParent.partitioner else None
+  override val partitioner: Option[Partitioner] = {
+    if (preservePartitoning) firstParent.partitioner else None
+  }
 
   override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
     val startTime = System.currentTimeMillis
@@ -92,7 +94,7 @@ private[spark] class PythonRDD(
     // Return an iterator that read lines from the process's stdout
     val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize))
     val stdoutIterator = new Iterator[Array[Byte]] {
-      def next(): Array[Byte] = {
+      override def next(): Array[Byte] = {
         val obj = _nextObj
         if (hasNext) {
           _nextObj = read()
@@ -175,7 +177,7 @@ private[spark] class PythonRDD(
 
       var _nextObj = read()
 
-      def hasNext = _nextObj != null
+      override def hasNext: Boolean = _nextObj != null
     }
     new InterruptibleIterator(context, stdoutIterator)
   }
@@ -303,11 +305,10 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce
  * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python.
  * This is used by PySpark's shuffle operations.
  */
-private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
-  RDD[(Long, Array[Byte])](prev) {
-  override def getPartitions = prev.partitions
-  override val partitioner = prev.partitioner
-  override def compute(split: Partition, context: TaskContext) =
+private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Long, Array[Byte])](prev) {
+  override def getPartitions: Array[Partition] = prev.partitions
+  override val partitioner: Option[Partitioner] = prev.partitioner
+  override def compute(split: Partition, context: TaskContext): Iterator[(Long, Array[Byte])] =
     prev.iterator(split, context).grouped(2).map {
       case Seq(a, b) => (Utils.deserializeLongValue(a), b)
       case x => throw new SparkException("PairwiseRDD: unexpected value: " + x)
@@ -435,7 +436,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       minSplits: Int,
-      batchSize: Int) = {
+      batchSize: Int): JavaRDD[Array[Byte]] = {
     val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text")
     val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text")
     val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]]
@@ -462,7 +463,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       confAsMap: java.util.HashMap[String, String],
-      batchSize: Int) = {
+      batchSize: Int): JavaRDD[Array[Byte]] = {
     val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration())
     val rdd =
       newAPIHadoopRDDFromClassNames[K, V, F](sc,
@@ -488,7 +489,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       confAsMap: java.util.HashMap[String, String],
-      batchSize: Int) = {
+      batchSize: Int): JavaRDD[Array[Byte]] = {
     val conf = PythonHadoopUtil.mapToConf(confAsMap)
     val rdd =
       newAPIHadoopRDDFromClassNames[K, V, F](sc,
@@ -505,7 +506,7 @@ private[spark] object PythonRDD extends Logging {
       inputFormatClass: String,
       keyClass: String,
       valueClass: String,
-      conf: Configuration) = {
+      conf: Configuration): RDD[(K, V)] = {
     val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]]
     val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]]
     val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]]
@@ -531,7 +532,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       confAsMap: java.util.HashMap[String, String],
-      batchSize: Int) = {
+      batchSize: Int): JavaRDD[Array[Byte]] = {
     val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration())
     val rdd =
       hadoopRDDFromClassNames[K, V, F](sc,
@@ -557,7 +558,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       confAsMap: java.util.HashMap[String, String],
-      batchSize: Int) = {
+      batchSize: Int): JavaRDD[Array[Byte]] = {
     val conf = PythonHadoopUtil.mapToConf(confAsMap)
     val rdd =
       hadoopRDDFromClassNames[K, V, F](sc,
@@ -686,7 +687,7 @@ private[spark] object PythonRDD extends Logging {
       pyRDD: JavaRDD[Array[Byte]],
       batchSerialized: Boolean,
       path: String,
-      compressionCodecClass: String) = {
+      compressionCodecClass: String): Unit = {
     saveAsHadoopFile(
       pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat",
       null, null, null, null, new java.util.HashMap(), compressionCodecClass)
@@ -711,7 +712,7 @@ private[spark] object PythonRDD extends Logging {
       keyConverterClass: String,
       valueConverterClass: String,
       confAsMap: java.util.HashMap[String, String],
-      compressionCodecClass: String) = {
+      compressionCodecClass: String): Unit = {
     val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized)
     val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse(
       inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass))
@@ -741,7 +742,7 @@ private[spark] object PythonRDD extends Logging {
       valueClass: String,
       keyConverterClass: String,
       valueConverterClass: String,
-      confAsMap: java.util.HashMap[String, String]) = {
+      confAsMap: java.util.HashMap[String, String]): Unit = {
     val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized)
     val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse(
       inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass))
@@ -766,7 +767,7 @@ private[spark] object PythonRDD extends Logging {
       confAsMap: java.util.HashMap[String, String],
       keyConverterClass: String,
       valueConverterClass: String,
-      useNewAPI: Boolean) = {
+      useNewAPI: Boolean): Unit = {
     val conf = PythonHadoopUtil.mapToConf(confAsMap)
     val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized),
       keyConverterClass, valueConverterClass, new JavaToWritableConverter)
diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
index fb52a960e0..257491e90d 100644
--- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala
@@ -84,7 +84,7 @@ private[spark] object SerDeUtil extends Logging {
   private var initialized = false
   // This should be called before trying to unpickle array.array from Python
   // In cluster mode, this should be put in closure
-  def initialize() = {
+  def initialize(): Unit = {
     synchronized{
       if (!initialized) {
         Unpickler.registerConstructor("array", "array", new ArrayConstructor())
diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
index cf289fb3ae..8f30ff9202 100644
--- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala
@@ -18,38 +18,37 @@
 package org.apache.spark.api.python
 
 import java.io.{DataOutput, DataInput}
+import java.{util => ju}
 
 import com.google.common.base.Charsets.UTF_8
 
 import org.apache.hadoop.io._
 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat
+
+import org.apache.spark.SparkException
 import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.{SparkContext, SparkException}
 
 /**
  * A class to test Pyrolite serialization on the Scala side, that will be deserialized
  * in Python
- * @param str
- * @param int
- * @param double
  */
 case class TestWritable(var str: String, var int: Int, var double: Double) extends Writable {
   def this() = this("", 0, 0.0)
 
-  def getStr = str
+  def getStr: String = str
   def setStr(str: String) { this.str = str }
-  def getInt = int
+  def getInt: Int = int
   def setInt(int: Int) { this.int = int }
-  def getDouble = double
+  def getDouble: Double = double
   def setDouble(double: Double) { this.double = double }
 
-  def write(out: DataOutput) = {
+  def write(out: DataOutput): Unit = {
     out.writeUTF(str)
     out.writeInt(int)
     out.writeDouble(double)
   }
 
-  def readFields(in: DataInput) = {
+  def readFields(in: DataInput): Unit = {
     str = in.readUTF()
     int = in.readInt()
     double = in.readDouble()
@@ -57,28 +56,28 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten
 }
 
 private[python] class TestInputKeyConverter extends Converter[Any, Any] {
-  override def convert(obj: Any) = {
+  override def convert(obj: Any): Char = {
     obj.asInstanceOf[IntWritable].get().toChar
   }
 }
 
 private[python] class TestInputValueConverter extends Converter[Any, Any] {
   import collection.JavaConversions._
-  override def convert(obj: Any) = {
+  override def convert(obj: Any): ju.List[Double] = {
     val m = obj.asInstanceOf[MapWritable]
     seqAsJavaList(m.keySet.map(w => w.asInstanceOf[DoubleWritable].get()).toSeq)
   }
 }
 
 private[python] class TestOutputKeyConverter extends Converter[Any, Any] {
-  override def convert(obj: Any) = {
+  override def convert(obj: Any): Text = {
     new Text(obj.asInstanceOf[Int].toString)
   }
 }
 
 private[python] class TestOutputValueConverter extends Converter[Any, Any] {
   import collection.JavaConversions._
-  override def convert(obj: Any) = {
+  override def convert(obj: Any): DoubleWritable = {
     new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head)
   }
 }
@@ -86,7 +85,7 @@ private[python] class TestOutputValueConverter extends Converter[Any, Any] {
 private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable])
 
 private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] {
-  override def convert(obj: Any) = obj match {
+  override def convert(obj: Any): DoubleArrayWritable = obj match {
     case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] =>
       val daw = new DoubleArrayWritable
       daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_)))
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index a5ea478f23..12d79f6ed3 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -146,5 +146,5 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo
     }
   }
 
-  override def toString = "Broadcast(" + id + ")"
+  override def toString: String = "Broadcast(" + id + ")"
 }
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
index 8f8a0b11f9..685313ac00 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala
@@ -58,7 +58,7 @@ private[spark] class BroadcastManager(
 
   private val nextBroadcastId = new AtomicLong(0)
 
-  def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean) = {
+  def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = {
     broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement())
   }
 
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 1444c0dd3d..74ccfa6d3c 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -160,7 +160,7 @@ private[broadcast] object HttpBroadcast extends Logging {
     logInfo("Broadcast server started at " + serverUri)
   }
 
-  def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name)
+  def getFile(id: Long): File = new File(broadcastDir, BroadcastBlockId(id).name)
 
   private def write(id: Long, value: Any) {
     val file = getFile(id)
@@ -222,7 +222,7 @@ private[broadcast] object HttpBroadcast extends Logging {
    * If removeFromDriver is true, also remove these persisted blocks on the driver
    * and delete the associated broadcast file.
    */
-  def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized {
+  def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = synchronized {
     SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking)
     if (removeFromDriver) {
       val file = getFile(id)
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
index c7ef02d572..cf3ae36f27 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala
@@ -31,7 +31,7 @@ class HttpBroadcastFactory extends BroadcastFactory {
     HttpBroadcast.initialize(isDriver, conf, securityMgr)
   }
 
-  override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) =
+  override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] =
     new HttpBroadcast[T](value_, isLocal, id)
 
   override def stop() { HttpBroadcast.stop() }
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 94142d3336..23b02e6033 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -222,7 +222,7 @@ private object TorrentBroadcast extends Logging {
    * Remove all persisted blocks associated with this torrent broadcast on the executors.
    * If removeFromDriver is true, also remove these persisted blocks on the driver.
    */
-  def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = {
+  def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = {
     logDebug(s"Unpersisting TorrentBroadcast $id")
     SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking)
   }
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
index fb024c1209..96d8dd7990 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala
@@ -30,7 +30,7 @@ class TorrentBroadcastFactory extends BroadcastFactory {
 
   override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { }
 
-  override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = {
+  override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long): Broadcast[T] = {
     new TorrentBroadcast[T](value_, id)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala
index 237d26fc6b..65238af2ca 100644
--- a/core/src/main/scala/org/apache/spark/deploy/Client.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala
@@ -38,7 +38,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
   var masterActor: ActorSelection = _
   val timeout = AkkaUtils.askTimeout(conf)
 
-  override def preStart() = {
+  override def preStart(): Unit = {
     masterActor = context.actorSelection(
       Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system)))
 
@@ -118,7 +118,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf)
     }
   }
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
 
     case SubmitDriverResponse(success, driverId, message) =>
       println(message)
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 53bc62aff7..5cbac787dc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -42,7 +42,7 @@ private[deploy] class ClientArguments(args: Array[String]) {
   var memory: Int = DEFAULT_MEMORY
   var cores: Int = DEFAULT_CORES
   private var _driverOptions = ListBuffer[String]()
-  def driverOptions = _driverOptions.toSeq
+  def driverOptions: Seq[String] = _driverOptions.toSeq
 
   // kill parameters
   var driverId: String = ""
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index 7f600d8960..0997507d01 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -162,7 +162,7 @@ private[deploy] object DeployMessages {
     Utils.checkHost(host, "Required hostname")
     assert (port > 0)
 
-    def uri = "spark://" + host + ":" + port
+    def uri: String = "spark://" + host + ":" + port
     def restUri: Option[String] = restPort.map { p => "spark://" + host + ":" + p }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
index 5668b53fc6..a7c89276a0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -426,7 +426,7 @@ private object SparkDocker {
 }
 
 private class DockerId(val id: String) {
-  override def toString = id
+  override def toString: String = id
 }
 
 private object Docker extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index 458a7c3a45..dfc5b97e6a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.deploy
 
+import org.json4s.JsonAST.JObject
 import org.json4s.JsonDSL._
 
 import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
@@ -24,7 +25,7 @@ import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo}
 import org.apache.spark.deploy.worker.ExecutorRunner
 
 private[deploy] object JsonProtocol {
- def writeWorkerInfo(obj: WorkerInfo) = {
+ def writeWorkerInfo(obj: WorkerInfo): JObject = {
    ("id" -> obj.id) ~
    ("host" -> obj.host) ~
    ("port" -> obj.port) ~
@@ -39,7 +40,7 @@ private[deploy] object JsonProtocol {
    ("lastheartbeat" -> obj.lastHeartbeat)
  }
 
-  def writeApplicationInfo(obj: ApplicationInfo) = {
+  def writeApplicationInfo(obj: ApplicationInfo): JObject = {
     ("starttime" -> obj.startTime) ~
     ("id" -> obj.id) ~
     ("name" -> obj.desc.name) ~
@@ -51,7 +52,7 @@ private[deploy] object JsonProtocol {
     ("duration" -> obj.duration)
   }
 
-  def writeApplicationDescription(obj: ApplicationDescription) = {
+  def writeApplicationDescription(obj: ApplicationDescription): JObject = {
     ("name" -> obj.name) ~
     ("cores" -> obj.maxCores) ~
     ("memoryperslave" -> obj.memoryPerSlave) ~
@@ -59,14 +60,14 @@ private[deploy] object JsonProtocol {
     ("command" -> obj.command.toString)
   }
 
-  def writeExecutorRunner(obj: ExecutorRunner) = {
+  def writeExecutorRunner(obj: ExecutorRunner): JObject = {
     ("id" -> obj.execId) ~
     ("memory" -> obj.memory) ~
     ("appid" -> obj.appId) ~
     ("appdesc" -> writeApplicationDescription(obj.appDesc))
   }
 
-  def writeDriverInfo(obj: DriverInfo) = {
+  def writeDriverInfo(obj: DriverInfo): JObject = {
     ("id" -> obj.id) ~
     ("starttime" -> obj.startTime.toString) ~
     ("state" -> obj.state.toString) ~
@@ -74,7 +75,7 @@ private[deploy] object JsonProtocol {
     ("memory" -> obj.desc.mem)
   }
 
-  def writeMasterState(obj: MasterStateResponse) = {
+  def writeMasterState(obj: MasterStateResponse): JObject = {
     ("url" -> obj.uri) ~
     ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~
     ("cores" -> obj.workers.map(_.cores).sum) ~
@@ -87,7 +88,7 @@ private[deploy] object JsonProtocol {
     ("status" -> obj.status.toString)
   }
 
-  def writeWorkerState(obj: WorkerStateResponse) = {
+  def writeWorkerState(obj: WorkerStateResponse): JObject = {
     ("id" -> obj.workerId) ~
     ("masterurl" -> obj.masterUrl) ~
     ("masterwebuiurl" -> obj.masterWebUiUrl) ~
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index e0a32fb65c..c2568eb4b6 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -193,7 +193,7 @@ class SparkHadoopUtil extends Logging {
    * that file.
    */
   def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = {
-    def recurse(path: Path) = {
+    def recurse(path: Path): Array[FileStatus] = {
       val (directories, leaves) = fs.listStatus(path).partition(_.isDir)
       leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath))
     }
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index 4f506be63f..660307d19e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -777,7 +777,7 @@ private[deploy] object SparkSubmitUtils {
   }
 
   /** A nice function to use in tests as well. Values are dummy strings. */
-  def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance(
+  def getModuleDescriptor: DefaultModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance(
     ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0"))
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index 2250d5a28e..6eb73c4347 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -252,7 +252,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
     master.startsWith("spark://") && deployMode == "cluster"
   }
 
-  override def toString = {
+  override def toString: String = {
     s"""Parsed arguments:
     |  master                  $master
     |  deployMode              $deployMode
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 2d24083a77..3b72972525 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -116,7 +116,7 @@ private[spark] class AppClient(
       masterAkkaUrls.map(AddressFromURIString(_).hostPort).contains(remoteUrl.hostPort)
     }
 
-    override def receiveWithLogging = {
+    override def receiveWithLogging: PartialFunction[Any, Unit] = {
       case RegisteredApplication(appId_, masterUrl) =>
         appId = appId_
         registered = true
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index db7c499661..80c9c13dde 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -93,7 +93,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
    */
   private def getRunner(operateFun: () => Unit): Runnable = {
     new Runnable() {
-      override def run() = Utils.tryOrExit {
+      override def run(): Unit = Utils.tryOrExit {
         operateFun()
       }
     }
@@ -141,7 +141,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis
     }
   }
 
-  override def getListing() = applications.values
+  override def getListing(): Iterable[FsApplicationHistoryInfo] = applications.values
 
   override def getAppUI(appId: String): Option[SparkUI] = {
     try {
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index af483d560b..72f6048239 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -61,7 +61,7 @@ class HistoryServer(
   private val appCache = CacheBuilder.newBuilder()
     .maximumSize(retainedApplications)
     .removalListener(new RemovalListener[String, SparkUI] {
-      override def onRemoval(rm: RemovalNotification[String, SparkUI]) = {
+      override def onRemoval(rm: RemovalNotification[String, SparkUI]): Unit = {
         detachSparkUI(rm.getValue())
       }
     })
@@ -149,14 +149,14 @@ class HistoryServer(
    *
    * @return List of all known applications.
    */
-  def getApplicationList() = provider.getListing()
+  def getApplicationList(): Iterable[ApplicationHistoryInfo] = provider.getListing()
 
   /**
    * Returns the provider configuration to show in the listing page.
    *
    * @return A map with the provider's configuration.
    */
-  def getProviderConfig() = provider.getConfig()
+  def getProviderConfig(): Map[String, String] = provider.getConfig()
 
 }
 
@@ -195,9 +195,7 @@ object HistoryServer extends Logging {
     server.bind()
 
     Runtime.getRuntime().addShutdownHook(new Thread("HistoryServerStopper") {
-      override def run() = {
-        server.stop()
-      }
+      override def run(): Unit = server.stop()
     })
 
     // Wait until the end of the world... or if the HistoryServer process is manually stopped
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
index d2d30bfd7f..32499b3a78 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -48,7 +48,7 @@ private[master] class FileSystemPersistenceEngine(
     new File(dir + File.separator + name).delete()
   }
 
-  override def read[T: ClassTag](prefix: String) = {
+  override def read[T: ClassTag](prefix: String): Seq[T] = {
     val files = new File(dir).listFiles().filter(_.getName.startsWith(prefix))
     files.map(deserializeFromFile[T])
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index 1b42121c8d..80506621f4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -204,7 +204,7 @@ private[master] class Master(
     self ! RevokedLeadership
   }
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case ElectedLeader => {
       val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData()
       state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala
index 1583bf1f60..351db8fab2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala
@@ -51,20 +51,27 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial
  */
 private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization)
   extends StandaloneRecoveryModeFactory(conf, serializer) with Logging {
+
   val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
 
-  def createPersistenceEngine() = {
+  def createPersistenceEngine(): PersistenceEngine = {
     logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
     new FileSystemPersistenceEngine(RECOVERY_DIR, serializer)
   }
 
-  def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master)
+  def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = {
+    new MonarchyLeaderAgent(master)
+  }
 }
 
 private[master] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization)
   extends StandaloneRecoveryModeFactory(conf, serializer) {
-  def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer)
 
-  def createLeaderElectionAgent(master: LeaderElectable) =
+  def createPersistenceEngine(): PersistenceEngine = {
+    new ZooKeeperPersistenceEngine(conf, serializer)
+  }
+
+  def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = {
     new ZooKeeperLeaderElectionAgent(master, conf)
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index e94aae93e4..9b3d48c6ed 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -104,7 +104,7 @@ private[spark] class WorkerInfo(
     "http://" + this.publicAddress + ":" + this.webUiPort
   }
 
-  def setState(state: WorkerState.Value) = {
+  def setState(state: WorkerState.Value): Unit = {
     this.state = state
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 1ac6677ad2..a285783f72 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -46,7 +46,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializat
     zk.delete().forPath(WORKING_DIR + "/" + name)
   }
 
-  override def read[T: ClassTag](prefix: String) = {
+  override def read[T: ClassTag](prefix: String): Seq[T] = {
     val file = zk.getChildren.forPath(WORKING_DIR).filter(_.startsWith(prefix))
     file.map(deserializeFromFile[T]).flatten
   }
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index dee2e4a447..46509e39c0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -95,7 +95,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
 
     // For now we only show driver information if the user has submitted drivers to the cluster.
     // This is until we integrate the notion of drivers and applications in the UI.
-    def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
+    def hasDrivers: Boolean = activeDrivers.length > 0 || completedDrivers.length > 0
 
     val content =
         <div class="row-fluid">
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
index 27a9eabb1e..e0948e16ef 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala
@@ -56,8 +56,14 @@ private[deploy] class DriverRunner(
   private var finalExitCode: Option[Int] = None
 
   // Decoupled for testing
-  def setClock(_clock: Clock) = clock = _clock
-  def setSleeper(_sleeper: Sleeper) = sleeper = _sleeper
+  def setClock(_clock: Clock): Unit = {
+    clock = _clock
+  }
+
+  def setSleeper(_sleeper: Sleeper): Unit = {
+    sleeper = _sleeper
+  }
+
   private var clock: Clock = new SystemClock()
   private var sleeper = new Sleeper {
     def sleep(seconds: Int): Unit = (0 until seconds).takeWhile(f => {Thread.sleep(1000); !killed})
@@ -155,7 +161,7 @@ private[deploy] class DriverRunner(
 
   private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) {
     builder.directory(baseDir)
-    def initialize(process: Process) = {
+    def initialize(process: Process): Unit = {
       // Redirect stdout and stderr to files
       val stdout = new File(baseDir, "stdout")
       CommandUtils.redirectStream(process.getInputStream, stdout)
@@ -169,8 +175,8 @@ private[deploy] class DriverRunner(
     runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise)
   }
 
-  def runCommandWithRetry(command: ProcessBuilderLike, initialize: Process => Unit,
-    supervise: Boolean) {
+  def runCommandWithRetry(
+      command: ProcessBuilderLike, initialize: Process => Unit, supervise: Boolean): Unit = {
     // Time to wait between submission retries.
     var waitSeconds = 1
     // A run of this many seconds resets the exponential back-off.
@@ -216,8 +222,8 @@ private[deploy] trait ProcessBuilderLike {
 }
 
 private[deploy] object ProcessBuilderLike {
-  def apply(processBuilder: ProcessBuilder) = new ProcessBuilderLike {
-    def start() = processBuilder.start()
-    def command = processBuilder.command()
+  def apply(processBuilder: ProcessBuilder): ProcessBuilderLike = new ProcessBuilderLike {
+    override def start(): Process = processBuilder.start()
+    override def command: Seq[String] = processBuilder.command()
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index c1b0a295f9..c4c24a7866 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -275,7 +275,7 @@ private[worker] class Worker(
     }
   }
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case RegisteredWorker(masterUrl, masterWebUiUrl) =>
       logInfo("Successfully registered with master " + masterUrl)
       registered = true
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
index 09d866fb0c..e0790274d7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -50,7 +50,7 @@ private[spark] class WorkerWatcher(workerUrl: String)
 
   private def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1)
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) =>
       logInfo(s"Successfully connected to $workerUrl")
 
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index dd19e4947d..b5205d4e99 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -62,7 +62,7 @@ private[spark] class CoarseGrainedExecutorBackend(
       .map(e => (e._1.substring(prefix.length).toLowerCase, e._2))
   }
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case RegisteredExecutor =>
       logInfo("Successfully registered with driver")
       val (hostname, _) = Utils.parseHostPort(hostPort)
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala
index 41925f7e97..3e47d13f75 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala
@@ -33,7 +33,7 @@ private[spark] case object TriggerThreadDump
 private[spark]
 class ExecutorActor(executorId: String) extends Actor with ActorLogReceive with Logging {
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case TriggerThreadDump =>
       sender ! Utils.getThreadDump()
   }
diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index 07b152651d..06152f16ae 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -17,13 +17,10 @@
 
 package org.apache.spark.executor
 
-import java.util.concurrent.atomic.AtomicLong
-
-import org.apache.spark.executor.DataReadMethod.DataReadMethod
-
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.executor.DataReadMethod.DataReadMethod
 import org.apache.spark.storage.{BlockId, BlockStatus}
 
 /**
@@ -44,14 +41,14 @@ class TaskMetrics extends Serializable {
    * Host's name the task runs on
    */
   private var _hostname: String = _
-  def hostname = _hostname
+  def hostname: String = _hostname
   private[spark] def setHostname(value: String) = _hostname = value
   
   /**
    * Time taken on the executor to deserialize this task
    */
   private var _executorDeserializeTime: Long = _
-  def executorDeserializeTime = _executorDeserializeTime
+  def executorDeserializeTime: Long = _executorDeserializeTime
   private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value
   
   
@@ -59,14 +56,14 @@ class TaskMetrics extends Serializable {
    * Time the executor spends actually running the task (including fetching shuffle data)
    */
   private var _executorRunTime: Long = _
-  def executorRunTime = _executorRunTime
+  def executorRunTime: Long = _executorRunTime
   private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value
   
   /**
    * The number of bytes this task transmitted back to the driver as the TaskResult
    */
   private var _resultSize: Long = _
-  def resultSize = _resultSize
+  def resultSize: Long = _resultSize
   private[spark] def setResultSize(value: Long) = _resultSize = value
 
 
@@ -74,31 +71,31 @@ class TaskMetrics extends Serializable {
    * Amount of time the JVM spent in garbage collection while executing this task
    */
   private var _jvmGCTime: Long = _
-  def jvmGCTime = _jvmGCTime
+  def jvmGCTime: Long = _jvmGCTime
   private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value
 
   /**
    * Amount of time spent serializing the task result
    */
   private var _resultSerializationTime: Long = _
-  def resultSerializationTime = _resultSerializationTime
+  def resultSerializationTime: Long = _resultSerializationTime
   private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value
 
   /**
    * The number of in-memory bytes spilled by this task
    */
   private var _memoryBytesSpilled: Long = _
-  def memoryBytesSpilled = _memoryBytesSpilled
-  private[spark] def incMemoryBytesSpilled(value: Long) = _memoryBytesSpilled += value
-  private[spark] def decMemoryBytesSpilled(value: Long) = _memoryBytesSpilled -= value
+  def memoryBytesSpilled: Long = _memoryBytesSpilled
+  private[spark] def incMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled += value
+  private[spark] def decMemoryBytesSpilled(value: Long): Unit = _memoryBytesSpilled -= value
 
   /**
    * The number of on-disk bytes spilled by this task
    */
   private var _diskBytesSpilled: Long = _
-  def diskBytesSpilled = _diskBytesSpilled
-  def incDiskBytesSpilled(value: Long) = _diskBytesSpilled += value
-  def decDiskBytesSpilled(value: Long) = _diskBytesSpilled -= value
+  def diskBytesSpilled: Long = _diskBytesSpilled
+  def incDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled += value
+  def decDiskBytesSpilled(value: Long): Unit = _diskBytesSpilled -= value
 
   /**
    * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read
@@ -106,7 +103,7 @@ class TaskMetrics extends Serializable {
    */
   private var _inputMetrics: Option[InputMetrics] = None
 
-  def inputMetrics = _inputMetrics
+  def inputMetrics: Option[InputMetrics] = _inputMetrics
 
   /**
    * This should only be used when recreating TaskMetrics, not when updating input metrics in
@@ -128,7 +125,7 @@ class TaskMetrics extends Serializable {
    */
   private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None
 
-  def shuffleReadMetrics = _shuffleReadMetrics
+  def shuffleReadMetrics: Option[ShuffleReadMetrics] = _shuffleReadMetrics
 
   /**
    * This should only be used when recreating TaskMetrics, not when updating read metrics in
@@ -177,17 +174,18 @@ class TaskMetrics extends Serializable {
    * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed,
    * we can store all the different inputMetrics (one per readMethod).
    */
-  private[spark] def getInputMetricsForReadMethod(
-      readMethod: DataReadMethod): InputMetrics = synchronized {
-    _inputMetrics match {
-      case None =>
-        val metrics = new InputMetrics(readMethod)
-        _inputMetrics = Some(metrics)
-        metrics
-      case Some(metrics @ InputMetrics(method)) if method == readMethod =>
-        metrics
-      case Some(InputMetrics(method)) =>
-       new InputMetrics(readMethod)
+  private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): InputMetrics = {
+    synchronized {
+      _inputMetrics match {
+        case None =>
+          val metrics = new InputMetrics(readMethod)
+          _inputMetrics = Some(metrics)
+          metrics
+        case Some(metrics @ InputMetrics(method)) if method == readMethod =>
+          metrics
+        case Some(InputMetrics(method)) =>
+          new InputMetrics(readMethod)
+      }
     }
   }
 
@@ -256,14 +254,14 @@ case class InputMetrics(readMethod: DataReadMethod.Value) {
    */
   private var _bytesRead: Long = _
   def bytesRead: Long = _bytesRead
-  def incBytesRead(bytes: Long) = _bytesRead += bytes
+  def incBytesRead(bytes: Long): Unit = _bytesRead += bytes
 
   /**
    * Total records read.
    */
   private var _recordsRead: Long = _
   def recordsRead: Long = _recordsRead
-  def incRecordsRead(records: Long) =  _recordsRead += records
+  def incRecordsRead(records: Long): Unit =  _recordsRead += records
 
   /**
    * Invoke the bytesReadCallback and mutate bytesRead.
@@ -293,15 +291,15 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) {
    * Total bytes written
    */
   private var _bytesWritten: Long = _
-  def bytesWritten = _bytesWritten
-  private[spark] def setBytesWritten(value : Long) = _bytesWritten = value
+  def bytesWritten: Long = _bytesWritten
+  private[spark] def setBytesWritten(value : Long): Unit = _bytesWritten = value
 
   /**
    * Total records written
    */
   private var _recordsWritten: Long = 0L
-  def recordsWritten = _recordsWritten
-  private[spark] def setRecordsWritten(value: Long) = _recordsWritten = value
+  def recordsWritten: Long = _recordsWritten
+  private[spark] def setRecordsWritten(value: Long): Unit = _recordsWritten = value
 }
 
 /**
@@ -314,7 +312,7 @@ class ShuffleReadMetrics extends Serializable {
    * Number of remote blocks fetched in this shuffle by this task
    */
   private var _remoteBlocksFetched: Int = _
-  def remoteBlocksFetched = _remoteBlocksFetched
+  def remoteBlocksFetched: Int = _remoteBlocksFetched
   private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value
   private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value
   
@@ -322,7 +320,7 @@ class ShuffleReadMetrics extends Serializable {
    * Number of local blocks fetched in this shuffle by this task
    */
   private var _localBlocksFetched: Int = _
-  def localBlocksFetched = _localBlocksFetched
+  def localBlocksFetched: Int = _localBlocksFetched
   private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value
   private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value
 
@@ -332,7 +330,7 @@ class ShuffleReadMetrics extends Serializable {
    * still not finished processing block A, it is not considered to be blocking on block B.
    */
   private var _fetchWaitTime: Long = _
-  def fetchWaitTime = _fetchWaitTime
+  def fetchWaitTime: Long = _fetchWaitTime
   private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value
   private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value
   
@@ -340,7 +338,7 @@ class ShuffleReadMetrics extends Serializable {
    * Total number of remote bytes read from the shuffle by this task
    */
   private var _remoteBytesRead: Long = _
-  def remoteBytesRead = _remoteBytesRead
+  def remoteBytesRead: Long = _remoteBytesRead
   private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value
   private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value
 
@@ -348,24 +346,24 @@ class ShuffleReadMetrics extends Serializable {
    * Shuffle data that was read from the local disk (as opposed to from a remote executor).
    */
   private var _localBytesRead: Long = _
-  def localBytesRead = _localBytesRead
+  def localBytesRead: Long = _localBytesRead
   private[spark] def incLocalBytesRead(value: Long) = _localBytesRead += value
 
   /**
    * Total bytes fetched in the shuffle by this task (both remote and local).
    */
-  def totalBytesRead = _remoteBytesRead + _localBytesRead
+  def totalBytesRead: Long = _remoteBytesRead + _localBytesRead
 
   /**
    * Number of blocks fetched in this shuffle by this task (remote or local)
    */
-  def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched
+  def totalBlocksFetched: Int = _remoteBlocksFetched + _localBlocksFetched
 
   /**
    * Total number of records read from the shuffle by this task
    */
   private var _recordsRead: Long = _
-  def recordsRead = _recordsRead
+  def recordsRead: Long = _recordsRead
   private[spark] def incRecordsRead(value: Long) = _recordsRead += value
   private[spark] def decRecordsRead(value: Long) = _recordsRead -= value
 }
@@ -380,7 +378,7 @@ class ShuffleWriteMetrics extends Serializable {
    * Number of bytes written for the shuffle by this task
    */
   @volatile private var _shuffleBytesWritten: Long = _
-  def shuffleBytesWritten = _shuffleBytesWritten
+  def shuffleBytesWritten: Long = _shuffleBytesWritten
   private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value
   private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value
   
@@ -388,7 +386,7 @@ class ShuffleWriteMetrics extends Serializable {
    * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds
    */
   @volatile private var _shuffleWriteTime: Long = _
-  def shuffleWriteTime= _shuffleWriteTime
+  def shuffleWriteTime: Long = _shuffleWriteTime
   private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value
   private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value
   
@@ -396,7 +394,7 @@ class ShuffleWriteMetrics extends Serializable {
    * Total number of records written to the shuffle by this task
    */
   @volatile private var _shuffleRecordsWritten: Long = _
-  def shuffleRecordsWritten = _shuffleRecordsWritten
+  def shuffleRecordsWritten: Long = _shuffleRecordsWritten
   private[spark] def incShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten += value
   private[spark] def decShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten -= value
   private[spark] def setShuffleRecordsWritten(value: Long) = _shuffleRecordsWritten = value
diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
index 593a62b3e3..6cda7772f7 100644
--- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
+++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
@@ -73,16 +73,16 @@ private[spark] abstract class StreamBasedRecordReader[T](
   private var key = ""
   private var value: T = null.asInstanceOf[T]
 
-  override def initialize(split: InputSplit, context: TaskAttemptContext) = {}
-  override def close() = {}
+  override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {}
+  override def close(): Unit = {}
 
-  override def getProgress = if (processed) 1.0f else 0.0f
+  override def getProgress: Float = if (processed) 1.0f else 0.0f
 
-  override def getCurrentKey = key
+  override def getCurrentKey: String = key
 
-  override def getCurrentValue = value
+  override def getCurrentValue: T = value
 
-  override def nextKeyValue = {
+  override def nextKeyValue: Boolean = {
     if (!processed) {
       val fileIn = new PortableDataStream(split, context, index)
       value = parseStream(fileIn)
@@ -119,7 +119,8 @@ private[spark] class StreamRecordReader(
  * The format for the PortableDataStream files
  */
 private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] {
-  override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = {
+  override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext)
+    : CombineFileRecordReader[String, PortableDataStream] = {
     new CombineFileRecordReader[String, PortableDataStream](
       split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader])
   }
@@ -204,7 +205,7 @@ class PortableDataStream(
   /**
    * Close the file (if it is currently open)
    */
-  def close() = {
+  def close(): Unit = {
     if (isOpen) {
       try {
         fileIn.close()
diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
index 21b782edd2..87c2aa4810 100644
--- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
+++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala
@@ -52,7 +52,7 @@ trait SparkHadoopMapRedUtil {
       jobId: Int,
       isMap: Boolean,
       taskId: Int,
-      attemptId: Int) = {
+      attemptId: Int): TaskAttemptID = {
     new TaskAttemptID(jtIdentifier, jobId, isMap, taskId, attemptId)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
index 3340673f91..cfd20392d1 100644
--- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
+++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala
@@ -45,7 +45,7 @@ trait SparkHadoopMapReduceUtil {
       jobId: Int,
       isMap: Boolean,
       taskId: Int,
-      attemptId: Int) = {
+      attemptId: Int): TaskAttemptID = {
     val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID")
     try {
       // First, attempt to use the old-style constructor that takes a boolean isMap
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index 345db36630..9150ad3571 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit
 import scala.collection.mutable
 
 import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry}
+import org.eclipse.jetty.servlet.ServletContextHandler
 
 import org.apache.spark.{Logging, SecurityManager, SparkConf}
 import org.apache.spark.metrics.sink.{MetricsServlet, Sink}
@@ -84,7 +85,7 @@ private[spark] class MetricsSystem private (
   /**
    * Get any UI handlers used by this metrics system; can only be called after start().
    */
-  def getServletHandlers = {
+  def getServletHandlers: Array[ServletContextHandler] = {
     require(running, "Can only call getServletHandlers on a running MetricsSystem")
     metricsServlet.map(_.getHandlers).getOrElse(Array())
   }
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
index 2f65bc8b46..0c2e212a33 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
@@ -30,8 +30,12 @@ import org.eclipse.jetty.servlet.ServletContextHandler
 import org.apache.spark.SecurityManager
 import org.apache.spark.ui.JettyUtils._
 
-private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry,
-    securityMgr: SecurityManager) extends Sink {
+private[spark] class MetricsServlet(
+    val property: Properties,
+    val registry: MetricRegistry,
+    securityMgr: SecurityManager)
+  extends Sink {
+
   val SERVLET_KEY_PATH = "path"
   val SERVLET_KEY_SAMPLE = "sample"
 
@@ -45,10 +49,12 @@ private[spark] class MetricsServlet(val property: Properties, val registry: Metr
   val mapper = new ObjectMapper().registerModule(
     new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample))
 
-  def getHandlers = Array[ServletContextHandler](
-    createServletHandler(servletPath,
-      new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr)
-  )
+  def getHandlers: Array[ServletContextHandler] = {
+    Array[ServletContextHandler](
+      createServletHandler(servletPath,
+        new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr)
+    )
+  }
 
   def getMetricsSnapshot(request: HttpServletRequest): String = {
     mapper.writeValueAsString(registry)
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
index 0d83d8c425..9fad4e7dea 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.metrics.sink
 
 private[spark] trait Sink {
-  def start: Unit
-  def stop: Unit
+  def start(): Unit
+  def stop(): Unit
   def report(): Unit
 }
diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
index a1a2c00ed1..1ba25aa74a 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala
@@ -32,11 +32,11 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage])
 
   def this() = this(null.asInstanceOf[Seq[BlockMessage]])
 
-  def apply(i: Int) = blockMessages(i)
+  def apply(i: Int): BlockMessage = blockMessages(i)
 
-  def iterator = blockMessages.iterator
+  def iterator: Iterator[BlockMessage] = blockMessages.iterator
 
-  def length = blockMessages.length
+  def length: Int = blockMessages.length
 
   def set(bufferMessage: BufferMessage) {
     val startTime = System.currentTimeMillis
diff --git a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala
index 3b245c5c7a..9a9e22b0c2 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala
@@ -31,9 +31,9 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId:
   val initialSize = currentSize()
   var gotChunkForSendingOnce = false
 
-  def size = initialSize
+  def size: Int = initialSize
 
-  def currentSize() = {
+  def currentSize(): Int = {
     if (buffers == null || buffers.isEmpty) {
       0
     } else {
@@ -100,11 +100,11 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId:
     buffers.foreach(_.flip)
   }
 
-  def hasAckId() = (ackId != 0)
+  def hasAckId(): Boolean = ackId != 0
 
-  def isCompletelyReceived() = !buffers(0).hasRemaining
+  def isCompletelyReceived: Boolean = !buffers(0).hasRemaining
 
-  override def toString = {
+  override def toString: String = {
     if (hasAckId) {
       "BufferAckMessage(aid = " + ackId + ", id = " + id + ", size = " + size + ")"
     } else {
diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala
index c2d9578be7..04eb2bf9ba 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala
@@ -101,9 +101,11 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
     socketRemoteConnectionManagerId
   }
 
-  def key() = channel.keyFor(selector)
+  def key(): SelectionKey = channel.keyFor(selector)
 
-  def getRemoteAddress() = channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]
+  def getRemoteAddress(): InetSocketAddress = {
+    channel.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]
+  }
 
   // Returns whether we have to register for further reads or not.
   def read(): Boolean = {
@@ -280,7 +282,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector,
 
   /* channel.socket.setSendBufferSize(256 * 1024) */
 
-  override def getRemoteAddress() = address
+  override def getRemoteAddress(): InetSocketAddress = address
 
   val DEFAULT_INTEREST = SelectionKey.OP_READ
 
diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala
index 764dc5e550..b3b281ff46 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala
@@ -18,7 +18,9 @@
 package org.apache.spark.network.nio
 
 private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) {
-  override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId
+  override def toString: String = {
+    connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId
+  }
 }
 
 private[nio] object ConnectionId {
diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
index ee22c6656e..741fe3e1ea 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala
@@ -188,7 +188,7 @@ private[nio] class ConnectionManager(
   private val readRunnableStarted: HashSet[SelectionKey] = new HashSet[SelectionKey]()
 
   private val selectorThread = new Thread("connection-manager-thread") {
-    override def run() = ConnectionManager.this.run()
+    override def run(): Unit = ConnectionManager.this.run()
   }
   selectorThread.setDaemon(true)
   // start this thread last, since it invokes run(), which accesses members above
diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala
index cbb37ec5ce..1cd13d887c 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala
@@ -26,7 +26,7 @@ private[nio] case class ConnectionManagerId(host: String, port: Int) {
   Utils.checkHost(host)
   assert (port > 0)
 
-  def toSocketAddress() = new InetSocketAddress(host, port)
+  def toSocketAddress(): InetSocketAddress = new InetSocketAddress(host, port)
 }
 
 
diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala
index fb4a979b82..85d2fe2bf9 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala
@@ -42,7 +42,9 @@ private[nio] abstract class Message(val typ: Long, val id: Int) {
 
   def timeTaken(): String = (finishTime - startTime).toString + " ms"
 
-  override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")"
+  override def toString: String = {
+    this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")"
+  }
 }
 
 
@@ -51,7 +53,7 @@ private[nio] object Message {
 
   var lastId = 1
 
-  def getNewId() = synchronized {
+  def getNewId(): Int = synchronized {
     lastId += 1
     if (lastId == 0) {
       lastId += 1
diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala
index 278c5ac356..a4568e849f 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala
@@ -24,9 +24,9 @@ import scala.collection.mutable.ArrayBuffer
 private[nio]
 class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
 
-  val size = if (buffer == null) 0 else buffer.remaining
+  val size: Int = if (buffer == null) 0 else buffer.remaining
 
-  lazy val buffers = {
+  lazy val buffers: ArrayBuffer[ByteBuffer] = {
     val ab = new ArrayBuffer[ByteBuffer]()
     ab += header.buffer
     if (buffer != null) {
@@ -35,7 +35,7 @@ class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
     ab
   }
 
-  override def toString = {
+  override def toString: String = {
     "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")"
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala
index 6e20f291c5..7b3da4bb9d 100644
--- a/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala
+++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala
@@ -50,8 +50,10 @@ private[nio] class MessageChunkHeader(
       flip.asInstanceOf[ByteBuffer]
   }
 
-  override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ +
+  override def toString: String = {
+    "" + this.getClass.getSimpleName + ":" + id + " of type " + typ +
       " and sizes " + totalSize + " / " + chunkSize + " bytes, securityNeg: " + securityNeg
+  }
 
 }
 
diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala
index cadd0c7ed1..53c4b32c95 100644
--- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala
+++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala
@@ -99,7 +99,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) {
           case None => "(partial: " + initialValue + ")"
         }
       }
-      def getFinalValueInternal() = PartialResult.this.getFinalValueInternal().map(f)
+      def getFinalValueInternal(): Option[T] = PartialResult.this.getFinalValueInternal().map(f)
     }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 1cbd684224..9059eb13bb 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -70,7 +70,7 @@ class CartesianRDD[T: ClassTag, U: ClassTag](
     (rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)).distinct
   }
 
-  override def compute(split: Partition, context: TaskContext) = {
+  override def compute(split: Partition, context: TaskContext): Iterator[(T, U)] = {
     val currSplit = split.asInstanceOf[CartesianPartition]
     for (x <- rdd1.iterator(currSplit.s1, context);
          y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index b073eba8a1..5117ccfabf 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -186,7 +186,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack:
     override val isEmpty = !it.hasNext
 
     // initializes/resets to start iterating from the beginning
-    def resetIterator() = {
+    def resetIterator(): Iterator[(String, Partition)] = {
       val iterators = (0 to 2).map( x =>
         prev.partitions.iterator.flatMap(p => {
           if (currPrefLocs(p).size > x) Some((currPrefLocs(p)(x), p)) else None
@@ -196,10 +196,10 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack:
     }
 
     // hasNext() is false iff there are no preferredLocations for any of the partitions of the RDD
-    def hasNext(): Boolean = { !isEmpty }
+    override def hasNext: Boolean = { !isEmpty }
 
     // return the next preferredLocation of some partition of the RDD
-    def next(): (String, Partition) = {
+    override def next(): (String, Partition) = {
       if (it.hasNext) {
         it.next()
       } else {
@@ -237,7 +237,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack:
     val rotIt = new LocationIterator(prev)
 
     // deal with empty case, just create targetLen partition groups with no preferred location
-    if (!rotIt.hasNext()) {
+    if (!rotIt.hasNext) {
       (1 to targetLen).foreach(x => groupArr += PartitionGroup())
       return
     }
@@ -343,7 +343,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack:
 
 private case class PartitionGroup(prefLoc: Option[String] = None) {
   var arr = mutable.ArrayBuffer[Partition]()
-  def size = arr.size
+  def size: Int = arr.size
 }
 
 private object PartitionGroup {
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 486e86ce1b..f77abac42b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -215,8 +215,7 @@ class HadoopRDD[K, V](
       logInfo("Input split: " + split.inputSplit)
       val jobConf = getJobConf()
 
-      val inputMetrics = context.taskMetrics
-        .getInputMetricsForReadMethod(DataReadMethod.Hadoop)
+      val inputMetrics = context.taskMetrics.getInputMetricsForReadMethod(DataReadMethod.Hadoop)
 
       // Find a function that will return the FileSystem bytes read by this thread. Do this before
       // creating RecordReader, because RecordReader's constructor might read some bytes
@@ -240,7 +239,7 @@ class HadoopRDD[K, V](
       val key: K = reader.createKey()
       val value: V = reader.createValue()
 
-      override def getNext() = {
+      override def getNext(): (K, V) = {
         try {
           finished = !reader.next(key, value)
         } catch {
@@ -337,11 +336,11 @@ private[spark] object HadoopRDD extends Logging {
    * The three methods below are helpers for accessing the local map, a property of the SparkEnv of
    * the local process.
    */
-  def getCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.get(key)
+  def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key)
 
-  def containsCachedMetadata(key: String) = SparkEnv.get.hadoopJobMetadata.containsKey(key)
+  def containsCachedMetadata(key: String): Boolean = SparkEnv.get.hadoopJobMetadata.containsKey(key)
 
-  def putCachedMetadata(key: String, value: Any) =
+  private def putCachedMetadata(key: String, value: Any): Unit =
     SparkEnv.get.hadoopJobMetadata.put(key, value)
 
   /** Add Hadoop configuration specific to a single partition and attempt. */
@@ -371,7 +370,7 @@ private[spark] object HadoopRDD extends Logging {
 
     override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-    override def compute(split: Partition, context: TaskContext) = {
+    override def compute(split: Partition, context: TaskContext): Iterator[U] = {
       val partition = split.asInstanceOf[HadoopPartition]
       val inputSplit = partition.inputSplit.value
       f(inputSplit, firstParent[T].iterator(split, context))
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index e2267861e7..0c28f045e4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.rdd
 
-import java.sql.{Connection, ResultSet}
+import java.sql.{PreparedStatement, Connection, ResultSet}
 
 import scala.reflect.ClassTag
 
@@ -28,8 +28,9 @@ import org.apache.spark.util.NextIterator
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 
 private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition {
-  override def index = idx
+  override def index: Int = idx
 }
+
 // TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private
 /**
  * An RDD that executes an SQL query on a JDBC connection and reads results.
@@ -70,7 +71,8 @@ class JdbcRDD[T: ClassTag](
     }).toArray
   }
 
-  override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] {
+  override def compute(thePart: Partition, context: TaskContext): Iterator[T] = new NextIterator[T]
+  {
     context.addTaskCompletionListener{ context => closeIfNeeded() }
     val part = thePart.asInstanceOf[JdbcPartition]
     val conn = getConnection()
@@ -88,7 +90,7 @@ class JdbcRDD[T: ClassTag](
     stmt.setLong(2, part.upper)
     val rs = stmt.executeQuery()
 
-    override def getNext: T = {
+    override def getNext(): T = {
       if (rs.next()) {
         mapRow(rs)
       } else {
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index 4883fb8288..a838aac6e8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -31,6 +31,6 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
 
   override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-  override def compute(split: Partition, context: TaskContext) =
+  override def compute(split: Partition, context: TaskContext): Iterator[U] =
     f(context, split.index, firstParent[T].iterator(split, context))
 }
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 7fb94840df..2ab967f4bb 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -238,7 +238,7 @@ private[spark] object NewHadoopRDD {
 
     override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-    override def compute(split: Partition, context: TaskContext) = {
+    override def compute(split: Partition, context: TaskContext): Iterator[U] = {
       val partition = split.asInstanceOf[NewHadoopPartition]
       val inputSplit = partition.serializableHadoopSplit.value
       f(inputSplit, firstParent[T].iterator(split, context))
diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index f12d0cffab..e2394e28f8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -98,7 +98,7 @@ private[spark] class ParallelCollectionRDD[T: ClassTag](
     slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray
   }
 
-  override def compute(s: Partition, context: TaskContext) = {
+  override def compute(s: Partition, context: TaskContext): Iterator[T] = {
     new InterruptibleIterator(context, s.asInstanceOf[ParallelCollectionPartition[T]].iterator)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index f781a8d776..a00f4c1cdf 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -40,7 +40,7 @@ private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterF
     .filter(s => partitionFilterFunc(s.index)).zipWithIndex
     .map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition }
 
-  override def getParents(partitionId: Int) = {
+  override def getParents(partitionId: Int): List[Int] = {
     List(partitions(partitionId).asInstanceOf[PartitionPruningRDDPartition].parentSplit.index)
   }
 }
@@ -59,8 +59,10 @@ class PartitionPruningRDD[T: ClassTag](
     @transient partitionFilterFunc: Int => Boolean)
   extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) {
 
-  override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(
-    split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context)
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    firstParent[T].iterator(
+      split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context)
+  }
 
   override protected def getPartitions: Array[Partition] =
     getDependencies.head.asInstanceOf[PruneDependency[T]].partitions
@@ -74,7 +76,7 @@ object PartitionPruningRDD {
    * Create a PartitionPruningRDD. This function can be used to create the PartitionPruningRDD
    * when its type T is not known at compile time.
    */
-  def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = {
+  def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean): PartitionPruningRDD[T] = {
     new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index ed79032893..dc60d48927 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -149,10 +149,10 @@ private[spark] class PipedRDD[T: ClassTag](
     }.start()
 
     // Return an iterator that read lines from the process's stdout
-    val lines = Source.fromInputStream(proc.getInputStream).getLines
+    val lines = Source.fromInputStream(proc.getInputStream).getLines()
     new Iterator[String] {
-      def next() = lines.next()
-      def hasNext = {
+      def next(): String = lines.next()
+      def hasNext: Boolean = {
         if (lines.hasNext) {
           true
         } else {
@@ -162,7 +162,7 @@ private[spark] class PipedRDD[T: ClassTag](
           }
 
           // cleanup task working directory if used
-          if (workInTaskDirectory == true) {
+          if (workInTaskDirectory) {
             scala.util.control.Exception.ignoring(classOf[IOException]) {
               Utils.deleteRecursively(new File(taskDirectory))
             }
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index a4c74ed03e..ddbfd5624e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -186,7 +186,7 @@ abstract class RDD[T: ClassTag](
   }
 
   /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
-  def getStorageLevel = storageLevel
+  def getStorageLevel: StorageLevel = storageLevel
 
   // Our dependencies and partitions will be gotten by calling subclass's methods below, and will
   // be overwritten when we're checkpointed
@@ -746,13 +746,13 @@ abstract class RDD[T: ClassTag](
   def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = {
     zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) =>
       new Iterator[(T, U)] {
-        def hasNext = (thisIter.hasNext, otherIter.hasNext) match {
+        def hasNext: Boolean = (thisIter.hasNext, otherIter.hasNext) match {
           case (true, true) => true
           case (false, false) => false
           case _ => throw new SparkException("Can only zip RDDs with " +
             "same number of elements in each partition")
         }
-        def next = (thisIter.next, otherIter.next)
+        def next(): (T, U) = (thisIter.next(), otherIter.next())
       }
     }
   }
@@ -868,8 +868,8 @@ abstract class RDD[T: ClassTag](
       // Our partitioner knows how to handle T (which, since we have a partitioner, is
       // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples
       val p2 = new Partitioner() {
-        override def numPartitions = p.numPartitions
-        override def getPartition(k: Any) = p.getPartition(k.asInstanceOf[(Any, _)]._1)
+        override def numPartitions: Int = p.numPartitions
+        override def getPartition(k: Any): Int = p.getPartition(k.asInstanceOf[(Any, _)]._1)
       }
       // Unfortunately, since we're making a new p2, we'll get ShuffleDependencies
       // anyway, and when calling .keys, will not have a partitioner set, even though
@@ -1394,7 +1394,7 @@ abstract class RDD[T: ClassTag](
   }
 
   /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */
-  def context = sc
+  def context: SparkContext = sc
 
   /**
    * Private API for changing an RDD's ClassTag.
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index d9fe684725..2dc47f9593 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -17,14 +17,12 @@
 
 package org.apache.spark.rdd
 
-import scala.reflect.ClassTag
-
 import org.apache.spark._
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.serializer.Serializer
 
 private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
-  override val index = idx
+  override val index: Int = idx
   override def hashCode(): Int = idx
 }
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index ed24ea22a6..c27f435eb9 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -105,7 +105,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
         seq
       }
     }
-    def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match {
+    def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit): Unit = dep match {
       case NarrowCoGroupSplitDep(rdd, _, itsSplit) =>
         rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op)
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
index aece683ff3..4239e7e22a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
@@ -44,7 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](
 
   var parentPartition: Partition = rdd.partitions(parentRddPartitionIndex)
 
-  def preferredLocations() = rdd.preferredLocations(parentPartition)
+  def preferredLocations(): Seq[String] = rdd.preferredLocations(parentPartition)
 
   override val index: Int = idx
 
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
index 95b2dd954e..d0be304762 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
@@ -32,7 +32,7 @@ private[spark] class ZippedPartitionsPartition(
 
   override val index: Int = idx
   var partitionValues = rdds.map(rdd => rdd.partitions(idx))
-  def partitions = partitionValues
+  def partitions: Seq[Partition] = partitionValues
 
   @throws(classOf[IOException])
   private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala
index fa83372bb4..e0edd7d4ae 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala
@@ -39,8 +39,11 @@ class AccumulableInfo (
 }
 
 object AccumulableInfo {
-  def apply(id: Long, name: String, update: Option[String], value: String) =
+  def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = {
     new AccumulableInfo(id, name, update, value)
+  }
 
-  def apply(id: Long, name: String, value: String) = new AccumulableInfo(id, name, None, value)
+  def apply(id: Long, name: String, value: String): AccumulableInfo = {
+    new AccumulableInfo(id, name, None, value)
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index 8feac6cb6b..b405bd3338 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -946,7 +946,7 @@ class DAGScheduler(
 
     val stage = stageIdToStage(task.stageId)
 
-    def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = {
+    def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None): Unit = {
       val serviceTime = stage.latestInfo.submissionTime match {
         case Some(t) => "%.03f".format((clock.getTimeMillis() - t) / 1000.0)
         case _ => "Unknown"
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 34fa6d27c3..c0d889360a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -149,47 +149,60 @@ private[spark] class EventLoggingListener(
   }
 
   // Events that do not trigger a flush
-  override def onStageSubmitted(event: SparkListenerStageSubmitted) =
-    logEvent(event)
-  override def onTaskStart(event: SparkListenerTaskStart) =
-    logEvent(event)
-  override def onTaskGettingResult(event: SparkListenerTaskGettingResult) =
-    logEvent(event)
-  override def onTaskEnd(event: SparkListenerTaskEnd) =
-    logEvent(event)
-  override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) =
-    logEvent(event)
+  override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event)
+
+  override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event)
+
+  override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = logEvent(event)
+
+  override def onTaskEnd(event: SparkListenerTaskEnd): Unit = logEvent(event)
+
+  override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = logEvent(event)
 
   // Events that trigger a flush
-  override def onStageCompleted(event: SparkListenerStageCompleted) =
-    logEvent(event, flushLogger = true)
-  override def onJobStart(event: SparkListenerJobStart) =
-    logEvent(event, flushLogger = true)
-  override def onJobEnd(event: SparkListenerJobEnd) =
+  override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
     logEvent(event, flushLogger = true)
-  override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) =
+  }
+
+  override def onJobStart(event: SparkListenerJobStart): Unit = logEvent(event, flushLogger = true)
+
+  override def onJobEnd(event: SparkListenerJobEnd): Unit = logEvent(event, flushLogger = true)
+
+  override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = {
     logEvent(event, flushLogger = true)
-  override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved) =
+  }
+
+  override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = {
     logEvent(event, flushLogger = true)
-  override def onUnpersistRDD(event: SparkListenerUnpersistRDD) =
+  }
+
+  override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = {
     logEvent(event, flushLogger = true)
-  override def onApplicationStart(event: SparkListenerApplicationStart) =
+  }
+
+  override def onApplicationStart(event: SparkListenerApplicationStart): Unit = {
     logEvent(event, flushLogger = true)
-  override def onApplicationEnd(event: SparkListenerApplicationEnd) =
+  }
+
+  override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = {
     logEvent(event, flushLogger = true)
-  override def onExecutorAdded(event: SparkListenerExecutorAdded) =
+  }
+  override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = {
     logEvent(event, flushLogger = true)
-  override def onExecutorRemoved(event: SparkListenerExecutorRemoved) =
+  }
+
+  override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = {
     logEvent(event, flushLogger = true)
+  }
 
   // No-op because logging every update would be overkill
-  override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { }
+  override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { }
 
   /**
    * Stop logging events. The event log file will be renamed so that it loses the
    * ".inprogress" suffix.
    */
-  def stop() = {
+  def stop(): Unit = {
     writer.foreach(_.close())
 
     val target = new Path(logPath)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index 8aa528ac57..e55b76c36c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -57,7 +57,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener
   private val stageIdToJobId = new HashMap[Int, Int]
   private val jobIdToStageIds = new HashMap[Int, Seq[Int]]
   private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
-    override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
+    override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
   }
 
   createLogDir()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
index 29879b374b..382b09422a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
@@ -34,7 +34,7 @@ private[spark] class JobWaiter[T](
   @volatile
   private var _jobFinished = totalTasks == 0
 
-  def jobFinished = _jobFinished
+  def jobFinished: Boolean = _jobFinished
 
   // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero
   // partition RDDs), we set the jobResult directly to JobSucceeded.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
index 759df023a6..a3caa9f000 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
@@ -160,7 +160,7 @@ private[spark] object OutputCommitCoordinator {
   class OutputCommitCoordinatorActor(outputCommitCoordinator: OutputCommitCoordinator)
     extends Actor with ActorLogReceive with Logging {
 
-    override def receiveWithLogging = {
+    override def receiveWithLogging: PartialFunction[Any, Unit] = {
       case AskPermissionToCommitOutput(stage, partition, taskAttempt) =>
         sender ! outputCommitCoordinator.handleAskPermissionToCommit(stage, partition, taskAttempt)
       case StopCoordinator =>
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 4a9ff918af..e074ce6ebf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -64,5 +64,5 @@ private[spark] class ResultTask[T, U](
   // This is only callable on the driver side.
   override def preferredLocations: Seq[TaskLocation] = preferredLocs
 
-  override def toString = "ResultTask(" + stageId + ", " + partitionId + ")"
+  override def toString: String = "ResultTask(" + stageId + ", " + partitionId + ")"
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 79709089c0..fd0d484b45 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -47,7 +47,7 @@ private[spark] class ShuffleMapTask(
 
   /** A constructor used only in test suites. This does not require passing in an RDD. */
   def this(partitionId: Int) {
-    this(0, null, new Partition { override def index = 0 }, null)
+    this(0, null, new Partition { override def index: Int = 0 }, null)
   }
 
   @transient private val preferredLocs: Seq[TaskLocation] = {
@@ -83,5 +83,5 @@ private[spark] class ShuffleMapTask(
 
   override def preferredLocations: Seq[TaskLocation] = preferredLocs
 
-  override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partitionId)
+  override def toString: String = "ShuffleMapTask(%d, %d)".format(stageId, partitionId)
 }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 52720d48ca..b711ff209a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -300,7 +300,7 @@ private[spark] object StatsReportListener extends Logging {
   }
 
   def showDistribution(heading: String, dOpt: Option[Distribution], format:String) {
-    def f(d: Double) = format.format(d)
+    def f(d: Double): String = format.format(d)
     showDistribution(heading, dOpt, f _)
   }
 
@@ -346,7 +346,7 @@ private[spark] object StatsReportListener extends Logging {
   /**
    * Reformat a time interval in milliseconds to a prettier format for output
    */
-  def millisToString(ms: Long) = {
+  def millisToString(ms: Long): String = {
     val (size, units) =
       if (ms > hours) {
         (ms.toDouble / hours, "hours")
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index cc13f57a49..4cbc6e84a6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -133,7 +133,7 @@ private[spark] class Stage(
 
   def attemptId: Int = nextAttemptId
 
-  override def toString = "Stage " + id
+  override def toString: String = "Stage " + id
 
   override def hashCode(): Int = id
 
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
index 10c685f29d..da07ce2c6e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala
@@ -29,23 +29,22 @@ private[spark] sealed trait TaskLocation {
 /**
  * A location that includes both a host and an executor id on that host.
  */
-private [spark] case class ExecutorCacheTaskLocation(override val host: String,
-    val executorId: String) extends TaskLocation {
-}
+private [spark]
+case class ExecutorCacheTaskLocation(override val host: String, executorId: String)
+  extends TaskLocation
 
 /**
  * A location on a host.
  */
 private [spark] case class HostTaskLocation(override val host: String) extends TaskLocation {
-  override def toString = host
+  override def toString: String = host
 }
 
 /**
  * A location on a host that is cached by HDFS.
  */
-private [spark] case class HDFSCacheTaskLocation(override val host: String)
-    extends TaskLocation {
-  override def toString = TaskLocation.inMemoryLocationTag + host
+private [spark] case class HDFSCacheTaskLocation(override val host: String) extends TaskLocation {
+  override def toString: String = TaskLocation.inMemoryLocationTag + host
 }
 
 private[spark] object TaskLocation {
@@ -54,14 +53,16 @@ private[spark] object TaskLocation {
   // confusion.  See  RFC 952 and RFC 1123 for information about the format of hostnames.
   val inMemoryLocationTag = "hdfs_cache_"
 
-  def apply(host: String, executorId: String) = new ExecutorCacheTaskLocation(host, executorId)
+  def apply(host: String, executorId: String): TaskLocation = {
+    new ExecutorCacheTaskLocation(host, executorId)
+  }
 
   /**
    * Create a TaskLocation from a string returned by getPreferredLocations.
    * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the
    * location is cached.
    */
-  def apply(str: String) = {
+  def apply(str: String): TaskLocation = {
     val hstr = str.stripPrefix(inMemoryLocationTag)
     if (hstr.equals(str)) {
       new HostTaskLocation(str)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index f33fd4450b..076b36e86c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -373,17 +373,17 @@ private[spark] class TaskSchedulerImpl(
   }
 
   def handleSuccessfulTask(
-    taskSetManager: TaskSetManager,
-    tid: Long,
-    taskResult: DirectTaskResult[_]) = synchronized {
+      taskSetManager: TaskSetManager,
+      tid: Long,
+      taskResult: DirectTaskResult[_]): Unit = synchronized {
     taskSetManager.handleSuccessfulTask(tid, taskResult)
   }
 
   def handleFailedTask(
-    taskSetManager: TaskSetManager,
-    tid: Long,
-    taskState: TaskState,
-    reason: TaskEndReason) = synchronized {
+      taskSetManager: TaskSetManager,
+      tid: Long,
+      taskState: TaskState,
+      reason: TaskEndReason): Unit = synchronized {
     taskSetManager.handleFailedTask(tid, taskState, reason)
     if (!taskSetManager.isZombie && taskState != TaskState.KILLED) {
       // Need to revive offers again now that the task set manager state has been updated to
@@ -423,7 +423,7 @@ private[spark] class TaskSchedulerImpl(
     starvationTimer.cancel()
   }
 
-  override def defaultParallelism() = backend.defaultParallelism()
+  override def defaultParallelism(): Int = backend.defaultParallelism()
 
   // Check for speculatable tasks in all our active jobs.
   def checkSpeculatableTasks() {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 529237f0d3..d509881c74 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -20,6 +20,7 @@ package org.apache.spark.scheduler
 import java.io.NotSerializableException
 import java.nio.ByteBuffer
 import java.util.Arrays
+import java.util.concurrent.ConcurrentLinkedQueue
 
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
@@ -29,6 +30,7 @@ import scala.util.control.NonFatal
 
 import org.apache.spark._
 import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.SchedulingMode._
 import org.apache.spark.TaskState.TaskState
 import org.apache.spark.util.{Clock, SystemClock, Utils}
 
@@ -97,7 +99,8 @@ private[spark] class TaskSetManager(
   var calculatedTasks = 0
 
   val runningTasksSet = new HashSet[Long]
-  override def runningTasks = runningTasksSet.size
+
+  override def runningTasks: Int = runningTasksSet.size
 
   // True once no more tasks should be launched for this task set manager. TaskSetManagers enter
   // the zombie state once at least one attempt of each task has completed successfully, or if the
@@ -168,9 +171,9 @@ private[spark] class TaskSetManager(
   var currentLocalityIndex = 0    // Index of our current locality level in validLocalityLevels
   var lastLaunchTime = clock.getTimeMillis()  // Time we last launched a task at this level
 
-  override def schedulableQueue = null
+  override def schedulableQueue: ConcurrentLinkedQueue[Schedulable] = null
 
-  override def schedulingMode = SchedulingMode.NONE
+  override def schedulingMode: SchedulingMode = SchedulingMode.NONE
 
   var emittedTaskSizeWarning = false
 
@@ -585,7 +588,7 @@ private[spark] class TaskSetManager(
   /**
    * Marks the task as getting result and notifies the DAG Scheduler
    */
-  def handleTaskGettingResult(tid: Long) = {
+  def handleTaskGettingResult(tid: Long): Unit = {
     val info = taskInfos(tid)
     info.markGettingResult()
     sched.dagScheduler.taskGettingResult(info)
@@ -612,7 +615,7 @@ private[spark] class TaskSetManager(
   /**
    * Marks the task as successful and notifies the DAGScheduler that a task has ended.
    */
-  def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = {
+  def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
     val info = taskInfos(tid)
     val index = info.index
     info.markSuccessful()
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 87ebf31139..5d258d9da4 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -85,7 +85,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
       context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
     }
 
-    def receiveWithLogging = {
+    def receiveWithLogging: PartialFunction[Any, Unit] = {
       case RegisterExecutor(executorId, hostPort, cores, logUrls) =>
         Utils.checkHostPort(hostPort, "Host port expected " + hostPort)
         if (executorDataMap.contains(executorId)) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
index f14aaeea0a..5a38ad9f2b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -109,7 +109,7 @@ private[spark] abstract class YarnSchedulerBackend(
       context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
     }
 
-    override def receive = {
+    override def receive: PartialFunction[Any, Unit] = {
       case RegisterClusterManager =>
         logInfo(s"ApplicationMaster registered as $sender")
         amActor = Some(sender)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
index aa3ec0f8cf..8df4f3b554 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala
@@ -24,7 +24,7 @@ private[spark] object MemoryUtils {
   val OVERHEAD_FRACTION = 0.10
   val OVERHEAD_MINIMUM = 384
 
-  def calculateTotalMemory(sc: SparkContext) = {
+  def calculateTotalMemory(sc: SparkContext): Int = {
     sc.conf.getInt("spark.mesos.executor.memoryOverhead",
       math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory
   }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 06bb527522..b381436839 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -387,7 +387,7 @@ private[spark] class MesosSchedulerBackend(
   }
 
   // TODO: query Mesos for number of cores
-  override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8)
+  override def defaultParallelism(): Int = sc.conf.getInt("spark.default.parallelism", 8)
 
   override def applicationId(): String =
     Option(appId).getOrElse {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
index d95426d918..eb3f999b5b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -59,7 +59,7 @@ private[spark] class LocalActor(
   private val executor = new Executor(
     localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true)
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case ReviveOffers =>
       reviveOffers()
 
@@ -117,7 +117,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores:
     localActor ! ReviveOffers
   }
 
-  override def defaultParallelism() =
+  override def defaultParallelism(): Int =
     scheduler.conf.getInt("spark.default.parallelism", totalCores)
 
   override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) {
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 1baa0e009f..dfbde7c8a1 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -59,9 +59,10 @@ private[spark] class JavaSerializationStream(
 }
 
 private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader)
-extends DeserializationStream {
+  extends DeserializationStream {
+
   private val objIn = new ObjectInputStream(in) {
-    override def resolveClass(desc: ObjectStreamClass) =
+    override def resolveClass(desc: ObjectStreamClass): Class[_] =
       Class.forName(desc.getName, false, loader)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index dc7aa99738..f83bcaa5cc 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -60,7 +60,7 @@ class KryoSerializer(conf: SparkConf)
     .split(',')
     .filter(!_.isEmpty)
 
-  def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize))
+  def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize))
 
   def newKryo(): Kryo = {
     val instantiator = new EmptyScalaKryoInstantiator
diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
index 7de2f9cbb2..660df00bc3 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala
@@ -106,7 +106,7 @@ class FileShuffleBlockManager(conf: SparkConf)
    * when the writers are closed successfully
    */
   def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer,
-      writeMetrics: ShuffleWriteMetrics) = {
+      writeMetrics: ShuffleWriteMetrics): ShuffleWriterGroup = {
     new ShuffleWriterGroup {
       shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets))
       private val shuffleState = shuffleStates(shuffleId)
@@ -268,7 +268,7 @@ object FileShuffleBlockManager {
       new PrimitiveVector[Long]()
     }
 
-    def apply(bucketId: Int) = files(bucketId)
+    def apply(bucketId: Int): File = files(bucketId)
 
     def recordMapOutput(mapId: Int, offsets: Array[Long], lengths: Array[Long]) {
       assert(offsets.length == lengths.length)
diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala
index b292587d37..87fd161e06 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala
@@ -80,7 +80,7 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager {
    * end of the output file. This will be used by getBlockLocation to figure out where each block
    * begins and ends.
    * */
-  def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]) = {
+  def writeIndexFile(shuffleId: Int, mapId: Int, lengths: Array[Long]): Unit = {
     val indexFile = getIndexFile(shuffleId, mapId)
     val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile)))
     try {
@@ -121,5 +121,5 @@ class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager {
     }
   }
 
-  override def stop() = {}
+  override def stop(): Unit = {}
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
index 1f012941c8..c186fd360f 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -35,13 +35,13 @@ sealed abstract class BlockId {
   def name: String
 
   // convenience methods
-  def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None
-  def isRDD = isInstanceOf[RDDBlockId]
-  def isShuffle = isInstanceOf[ShuffleBlockId]
-  def isBroadcast = isInstanceOf[BroadcastBlockId]
+  def asRDDId: Option[RDDBlockId] = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None
+  def isRDD: Boolean = isInstanceOf[RDDBlockId]
+  def isShuffle: Boolean = isInstanceOf[ShuffleBlockId]
+  def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId]
 
-  override def toString = name
-  override def hashCode = name.hashCode
+  override def toString: String = name
+  override def hashCode: Int = name.hashCode
   override def equals(other: Any): Boolean = other match {
     case o: BlockId => getClass == o.getClass && name.equals(o.name)
     case _ => false
@@ -50,54 +50,54 @@ sealed abstract class BlockId {
 
 @DeveloperApi
 case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId {
-  def name = "rdd_" + rddId + "_" + splitIndex
+  override def name: String = "rdd_" + rddId + "_" + splitIndex
 }
 
 // Format of the shuffle block ids (including data and index) should be kept in sync with
 // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getBlockData().
 @DeveloperApi
 case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId {
-  def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId
+  override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId
 }
 
 @DeveloperApi
 case class ShuffleDataBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId {
-  def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data"
+  override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".data"
 }
 
 @DeveloperApi
 case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId {
-  def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index"
+  override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index"
 }
 
 @DeveloperApi
 case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId {
-  def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field)
+  override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field)
 }
 
 @DeveloperApi
 case class TaskResultBlockId(taskId: Long) extends BlockId {
-  def name = "taskresult_" + taskId
+  override def name: String = "taskresult_" + taskId
 }
 
 @DeveloperApi
 case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId {
-  def name = "input-" + streamId + "-" + uniqueId
+  override def name: String = "input-" + streamId + "-" + uniqueId
 }
 
 /** Id associated with temporary local data managed as blocks. Not serializable. */
 private[spark] case class TempLocalBlockId(id: UUID) extends BlockId {
-  def name = "temp_local_" + id
+  override def name: String = "temp_local_" + id
 }
 
 /** Id associated with temporary shuffle data managed as blocks. Not serializable. */
 private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId {
-  def name = "temp_shuffle_" + id
+  override def name: String = "temp_shuffle_" + id
 }
 
 // Intended only for testing purposes
 private[spark] case class TestBlockId(id: String) extends BlockId {
-  def name = "test_" + id
+  override def name: String = "test_" + id
 }
 
 @DeveloperApi
@@ -112,7 +112,7 @@ object BlockId {
   val TEST = "test_(.*)".r
 
   /** Converts a BlockId "name" String back into a BlockId. */
-  def apply(id: String) = id match {
+  def apply(id: String): BlockId = id match {
     case RDD(rddId, splitIndex) =>
       RDDBlockId(rddId.toInt, splitIndex.toInt)
     case SHUFFLE(shuffleId, mapId, reduceId) =>
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
index b177a59c72..a6f1ebf325 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala
@@ -77,11 +77,11 @@ class BlockManagerId private (
   @throws(classOf[IOException])
   private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this)
 
-  override def toString = s"BlockManagerId($executorId, $host, $port)"
+  override def toString: String = s"BlockManagerId($executorId, $host, $port)"
 
   override def hashCode: Int = (executorId.hashCode * 41 + host.hashCode) * 41 + port
 
-  override def equals(that: Any) = that match {
+  override def equals(that: Any): Boolean = that match {
     case id: BlockManagerId =>
       executorId == id.executorId && port == id.port && host == id.host
     case _ =>
@@ -100,10 +100,10 @@ private[spark] object BlockManagerId {
    * @param port Port of the block manager.
    * @return A new [[org.apache.spark.storage.BlockManagerId]].
    */
-  def apply(execId: String, host: String, port: Int) =
+  def apply(execId: String, host: String, port: Int): BlockManagerId =
     getCachedBlockManagerId(new BlockManagerId(execId, host, port))
 
-  def apply(in: ObjectInput) = {
+  def apply(in: ObjectInput): BlockManagerId = {
     val obj = new BlockManagerId()
     obj.readExternal(in)
     getCachedBlockManagerId(obj)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index 654796f23c..061964826f 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -79,7 +79,7 @@ class BlockManagerMaster(
    * Check if block manager master has a block. Note that this can be used to check for only
    * those blocks that are reported to block manager master.
    */
-  def contains(blockId: BlockId) = {
+  def contains(blockId: BlockId): Boolean = {
     !getLocations(blockId).isEmpty
   }
 
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index 787b0f96be..5b53280161 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -52,7 +52,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus
 
   private val akkaTimeout = AkkaUtils.askTimeout(conf)
 
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) =>
       register(blockManagerId, maxMemSize, slaveActor)
       sender ! true
@@ -421,7 +421,7 @@ private[spark] class BlockManagerInfo(
   // Mapping from block id to its status.
   private val _blocks = new JHashMap[BlockId, BlockStatus]
 
-  def getStatus(blockId: BlockId) = Option(_blocks.get(blockId))
+  def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId))
 
   def updateLastSeenMs() {
     _lastSeenMs = System.currentTimeMillis()
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
index 8462871e79..52fb896c4e 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala
@@ -38,7 +38,7 @@ class BlockManagerSlaveActor(
   import context.dispatcher
 
   // Operations that involve removing blocks may be slow and should be done asynchronously
-  override def receiveWithLogging = {
+  override def receiveWithLogging: PartialFunction[Any, Unit] = {
     case RemoveBlock(blockId) =>
       doAsync[Boolean]("removing block " + blockId, sender) {
         blockManager.removeBlock(blockId)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
index 81164178b9..f703e50b6b 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -82,11 +82,13 @@ private[spark] class DiskBlockObjectWriter(
 {
   /** Intercepts write calls and tracks total time spent writing. Not thread safe. */
   private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream {
-    def write(i: Int): Unit = callWithTiming(out.write(i))
-    override def write(b: Array[Byte]) = callWithTiming(out.write(b))
-    override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len))
-    override def close() = out.close()
-    override def flush() = out.flush()
+    override def write(i: Int): Unit = callWithTiming(out.write(i))
+    override def write(b: Array[Byte]): Unit = callWithTiming(out.write(b))
+    override def write(b: Array[Byte], off: Int, len: Int): Unit = {
+      callWithTiming(out.write(b, off, len))
+    }
+    override def close(): Unit = out.close()
+    override def flush(): Unit = out.flush()
   }
 
   /** The file channel, used for repositioning / truncating the file. */
@@ -141,8 +143,9 @@ private[spark] class DiskBlockObjectWriter(
       if (syncWrites) {
         // Force outstanding writes to disk and track how long it takes
         objOut.flush()
-        def sync = fos.getFD.sync()
-        callWithTiming(sync)
+        callWithTiming {
+          fos.getFD.sync()
+        }
       }
       objOut.close()
 
diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
index 132502b75f..95e2d688d9 100644
--- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
+++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
@@ -24,5 +24,7 @@ import java.io.File
  * based off an offset and a length.
  */
 private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) {
-  override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length)
+  override def toString: String = {
+    "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length)
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
index 120c327a7e..0186eb30a1 100644
--- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
+++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
@@ -36,7 +36,7 @@ class RDDInfo(
 
   def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0
 
-  override def toString = {
+  override def toString: String = {
     import Utils.bytesToString
     ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " +
       "MemorySize: %s; TachyonSize: %s; DiskSize: %s").format(
@@ -44,7 +44,7 @@ class RDDInfo(
         bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize))
   }
 
-  override def compare(that: RDDInfo) = {
+  override def compare(that: RDDInfo): Int = {
     this.id - that.id
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index e5e1cf5a69..134abea866 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -50,11 +50,11 @@ class StorageLevel private(
 
   def this() = this(false, true, false, false)  // For deserialization
 
-  def useDisk = _useDisk
-  def useMemory = _useMemory
-  def useOffHeap = _useOffHeap
-  def deserialized = _deserialized
-  def replication = _replication
+  def useDisk: Boolean = _useDisk
+  def useMemory: Boolean = _useMemory
+  def useOffHeap: Boolean = _useOffHeap
+  def deserialized: Boolean = _deserialized
+  def replication: Int = _replication
 
   assert(replication < 40, "Replication restricted to be less than 40 for calculating hash codes")
 
@@ -80,7 +80,7 @@ class StorageLevel private(
       false
   }
 
-  def isValid = (useMemory || useDisk || useOffHeap) && (replication > 0)
+  def isValid: Boolean = (useMemory || useDisk || useOffHeap) && (replication > 0)
 
   def toInt: Int = {
     var ret = 0
@@ -183,7 +183,7 @@ object StorageLevel {
       useMemory: Boolean,
       useOffHeap: Boolean,
       deserialized: Boolean,
-      replication: Int) = {
+      replication: Int): StorageLevel = {
     getCachedStorageLevel(
       new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication))
   }
@@ -197,7 +197,7 @@ object StorageLevel {
       useDisk: Boolean,
       useMemory: Boolean,
       deserialized: Boolean,
-      replication: Int = 1) = {
+      replication: Int = 1): StorageLevel = {
     getCachedStorageLevel(new StorageLevel(useDisk, useMemory, false, deserialized, replication))
   }
 
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
index def49e80a3..7d75929b96 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
@@ -19,7 +19,6 @@ package org.apache.spark.storage
 
 import scala.collection.mutable
 
-import org.apache.spark.SparkContext
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler._
 
@@ -32,7 +31,7 @@ class StorageStatusListener extends SparkListener {
   // This maintains only blocks that are cached (i.e. storage level is not StorageLevel.NONE)
   private[storage] val executorIdToStorageStatus = mutable.Map[String, StorageStatus]()
 
-  def storageStatusList = executorIdToStorageStatus.values.toSeq
+  def storageStatusList: Seq[StorageStatus] = executorIdToStorageStatus.values.toSeq
 
   /** Update storage status list to reflect updated block statuses */
   private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) {
@@ -56,7 +55,7 @@ class StorageStatusListener extends SparkListener {
     }
   }
 
-  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
+  override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
     val info = taskEnd.taskInfo
     val metrics = taskEnd.taskMetrics
     if (info != null && metrics != null) {
@@ -67,7 +66,7 @@ class StorageStatusListener extends SparkListener {
     }
   }
 
-  override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized {
+  override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized {
     updateStorageStatus(unpersistRDD.rddId)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala
index b86abbda1d..65fa81704c 100644
--- a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala
+++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala
@@ -24,5 +24,7 @@ import tachyon.client.TachyonFile
  * a length.
  */
 private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) {
-  override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length)
+  override def toString: String = {
+    "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length)
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 0c24ad2760..adfa6bbada 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -60,7 +60,7 @@ private[spark] class SparkUI private (
   }
   initialize()
 
-  def getAppName = appName
+  def getAppName: String = appName
 
   /** Set the app name for this UI. */
   def setAppName(name: String) {
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index b5022fe853..f07864141a 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -149,9 +149,11 @@ private[spark] object UIUtils extends Logging {
     }
   }
 
-  def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource
+  def prependBaseUri(basePath: String = "", resource: String = ""): String = {
+    uiRoot + basePath + resource
+  }
 
-  def commonHeaderNodes = {
+  def commonHeaderNodes: Seq[Node] = {
     <meta http-equiv="Content-type" content="text/html; charset=utf-8" />
     <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")}
           type="text/css" />
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index fc1844600f..19ac7a826e 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -51,7 +51,7 @@ private[spark] object UIWorkloadGenerator {
     val nJobSet = args(2).toInt
     val sc = new SparkContext(conf)
 
-    def setProperties(s: String) = {
+    def setProperties(s: String): Unit = {
       if(schedulingMode == SchedulingMode.FAIR) {
         sc.setLocalProperty("spark.scheduler.pool", s)
       }
@@ -59,7 +59,7 @@ private[spark] object UIWorkloadGenerator {
     }
 
     val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS)
-    def nextFloat() = new Random().nextFloat()
+    def nextFloat(): Float = new Random().nextFloat()
 
     val jobs = Seq[(String, () => Long)](
       ("Count", baseData.count),
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
index 3afd7ef07d..69053fe44d 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.HashMap
 import org.apache.spark.ExceptionFailure
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.scheduler._
-import org.apache.spark.storage.StorageStatusListener
+import org.apache.spark.storage.{StorageStatus, StorageStatusListener}
 import org.apache.spark.ui.{SparkUI, SparkUITab}
 
 private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") {
@@ -55,19 +55,19 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp
   val executorToShuffleWrite = HashMap[String, Long]()
   val executorToLogUrls = HashMap[String, Map[String, String]]()
 
-  def storageStatusList = storageStatusListener.storageStatusList
+  def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList
 
-  override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) = synchronized {
+  override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized {
     val eid = executorAdded.executorId
     executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap
   }
 
-  override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
+  override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {
     val eid = taskStart.taskInfo.executorId
     executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1
   }
 
-  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
+  override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
     val info = taskEnd.taskInfo
     if (info != null) {
       val eid = info.executorId
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index 937d95a934..949e80d30f 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -73,7 +73,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
 
   // Misc:
   val executorIdToBlockManagerId = HashMap[ExecutorId, BlockManagerId]()
-  def blockManagerIds = executorIdToBlockManagerId.values.toSeq
+  def blockManagerIds: Seq[BlockManagerId] = executorIdToBlockManagerId.values.toSeq
 
   var schedulingMode: Option[SchedulingMode] = None
 
@@ -146,7 +146,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     }
   }
 
-  override def onJobStart(jobStart: SparkListenerJobStart) = synchronized {
+  override def onJobStart(jobStart: SparkListenerJobStart): Unit = synchronized {
     val jobGroup = for (
       props <- Option(jobStart.properties);
       group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID))
@@ -182,7 +182,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     }
   }
 
-  override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized {
+  override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = synchronized {
     val jobData = activeJobs.remove(jobEnd.jobId).getOrElse {
       logWarning(s"Job completed for unknown job ${jobEnd.jobId}")
       new JobUIData(jobId = jobEnd.jobId)
@@ -219,7 +219,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     }
   }
 
-  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized {
+  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized {
     val stage = stageCompleted.stageInfo
     stageIdToInfo(stage.stageId) = stage
     val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), {
@@ -260,7 +260,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
   }
 
   /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */
-  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized {
+  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
     val stage = stageSubmitted.stageInfo
     activeStages(stage.stageId) = stage
     pendingStages.remove(stage.stageId)
@@ -288,7 +288,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     }
   }
 
-  override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
+  override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized {
     val taskInfo = taskStart.taskInfo
     if (taskInfo != null) {
       val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), {
@@ -312,7 +312,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     // stageToTaskInfos already has the updated status.
   }
 
-  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
+  override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
     val info = taskEnd.taskInfo
     // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task
     // completion event is for. Let's just drop it here. This means we might have some speculation
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index b2bbfdee56..7ffcf291b5 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -24,7 +24,7 @@ import org.apache.spark.ui.{SparkUI, SparkUITab}
 private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") {
   val sc = parent.sc
   val killEnabled = parent.killEnabled
-  def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+  def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
   val listener = parent.jobProgressListener
 
   attachPage(new AllJobsPage(this))
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 110f8780a9..e03442894c 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs
 import java.util.Date
 import javax.servlet.http.HttpServletRequest
 
-import scala.xml.{Node, Unparsed}
+import scala.xml.{Elem, Node, Unparsed}
 
 import org.apache.commons.lang3.StringEscapeUtils
 
@@ -170,7 +170,8 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
         </div>
 
       val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
-      def accumulableRow(acc: AccumulableInfo) = <tr><td>{acc.name}</td><td>{acc.value}</td></tr>
+      def accumulableRow(acc: AccumulableInfo): Elem =
+        <tr><td>{acc.name}</td><td>{acc.value}</td></tr>
       val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow,
         accumulables.values.toSeq)
 
@@ -293,10 +294,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
           val schedulerDelayQuantiles = schedulerDelayTitle +:
             getFormattedTimeQuantiles(schedulerDelays)
 
-          def getFormattedSizeQuantiles(data: Seq[Double]) =
+          def getFormattedSizeQuantiles(data: Seq[Double]): Seq[Elem] =
             getDistributionQuantiles(data).map(d => <td>{Utils.bytesToString(d.toLong)}</td>)
 
-          def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = {
+          def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double])
+            : Seq[Elem] = {
             val recordDist = getDistributionQuantiles(records).iterator
             getDistributionQuantiles(data).map(d =>
               <td>{s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"}</td>
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
index 937261de00..1bd2d87e00 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
@@ -32,10 +32,10 @@ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages"
   attachPage(new StagePage(this))
   attachPage(new PoolPage(this))
 
-  def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+  def isFairScheduler: Boolean = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
 
-  def handleKillRequest(request: HttpServletRequest) =  {
-    if ((killEnabled) && (parent.securityManager.checkModifyPermissions(request.getRemoteUser))) {
+  def handleKillRequest(request: HttpServletRequest): Unit = {
+    if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) {
       val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean
       val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt
       if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
index dbf1ceeda1..711a3697bd 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
@@ -94,11 +94,11 @@ private[jobs] object UIData {
     var taskData = new HashMap[Long, TaskUIData]
     var executorSummary = new HashMap[String, ExecutorSummary]
 
-    def hasInput = inputBytes > 0
-    def hasOutput = outputBytes > 0
-    def hasShuffleRead = shuffleReadTotalBytes > 0
-    def hasShuffleWrite = shuffleWriteBytes > 0
-    def hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0
+    def hasInput: Boolean = inputBytes > 0
+    def hasOutput: Boolean = outputBytes > 0
+    def hasShuffleRead: Boolean = shuffleReadTotalBytes > 0
+    def hasShuffleWrite: Boolean = shuffleWriteBytes > 0
+    def hasBytesSpilled: Boolean = memoryBytesSpilled > 0 && diskBytesSpilled > 0
   }
 
   /**
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
index a81291d505..045bd78499 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
@@ -40,10 +40,10 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag
 class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener {
   private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing
 
-  def storageStatusList = storageStatusListener.storageStatusList
+  def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList
 
   /** Filter RDD info to include only those with cached partitions */
-  def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
+  def rddInfoList: Seq[RDDInfo] = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq
 
   /** Update the storage info of the RDDs whose blocks are among the given updated blocks */
   private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = {
@@ -56,19 +56,19 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar
    * Assumes the storage status list is fully up-to-date. This implies the corresponding
    * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener.
    */
-  override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
+  override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
     val metrics = taskEnd.taskMetrics
     if (metrics != null && metrics.updatedBlocks.isDefined) {
       updateRDDInfo(metrics.updatedBlocks.get)
     }
   }
 
-  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized {
+  override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = synchronized {
     val rddInfos = stageSubmitted.stageInfo.rddInfos
     rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) }
   }
 
-  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized {
+  override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized {
     // Remove all partitions that are no longer cached in current completed stage
     val completedRddIds = stageCompleted.stageInfo.rddInfos.map(r => r.id).toSet
     _rddInfoMap.retain { case (id, info) =>
@@ -76,7 +76,7 @@ class StorageListener(storageStatusListener: StorageStatusListener) extends Spar
     }
   }
 
-  override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized {
+  override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD): Unit = synchronized {
     _rddInfoMap.remove(unpersistRDD.rddId)
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
index 390310243e..9044aaeef2 100644
--- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
+++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
@@ -27,8 +27,8 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat
 // scalastyle:on
 
   private[this] var completed = false
-  def next() = sub.next()
-  def hasNext = {
+  def next(): A = sub.next()
+  def hasNext: Boolean = {
     val r = sub.hasNext
     if (!r && !completed) {
       completed = true
@@ -37,13 +37,13 @@ abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterat
     r
   }
 
-  def completion()
+  def completion(): Unit
 }
 
 private[spark] object CompletionIterator {
-  def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A,I] = {
+  def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A, I] = {
     new CompletionIterator[A,I](sub) {
-      def completion() = completionFunction
+      def completion(): Unit = completionFunction
     }
   }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala
index a465298c8c..9aea8efa38 100644
--- a/core/src/main/scala/org/apache/spark/util/Distribution.scala
+++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala
@@ -57,7 +57,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va
     out.println
   }
 
-  def statCounter = StatCounter(data.slice(startIdx, endIdx))
+  def statCounter: StatCounter = StatCounter(data.slice(startIdx, endIdx))
 
   /**
    * print a summary of this distribution to the given PrintStream.
diff --git a/core/src/main/scala/org/apache/spark/util/ManualClock.scala b/core/src/main/scala/org/apache/spark/util/ManualClock.scala
index cf89c1782f..1718554061 100644
--- a/core/src/main/scala/org/apache/spark/util/ManualClock.scala
+++ b/core/src/main/scala/org/apache/spark/util/ManualClock.scala
@@ -39,31 +39,27 @@ private[spark] class ManualClock(private var time: Long) extends Clock {
   /**
    * @param timeToSet new time (in milliseconds) that the clock should represent
    */
-  def setTime(timeToSet: Long) =
-    synchronized {
-      time = timeToSet
-      notifyAll()
-    }
+  def setTime(timeToSet: Long): Unit = synchronized {
+    time = timeToSet
+    notifyAll()
+  }
 
   /**
    * @param timeToAdd time (in milliseconds) to add to the clock's time
    */
-  def advance(timeToAdd: Long) =
-    synchronized {
-      time += timeToAdd
-      notifyAll()
-    }
+  def advance(timeToAdd: Long): Unit = synchronized {
+    time += timeToAdd
+    notifyAll()
+  }
 
   /**
    * @param targetTime block until the clock time is set or advanced to at least this time
    * @return current time reported by the clock when waiting finishes
    */
-  def waitTillTime(targetTime: Long): Long =
-    synchronized {
-      while (time < targetTime) {
-        wait(100)
-      }
-      getTimeMillis()
+  def waitTillTime(targetTime: Long): Long = synchronized {
+    while (time < targetTime) {
+      wait(100)
     }
-
+    getTimeMillis()
+  }
 }
diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index ac40f19ed6..375ed430bd 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -67,14 +67,15 @@ private[spark] object MetadataCleanerType extends Enumeration {
 
   type MetadataCleanerType = Value
 
-  def systemProperty(which: MetadataCleanerType.MetadataCleanerType) =
-      "spark.cleaner.ttl." + which.toString
+  def systemProperty(which: MetadataCleanerType.MetadataCleanerType): String = {
+    "spark.cleaner.ttl." + which.toString
+  }
 }
 
 // TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the
 // initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
 private[spark] object MetadataCleaner {
-  def getDelaySeconds(conf: SparkConf) = {
+  def getDelaySeconds(conf: SparkConf): Int = {
     conf.getInt("spark.cleaner.ttl", -1)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/util/MutablePair.scala b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
index 74fa77b68d..dad888548e 100644
--- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala
+++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala
@@ -43,7 +43,7 @@ case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef
     this
   }
 
-  override def toString = "(" + _1 + "," + _2 + ")"
+  override def toString: String = "(" + _1 + "," + _2 + ")"
 
   override def canEqual(that: Any): Boolean = that.isInstanceOf[MutablePair[_,_]]
 }
diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
index 6d8d9e8da3..73d126ff62 100644
--- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
+++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
@@ -22,7 +22,7 @@ package org.apache.spark.util
  */
 private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) {
 
-  override def findClass(name: String) = {
+  override def findClass(name: String): Class[_] = {
     super.findClass(name)
   }
 
diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
index 770ff9d5ad..a06b6f84ef 100644
--- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
@@ -27,7 +27,7 @@ import java.nio.channels.Channels
  */
 private[spark]
 class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable {
-  def value = buffer
+  def value: ByteBuffer = buffer
 
   private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException {
     val length = in.readInt()
diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala
index d80eed455c..8586da1996 100644
--- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala
+++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala
@@ -141,8 +141,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
 
 object StatCounter {
   /** Build a StatCounter from a list of values. */
-  def apply(values: TraversableOnce[Double]) = new StatCounter(values)
+  def apply(values: TraversableOnce[Double]): StatCounter = new StatCounter(values)
 
   /** Build a StatCounter from a list of values passed as variable-length arguments. */
-  def apply(values: Double*) = new StatCounter(values)
+  def apply(values: Double*): StatCounter = new StatCounter(values)
 }
diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala
index f5be5856c2..310c0c1094 100644
--- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala
@@ -82,7 +82,7 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo
     this
   }
 
-  override def update(key: A, value: B) = this += ((key, value))
+  override def update(key: A, value: B): Unit = this += ((key, value))
 
   override def apply(key: A): B = internalMap.apply(key)
 
@@ -92,14 +92,14 @@ private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boo
 
   override def size: Int = internalMap.size
 
-  override def foreach[U](f: ((A, B)) => U) = nonNullReferenceMap.foreach(f)
+  override def foreach[U](f: ((A, B)) => U): Unit = nonNullReferenceMap.foreach(f)
 
   def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value)
 
   def toMap: Map[A, B] = iterator.toMap
 
   /** Remove old key-value pairs with timestamps earlier than `threshTime`. */
-  def clearOldValues(threshTime: Long) = internalMap.clearOldValues(threshTime)
+  def clearOldValues(threshTime: Long): Unit = internalMap.clearOldValues(threshTime)
 
   /** Remove entries with values that are no longer strongly reachable. */
   def clearNullValues() {
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index fa56bb09e4..d9a671687a 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -85,7 +85,7 @@ private[spark] object Utils extends Logging {
   def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = {
     val bis = new ByteArrayInputStream(bytes)
     val ois = new ObjectInputStream(bis) {
-      override def resolveClass(desc: ObjectStreamClass) =
+      override def resolveClass(desc: ObjectStreamClass): Class[_] =
         Class.forName(desc.getName, false, loader)
     }
     ois.readObject.asInstanceOf[T]
@@ -106,11 +106,10 @@ private[spark] object Utils extends Logging {
 
   /** Serialize via nested stream using specific serializer */
   def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)(
-      f: SerializationStream => Unit) = {
+      f: SerializationStream => Unit): Unit = {
     val osWrapper = ser.serializeStream(new OutputStream {
-      def write(b: Int) = os.write(b)
-
-      override def write(b: Array[Byte], off: Int, len: Int) = os.write(b, off, len)
+      override def write(b: Int): Unit = os.write(b)
+      override def write(b: Array[Byte], off: Int, len: Int): Unit = os.write(b, off, len)
     })
     try {
       f(osWrapper)
@@ -121,10 +120,9 @@ private[spark] object Utils extends Logging {
 
   /** Deserialize via nested stream using specific serializer */
   def deserializeViaNestedStream(is: InputStream, ser: SerializerInstance)(
-      f: DeserializationStream => Unit) = {
+      f: DeserializationStream => Unit): Unit = {
     val isWrapper = ser.deserializeStream(new InputStream {
-      def read(): Int = is.read()
-
+      override def read(): Int = is.read()
       override def read(b: Array[Byte], off: Int, len: Int): Int = is.read(b, off, len)
     })
     try {
@@ -137,7 +135,7 @@ private[spark] object Utils extends Logging {
   /**
    * Get the ClassLoader which loaded Spark.
    */
-  def getSparkClassLoader = getClass.getClassLoader
+  def getSparkClassLoader: ClassLoader = getClass.getClassLoader
 
   /**
    * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that
@@ -146,7 +144,7 @@ private[spark] object Utils extends Logging {
    * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently
    * active loader when setting up ClassLoader delegation chains.
    */
-  def getContextOrSparkClassLoader =
+  def getContextOrSparkClassLoader: ClassLoader =
     Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader)
 
   /** Determines whether the provided class is loadable in the current thread. */
@@ -155,12 +153,14 @@ private[spark] object Utils extends Logging {
   }
 
   /** Preferred alternative to Class.forName(className) */
-  def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader)
+  def classForName(className: String): Class[_] = {
+    Class.forName(className, true, getContextOrSparkClassLoader)
+  }
 
   /**
    * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]]
    */
-  def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = {
+  def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput): Unit = {
     if (bb.hasArray) {
       out.write(bb.array(), bb.arrayOffset() + bb.position(), bb.remaining())
     } else {
@@ -1557,7 +1557,7 @@ private[spark] object Utils extends Logging {
 
 
   /** Return the class name of the given object, removing all dollar signs */
-  def getFormattedClassName(obj: AnyRef) = {
+  def getFormattedClassName(obj: AnyRef): String = {
     obj.getClass.getSimpleName.replace("$", "")
   }
 
@@ -1570,7 +1570,7 @@ private[spark] object Utils extends Logging {
   }
 
   /** Return an empty JSON object */
-  def emptyJson = JObject(List[JField]())
+  def emptyJson: JsonAST.JObject = JObject(List[JField]())
 
   /**
    * Return a Hadoop FileSystem with the scheme encoded in the given path.
@@ -1618,7 +1618,7 @@ private[spark] object Utils extends Logging {
   /**
    * Indicates whether Spark is currently running unit tests.
    */
-  def isTesting = {
+  def isTesting: Boolean = {
     sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing")
   }
 
diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
index af1f64649f..f79e8e0491 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
@@ -156,10 +156,10 @@ class BitSet(numBits: Int) extends Serializable {
   /**
    * Get an iterator over the set bits.
    */
-  def iterator = new Iterator[Int] {
+  def iterator: Iterator[Int] = new Iterator[Int] {
     var ind = nextSetBit(0)
     override def hasNext: Boolean = ind >= 0
-    override def next() = {
+    override def next(): Int = {
       val tmp = ind
       ind  = nextSetBit(ind + 1)
       tmp
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 8a0f5a602d..9ff4744593 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -159,7 +159,7 @@ class ExternalAppendOnlyMap[K, V, C](
     val batchSizes = new ArrayBuffer[Long]
 
     // Flush the disk writer's contents to disk, and update relevant variables
-    def flush() = {
+    def flush(): Unit = {
       val w = writer
       writer = null
       w.commitAndClose()
@@ -355,7 +355,7 @@ class ExternalAppendOnlyMap[K, V, C](
         val pairs: ArrayBuffer[(K, C)])
       extends Comparable[StreamBuffer] {
 
-      def isEmpty = pairs.length == 0
+      def isEmpty: Boolean = pairs.length == 0
 
       // Invalid if there are no more pairs in this stream
       def minKeyHash: Int = {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index d69f2d9048..3262e670c2 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -283,7 +283,7 @@ private[spark] class ExternalSorter[K, V, C](
 
     // Flush the disk writer's contents to disk, and update relevant variables.
     // The writer is closed at the end of this process, and cannot be reused.
-    def flush() = {
+    def flush(): Unit = {
       val w = writer
       writer = null
       w.commitAndClose()
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
index b8de4ff9aa..c52591b352 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
@@ -109,7 +109,7 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag](
     }
   }
 
-  override def iterator = new Iterator[(K, V)] {
+  override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] {
     var pos = -1
     var nextPair: (K, V) = computeNextPair()
 
@@ -132,9 +132,9 @@ class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag](
       }
     }
 
-    def hasNext = nextPair != null
+    def hasNext: Boolean = nextPair != null
 
-    def next() = {
+    def next(): (K, V) = {
       val pair = nextPair
       nextPair = computeNextPair()
       pair
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
index 4e363b74f4..c80057f95e 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
@@ -85,7 +85,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag](
 
   protected var _bitset = new BitSet(_capacity)
 
-  def getBitSet = _bitset
+  def getBitSet: BitSet = _bitset
 
   // Init of the array in constructor (instead of in declaration) to work around a Scala compiler
   // specialization bug that would generate two arrays (one for Object and one for specialized T).
@@ -183,7 +183,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag](
   /** Return the value at the specified position. */
   def getValue(pos: Int): T = _data(pos)
 
-  def iterator = new Iterator[T] {
+  def iterator: Iterator[T] = new Iterator[T] {
     var pos = nextPos(0)
     override def hasNext: Boolean = pos != INVALID_POS
     override def next(): T = {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
index 2e1ef06cbc..61e2264276 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
@@ -46,7 +46,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
 
   private var _oldValues: Array[V] = null
 
-  override def size = _keySet.size
+  override def size: Int = _keySet.size
 
   /** Get the value for a given key */
   def apply(k: K): V = {
@@ -87,7 +87,7 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
     }
   }
 
-  override def iterator = new Iterator[(K, V)] {
+  override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] {
     var pos = 0
     var nextPair: (K, V) = computeNextPair()
 
@@ -103,9 +103,9 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
       }
     }
 
-    def hasNext = nextPair != null
+    def hasNext: Boolean = nextPair != null
 
-    def next() = {
+    def next(): (K, V) = {
       val pair = nextPair
       nextPair = computeNextPair()
       pair
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
index c5268c0fae..bdbca00a00 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
@@ -32,7 +32,7 @@ private[spark] object Utils {
    */
   def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = {
     val ordering = new GuavaOrdering[T] {
-      override def compare(l: T, r: T) = ord.compare(l, r)
+      override def compare(l: T, r: T): Int = ord.compare(l, r)
     }
     collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator
   }
diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala
index 1d54670606..14b6ba4af4 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala
@@ -121,7 +121,7 @@ private[spark] object FileAppender extends Logging {
     val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT)
     val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT)
 
-    def createTimeBasedAppender() = {
+    def createTimeBasedAppender(): FileAppender = {
       val validatedParams: Option[(Long, String)] = rollingInterval match {
         case "daily" =>
           logInfo(s"Rolling executor logs enabled for $file with daily rolling")
@@ -149,7 +149,7 @@ private[spark] object FileAppender extends Logging {
       }
     }
 
-    def createSizeBasedAppender() = {
+    def createSizeBasedAppender(): FileAppender = {
       rollingSizeBytes match {
         case IntParam(bytes) =>
           logInfo(s"Rolling executor logs enabled for $file with rolling every $bytes bytes")
diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
index 76e7a2760b..786b97ad7b 100644
--- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
@@ -105,7 +105,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals
 
   private val rng: Random = new XORShiftRandom
 
-  override def setSeed(seed: Long) = rng.setSeed(seed)
+  override def setSeed(seed: Long): Unit = rng.setSeed(seed)
 
   override def sample(items: Iterator[T]): Iterator[T] = {
     if (ub - lb <= 0.0) {
@@ -131,7 +131,7 @@ class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = fals
   def cloneComplement(): BernoulliCellSampler[T] =
     new BernoulliCellSampler[T](lb, ub, !complement)
 
-  override def clone = new BernoulliCellSampler[T](lb, ub, complement)
+  override def clone: BernoulliCellSampler[T] = new BernoulliCellSampler[T](lb, ub, complement)
 }
 
 
@@ -153,7 +153,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T
 
   private val rng: Random = RandomSampler.newDefaultRNG
 
-  override def setSeed(seed: Long) = rng.setSeed(seed)
+  override def setSeed(seed: Long): Unit = rng.setSeed(seed)
 
   override def sample(items: Iterator[T]): Iterator[T] = {
     if (fraction <= 0.0) {
@@ -167,7 +167,7 @@ class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T
     }
   }
 
-  override def clone = new BernoulliSampler[T](fraction)
+  override def clone: BernoulliSampler[T] = new BernoulliSampler[T](fraction)
 }
 
 
@@ -209,7 +209,7 @@ class PoissonSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T]
     }
   }
 
-  override def clone = new PoissonSampler[T](fraction)
+  override def clone: PoissonSampler[T] = new PoissonSampler[T](fraction)
 }
 
 
@@ -228,15 +228,18 @@ class GapSamplingIterator[T: ClassTag](
     val arrayClass = Array.empty[T].iterator.getClass
     val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass
     data.getClass match {
-      case `arrayClass` => ((n: Int) => { data = data.drop(n) })
-      case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) })
-      case _ => ((n: Int) => {
+      case `arrayClass` =>
+        (n: Int) => { data = data.drop(n) }
+      case `arrayBufferClass` =>
+        (n: Int) => { data = data.drop(n) }
+      case _ =>
+        (n: Int) => {
           var j = 0
           while (j < n && data.hasNext) {
             data.next()
             j += 1
           }
-        })
+        }
     }
   }
 
@@ -244,21 +247,21 @@ class GapSamplingIterator[T: ClassTag](
 
   override def next(): T = {
     val r = data.next()
-    advance
+    advance()
     r
   }
 
   private val lnq = math.log1p(-f)
 
   /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */
-  private def advance: Unit = {
+  private def advance(): Unit = {
     val u = math.max(rng.nextDouble(), epsilon)
     val k = (math.log(u) / lnq).toInt
     iterDrop(k)
   }
 
   /** advance to first sample as part of object construction. */
-  advance
+  advance()
   // Attempting to invoke this closer to the top with other object initialization
   // was causing it to break in strange ways, so I'm invoking it last, which seems to
   // work reliably.
@@ -279,15 +282,18 @@ class GapSamplingReplacementIterator[T: ClassTag](
     val arrayClass = Array.empty[T].iterator.getClass
     val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass
     data.getClass match {
-      case `arrayClass` => ((n: Int) => { data = data.drop(n) })
-      case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) })
-      case _ => ((n: Int) => {
+      case `arrayClass` =>
+        (n: Int) => { data = data.drop(n) }
+      case `arrayBufferClass` =>
+        (n: Int) => { data = data.drop(n) }
+      case _ =>
+        (n: Int) => {
           var j = 0
           while (j < n && data.hasNext) {
             data.next()
             j += 1
           }
-        })
+        }
     }
   }
 
@@ -300,7 +306,7 @@ class GapSamplingReplacementIterator[T: ClassTag](
   override def next(): T = {
     val r = v
     rep -= 1
-    if (rep <= 0) advance
+    if (rep <= 0) advance()
     r
   }
 
@@ -309,7 +315,7 @@ class GapSamplingReplacementIterator[T: ClassTag](
    * Samples 'k' from geometric distribution  P(k) = (1-q)(q)^k, where q = e^(-f), that is
    * q is the probabililty of Poisson(0; f)
    */
-  private def advance: Unit = {
+  private def advance(): Unit = {
     val u = math.max(rng.nextDouble(), epsilon)
     val k = (math.log(u) / (-f)).toInt
     iterDrop(k)
@@ -343,7 +349,7 @@ class GapSamplingReplacementIterator[T: ClassTag](
   }
 
   /** advance to first sample as part of object construction. */
-  advance
+  advance()
   // Attempting to invoke this closer to the top with other object initialization
   // was causing it to break in strange ways, so I'm invoking it last, which seems to
   // work reliably.
diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
index 2ae308dacf..9e29bf9d61 100644
--- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
@@ -311,7 +311,7 @@ private[random] class AcceptanceResult(var numItems: Long = 0L, var numAccepted:
   var acceptBound: Double = Double.NaN // upper bound for accepting item instantly
   var waitListBound: Double = Double.NaN // upper bound for adding item to waitlist
 
-  def areBoundsEmpty = acceptBound.isNaN || waitListBound.isNaN
+  def areBoundsEmpty: Boolean = acceptBound.isNaN || waitListBound.isNaN
 
   def merge(other: Option[AcceptanceResult]): Unit = {
     if (other.isDefined) {
diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
index 467b890fb4..c4a7b4441c 100644
--- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
+++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
@@ -83,7 +83,7 @@ private[spark] object XORShiftRandom {
    * @return Map of execution times for {@link java.util.Random java.util.Random}
    * and XORShift
    */
-  def benchmark(numIters: Int) = {
+  def benchmark(numIters: Int): Map[String, Long] = {
 
     val seed = 1L
     val million = 1e6.toInt
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 328d59485a..56f5dbe53f 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -44,7 +44,13 @@ object MimaExcludes {
             // the maven-generated artifacts in 1.3.
             excludePackage("org.spark-project.jetty"),
             MimaBuild.excludeSparkPackage("unused"),
-            ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional")
+            ProblemFilters.exclude[MissingClassProblem]("com.google.common.base.Optional"),
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.rdd.JdbcRDD.compute"),
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.broadcast.HttpBroadcastFactory.newBroadcast"),
+            ProblemFilters.exclude[IncompatibleResultTypeProblem](
+              "org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast")
           )
 
         case v if v.startsWith("1.3") =>
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 0ff521706c..459a5035d4 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -137,9 +137,9 @@
  <!--   <parameter name="maxMethods"><![CDATA[30]]></parameter> -->
  <!--  </parameters> -->
  <!-- </check> -->
- <!-- <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="true"></check> -->
+ <check level="error" class="org.scalastyle.scalariform.PublicMethodsHaveTypeChecker" enabled="false"></check>
  <check level="error" class="org.scalastyle.file.NewLineAtEofChecker" enabled="true"></check>
  <check level="error" class="org.scalastyle.file.NoNewLineAtEofChecker" enabled="false"></check>
- <check level="error" class="org.apache.spark.scalastyle.NonASCIICharacterChecker" enabled="true"></check>
+ <check level="error" class="org.scalastyle.scalariform.NonASCIICharacterChecker" enabled="true"></check>
  <check level="error" class="org.scalastyle.scalariform.SpaceAfterCommentStartChecker" enabled="true"></check>
 </scalastyle>
-- 
GitLab