Repository: spark
Updated Branches:
  refs/heads/master bfd3ee9f7 -> 4ce2782a6


http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
----------------------------------------------------------------------
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 8116417..f703e50 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()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala
----------------------------------------------------------------------
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 132502b..95e2d68 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)
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
----------------------------------------------------------------------
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 120c327..0186eb3 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
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
----------------------------------------------------------------------
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 e5e1cf5..134abea 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))
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
----------------------------------------------------------------------
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 def49e8..7d75929 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)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala
----------------------------------------------------------------------
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 b86abbd..65fa817 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)
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
----------------------------------------------------------------------
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 0c24ad2..adfa6bb 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) {

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
----------------------------------------------------------------------
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 b5022fe..f078641 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" />

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
----------------------------------------------------------------------
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 fc18446..19ac7a8 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),

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala
----------------------------------------------------------------------
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 3afd7ef..69053fe 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
----------------------------------------------------------------------
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 937d95a..949e80d 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
----------------------------------------------------------------------
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 b2bbfde..7ffcf29 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))

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
----------------------------------------------------------------------
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 110f878..e034428 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>

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
----------------------------------------------------------------------
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 937261d..1bd2d87 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)) 
{

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala
----------------------------------------------------------------------
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 dbf1cee..711a369 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
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala
----------------------------------------------------------------------
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 a81291d..045bd78 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)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala
----------------------------------------------------------------------
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 3903102..9044aae 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
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/Distribution.scala
----------------------------------------------------------------------
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 a465298..9aea8ef 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.

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/ManualClock.scala
----------------------------------------------------------------------
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 cf89c17..1718554 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()
+  }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
----------------------------------------------------------------------
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 ac40f19..375ed43 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)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/MutablePair.scala
----------------------------------------------------------------------
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 74fa77b..dad8885 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[_,_]]
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala
----------------------------------------------------------------------
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 6d8d9e8..73d126f 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)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala
----------------------------------------------------------------------
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 770ff9d..a06b6f8 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/StatCounter.scala
----------------------------------------------------------------------
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 d80eed4..8586da1 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)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala
----------------------------------------------------------------------
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 f5be585..310c0c1 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() {

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/Utils.scala
----------------------------------------------------------------------
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 fa56bb0..d9a6716 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")
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala
----------------------------------------------------------------------
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 af1f646..f79e8e0 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
----------------------------------------------------------------------
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 8a0f5a6..9ff4744 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 = {

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
----------------------------------------------------------------------
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 d69f2d9..3262e67 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
----------------------------------------------------------------------
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 b8de4ff..c52591b 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
----------------------------------------------------------------------
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 4e363b7..c80057f 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 = {

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
----------------------------------------------------------------------
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 2e1ef06..61e2264 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
----------------------------------------------------------------------
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 c5268c0..bdbca00 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
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala
----------------------------------------------------------------------
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 1d54670..14b6ba4 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")

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala
----------------------------------------------------------------------
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 76e7a27..786b97a 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.

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala
----------------------------------------------------------------------
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 2ae308d..9e29bf9 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) {

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala
----------------------------------------------------------------------
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 467b890..c4a7b44 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

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/project/MimaExcludes.scala
----------------------------------------------------------------------
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 328d594..56f5dbe 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") =>

http://git-wip-us.apache.org/repos/asf/spark/blob/4ce2782a/scalastyle-config.xml
----------------------------------------------------------------------
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 0ff5217..459a503 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>


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to