diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
index 39fb3b249d7315c86761709dfe4d0835a7d377ff..aa0b37323132747376702dd387d5416905610ebf 100644
--- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
+++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java
@@ -155,11 +155,7 @@ public class TaskMemoryManager {
         for (MemoryConsumer c: consumers) {
           if (c != consumer && c.getUsed() > 0 && c.getMode() == mode) {
             long key = c.getUsed();
-            List<MemoryConsumer> list = sortedConsumers.get(key);
-            if (list == null) {
-              list = new ArrayList<>(1);
-              sortedConsumers.put(key, list);
-            }
+            List<MemoryConsumer> list = sortedConsumers.computeIfAbsent(key, k -> new ArrayList<>(1));
             list.add(c);
           }
         }
diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
index 9307eb93a5b202ac9a102d375351552bc88017ba..b38639e854815f1af971fdf65d0dd33f512cca24 100644
--- a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
+++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java
@@ -19,6 +19,7 @@ package org.apache.spark.status.api.v1;
 
 import org.apache.spark.util.EnumUtil;
 
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -30,9 +31,7 @@ public enum TaskSorting {
   private final Set<String> alternateNames;
   TaskSorting(String... names) {
     alternateNames = new HashSet<>();
-    for (String n: names) {
-      alternateNames.add(n);
-    }
+    Collections.addAll(alternateNames, names);
   }
 
   public static TaskSorting fromString(String str) {
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 2e991ce394c42bb59feffac1a82ef34637ab82fe..c216fe477fd15275e4e4ba5b076f55121f293acb 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -71,8 +71,7 @@ private[spark] object CompressionCodec {
       val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf])
       Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec])
     } catch {
-      case e: ClassNotFoundException => None
-      case e: IllegalArgumentException => None
+      case _: ClassNotFoundException | _: IllegalArgumentException => None
     }
     codec.getOrElse(throw new IllegalArgumentException(s"Codec [$codecName] is not available. " +
       s"Consider setting $configKey=$FALLBACK_COMPRESSION_CODEC"))
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index a9480cc220c8de17ba5c4bb0155dd8d83e2ca8fc..8b75f5d8fe1a8895892766cf4c77a5dbffa287e5 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -124,7 +124,7 @@ private[spark] abstract class WebUI(
 
   /** Bind to the HTTP server behind this web interface. */
   def bind(): Unit = {
-    assert(!serverInfo.isDefined, s"Attempted to bind $className more than once!")
+    assert(serverInfo.isEmpty, s"Attempted to bind $className more than once!")
     try {
       val host = Option(conf.getenv("SPARK_LOCAL_IP")).getOrElse("0.0.0.0")
       serverInfo = Some(startJettyServer(host, port, sslOptions, handlers, conf, name))
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
index c6a07445f2a35e70a8208ddf492e918177d1431e..dbcc6402bc309f62bc286abb8ee132f07d731766 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala
@@ -49,7 +49,7 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage
       }.map { thread =>
         val threadId = thread.threadId
         val blockedBy = thread.blockedByThreadId match {
-          case Some(blockedByThreadId) =>
+          case Some(_) =>
             <div>
               Blocked by <a href={s"#${thread.blockedByThreadId}_td_id"}>
               Thread {thread.blockedByThreadId} {thread.blockedByLock}</a>
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
index 2d1691e55c428466daea7900f391e9611a512d22..d849ce76a9e3c3cf3badf917dad5965280163db3 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -48,7 +48,6 @@ private[ui] class ExecutorsPage(
     parent: ExecutorsTab,
     threadDumpEnabled: Boolean)
   extends WebUIPage("") {
-  private val listener = parent.listener
 
   def render(request: HttpServletRequest): Seq[Node] = {
     val content =
@@ -59,7 +58,7 @@ private[ui] class ExecutorsPage(
           <script src={UIUtils.prependBaseUri("/static/executorspage.js")}></script> ++
           <script>setThreadDumpEnabled({threadDumpEnabled})</script>
         }
-      </div>;
+      </div>
 
     UIUtils.headerSparkPage("Executors", content, parent, useDataTables = true)
   }
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 8ae712f8ed323642eb157ff690636f6526d866b2..03851293eb2f1c222d0a38ab833a276c3e6062b9 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
@@ -64,7 +64,7 @@ private[ui] case class ExecutorTaskSummary(
 @DeveloperApi
 class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: SparkConf)
     extends SparkListener {
-  var executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]()
+  val executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]()
   var executorEvents = new ListBuffer[SparkListenerEvent]()
 
   private val maxTimelineExecutors = conf.getInt("spark.ui.timeline.executors.maximum", 1000)
@@ -137,7 +137,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar
           // could have failed half-way through. The correct fix would be to keep track of the
           // metrics added by each attempt, but this is much more complicated.
           return
-        case e: ExceptionFailure =>
+        case _: ExceptionFailure =>
           taskSummary.tasksFailed += 1
         case _ =>
           taskSummary.tasksComplete += 1
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
index fe6ca1099e6b0154f551fac6c94701a8faf615a0..2b0816e35747d966679e5f7bd41e298a886299f4 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala
@@ -34,9 +34,9 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") {
     listener.synchronized {
       val activeStages = listener.activeStages.values.toSeq
       val pendingStages = listener.pendingStages.values.toSeq
-      val completedStages = listener.completedStages.reverse.toSeq
+      val completedStages = listener.completedStages.reverse
       val numCompletedStages = listener.numCompletedStages
-      val failedStages = listener.failedStages.reverse.toSeq
+      val failedStages = listener.failedStages.reverse
       val numFailedStages = listener.numFailedStages
       val subPath = "stages"
 
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
index 52f41298a17295cc1172eb5d31b41a0ab3b5b7bc..382a6f979f2e60f3a041926d85f3b70e947f9170 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
@@ -133,9 +133,9 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
             </td>
             <td>{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}</td>
             <td sorttable_customkey={v.taskTime.toString}>{UIUtils.formatDuration(v.taskTime)}</td>
-            <td>{v.failedTasks + v.succeededTasks + v.reasonToNumKilled.map(_._2).sum}</td>
+            <td>{v.failedTasks + v.succeededTasks + v.reasonToNumKilled.values.sum}</td>
             <td>{v.failedTasks}</td>
-            <td>{v.reasonToNumKilled.map(_._2).sum}</td>
+            <td>{v.reasonToNumKilled.values.sum}</td>
             <td>{v.succeededTasks}</td>
             {if (stageData.hasInput) {
               <td sorttable_customkey={v.inputBytes.toString}>
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 1cf03e1541d141cd4a6daeed6f4ab63d6a7fc659..f78db5ab80d15c03c3b0da68a2658e4d71f448c6 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
@@ -226,7 +226,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
         trimJobsIfNecessary(completedJobs)
         jobData.status = JobExecutionStatus.SUCCEEDED
         numCompletedJobs += 1
-      case JobFailed(exception) =>
+      case JobFailed(_) =>
         failedJobs += jobData
         trimJobsIfNecessary(failedJobs)
         jobData.status = JobExecutionStatus.FAILED
@@ -284,7 +284,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging {
     ) {
       jobData.numActiveStages -= 1
       if (stage.failureReason.isEmpty) {
-        if (!stage.submissionTime.isEmpty) {
+        if (stage.submissionTime.isDefined) {
           jobData.completedStageIndices.add(stage.stageId)
         }
       } else {
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 ff17775008acc48d2a26ce9d7084c5ae6b1d0793..19325a2dc9169e7fbdeeb2de115478425db1b1bd 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
@@ -142,7 +142,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
 
       val allAccumulables = progressListener.stageIdToData((stageId, stageAttemptId)).accumulables
       val externalAccumulables = allAccumulables.values.filter { acc => !acc.internal }
-      val hasAccumulators = externalAccumulables.size > 0
+      val hasAccumulators = externalAccumulables.nonEmpty
 
       val summary =
         <div>
@@ -339,7 +339,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
       val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.metrics.isDefined)
 
       val summaryTable: Option[Seq[Node]] =
-        if (validTasks.size == 0) {
+        if (validTasks.isEmpty) {
           None
         }
         else {
@@ -786,8 +786,8 @@ private[ui] object StagePage {
       info: TaskInfo, metrics: TaskMetricsUIData, currentTime: Long): Long = {
     if (info.finished) {
       val totalExecutionTime = info.finishTime - info.launchTime
-      val executorOverhead = (metrics.executorDeserializeTime +
-        metrics.resultSerializationTime)
+      val executorOverhead = metrics.executorDeserializeTime +
+        metrics.resultSerializationTime
       math.max(
         0,
         totalExecutionTime - metrics.executorRunTime - executorOverhead -
@@ -872,7 +872,7 @@ private[ui] class TaskDataSource(
   // so that we can avoid creating duplicate contents during sorting the data
   private val data = tasks.map(taskRow).sorted(ordering(sortColumn, desc))
 
-  private var _slicedTaskIds: Set[Long] = null
+  private var _slicedTaskIds: Set[Long] = _
 
   override def dataSize: Int = data.size
 
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index f4caad0f58715900283f5e620d5020529c19685f..256b726fa7eea54cb6fd3f397b14d9a102d40e82 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -412,7 +412,7 @@ private[ui] class StageDataSource(
   // so that we can avoid creating duplicate contents during sorting the data
   private val data = stages.map(stageRow).sorted(ordering(sortColumn, desc))
 
-  private var _slicedStageIds: Set[Int] = null
+  private var _slicedStageIds: Set[Int] = _
 
   override def dataSize: Int = data.size
 
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
index 76d7c6d414bcfeedb8d2ca22b2b4d595143669be..aa84788f1df880a6d95ecf4d73c786d730ad19ee 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
@@ -151,7 +151,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") {
   /** Render a stream block */
   private def streamBlockTableRow(block: (BlockId, Seq[BlockUIData])): Seq[Node] = {
     val replications = block._2
-    assert(replications.size > 0) // This must be true because it's the result of "groupBy"
+    assert(replications.nonEmpty) // This must be true because it's the result of "groupBy"
     if (replications.size == 1) {
       streamBlockTableSubrow(block._1, replications.head, replications.size, true)
     } else {