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 d3bbbaffd59a..fd846545d689 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -104,7 +104,8 @@ case class SparkListenerJobEnd( extends SparkListenerEvent @DeveloperApi -case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) +case class SparkListenerEnvironmentUpdate( + environmentDetails: Map[String, collection.Seq[(String, String)]]) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 287bf2165c90..0e5688bb404e 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -168,7 +168,7 @@ private[spark] class AppStatusListener( override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate): Unit = { val details = event.environmentDetails - val jvmInfo = Map(details("JVM Information"): _*) + val jvmInfo = details("JVM Information").toMap val runtime = new v1.RuntimeInfo( jvmInfo.get("Java Version").orNull, jvmInfo.get("Java Home").orNull, diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index ebf52189796a..70fcbfd2d519 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -704,7 +704,7 @@ private[spark] class AppStatusStore( store.read(classOf[RDDOperationGraphWrapper], stageId).toRDDOperationGraph() } - def operationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = { + def operationGraphForJob(jobId: Int): collection.Seq[RDDOperationGraph] = { val job = store.read(classOf[JobDataWrapper], jobId) val stages = job.info.stageIds.sorted diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 6437d2f0732b..884d5337012a 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -543,14 +543,14 @@ private class LiveRDDPartition(val blockName: String, rddLevel: StorageLevel) { var value: v1.RDDPartitionInfo = null - def executors: Seq[String] = value.executors + def executors: collection.Seq[String] = value.executors def memoryUsed: Long = value.memoryUsed def diskUsed: Long = value.diskUsed def update( - executors: Seq[String], + executors: collection.Seq[String], memoryUsed: Long, diskUsed: Long): Unit = { val level = StorageLevel(diskUsed > 0, memoryUsed > 0, rddLevel.useOffHeap, diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 70c74b418e3c..025943f628be 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -39,7 +39,7 @@ case class ApplicationInfo private[spark]( maxCores: Option[Int], coresPerExecutor: Option[Int], memoryPerExecutorMB: Option[Int], - attempts: Seq[ApplicationAttemptInfo]) + attempts: collection.Seq[ApplicationAttemptInfo]) @JsonIgnoreProperties( value = Array("startTimeEpoch", "endTimeEpoch", "lastUpdatedEpoch"), @@ -196,7 +196,7 @@ class JobData private[spark]( val description: Option[String], val submissionTime: Option[Date], val completionTime: Option[Date], - val stageIds: Seq[Int], + val stageIds: collection.Seq[Int], val jobGroup: Option[String], val status: JobExecutionStatus, val numTasks: Int, @@ -220,8 +220,8 @@ class RDDStorageInfo private[spark]( val storageLevel: String, val memoryUsed: Long, val diskUsed: Long, - val dataDistribution: Option[Seq[RDDDataDistribution]], - val partitions: Option[Seq[RDDPartitionInfo]]) + val dataDistribution: Option[collection.Seq[RDDDataDistribution]], + val partitions: Option[collection.Seq[RDDPartitionInfo]]) class RDDDataDistribution private[spark]( val address: String, @@ -242,7 +242,7 @@ class RDDPartitionInfo private[spark]( val storageLevel: String, val memoryUsed: Long, val diskUsed: Long, - val executors: Seq[String]) + val executors: collection.Seq[String]) class StageData private[spark]( val status: StageStatus, @@ -318,7 +318,7 @@ class TaskData private[spark]( val status: String, val taskLocality: String, val speculative: Boolean, - val accumulatorUpdates: Seq[AccumulableInfo], + val accumulatorUpdates: collection.Seq[AccumulableInfo], val errorMessage: Option[String] = None, val taskMetrics: Option[TaskMetrics] = None, val executorLogs: Map[String, String], @@ -456,12 +456,12 @@ class VersionInfo private[spark]( // REST call, they are not stored with it. class ApplicationEnvironmentInfo private[spark] ( val runtime: RuntimeInfo, - val sparkProperties: Seq[(String, String)], - val hadoopProperties: Seq[(String, String)], - val systemProperties: Seq[(String, String)], - val metricsProperties: Seq[(String, String)], - val classpathEntries: Seq[(String, String)], - val resourceProfiles: Seq[ResourceProfileInfo]) + val sparkProperties: collection.Seq[(String, String)], + val hadoopProperties: collection.Seq[(String, String)], + val systemProperties: collection.Seq[(String, String)], + val metricsProperties: collection.Seq[(String, String)], + val classpathEntries: collection.Seq[(String, String)], + val resourceProfiles: collection.Seq[ResourceProfileInfo]) class RuntimeInfo private[spark]( val javaVersion: String, diff --git a/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationEnvironmentInfoWrapperSerializer.scala b/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationEnvironmentInfoWrapperSerializer.scala index 89fb4120d8db..33a18daacbc6 100644 --- a/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationEnvironmentInfoWrapperSerializer.scala +++ b/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationEnvironmentInfoWrapperSerializer.scala @@ -86,13 +86,13 @@ class ApplicationEnvironmentInfoWrapperSerializer extends ProtobufSerDe { } new ApplicationEnvironmentInfo( runtime = runtime, - sparkProperties = info.getSparkPropertiesList.asScala.map(pairSSToTuple).toSeq, - hadoopProperties = info.getHadoopPropertiesList.asScala.map(pairSSToTuple).toSeq, - systemProperties = info.getSystemPropertiesList.asScala.map(pairSSToTuple).toSeq, - metricsProperties = info.getMetricsPropertiesList.asScala.map(pairSSToTuple).toSeq, - classpathEntries = info.getClasspathEntriesList.asScala.map(pairSSToTuple).toSeq, + sparkProperties = info.getSparkPropertiesList.asScala.map(pairSSToTuple), + hadoopProperties = info.getHadoopPropertiesList.asScala.map(pairSSToTuple), + systemProperties = info.getSystemPropertiesList.asScala.map(pairSSToTuple), + metricsProperties = info.getMetricsPropertiesList.asScala.map(pairSSToTuple), + classpathEntries = info.getClasspathEntriesList.asScala.map(pairSSToTuple), resourceProfiles = - info.getResourceProfilesList.asScala.map(deserializeResourceProfileInfo).toSeq + info.getResourceProfilesList.asScala.map(deserializeResourceProfileInfo) ) } diff --git a/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationInfoWrapperSerializer.scala b/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationInfoWrapperSerializer.scala index cfb9fa20764c..5a2accb75067 100644 --- a/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationInfoWrapperSerializer.scala +++ b/core/src/main/scala/org/apache/spark/status/protobuf/ApplicationInfoWrapperSerializer.scala @@ -74,7 +74,7 @@ class ApplicationInfoWrapperSerializer extends ProtobufSerDe { val maxCores = getOptional(info.hasMaxCores, info.getMaxCores) val coresPerExecutor = getOptional(info.hasCoresPerExecutor, info.getCoresPerExecutor) val memoryPerExecutorMB = getOptional(info.hasMemoryPerExecutorMb, info.getMemoryPerExecutorMb) - val attempts = info.getAttemptsList.asScala.map(deserializeApplicationAttemptInfo).toSeq + val attempts = info.getAttemptsList.asScala.map(deserializeApplicationAttemptInfo) ApplicationInfo( id = info.getId, name = info.getName, diff --git a/core/src/main/scala/org/apache/spark/status/protobuf/JobDataWrapperSerializer.scala b/core/src/main/scala/org/apache/spark/status/protobuf/JobDataWrapperSerializer.scala index a5db51345245..98ac2d643c96 100644 --- a/core/src/main/scala/org/apache/spark/status/protobuf/JobDataWrapperSerializer.scala +++ b/core/src/main/scala/org/apache/spark/status/protobuf/JobDataWrapperSerializer.scala @@ -97,7 +97,7 @@ class JobDataWrapperSerializer extends ProtobufSerDe { description = description, submissionTime = submissionTime, completionTime = completionTime, - stageIds = info.getStageIdsList.asScala.map(_.toInt).toSeq, + stageIds = info.getStageIdsList.asScala.map(_.toInt), jobGroup = jobGroup, status = status, numTasks = info.getNumTasks, diff --git a/core/src/main/scala/org/apache/spark/status/protobuf/RDDStorageInfoWrapperSerializer.scala b/core/src/main/scala/org/apache/spark/status/protobuf/RDDStorageInfoWrapperSerializer.scala index 5779c1e62428..be6fe1f83cdf 100644 --- a/core/src/main/scala/org/apache/spark/status/protobuf/RDDStorageInfoWrapperSerializer.scala +++ b/core/src/main/scala/org/apache/spark/status/protobuf/RDDStorageInfoWrapperSerializer.scala @@ -96,10 +96,10 @@ class RDDStorageInfoWrapperSerializer extends ProtobufSerDe { if (info.getDataDistributionList.isEmpty) { None } else { - Some(info.getDataDistributionList.asScala.map(deserializeRDDDataDistribution).toSeq) + Some(info.getDataDistributionList.asScala.map(deserializeRDDDataDistribution)) }, partitions = - Some(info.getPartitionsList.asScala.map(deserializeRDDPartitionInfo).toSeq) + Some(info.getPartitionsList.asScala.map(deserializeRDDPartitionInfo)) ) } @@ -126,7 +126,7 @@ class RDDStorageInfoWrapperSerializer extends ProtobufSerDe { storageLevel = info.getStorageLevel, memoryUsed = info.getMemoryUsed, diskUsed = info.getDiskUsed, - executors = info.getExecutorsList.asScala.toSeq + executors = info.getExecutorsList.asScala ) } } diff --git a/core/src/main/scala/org/apache/spark/status/protobuf/TaskDataWrapperSerializer.scala b/core/src/main/scala/org/apache/spark/status/protobuf/TaskDataWrapperSerializer.scala index c4c8fd0b5938..80f258aead65 100644 --- a/core/src/main/scala/org/apache/spark/status/protobuf/TaskDataWrapperSerializer.scala +++ b/core/src/main/scala/org/apache/spark/status/protobuf/TaskDataWrapperSerializer.scala @@ -102,7 +102,7 @@ class TaskDataWrapperSerializer extends ProtobufSerDe { status = weakIntern(binary.getStatus), taskLocality = weakIntern(binary.getTaskLocality), speculative = binary.getSpeculative, - accumulatorUpdates = accumulatorUpdates.toSeq, + accumulatorUpdates = accumulatorUpdates, errorMessage = getOptional(binary.hasErrorMessage, binary.getErrorMessage), hasMetrics = binary.getHasMetrics, executorDeserializeTime = binary.getExecutorDeserializeTime, diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 895fb586536b..d135986855f4 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -180,7 +180,7 @@ private[spark] class TaskDataWrapper( @KVIndexParam(value = TaskIndexNames.LOCALITY, parent = TaskIndexNames.STAGE) val taskLocality: String, val speculative: Boolean, - val accumulatorUpdates: Seq[AccumulableInfo], + val accumulatorUpdates: collection.Seq[AccumulableInfo], val errorMessage: Option[String], val hasMetrics: Boolean, diff --git a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala index a002af70a919..7155726ed427 100644 --- a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala @@ -43,7 +43,7 @@ private[spark] abstract class PagedDataSource[T](val pageSize: Int) { /** * Slice a range of data. */ - protected def sliceData(from: Int, to: Int): Seq[T] + protected def sliceData(from: Int, to: Int): collection.Seq[T] /** * Slice the data for this page @@ -76,7 +76,7 @@ private[spark] abstract class PagedDataSource[T](val pageSize: Int) { * The data returned by `PagedDataSource.pageData`, including the page number, the number of total * pages and the data in this page. */ -private[ui] case class PageData[T](totalPage: Int, data: Seq[T]) +private[ui] case class PageData[T](totalPage: Int, data: collection.Seq[T]) /** * A paged table that will generate a HTML table for a specified page and also the page navigation. 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 aca03fde4c96..e03324ceb1b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -492,7 +492,8 @@ private[spark] object UIUtils extends Logging { } /** Return a "DAG visualization" DOM element that expands into a visualization for a job. */ - def showDagVizForJob(jobId: Int, graphs: Seq[RDDOperationGraph]): Seq[Node] = { + def showDagVizForJob(jobId: Int, + graphs: collection.Seq[RDDOperationGraph]): collection.Seq[Node] = { showDagViz(graphs, forJob = true) } @@ -503,7 +504,8 @@ private[spark] object UIUtils extends Logging { * a format that is expected by spark-dag-viz.js. Any changes in the format here must be * reflected there. */ - private def showDagViz(graphs: Seq[RDDOperationGraph], forJob: Boolean): Seq[Node] = { + private def showDagViz( + graphs: collection.Seq[RDDOperationGraph], forJob: Boolean): collection.Seq[Node] = {